You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by sr...@apache.org on 2017/08/24 18:22:32 UTC

[01/11] flink git commit: [FLINK-7460] [state backends] Close all ColumnFamilyHandles when restoring from rescaled incremental checkpoints

Repository: flink
Updated Branches:
  refs/heads/master 3f4de57b1 -> 6642768ad


[FLINK-7460] [state backends] Close all ColumnFamilyHandles when restoring from rescaled incremental checkpoints


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

Branch: refs/heads/master
Commit: ca87bec4f79c32c9f6cf7a4aa96866f6fac957e0
Parents: 3f4de57
Author: Stefan Richter <s....@data-artisans.com>
Authored: Mon Aug 14 14:01:03 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Thu Aug 24 17:17:39 2017 +0200

----------------------------------------------------------------------
 .../state/RocksDBKeyedStateBackend.java         | 453 ++++++++++---------
 .../state/RocksDBStateBackendTest.java          | 313 +++++++------
 .../runtime/state/StateBackendTestBase.java     | 244 +++++-----
 3 files changed, 546 insertions(+), 464 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ca87bec4/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index 756cfdd..b7f386d 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -105,7 +105,9 @@ import java.io.ObjectInputStream;
 import java.io.OutputStream;
 import java.nio.file.Files;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -138,6 +140,9 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	/** The name of the merge operator in RocksDB. Do not change except you know exactly what you do. */
 	public static final String MERGE_OPERATOR_NAME = "stringappendtest";
 
+	/** Bytes for the name of the column decriptor for the default column family. */
+	public static final byte[] DEFAULT_COLUMN_FAMILY_NAME_BYTES = "default".getBytes(ConfigConstants.DEFAULT_CHARSET);
+
 	private final String operatorIdentifier;
 
 	/** The column family options from the options factory. */
@@ -196,7 +201,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	private final SortedMap<Long, Set<StateHandleID>> materializedSstFiles;
 
 	/** The identifier of the last completed checkpoint. */
-	private long lastCompletedCheckpointId = -1;
+	private long lastCompletedCheckpointId = -1L;
 
 	/** Unique ID of this backend. */
 	private UUID backendUID;
@@ -204,17 +209,17 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	private static final String SST_FILE_SUFFIX = ".sst";
 
 	public RocksDBKeyedStateBackend(
-			String operatorIdentifier,
-			ClassLoader userCodeClassLoader,
-			File instanceBasePath,
-			DBOptions dbOptions,
-			ColumnFamilyOptions columnFamilyOptions,
-			TaskKvStateRegistry kvStateRegistry,
-			TypeSerializer<K> keySerializer,
-			int numberOfKeyGroups,
-			KeyGroupRange keyGroupRange,
-			ExecutionConfig executionConfig,
-			boolean enableIncrementalCheckpointing
+		String operatorIdentifier,
+		ClassLoader userCodeClassLoader,
+		File instanceBasePath,
+		DBOptions dbOptions,
+		ColumnFamilyOptions columnFamilyOptions,
+		TaskKvStateRegistry kvStateRegistry,
+		TypeSerializer<K> keySerializer,
+		int numberOfKeyGroups,
+		KeyGroupRange keyGroupRange,
+		ExecutionConfig executionConfig,
+		boolean enableIncrementalCheckpointing
 	) throws IOException {
 
 		super(kvStateRegistry, keySerializer, userCodeClassLoader, numberOfKeyGroups, keyGroupRange, executionConfig);
@@ -253,10 +258,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		this.restoredKvStateMetaInfos = new HashMap<>();
 		this.materializedSstFiles = new TreeMap<>();
 		this.backendUID = UUID.randomUUID();
-
-		LOG.debug("Setting initial backend ID in RocksDBKeyedStateBackend for operator {} to {}.",
-			this.operatorIdentifier,
-			this.backendUID);
+		LOG.debug("Setting initial keyed backend uid for operator {} to {}.", this.operatorIdentifier, this.backendUID);
 	}
 
 	/**
@@ -277,7 +279,6 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				// DB is closed. So we start with the ones created by Flink...
 				for (Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>> columnMetaData :
 					kvStateInformation.values()) {
-
 					IOUtils.closeQuietly(columnMetaData.f0);
 				}
 
@@ -328,10 +329,10 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	 */
 	@Override
 	public RunnableFuture<KeyedStateHandle> snapshot(
-			final long checkpointId,
-			final long timestamp,
-			final CheckpointStreamFactory streamFactory,
-			CheckpointOptions checkpointOptions) throws Exception {
+		final long checkpointId,
+		final long timestamp,
+		final CheckpointStreamFactory streamFactory,
+		CheckpointOptions checkpointOptions) throws Exception {
 
 		if (checkpointOptions.getCheckpointType() != CheckpointOptions.CheckpointType.SAVEPOINT &&
 			enableIncrementalCheckpointing) {
@@ -342,9 +343,9 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	private RunnableFuture<KeyedStateHandle> snapshotIncrementally(
-			final long checkpointId,
-			final long checkpointTimestamp,
-			final CheckpointStreamFactory checkpointStreamFactory) throws Exception {
+		final long checkpointId,
+		final long checkpointTimestamp,
+		final CheckpointStreamFactory checkpointStreamFactory) throws Exception {
 
 		final RocksDBIncrementalSnapshotOperation<K> snapshotOperation =
 			new RocksDBIncrementalSnapshotOperation<>(
@@ -361,7 +362,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			if (!hasRegisteredState()) {
 				if (LOG.isDebugEnabled()) {
 					LOG.debug("Asynchronous RocksDB snapshot performed on empty keyed state at " +
-							checkpointTimestamp + " . Returning null.");
+						checkpointTimestamp + " . Returning null.");
 				}
 				return DoneFuture.nullValue();
 			}
@@ -391,9 +392,9 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	private RunnableFuture<KeyedStateHandle> snapshotFully(
-			final long checkpointId,
-			final long timestamp,
-			final CheckpointStreamFactory streamFactory) throws Exception {
+		final long checkpointId,
+		final long timestamp,
+		final CheckpointStreamFactory streamFactory) throws Exception {
 
 		long startTime = System.currentTimeMillis();
 
@@ -406,7 +407,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				if (!hasRegisteredState()) {
 					if (LOG.isDebugEnabled()) {
 						LOG.debug("Asynchronous RocksDB snapshot performed on empty keyed state at " + timestamp +
-								" . Returning null.");
+							" . Returning null.");
 					}
 					return DoneFuture.nullValue();
 				}
@@ -419,52 +420,52 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 		// implementation of the async IO operation, based on FutureTask
 		AbstractAsyncIOCallable<KeyedStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream> ioCallable =
-				new AbstractAsyncIOCallable<KeyedStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream>() {
+			new AbstractAsyncIOCallable<KeyedStateHandle, CheckpointStreamFactory.CheckpointStateOutputStream>() {
 
-					@Override
-					public CheckpointStreamFactory.CheckpointStateOutputStream openIOHandle() throws Exception {
-						snapshotOperation.openCheckpointStream();
-						return snapshotOperation.getOutStream();
-					}
+				@Override
+				public CheckpointStreamFactory.CheckpointStateOutputStream openIOHandle() throws Exception {
+					snapshotOperation.openCheckpointStream();
+					return snapshotOperation.getOutStream();
+				}
 
-					@Override
-					public KeyGroupsStateHandle performOperation() throws Exception {
-						long startTime = System.currentTimeMillis();
-						synchronized (asyncSnapshotLock) {
-							try {
-								// hold the db lock while operation on the db to guard us against async db disposal
-								if (db == null) {
-									throw new IOException("RocksDB closed.");
-								}
+				@Override
+				public KeyGroupsStateHandle performOperation() throws Exception {
+					long startTime = System.currentTimeMillis();
+					synchronized (asyncSnapshotLock) {
+						try {
+							// hold the db lock while operation on the db to guard us against async db disposal
+							if (db == null) {
+								throw new IOException("RocksDB closed.");
+							}
 
-								snapshotOperation.writeDBSnapshot();
+							snapshotOperation.writeDBSnapshot();
 
-							} finally {
-								snapshotOperation.closeCheckpointStream();
-							}
+						} finally {
+							snapshotOperation.closeCheckpointStream();
 						}
+					}
 
-						LOG.info("Asynchronous RocksDB snapshot ({}, asynchronous part) in thread {} took {} ms.",
-							streamFactory, Thread.currentThread(), (System.currentTimeMillis() - startTime));
+					LOG.info("Asynchronous RocksDB snapshot ({}, asynchronous part) in thread {} took {} ms.",
+						streamFactory, Thread.currentThread(), (System.currentTimeMillis() - startTime));
 
-						return snapshotOperation.getSnapshotResultStateHandle();
-					}
+					return snapshotOperation.getSnapshotResultStateHandle();
+				}
 
-					private void releaseSnapshotOperationResources(boolean canceled) {
-						// hold the db lock while operation on the db to guard us against async db disposal
-						synchronized (asyncSnapshotLock) {
-							snapshotOperation.releaseSnapshotResources(canceled);
-						}
+				private void releaseSnapshotOperationResources(boolean canceled) {
+					// hold the db lock while operation on the db to guard us against async db disposal
+					synchronized (asyncSnapshotLock) {
+						snapshotOperation.releaseSnapshotResources(canceled);
 					}
+				}
 
-					@Override
-					public void done(boolean canceled) {
-						releaseSnapshotOperationResources(canceled);
-					}
-				};
+				@Override
+				public void done(boolean canceled) {
+					releaseSnapshotOperationResources(canceled);
+				}
+			};
 
 		LOG.info("Asynchronous RocksDB snapshot (" + streamFactory + ", synchronous part) in thread " +
-				Thread.currentThread() + " took " + (System.currentTimeMillis() - startTime) + " ms.");
+			Thread.currentThread() + " took " + (System.currentTimeMillis() - startTime) + " ms.");
 
 		return AsyncStoppableTaskWithCallback.from(ioCallable);
 	}
@@ -493,8 +494,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		private KeyGroupsStateHandle snapshotResultStateHandle;
 
 		RocksDBFullSnapshotOperation(
-				RocksDBKeyedStateBackend<K> stateBackend,
-				CheckpointStreamFactory checkpointStreamFactory) {
+			RocksDBKeyedStateBackend<K> stateBackend,
+			CheckpointStreamFactory checkpointStreamFactory) {
 
 			this.stateBackend = stateBackend;
 			this.checkpointStreamFactory = checkpointStreamFactory;
@@ -523,7 +524,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		public void openCheckpointStream() throws Exception {
 			Preconditions.checkArgument(outStream == null, "Output stream for snapshot is already set.");
 			outStream = checkpointStreamFactory.
-					createCheckpointStateOutputStream(checkpointId, checkpointTimeStamp);
+				createCheckpointStateOutputStream(checkpointId, checkpointTimeStamp);
 			stateBackend.cancelStreamRegistry.registerClosable(outStream);
 			outputView = new DataOutputViewStreamWrapper(outStream);
 		}
@@ -615,11 +616,11 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		private void writeKVStateMetaData() throws IOException {
 
 			List<RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?>> metaInfoSnapshots =
-					new ArrayList<>(stateBackend.kvStateInformation.size());
+				new ArrayList<>(stateBackend.kvStateInformation.size());
 
 			int kvStateId = 0;
 			for (Map.Entry<String, Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>>> column :
-					stateBackend.kvStateInformation.entrySet()) {
+				stateBackend.kvStateInformation.entrySet()) {
 
 				metaInfoSnapshots.add(column.getValue().f1.snapshot());
 
@@ -628,7 +629,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				readOptions.setSnapshot(snapshot);
 
 				kvStateIterators.add(
-						new Tuple2<>(stateBackend.db.newIterator(column.getValue().f0, readOptions), kvStateId));
+					new Tuple2<>(stateBackend.db.newIterator(column.getValue().f0, readOptions), kvStateId));
 
 				++kvStateId;
 			}
@@ -797,10 +798,10 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		private StreamStateHandle metaStateHandle = null;
 
 		private RocksDBIncrementalSnapshotOperation(
-				RocksDBKeyedStateBackend<K> stateBackend,
-				CheckpointStreamFactory checkpointStreamFactory,
-				long checkpointId,
-				long checkpointTimestamp) {
+			RocksDBKeyedStateBackend<K> stateBackend,
+			CheckpointStreamFactory checkpointStreamFactory,
+			long checkpointId,
+			long checkpointTimestamp) {
 
 			this.stateBackend = stateBackend;
 			this.checkpointStreamFactory = checkpointStreamFactory;
@@ -886,20 +887,14 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		void takeSnapshot() throws Exception {
 			assert (Thread.holdsLock(stateBackend.asyncSnapshotLock));
 
-			final long lastCompletedCheckpoint;
-
 			// use the last completed checkpoint as the comparison base.
 			synchronized (stateBackend.materializedSstFiles) {
-				lastCompletedCheckpoint = stateBackend.lastCompletedCheckpointId;
-				baseSstFiles = stateBackend.materializedSstFiles.get(lastCompletedCheckpoint);
+				baseSstFiles = stateBackend.materializedSstFiles.get(stateBackend.lastCompletedCheckpointId);
 			}
 
-			LOG.trace("Taking incremental snapshot for checkpoint {}. Snapshot is based on last completed checkpoint {} " +
-				"assuming the following (shared) files as base: {}.", checkpointId, lastCompletedCheckpoint, baseSstFiles);
-
 			// save meta data
 			for (Map.Entry<String, Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>>> stateMetaInfoEntry
-					: stateBackend.kvStateInformation.entrySet()) {
+				: stateBackend.kvStateInformation.entrySet()) {
 				stateMetaInfoSnapshots.add(stateMetaInfoEntry.getValue().f1.snapshot());
 			}
 
@@ -1054,47 +1049,39 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	private void createDB() throws IOException {
-		db = openDB(instanceRocksDBPath.getAbsolutePath(),
-			new ArrayList<ColumnFamilyDescriptor>(),
-			null);
+		List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>(1);
+		this.db = openDB(instanceRocksDBPath.getAbsolutePath(), Collections.emptyList(), columnFamilyHandles);
+		this.defaultColumnFamily = columnFamilyHandles.get(0);
 	}
 
 	private RocksDB openDB(
-			String path,
-			List<ColumnFamilyDescriptor> stateColumnFamilyDescriptors,
-			List<ColumnFamilyHandle> stateColumnFamilyHandles) throws IOException {
+		String path,
+		List<ColumnFamilyDescriptor> stateColumnFamilyDescriptors,
+		List<ColumnFamilyHandle> stateColumnFamilyHandles) throws IOException {
 
-		List<ColumnFamilyDescriptor> columnFamilyDescriptors = new ArrayList<>(stateColumnFamilyDescriptors);
+		List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+			new ArrayList<>(1 + stateColumnFamilyDescriptors.size());
 
-		// we add the required descriptor for the default CF in last position.
-		columnFamilyDescriptors.add(
-			new ColumnFamilyDescriptor(
-				"default".getBytes(ConfigConstants.DEFAULT_CHARSET), columnOptions));
+		columnFamilyDescriptors.addAll(stateColumnFamilyDescriptors);
 
-		List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>(columnFamilyDescriptors.size());
+		// we add the required descriptor for the default CF in last position.
+		columnFamilyDescriptors.add(new ColumnFamilyDescriptor(DEFAULT_COLUMN_FAMILY_NAME_BYTES, columnOptions));
 
 		RocksDB db;
 
 		try {
 			db = RocksDB.open(
-					Preconditions.checkNotNull(dbOptions),
-					Preconditions.checkNotNull(path),
-					columnFamilyDescriptors,
-					columnFamilyHandles);
+				Preconditions.checkNotNull(dbOptions),
+				Preconditions.checkNotNull(path),
+				columnFamilyDescriptors,
+				stateColumnFamilyHandles);
 		} catch (RocksDBException e) {
 			throw new IOException("Error while opening RocksDB instance.", e);
 		}
 
-		final int defaultColumnFamilyIndex = columnFamilyHandles.size() - 1;
-
-		// extract the default column family.
-		defaultColumnFamily = columnFamilyHandles.get(defaultColumnFamilyIndex);
-
-		if (stateColumnFamilyHandles != null) {
-			// return all CFs except the default CF which is kept separately because it is not used in Flink operations.
-			stateColumnFamilyHandles.addAll(
-				columnFamilyHandles.subList(0, defaultColumnFamilyIndex));
-		}
+		// requested + default CF
+		Preconditions.checkState(1 + stateColumnFamilyDescriptors.size() == stateColumnFamilyHandles.size(),
+			"Not all requested column family handles have been created");
 
 		return db;
 	}
@@ -1135,7 +1122,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		 * @throws RocksDBException
 		 */
 		public void doRestore(Collection<KeyedStateHandle> keyedStateHandles)
-				throws IOException, StateMigrationException, ClassNotFoundException, RocksDBException {
+			throws IOException, StateMigrationException, ClassNotFoundException, RocksDBException {
 
 			rocksDBKeyedStateBackend.createDB();
 
@@ -1144,8 +1131,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 					if (!(keyedStateHandle instanceof KeyGroupsStateHandle)) {
 						throw new IllegalStateException("Unexpected state handle type, " +
-								"expected: " + KeyGroupsStateHandle.class +
-								", but found: " + keyedStateHandle.getClass());
+							"expected: " + KeyGroupsStateHandle.class +
+							", but found: " + keyedStateHandle.getClass());
 					}
 					this.currentKeyGroupsStateHandle = (KeyGroupsStateHandle) keyedStateHandle;
 					restoreKeyGroupsInStateHandle();
@@ -1161,7 +1148,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		 * @throws ClassNotFoundException
 		 */
 		private void restoreKeyGroupsInStateHandle()
-				throws IOException, StateMigrationException, RocksDBException, ClassNotFoundException {
+			throws IOException, StateMigrationException, RocksDBException, ClassNotFoundException {
 			try {
 				currentStateHandleInStream = currentKeyGroupsStateHandle.openInputStream();
 				rocksDBKeyedStateBackend.cancelStreamRegistry.registerClosable(currentStateHandleInStream);
@@ -1186,17 +1173,17 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		private void restoreKVStateMetaData() throws IOException, StateMigrationException, RocksDBException {
 
 			KeyedBackendSerializationProxy<K> serializationProxy =
-					new KeyedBackendSerializationProxy<>(rocksDBKeyedStateBackend.userCodeClassLoader);
+				new KeyedBackendSerializationProxy<>(rocksDBKeyedStateBackend.userCodeClassLoader);
 
 			serializationProxy.read(currentStateHandleInView);
 
 			// check for key serializer compatibility; this also reconfigures the
 			// key serializer to be compatible, if it is required and is possible
 			if (CompatibilityUtil.resolveCompatibilityResult(
-					serializationProxy.getKeySerializer(),
-					UnloadableDummyTypeSerializer.class,
-					serializationProxy.getKeySerializerConfigSnapshot(),
-					rocksDBKeyedStateBackend.keySerializer)
+				serializationProxy.getKeySerializer(),
+				UnloadableDummyTypeSerializer.class,
+				serializationProxy.getKeySerializerConfigSnapshot(),
+				rocksDBKeyedStateBackend.keySerializer)
 				.isRequiresMigration()) {
 
 				// TODO replace with state migration; note that key hash codes need to remain the same after migration
@@ -1208,7 +1195,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				SnappyStreamCompressionDecorator.INSTANCE : UncompressedStreamCompressionDecorator.INSTANCE;
 
 			List<RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?>> restoredMetaInfos =
-					serializationProxy.getStateMetaInfoSnapshots();
+				serializationProxy.getStateMetaInfoSnapshots();
 			currentStateHandleKVStateColumnFamilies = new ArrayList<>(restoredMetaInfos.size());
 			//rocksDBKeyedStateBackend.restoredKvStateMetaInfos = new HashMap<>(restoredMetaInfos.size());
 
@@ -1218,22 +1205,24 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 					rocksDBKeyedStateBackend.kvStateInformation.get(restoredMetaInfo.getName());
 
 				if (registeredColumn == null) {
+					byte[] nameBytes = restoredMetaInfo.getName().getBytes(ConfigConstants.DEFAULT_CHARSET);
+
 					ColumnFamilyDescriptor columnFamilyDescriptor = new ColumnFamilyDescriptor(
-						restoredMetaInfo.getName().getBytes(ConfigConstants.DEFAULT_CHARSET),
+						nameBytes,
 						rocksDBKeyedStateBackend.columnOptions);
 
 					RegisteredKeyedBackendStateMetaInfo<?, ?> stateMetaInfo =
-							new RegisteredKeyedBackendStateMetaInfo<>(
-								restoredMetaInfo.getStateType(),
-								restoredMetaInfo.getName(),
-								restoredMetaInfo.getNamespaceSerializer(),
-								restoredMetaInfo.getStateSerializer());
+						new RegisteredKeyedBackendStateMetaInfo<>(
+							restoredMetaInfo.getStateType(),
+							restoredMetaInfo.getName(),
+							restoredMetaInfo.getNamespaceSerializer(),
+							restoredMetaInfo.getStateSerializer());
 
 					rocksDBKeyedStateBackend.restoredKvStateMetaInfos.put(restoredMetaInfo.getName(), restoredMetaInfo);
 
 					ColumnFamilyHandle columnFamily = rocksDBKeyedStateBackend.db.createColumnFamily(columnFamilyDescriptor);
 
-					registeredColumn = new Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>>(columnFamily, stateMetaInfo);
+					registeredColumn = new Tuple2<>(columnFamily, stateMetaInfo);
 					rocksDBKeyedStateBackend.kvStateInformation.put(stateMetaInfo.getName(), registeredColumn);
 
 				} else {
@@ -1303,7 +1292,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		}
 
 		private List<RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?>> readMetaData(
-				StreamStateHandle metaStateHandle) throws Exception {
+			StreamStateHandle metaStateHandle) throws Exception {
 
 			FSDataInputStream inputStream = null;
 
@@ -1319,10 +1308,10 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				// check for key serializer compatibility; this also reconfigures the
 				// key serializer to be compatible, if it is required and is possible
 				if (CompatibilityUtil.resolveCompatibilityResult(
-						serializationProxy.getKeySerializer(),
-						UnloadableDummyTypeSerializer.class,
-						serializationProxy.getKeySerializerConfigSnapshot(),
-						stateBackend.keySerializer)
+					serializationProxy.getKeySerializer(),
+					UnloadableDummyTypeSerializer.class,
+					serializationProxy.getKeySerializerConfigSnapshot(),
+					stateBackend.keySerializer)
 					.isRequiresMigration()) {
 
 					// TODO replace with state migration; note that key hash codes need to remain the same after migration
@@ -1340,8 +1329,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		}
 
 		private void readStateData(
-				Path restoreFilePath,
-				StreamStateHandle remoteFileHandle) throws IOException {
+			Path restoreFilePath,
+			StreamStateHandle remoteFileHandle) throws IOException {
 
 			FileSystem restoreFileSystem = restoreFilePath.getFileSystem();
 
@@ -1378,8 +1367,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		}
 
 		private void restoreInstance(
-				IncrementalKeyedStateHandle restoreStateHandle,
-				boolean hasExtraKeys) throws Exception {
+			IncrementalKeyedStateHandle restoreStateHandle,
+			boolean hasExtraKeys) throws Exception {
 
 			// read state data
 			Path restoreInstancePath = new Path(
@@ -1399,7 +1388,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				List<RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?>> stateMetaInfoSnapshots =
 					readMetaData(restoreStateHandle.getMetaStateHandle());
 
-				List<ColumnFamilyDescriptor> columnFamilyDescriptors = new ArrayList<>();
+				List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+					new ArrayList<>(1 + stateMetaInfoSnapshots.size());
 
 				for (RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?> stateMetaInfoSnapshot : stateMetaInfoSnapshots) {
 
@@ -1413,69 +1403,78 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 				if (hasExtraKeys) {
 
-					List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+					List<ColumnFamilyHandle> columnFamilyHandles =
+						new ArrayList<>(1 + columnFamilyDescriptors.size());
 
 					try (RocksDB restoreDb = stateBackend.openDB(
-							restoreInstancePath.getPath(),
-							columnFamilyDescriptors,
-							columnFamilyHandles)) {
-
-						for (int i = 0; i < columnFamilyHandles.size(); ++i) {
-							ColumnFamilyHandle columnFamilyHandle = columnFamilyHandles.get(i);
-							ColumnFamilyDescriptor columnFamilyDescriptor = columnFamilyDescriptors.get(i);
-							RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?> stateMetaInfoSnapshot = stateMetaInfoSnapshots.get(i);
-
-							Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>> registeredStateMetaInfoEntry =
-								stateBackend.kvStateInformation.get(stateMetaInfoSnapshot.getName());
-
-							if (null == registeredStateMetaInfoEntry) {
-
-								RegisteredKeyedBackendStateMetaInfo<?, ?> stateMetaInfo =
-									new RegisteredKeyedBackendStateMetaInfo<>(
-										stateMetaInfoSnapshot.getStateType(),
+						restoreInstancePath.getPath(),
+						columnFamilyDescriptors,
+						columnFamilyHandles)) {
+
+						try {
+							// iterating only the requested descriptors automatically skips the default column family handle
+							for (int i = 0; i < columnFamilyDescriptors.size(); ++i) {
+								ColumnFamilyHandle columnFamilyHandle = columnFamilyHandles.get(i);
+								ColumnFamilyDescriptor columnFamilyDescriptor = columnFamilyDescriptors.get(i);
+								RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?> stateMetaInfoSnapshot = stateMetaInfoSnapshots.get(i);
+
+								Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>> registeredStateMetaInfoEntry =
+									stateBackend.kvStateInformation.get(stateMetaInfoSnapshot.getName());
+
+								if (null == registeredStateMetaInfoEntry) {
+
+									RegisteredKeyedBackendStateMetaInfo<?, ?> stateMetaInfo =
+										new RegisteredKeyedBackendStateMetaInfo<>(
+											stateMetaInfoSnapshot.getStateType(),
+											stateMetaInfoSnapshot.getName(),
+											stateMetaInfoSnapshot.getNamespaceSerializer(),
+											stateMetaInfoSnapshot.getStateSerializer());
+
+									registeredStateMetaInfoEntry =
+										new Tuple2<>(
+											stateBackend.db.createColumnFamily(columnFamilyDescriptor),
+											stateMetaInfo);
+
+									stateBackend.kvStateInformation.put(
 										stateMetaInfoSnapshot.getName(),
-										stateMetaInfoSnapshot.getNamespaceSerializer(),
-										stateMetaInfoSnapshot.getStateSerializer());
-
-								registeredStateMetaInfoEntry =
-									new Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>>(
-										stateBackend.db.createColumnFamily(columnFamilyDescriptor),
-										stateMetaInfo);
+										registeredStateMetaInfoEntry);
+								}
 
-								stateBackend.kvStateInformation.put(
-									stateMetaInfoSnapshot.getName(),
-									registeredStateMetaInfoEntry);
-							}
+								ColumnFamilyHandle targetColumnFamilyHandle = registeredStateMetaInfoEntry.f0;
 
-							ColumnFamilyHandle targetColumnFamilyHandle = registeredStateMetaInfoEntry.f0;
+								try (RocksIterator iterator = restoreDb.newIterator(columnFamilyHandle)) {
 
-							try (RocksIterator iterator = restoreDb.newIterator(columnFamilyHandle)) {
+									int startKeyGroup = stateBackend.getKeyGroupRange().getStartKeyGroup();
+									byte[] startKeyGroupPrefixBytes = new byte[stateBackend.keyGroupPrefixBytes];
+									for (int j = 0; j < stateBackend.keyGroupPrefixBytes; ++j) {
+										startKeyGroupPrefixBytes[j] = (byte) (startKeyGroup >>> ((stateBackend.keyGroupPrefixBytes - j - 1) * Byte.SIZE));
+									}
 
-								int startKeyGroup = stateBackend.getKeyGroupRange().getStartKeyGroup();
-								byte[] startKeyGroupPrefixBytes = new byte[stateBackend.keyGroupPrefixBytes];
-								for (int j = 0; j < stateBackend.keyGroupPrefixBytes; ++j) {
-									startKeyGroupPrefixBytes[j] = (byte) (startKeyGroup >>> ((stateBackend.keyGroupPrefixBytes - j - 1) * Byte.SIZE));
-								}
+									iterator.seek(startKeyGroupPrefixBytes);
 
-								iterator.seek(startKeyGroupPrefixBytes);
+									while (iterator.isValid()) {
 
-								while (iterator.isValid()) {
+										int keyGroup = 0;
+										for (int j = 0; j < stateBackend.keyGroupPrefixBytes; ++j) {
+											keyGroup = (keyGroup << Byte.SIZE) + iterator.key()[j];
+										}
 
-									int keyGroup = 0;
-									for (int j = 0; j < stateBackend.keyGroupPrefixBytes; ++j) {
-										keyGroup = (keyGroup << Byte.SIZE) + iterator.key()[j];
-									}
+										if (stateBackend.keyGroupRange.contains(keyGroup)) {
+											stateBackend.db.put(targetColumnFamilyHandle,
+												iterator.key(), iterator.value());
+										}
 
-									if (stateBackend.keyGroupRange.contains(keyGroup)) {
-										stateBackend.db.put(targetColumnFamilyHandle,
-											iterator.key(), iterator.value());
+										iterator.next();
 									}
-
-									iterator.next();
-								}
+								} // releases native iterator resources
+							}
+						} finally {
+							//release native tmp db column family resources
+							for (ColumnFamilyHandle columnFamilyHandle : columnFamilyHandles) {
+								IOUtils.closeQuietly(columnFamilyHandle);
 							}
 						}
-					}
+					} // releases native tmp db resources
 				} else {
 					// pick up again the old backend id, so the we can reference existing state
 					stateBackend.backendUID = restoreStateHandle.getBackendIdentifier();
@@ -1491,11 +1490,16 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 					createFileHardLinksInRestorePath(sstFiles, restoreInstancePath);
 					createFileHardLinksInRestorePath(miscFiles, restoreInstancePath);
 
-					List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+					List<ColumnFamilyHandle> columnFamilyHandles =
+						new ArrayList<>(1 + columnFamilyDescriptors.size());
+
 					stateBackend.db = stateBackend.openDB(
 						stateBackend.instanceRocksDBPath.getAbsolutePath(),
 						columnFamilyDescriptors, columnFamilyHandles);
 
+					// extract and store the default column family which is located at the last index
+					stateBackend.defaultColumnFamily = columnFamilyHandles.remove(columnFamilyHandles.size() - 1);
+
 					for (int i = 0; i < columnFamilyDescriptors.size(); ++i) {
 						RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?> stateMetaInfoSnapshot = stateMetaInfoSnapshots.get(i);
 
@@ -1509,8 +1513,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 						stateBackend.kvStateInformation.put(
 							stateMetaInfoSnapshot.getName(),
-							new Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>>(
-								columnFamilyHandle, stateMetaInfo));
+							new Tuple2<>(columnFamilyHandle, stateMetaInfo));
 					}
 
 					// use the restore sst files as the base for succeeding checkpoints
@@ -1590,10 +1593,10 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	 */
 	@SuppressWarnings("rawtypes, unchecked")
 	protected <N, S> ColumnFamilyHandle getColumnFamily(
-			StateDescriptor<?, S> descriptor, TypeSerializer<N> namespaceSerializer) throws IOException, StateMigrationException {
+		StateDescriptor<?, S> descriptor, TypeSerializer<N> namespaceSerializer) throws IOException, StateMigrationException {
 
 		Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>> stateInfo =
-				kvStateInformation.get(descriptor.getName());
+			kvStateInformation.get(descriptor.getName());
 
 		RegisteredKeyedBackendStateMetaInfo<N, S> newMetaInfo = new RegisteredKeyedBackendStateMetaInfo<>(
 			descriptor.getType(),
@@ -1625,16 +1628,16 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 			// check compatibility results to determine if state migration is required
 			CompatibilityResult<N> namespaceCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-					restoredMetaInfo.getNamespaceSerializer(),
-					MigrationNamespaceSerializerProxy.class,
-					restoredMetaInfo.getNamespaceSerializerConfigSnapshot(),
-					newMetaInfo.getNamespaceSerializer());
+				restoredMetaInfo.getNamespaceSerializer(),
+				MigrationNamespaceSerializerProxy.class,
+				restoredMetaInfo.getNamespaceSerializerConfigSnapshot(),
+				newMetaInfo.getNamespaceSerializer());
 
 			CompatibilityResult<S> stateCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-					restoredMetaInfo.getStateSerializer(),
-					UnloadableDummyTypeSerializer.class,
-					restoredMetaInfo.getStateSerializerConfigSnapshot(),
-					newMetaInfo.getStateSerializer());
+				restoredMetaInfo.getStateSerializer(),
+				UnloadableDummyTypeSerializer.class,
+				restoredMetaInfo.getStateSerializerConfigSnapshot(),
+				newMetaInfo.getStateSerializer());
 
 			if (!namespaceCompatibility.isRequiresMigration() && !stateCompatibility.isRequiresMigration()) {
 				stateInfo.f1 = newMetaInfo;
@@ -1645,25 +1648,31 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			}
 		}
 
-		ColumnFamilyDescriptor columnDescriptor = new ColumnFamilyDescriptor(
-				descriptor.getName().getBytes(ConfigConstants.DEFAULT_CHARSET), columnOptions);
+		byte[] nameBytes = descriptor.getName().getBytes(ConfigConstants.DEFAULT_CHARSET);
+		Preconditions.checkState(!Arrays.equals(DEFAULT_COLUMN_FAMILY_NAME_BYTES, nameBytes),
+			"The chosen state name 'default' collides with the name of the default column family!");
+
+		ColumnFamilyDescriptor columnDescriptor = new ColumnFamilyDescriptor(nameBytes, columnOptions);
+
+		final ColumnFamilyHandle columnFamily;
 
 		try {
-			ColumnFamilyHandle columnFamily = db.createColumnFamily(columnDescriptor);
-			Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<N, S>> tuple =
-					new Tuple2<>(columnFamily, newMetaInfo);
-			Map rawAccess = kvStateInformation;
-			rawAccess.put(descriptor.getName(), tuple);
-			return columnFamily;
+			columnFamily = db.createColumnFamily(columnDescriptor);
 		} catch (RocksDBException e) {
 			throw new IOException("Error creating ColumnFamilyHandle.", e);
 		}
+
+		Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<N, S>> tuple =
+			new Tuple2<>(columnFamily, newMetaInfo);
+		Map rawAccess = kvStateInformation;
+		rawAccess.put(descriptor.getName(), tuple);
+		return columnFamily;
 	}
 
 	@Override
 	protected <N, T> InternalValueState<N, T> createValueState(
-			TypeSerializer<N> namespaceSerializer,
-			ValueStateDescriptor<T> stateDesc) throws Exception {
+		TypeSerializer<N> namespaceSerializer,
+		ValueStateDescriptor<T> stateDesc) throws Exception {
 
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
 
@@ -1672,8 +1681,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 	@Override
 	protected <N, T> InternalListState<N, T> createListState(
-			TypeSerializer<N> namespaceSerializer,
-			ListStateDescriptor<T> stateDesc) throws Exception {
+		TypeSerializer<N> namespaceSerializer,
+		ListStateDescriptor<T> stateDesc) throws Exception {
 
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
 
@@ -1682,8 +1691,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 	@Override
 	protected <N, T> InternalReducingState<N, T> createReducingState(
-			TypeSerializer<N> namespaceSerializer,
-			ReducingStateDescriptor<T> stateDesc) throws Exception {
+		TypeSerializer<N> namespaceSerializer,
+		ReducingStateDescriptor<T> stateDesc) throws Exception {
 
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
 
@@ -1692,8 +1701,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 	@Override
 	protected <N, T, ACC, R> InternalAggregatingState<N, T, R> createAggregatingState(
-			TypeSerializer<N> namespaceSerializer,
-			AggregatingStateDescriptor<T, ACC, R> stateDesc) throws Exception {
+		TypeSerializer<N> namespaceSerializer,
+		AggregatingStateDescriptor<T, ACC, R> stateDesc) throws Exception {
 
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
 		return new RocksDBAggregatingState<>(columnFamily, namespaceSerializer, stateDesc, this);
@@ -1701,8 +1710,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 	@Override
 	protected <N, T, ACC> InternalFoldingState<N, T, ACC> createFoldingState(
-			TypeSerializer<N> namespaceSerializer,
-			FoldingStateDescriptor<T, ACC> stateDesc) throws Exception {
+		TypeSerializer<N> namespaceSerializer,
+		FoldingStateDescriptor<T, ACC> stateDesc) throws Exception {
 
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
 
@@ -1711,7 +1720,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 	@Override
 	protected <N, UK, UV> InternalMapState<N, UK, UV> createMapState(TypeSerializer<N> namespaceSerializer,
-			MapStateDescriptor<UK, UV> stateDesc) throws Exception {
+	                                                                 MapStateDescriptor<UK, UV> stateDesc) throws Exception {
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
 
 		return new RocksDBMapState<>(columnFamily, namespaceSerializer, stateDesc, this);
@@ -1784,7 +1793,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 					@Override
 					public int compare(MergeIterator o1, MergeIterator o2) {
 						int arrayCmpRes = compareKeyGroupsForByteArrays(
-								o1.currentKey, o2.currentKey, currentBytes);
+							o1.currentKey, o2.currentKey, currentBytes);
 						return arrayCmpRes == 0 ? o1.getKvStateId() - o2.getKvStateId() : arrayCmpRes;
 					}
 				});
@@ -1799,7 +1808,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 			if (kvStateIterators.size() > 0) {
 				PriorityQueue<MergeIterator> iteratorPriorityQueue =
-						new PriorityQueue<>(kvStateIterators.size(), iteratorComparator);
+					new PriorityQueue<>(kvStateIterators.size(), iteratorComparator);
 
 				for (Tuple2<RocksIterator, Integer> rocksIteratorWithKVStateId : kvStateIterators) {
 					final RocksIterator rocksIterator = rocksIteratorWithKVStateId.f0;
@@ -1968,8 +1977,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		KeyedStateHandle keyedStateHandle = restoreState.iterator().next();
 		if (!(keyedStateHandle instanceof MigrationKeyGroupStateHandle)) {
 			throw new IllegalStateException("Unexpected state handle type, " +
-					"expected: " + MigrationKeyGroupStateHandle.class +
-					", but found: " + keyedStateHandle.getClass());
+				"expected: " + MigrationKeyGroupStateHandle.class +
+				", but found: " + keyedStateHandle.getClass());
 		}
 
 		MigrationKeyGroupStateHandle keyGroupStateHandle = (MigrationKeyGroupStateHandle) keyedStateHandle;
@@ -1989,8 +1998,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			byte mappingByte = inputView.readByte();
 
 			ObjectInputStream ooIn =
-					new InstantiationUtil.ClassLoaderObjectInputStream(
-							new DataInputViewStream(inputView), userCodeClassLoader);
+				new InstantiationUtil.ClassLoaderObjectInputStream(
+					new DataInputViewStream(inputView), userCodeClassLoader);
 
 			StateDescriptor<?, ?> stateDescriptor = (StateDescriptor<?, ?>) ooIn.readObject();
 
@@ -2015,7 +2024,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			while (true) {
 				byte mappingByte = inputView.readByte();
 				ColumnFamilyHandle handle = getColumnFamily(
-						columnFamilyMapping.get(mappingByte), MigrationNamespaceSerializerProxy.INSTANCE);
+					columnFamilyMapping.get(mappingByte), MigrationNamespaceSerializerProxy.INSTANCE);
 
 				byte[] keyAndNamespace = BytePrimitiveArraySerializer.INSTANCE.deserialize(inputView);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ca87bec4/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
index 991e0d4..08d661c 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
@@ -42,7 +42,9 @@ import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.runtime.util.BlockerCheckpointStreamFactory;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.filefilter.IOFileFilter;
+import org.junit.After;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -122,6 +124,22 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 		return backend;
 	}
 
+	// small safety net for instance cleanups, so that no native objects are left
+	@After
+	public void cleanupRocksDB() {
+		if (keyedStateBackend != null) {
+			IOUtils.closeQuietly(keyedStateBackend);
+			keyedStateBackend.dispose();
+		}
+
+		if (allCreatedCloseables != null) {
+			for (RocksObject rocksCloseable : allCreatedCloseables) {
+				verify(rocksCloseable, times(1)).close();
+			}
+			allCreatedCloseables = null;
+		}
+	}
+
 	public void setupRocksKeyedStateBackend() throws Exception {
 
 		blocker = new OneShotLatch();
@@ -238,149 +256,186 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 
 	@Test
 	public void testCorrectMergeOperatorSet() throws IOException {
-		ColumnFamilyOptions columnFamilyOptions = mock(ColumnFamilyOptions.class);
-
-		try (RocksDBKeyedStateBackend<Integer> test = new RocksDBKeyedStateBackend<>(
-			"test",
-			Thread.currentThread().getContextClassLoader(),
-			tempFolder.newFolder(),
-			mock(DBOptions.class),
-			columnFamilyOptions,
-			mock(TaskKvStateRegistry.class),
-			IntSerializer.INSTANCE,
-			1,
-			new KeyGroupRange(0, 0),
-			new ExecutionConfig(),
-			enableIncrementalCheckpointing)) {
+
+		final ColumnFamilyOptions columnFamilyOptions = spy(new ColumnFamilyOptions());
+		RocksDBKeyedStateBackend<Integer> test = null;
+		try {
+			test = new RocksDBKeyedStateBackend<>(
+				"test",
+				Thread.currentThread().getContextClassLoader(),
+				tempFolder.newFolder(),
+				mock(DBOptions.class),
+				columnFamilyOptions,
+				mock(TaskKvStateRegistry.class),
+				IntSerializer.INSTANCE,
+				1,
+				new KeyGroupRange(0, 0),
+				new ExecutionConfig(),
+				enableIncrementalCheckpointing);
 
 			verify(columnFamilyOptions, Mockito.times(1))
 				.setMergeOperatorName(RocksDBKeyedStateBackend.MERGE_OPERATOR_NAME);
+		} finally {
+			if (test != null) {
+				IOUtils.closeQuietly(test);
+				test.dispose();
+			}
+			columnFamilyOptions.close();
 		}
 	}
 
 	@Test
 	public void testReleasingSnapshotAfterBackendClosed() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory,
-			CheckpointOptions.forFullCheckpoint());
 
-		RocksDB spyDB = keyedStateBackend.db;
+		try {
+			RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory,
+				CheckpointOptions.forFullCheckpoint());
 
-		if (!enableIncrementalCheckpointing) {
-			verify(spyDB, times(1)).getSnapshot();
-			verify(spyDB, times(0)).releaseSnapshot(any(Snapshot.class));
-		}
+			RocksDB spyDB = keyedStateBackend.db;
 
-		this.keyedStateBackend.dispose();
-		verify(spyDB, times(1)).close();
-		assertEquals(null, keyedStateBackend.db);
+			if (!enableIncrementalCheckpointing) {
+				verify(spyDB, times(1)).getSnapshot();
+				verify(spyDB, times(0)).releaseSnapshot(any(Snapshot.class));
+			}
 
-		//Ensure every RocksObjects not closed yet
-		for (RocksObject rocksCloseable : allCreatedCloseables) {
-			verify(rocksCloseable, times(0)).close();
-		}
+			this.keyedStateBackend.dispose();
+			verify(spyDB, times(1)).close();
+			assertEquals(null, keyedStateBackend.db);
 
-		snapshot.cancel(true);
+			//Ensure every RocksObjects not closed yet
+			for (RocksObject rocksCloseable : allCreatedCloseables) {
+				verify(rocksCloseable, times(0)).close();
+			}
 
-		//Ensure every RocksObjects was closed exactly once
-		for (RocksObject rocksCloseable : allCreatedCloseables) {
-			verify(rocksCloseable, times(1)).close();
-		}
+			snapshot.cancel(true);
 
+			//Ensure every RocksObjects was closed exactly once
+			for (RocksObject rocksCloseable : allCreatedCloseables) {
+				verify(rocksCloseable, times(1)).close();
+			}
+		} finally {
+			keyedStateBackend.dispose();
+			keyedStateBackend = null;
+		}
 	}
 
 	@Test
 	public void testDismissingSnapshot() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
-		snapshot.cancel(true);
-		verifyRocksObjectsReleased();
+		try {
+			RunnableFuture<KeyedStateHandle> snapshot =
+				keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
+			snapshot.cancel(true);
+			verifyRocksObjectsReleased();
+		} finally {
+			this.keyedStateBackend.dispose();
+			this.keyedStateBackend = null;
+		}
 	}
 
 	@Test
 	public void testDismissingSnapshotNotRunnable() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
-		snapshot.cancel(true);
-		Thread asyncSnapshotThread = new Thread(snapshot);
-		asyncSnapshotThread.start();
 		try {
-			snapshot.get();
-			fail();
-		} catch (Exception ignored) {
+			RunnableFuture<KeyedStateHandle> snapshot =
+				keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
+			snapshot.cancel(true);
+			Thread asyncSnapshotThread = new Thread(snapshot);
+			asyncSnapshotThread.start();
+			try {
+				snapshot.get();
+				fail();
+			} catch (Exception ignored) {
 
+			}
+			asyncSnapshotThread.join();
+			verifyRocksObjectsReleased();
+		} finally {
+			this.keyedStateBackend.dispose();
+			this.keyedStateBackend = null;
 		}
-		asyncSnapshotThread.join();
-		verifyRocksObjectsReleased();
 	}
 
 	@Test
 	public void testCompletingSnapshot() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
-		Thread asyncSnapshotThread = new Thread(snapshot);
-		asyncSnapshotThread.start();
-		waiter.await(); // wait for snapshot to run
-		waiter.reset();
-		runStateUpdates();
-		blocker.trigger(); // allow checkpointing to start writing
-		waiter.await(); // wait for snapshot stream writing to run
-		KeyedStateHandle keyedStateHandle = snapshot.get();
-		assertNotNull(keyedStateHandle);
-		assertTrue(keyedStateHandle.getStateSize() > 0);
-		assertEquals(2, keyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups());
-		assertTrue(testStreamFactory.getLastCreatedStream().isClosed());
-		asyncSnapshotThread.join();
-		verifyRocksObjectsReleased();
+		try {
+			RunnableFuture<KeyedStateHandle> snapshot =
+				keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
+			Thread asyncSnapshotThread = new Thread(snapshot);
+			asyncSnapshotThread.start();
+			waiter.await(); // wait for snapshot to run
+			waiter.reset();
+			runStateUpdates();
+			blocker.trigger(); // allow checkpointing to start writing
+			waiter.await(); // wait for snapshot stream writing to run
+			KeyedStateHandle keyedStateHandle = snapshot.get();
+			assertNotNull(keyedStateHandle);
+			assertTrue(keyedStateHandle.getStateSize() > 0);
+			assertEquals(2, keyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups());
+			assertTrue(testStreamFactory.getLastCreatedStream().isClosed());
+			asyncSnapshotThread.join();
+			verifyRocksObjectsReleased();
+		} finally {
+			this.keyedStateBackend.dispose();
+			this.keyedStateBackend = null;
+		}
 	}
 
 	@Test
 	public void testCancelRunningSnapshot() throws Exception {
 		setupRocksKeyedStateBackend();
-		RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
-		Thread asyncSnapshotThread = new Thread(snapshot);
-		asyncSnapshotThread.start();
-		waiter.await(); // wait for snapshot to run
-		waiter.reset();
-		runStateUpdates();
-		snapshot.cancel(true);
-		blocker.trigger(); // allow checkpointing to start writing
-		assertTrue(testStreamFactory.getLastCreatedStream().isClosed());
-		waiter.await(); // wait for snapshot stream writing to run
 		try {
-			snapshot.get();
-			fail();
-		} catch (Exception ignored) {
-		}
+			RunnableFuture<KeyedStateHandle> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forFullCheckpoint());
+			Thread asyncSnapshotThread = new Thread(snapshot);
+			asyncSnapshotThread.start();
+			waiter.await(); // wait for snapshot to run
+			waiter.reset();
+			runStateUpdates();
+			snapshot.cancel(true);
+			blocker.trigger(); // allow checkpointing to start writing
+			assertTrue(testStreamFactory.getLastCreatedStream().isClosed());
+			waiter.await(); // wait for snapshot stream writing to run
+			try {
+				snapshot.get();
+				fail();
+			} catch (Exception ignored) {
+			}
 
-		asyncSnapshotThread.join();
-		verifyRocksObjectsReleased();
+			asyncSnapshotThread.join();
+			verifyRocksObjectsReleased();
+		} finally {
+			this.keyedStateBackend.dispose();
+			this.keyedStateBackend = null;
+		}
 	}
 
 	@Test
 	public void testDisposeDeletesAllDirectories() throws Exception {
 		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
-		ValueStateDescriptor<String> kvId =
+		Collection<File> allFilesInDbDir =
+			FileUtils.listFilesAndDirs(new File(dbPath), new AcceptAllFilter(), new AcceptAllFilter());
+		try {
+			ValueStateDescriptor<String> kvId =
 				new ValueStateDescriptor<>("id", String.class, null);
 
-		kvId.initializeSerializerUnlessSet(new ExecutionConfig());
+			kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
-		ValueState<String> state =
+			ValueState<String> state =
 				backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
-		backend.setCurrentKey(1);
-		state.update("Hello");
-
-		Collection<File> allFilesInDbDir =
-				FileUtils.listFilesAndDirs(new File(dbPath), new AcceptAllFilter(), new AcceptAllFilter());
-
-		// more than just the root directory
-		assertTrue(allFilesInDbDir.size() > 1);
-
-		backend.dispose();
+			backend.setCurrentKey(1);
+			state.update("Hello");
 
+			// more than just the root directory
+			assertTrue(allFilesInDbDir.size() > 1);
+		} finally {
+			IOUtils.closeQuietly(backend);
+			backend.dispose();
+		}
 		allFilesInDbDir =
-				FileUtils.listFilesAndDirs(new File(dbPath), new AcceptAllFilter(), new AcceptAllFilter());
+			FileUtils.listFilesAndDirs(new File(dbPath), new AcceptAllFilter(), new AcceptAllFilter());
 
 		// just the root directory left
 		assertEquals(1, allFilesInDbDir.size());
@@ -390,62 +445,64 @@ public class RocksDBStateBackendTest extends StateBackendTestBase<RocksDBStateBa
 	public void testSharedIncrementalStateDeRegistration() throws Exception {
 		if (enableIncrementalCheckpointing) {
 			AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
-			ValueStateDescriptor<String> kvId =
-				new ValueStateDescriptor<>("id", String.class, null);
+			try {
+				ValueStateDescriptor<String> kvId =
+					new ValueStateDescriptor<>("id", String.class, null);
 
-			kvId.initializeSerializerUnlessSet(new ExecutionConfig());
+				kvId.initializeSerializerUnlessSet(new ExecutionConfig());
 
-			ValueState<String> state =
-				backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+				ValueState<String> state =
+					backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
-			Queue<IncrementalKeyedStateHandle> previousStateHandles = new LinkedList<>();
-			SharedStateRegistry sharedStateRegistry = spy(new SharedStateRegistry());
-			for (int checkpointId = 0; checkpointId < 3; ++checkpointId) {
+				Queue<IncrementalKeyedStateHandle> previousStateHandles = new LinkedList<>();
+				SharedStateRegistry sharedStateRegistry = spy(new SharedStateRegistry());
+				for (int checkpointId = 0; checkpointId < 3; ++checkpointId) {
 
-				reset(sharedStateRegistry);
+					reset(sharedStateRegistry);
 
-				backend.setCurrentKey(checkpointId);
-				state.update("Hello-" + checkpointId);
+					backend.setCurrentKey(checkpointId);
+					state.update("Hello-" + checkpointId);
 
-				RunnableFuture<KeyedStateHandle> snapshot = backend.snapshot(
-					checkpointId,
-					checkpointId,
-					createStreamFactory(),
-					CheckpointOptions.forFullCheckpoint());
+					RunnableFuture<KeyedStateHandle> snapshot = backend.snapshot(
+						checkpointId,
+						checkpointId,
+						createStreamFactory(),
+						CheckpointOptions.forFullCheckpoint());
 
-				snapshot.run();
+					snapshot.run();
 
-				IncrementalKeyedStateHandle stateHandle = (IncrementalKeyedStateHandle) snapshot.get();
-				Map<StateHandleID, StreamStateHandle> sharedState =
-					new HashMap<>(stateHandle.getSharedState());
+					IncrementalKeyedStateHandle stateHandle = (IncrementalKeyedStateHandle) snapshot.get();
+					Map<StateHandleID, StreamStateHandle> sharedState =
+						new HashMap<>(stateHandle.getSharedState());
 
-				stateHandle.registerSharedStates(sharedStateRegistry);
+					stateHandle.registerSharedStates(sharedStateRegistry);
 
-				for (Map.Entry<StateHandleID, StreamStateHandle> e : sharedState.entrySet()) {
-					verify(sharedStateRegistry).registerReference(
-						stateHandle.createSharedStateRegistryKeyFromFileName(e.getKey()),
-						e.getValue());
-				}
+					for (Map.Entry<StateHandleID, StreamStateHandle> e : sharedState.entrySet()) {
+						verify(sharedStateRegistry).registerReference(
+							stateHandle.createSharedStateRegistryKeyFromFileName(e.getKey()),
+							e.getValue());
+					}
 
-				previousStateHandles.add(stateHandle);
-				backend.notifyCheckpointComplete(checkpointId);
+					previousStateHandles.add(stateHandle);
+					backend.notifyCheckpointComplete(checkpointId);
 
-				//-----------------------------------------------------------------
+					//-----------------------------------------------------------------
 
-				if (previousStateHandles.size() > 1) {
-					checkRemove(previousStateHandles.remove(), sharedStateRegistry);
+					if (previousStateHandles.size() > 1) {
+						checkRemove(previousStateHandles.remove(), sharedStateRegistry);
+					}
 				}
-			}
 
-			while (!previousStateHandles.isEmpty()) {
+				while (!previousStateHandles.isEmpty()) {
 
-				reset(sharedStateRegistry);
+					reset(sharedStateRegistry);
 
-				checkRemove(previousStateHandles.remove(), sharedStateRegistry);
+					checkRemove(previousStateHandles.remove(), sharedStateRegistry);
+				}
+			} finally {
+				IOUtils.closeQuietly(backend);
+				backend.dispose();
 			}
-
-			backend.close();
-			backend.dispose();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ca87bec4/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
index 6debff7..f6f73f2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
@@ -18,10 +18,6 @@
 
 package org.apache.flink.runtime.state;
 
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
-import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
@@ -64,14 +60,17 @@ import org.apache.flink.runtime.state.heap.StateTable;
 import org.apache.flink.runtime.state.internal.InternalKvState;
 import org.apache.flink.runtime.state.internal.InternalValueState;
 import org.apache.flink.runtime.util.BlockerCheckpointStreamFactory;
+import org.apache.flink.shaded.guava18.com.google.common.base.Joiner;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.util.FutureUtil;
 import org.apache.flink.util.IOUtils;
 import org.apache.flink.util.StateMigrationException;
 import org.apache.flink.util.TestLogger;
 
-import org.apache.flink.shaded.guava18.com.google.common.base.Joiner;
-
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -177,17 +176,15 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			Environment env) throws Exception {
 
 		AbstractKeyedStateBackend<K> backend = getStateBackend().createKeyedStateBackend(
-				env,
-				new JobID(),
-				"test_op",
-				keySerializer,
-				numberOfKeyGroups,
-				keyGroupRange,
-				env.getTaskKvStateRegistry());
+			env,
+			new JobID(),
+			"test_op",
+			keySerializer,
+			numberOfKeyGroups,
+			keyGroupRange,
+			env.getTaskKvStateRegistry());
 
-		if (null != state) {
-			backend.restore(state);
-		}
+		backend.restore(state);
 
 		return backend;
 	}
@@ -244,6 +241,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		}
 
 		assertEquals("Didn't see the expected Kryo exception.", 1, numExceptions);
+		backend.dispose();
 	}
 
 	@Test
@@ -303,6 +301,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		}
 
 		assertEquals("Didn't see the expected Kryo exception.", 1, numExceptions);
+		backend.dispose();
 	}
 
 	@Test
@@ -356,6 +355,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		}
 
 		assertEquals("Didn't see the expected Kryo exception.", 1, numExceptions);
+		backend.dispose();
 	}
 
 	@Test
@@ -411,6 +411,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		}
 
 		assertEquals("Didn't see the expected Kryo exception.", 1, numExceptions);
+		backend.dispose();
 	}
 
 
@@ -488,81 +489,91 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		CheckpointStreamFactory streamFactory = createStreamFactory();
 		SharedStateRegistry sharedStateRegistry = new SharedStateRegistry();
 		Environment env = new DummyEnvironment("test", 1, 0);
-		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
+		AbstractKeyedStateBackend<Integer> backend = null;
 
-		TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
+		try {
+			backend = createKeyedBackend(IntSerializer.INSTANCE, env);
 
-		// make sure that we are in fact using the KryoSerializer
-		assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
+			TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
 
-		ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
+			// make sure that we are in fact using the KryoSerializer
+			assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
 
-		ValueState<TestPojo> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+			ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
 
-		// ============== create snapshot - no Kryo registration or specific / default serializers ==============
+			ValueState<TestPojo> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
-		// make some more modifications
-		backend.setCurrentKey(1);
-		state.update(new TestPojo("u1", 1));
+			// ============== create snapshot - no Kryo registration or specific / default serializers ==============
 
-		backend.setCurrentKey(2);
-		state.update(new TestPojo("u2", 2));
+			// make some more modifications
+			backend.setCurrentKey(1);
+			state.update(new TestPojo("u1", 1));
 
-		KeyedStateHandle snapshot = runSnapshot(backend.snapshot(
+			backend.setCurrentKey(2);
+			state.update(new TestPojo("u2", 2));
+
+			KeyedStateHandle snapshot = runSnapshot(backend.snapshot(
 				682375462378L,
 				2,
 				streamFactory,
 				CheckpointOptions.forFullCheckpoint()));
 
-		snapshot.registerSharedStates(sharedStateRegistry);
-		backend.dispose();
+			snapshot.registerSharedStates(sharedStateRegistry);
+			backend.dispose();
 
-		// ========== restore snapshot - should use default serializer (ONLY SERIALIZATION) ==========
+			// ========== restore snapshot - should use default serializer (ONLY SERIALIZATION) ==========
 
-		// cast because our test serializer is not typed to TestPojo
-		env.getExecutionConfig().addDefaultKryoSerializer(TestPojo.class, (Class) CustomKryoTestSerializer.class);
+			// cast because our test serializer is not typed to TestPojo
+			env.getExecutionConfig().addDefaultKryoSerializer(TestPojo.class, (Class) CustomKryoTestSerializer.class);
 
-		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env);
+			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env);
 
-		// re-initialize to ensure that we create the KryoSerializer from scratch, otherwise
-		// initializeSerializerUnlessSet would not pick up our new config
-		kvId = new ValueStateDescriptor<>("id", pojoType);
-		state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+			// re-initialize to ensure that we create the KryoSerializer from scratch, otherwise
+			// initializeSerializerUnlessSet would not pick up our new config
+			kvId = new ValueStateDescriptor<>("id", pojoType);
+			state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
-		backend.setCurrentKey(1);
+			backend.setCurrentKey(1);
 
-		// update to test state backends that eagerly serialize, such as RocksDB
-		state.update(new TestPojo("u1", 11));
+			// update to test state backends that eagerly serialize, such as RocksDB
+			state.update(new TestPojo("u1", 11));
 
-		KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(
+			KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(
 				682375462378L,
 				2,
 				streamFactory,
 				CheckpointOptions.forFullCheckpoint()));
 
-		snapshot2.registerSharedStates(sharedStateRegistry);
+			snapshot2.registerSharedStates(sharedStateRegistry);
+			snapshot.discardState();
 
-		snapshot.discardState();
+			backend.dispose();
 
-		backend.dispose();
+			// ========= restore snapshot - should use default serializer (FAIL ON DESERIALIZATION) =========
 
-		// ========= restore snapshot - should use default serializer (FAIL ON DESERIALIZATION) =========
+			// cast because our test serializer is not typed to TestPojo
+			env.getExecutionConfig().addDefaultKryoSerializer(TestPojo.class, (Class) CustomKryoTestSerializer.class);
 
-		// cast because our test serializer is not typed to TestPojo
-		env.getExecutionConfig().addDefaultKryoSerializer(TestPojo.class, (Class) CustomKryoTestSerializer.class);
+			// on the second restore, since the custom serializer will be used for
+			// deserialization, we expect the deliberate failure to be thrown
+			expectedException.expect(ExpectedKryoTestException.class);
 
-		// on the second restore, since the custom serializer will be used for
-		// deserialization, we expect the deliberate failure to be thrown
-		expectedException.expect(ExpectedKryoTestException.class);
+			// state backends that eagerly deserializes (such as the memory state backend) will fail here
+			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
 
-		// state backends that eagerly deserializes (such as the memory state backend) will fail here
-		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
+			state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
-		state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+			backend.setCurrentKey(1);
+			// state backends that lazily deserializes (such as RocksDB) will fail here
+			state.value();
 
-		backend.setCurrentKey(1);
-		// state backends that lazily deserializes (such as RocksDB) will fail here
-		state.value();
+			snapshot2.discardState();
+			backend.dispose();
+		} finally {
+			// ensure to release native resources even when we exit through exception
+			IOUtils.closeQuietly(backend);
+			backend.dispose();
+		}
 	}
 
 	/**
@@ -581,78 +592,89 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		SharedStateRegistry sharedStateRegistry = new SharedStateRegistry();
 		Environment env = new DummyEnvironment("test", 1, 0);
 
-		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE, env);
+		AbstractKeyedStateBackend<Integer> backend = null;
 
-		TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
+		try {
+			backend = createKeyedBackend(IntSerializer.INSTANCE, env);
 
-		// make sure that we are in fact using the KryoSerializer
-		assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
+			TypeInformation<TestPojo> pojoType = new GenericTypeInfo<>(TestPojo.class);
 
-		ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
-		ValueState<TestPojo> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+			// make sure that we are in fact using the KryoSerializer
+			assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer);
 
-		// ============== create snapshot - no Kryo registration or specific / default serializers ==============
+			ValueStateDescriptor<TestPojo> kvId = new ValueStateDescriptor<>("id", pojoType);
+			ValueState<TestPojo> state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
-		// make some more modifications
-		backend.setCurrentKey(1);
-		state.update(new TestPojo("u1", 1));
+			// ============== create snapshot - no Kryo registration or specific / default serializers ==============
 
-		backend.setCurrentKey(2);
-		state.update(new TestPojo("u2", 2));
+			// make some more modifications
+			backend.setCurrentKey(1);
+			state.update(new TestPojo("u1", 1));
 
-		KeyedStateHandle snapshot = runSnapshot(backend.snapshot(
+			backend.setCurrentKey(2);
+			state.update(new TestPojo("u2", 2));
+
+			KeyedStateHandle snapshot = runSnapshot(backend.snapshot(
 				682375462378L,
 				2,
 				streamFactory,
 				CheckpointOptions.forFullCheckpoint()));
 
-		snapshot.registerSharedStates(sharedStateRegistry);
-		backend.dispose();
+			snapshot.registerSharedStates(sharedStateRegistry);
+			backend.dispose();
 
-		// ========== restore snapshot - should use specific serializer (ONLY SERIALIZATION) ==========
+			// ========== restore snapshot - should use specific serializer (ONLY SERIALIZATION) ==========
 
-		env.getExecutionConfig().registerTypeWithKryoSerializer(TestPojo.class, CustomKryoTestSerializer.class);
+			env.getExecutionConfig().registerTypeWithKryoSerializer(TestPojo.class, CustomKryoTestSerializer.class);
 
-		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env);
+			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env);
 
-		// re-initialize to ensure that we create the KryoSerializer from scratch, otherwise
-		// initializeSerializerUnlessSet would not pick up our new config
-		kvId = new ValueStateDescriptor<>("id", pojoType);
-		state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+			// re-initialize to ensure that we create the KryoSerializer from scratch, otherwise
+			// initializeSerializerUnlessSet would not pick up our new config
+			kvId = new ValueStateDescriptor<>("id", pojoType);
+			state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
-		backend.setCurrentKey(1);
+			backend.setCurrentKey(1);
 
-		// update to test state backends that eagerly serialize, such as RocksDB
-		state.update(new TestPojo("u1", 11));
+			// update to test state backends that eagerly serialize, such as RocksDB
+			state.update(new TestPojo("u1", 11));
 
-		KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(
+			KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(
 				682375462378L,
 				2,
 				streamFactory,
 				CheckpointOptions.forFullCheckpoint()));
 
-		snapshot2.registerSharedStates(sharedStateRegistry);
+			snapshot2.registerSharedStates(sharedStateRegistry);
 
-		snapshot.discardState();
+			snapshot.discardState();
 
-		backend.dispose();
+			backend.dispose();
 
-		// ========= restore snapshot - should use specific serializer (FAIL ON DESERIALIZATION) =========
+			// ========= restore snapshot - should use specific serializer (FAIL ON DESERIALIZATION) =========
 
-		env.getExecutionConfig().registerTypeWithKryoSerializer(TestPojo.class, CustomKryoTestSerializer.class);
+			env.getExecutionConfig().registerTypeWithKryoSerializer(TestPojo.class, CustomKryoTestSerializer.class);
 
-		// on the second restore, since the custom serializer will be used for
-		// deserialization, we expect the deliberate failure to be thrown
-		expectedException.expect(ExpectedKryoTestException.class);
+			// on the second restore, since the custom serializer will be used for
+			// deserialization, we expect the deliberate failure to be thrown
+			expectedException.expect(ExpectedKryoTestException.class);
 
-		// state backends that eagerly deserializes (such as the memory state backend) will fail here
-		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
+			// state backends that eagerly deserializes (such as the memory state backend) will fail here
+			backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2, env);
 
-		state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
+			state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId);
 
-		backend.setCurrentKey(1);
-		// state backends that lazily deserializes (such as RocksDB) will fail here
-		state.value();
+			backend.setCurrentKey(1);
+			// state backends that lazily deserializes (such as RocksDB) will fail here
+			state.value();
+
+			backend.dispose();
+		} finally {
+			// ensure that native resources are also released in case of exception
+			if (backend != null) {
+				backend.dispose();
+			}
+		}
 	}
 
 	@Test
@@ -1726,7 +1748,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		final int MAX_PARALLELISM = 10;
 
 		CheckpointStreamFactory streamFactory = createStreamFactory();
-		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(
+		final AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(
 				IntSerializer.INSTANCE,
 				MAX_PARALLELISM,
 				new KeyGroupRange(0, MAX_PARALLELISM - 1),
@@ -1770,7 +1792,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		backend.dispose();
 
 		// backend for the first half of the key group range
-		AbstractKeyedStateBackend<Integer> firstHalfBackend = restoreKeyedBackend(
+		final AbstractKeyedStateBackend<Integer> firstHalfBackend = restoreKeyedBackend(
 				IntSerializer.INSTANCE,
 				MAX_PARALLELISM,
 				new KeyGroupRange(0, 4),
@@ -1778,7 +1800,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 				new DummyEnvironment("test", 1, 0));
 
 		// backend for the second half of the key group range
-		AbstractKeyedStateBackend<Integer> secondHalfBackend = restoreKeyedBackend(
+		final AbstractKeyedStateBackend<Integer> secondHalfBackend = restoreKeyedBackend(
 				IntSerializer.INSTANCE,
 				MAX_PARALLELISM,
 				new KeyGroupRange(5, 9),
@@ -2017,7 +2039,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 
 	@Test
 	public void testCopyDefaultValue() throws Exception {
-		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+		final AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		ValueStateDescriptor<IntValue> kvId = new ValueStateDescriptor<>("id", IntValue.class, new IntValue(-1));
 
@@ -2044,7 +2066,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 	 */
 	@Test
 	public void testRequireNonNullNamespace() throws Exception {
-		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+		final AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 		ValueStateDescriptor<IntValue> kvId = new ValueStateDescriptor<>("id", IntValue.class, new IntValue(-1));
 
@@ -2076,7 +2098,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 	@SuppressWarnings("unchecked")
 	protected void testConcurrentMapIfQueryable() throws Exception {
 		final int numberOfKeyGroups = 1;
-		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(
+		final AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(
 				IntSerializer.INSTANCE,
 				numberOfKeyGroups,
 				new KeyGroupRange(0, 0),
@@ -2384,9 +2406,9 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 		streamFactory.setBlockerLatch(blocker);
 		streamFactory.setAfterNumberInvocations(10);
 
-		AbstractKeyedStateBackend<Integer> backend = null;
+		final AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+
 		try {
-			backend = createKeyedBackend(IntSerializer.INSTANCE);
 
 			if (!backend.supportsAsynchronousSnapshots()) {
 				return;
@@ -2413,14 +2435,11 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			waiter.await();
 
 			// close the backend to see if the close is propagated to the stream
-			backend.close();
+			IOUtils.closeQuietly(backend);
 
 			//unblock the stream so that it can run into the IOException
 			blocker.trigger();
 
-			//dispose the backend
-			backend.dispose();
-
 			runner.join();
 
 			try {
@@ -2431,10 +2450,7 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> exten
 			}
 
 		} finally {
-			if (null != backend) {
-				IOUtils.closeQuietly(backend);
-				backend.dispose();
-			}
+			backend.dispose();
 		}
 	}
 


[03/11] flink git commit: [FLINK-7461] Remove Backwards compatibility with <= Flink 1.1

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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
deleted file mode 100644
index da6e035..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom11MigrationITCase.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 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-rocksdb-savepoint",
-				new Tuple2<>(EXPECTED_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
-	}
-
-	@Test
-	public void testSavepointRestoreFromFlink11() throws Exception {
-
-		final int expectedSuccessfulChecks = 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, expectedSuccessfulChecks));
-	}
-
-	@Test
-	public void testSavepointRestoreFromFlink11FromRocksDB() throws Exception {
-
-		final int expectedSuccessfulChecks = 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-rocksdb-savepoint"),
-				new Tuple2<>(SUCCESSFUL_CHECK_ACCUMULATOR, expectedSuccessfulChecks));
-	}
-
-	private static class LegacyCheckpointedSource
-			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
-
-		public static String checkpointedString = "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(checkpointedString, state);
-		}
-
-		@Override
-		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return checkpointedString;
-		}
-	}
-
-	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.checkpointedString, 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;
-		}
-	}
-
-	private static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements Checkpointed<Tuple2<String, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		public static Tuple2<String, Long> checkpointedTuple =
-				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 checkpointedTuple;
-		}
-	}
-
-	private 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.checkpointedTuple, restoredState);
-			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
-
-		}
-
-		@Override
-		public void restoreState(Tuple2<String, Long> state) throws Exception {
-			restoredState = state;
-		}
-	}
-
-	private static class LegacyCheckpointedFlatMapWithKeyedState
-			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements Checkpointed<Tuple2<String, Long>> {
-
-		private static final long serialVersionUID = 1L;
-
-		public static Tuple2<String, Long> checkpointedTuple =
-				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 checkpointedTuple;
-		}
-	}
-
-	private 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.checkpointedTuple, restoredState);
-			getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1);
-		}
-
-		@Override
-		public void restoreState(Tuple2<String, Long> state) throws Exception {
-			restoredState = state;
-		}
-	}
-
-	private 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);
-		}
-	}
-
-	private 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);
-		}
-	}
-
-	private static class CheckpointedUdfOperator
-			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
-			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		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(checkpointedString);
-//
-//			result.setOperatorState(out.closeAndGetHandle());
-//
-//			return result;
-//		}
-	}
-
-	private static class RestoringCheckingUdfOperator
-			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
-			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		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();
-		}
-	}
-
-	private 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/6642768a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java
index 7dd1144..6859c2d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java
@@ -29,14 +29,8 @@ 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;
@@ -60,10 +54,13 @@ import static org.junit.Assert.assertEquals;
 /**
  * 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 test for checkpointed (legacy state) was removed from this test for Flink 1.4 because compatibility with
+ * Flink 1.1 is removed. The legacy state in the binary savepoints is ignored by the tests now.
  *
  * <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;
@@ -247,7 +244,7 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 	}
 
 	private static class LegacyCheckpointedSource
-			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
+			implements SourceFunction<Tuple2<Long, Long>> {
 
 		public static String checkpointedString = "Here be dragons!";
 
@@ -283,21 +280,10 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		public void cancel() {
 			isRunning = false;
 		}
-
-		@Override
-		public void restoreState(String state) throws Exception {
-			assertEquals(checkpointedString, state);
-		}
-
-		@Override
-		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return checkpointedString;
-		}
 	}
 
 	private static class CheckingRestoringSource
-			extends RichSourceFunction<Tuple2<Long, Long>>
-			implements CheckpointedRestoring<String> {
+			extends RichSourceFunction<Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -322,7 +308,6 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 		@Override
 		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
-			assertEquals(LegacyCheckpointedSource.checkpointedString, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
 
 			// immediately trigger any set timers
@@ -343,15 +328,9 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		public void cancel() {
 			isRunning = false;
 		}
-
-		@Override
-		public void restoreState(String state) throws Exception {
-			restoredState = state;
-		}
 	}
 
-	private static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements Checkpointed<Tuple2<String, Long>> {
+	private static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -362,19 +341,9 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		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 checkpointedTuple;
-		}
 	}
 
-	private static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements CheckpointedRestoring<Tuple2<String, Long>> {
+	private static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -393,20 +362,14 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
 			out.collect(value);
 
-			assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
 
 		}
 
-		@Override
-		public void restoreState(Tuple2<String, Long> state) throws Exception {
-			restoredState = state;
-		}
 	}
 
 	private static class LegacyCheckpointedFlatMapWithKeyedState
-			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements Checkpointed<Tuple2<String, Long>> {
+			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -424,19 +387,10 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 
 			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 checkpointedTuple;
-		}
 	}
 
-	private static class CheckingRestoringFlatMapWithKeyedState extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements CheckpointedRestoring<Tuple2<String, Long>> {
+	private static class CheckingRestoringFlatMapWithKeyedState
+		extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -464,18 +418,12 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 			}
 
 			assertEquals(value.f1, state.value());
-			assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
 		}
-
-		@Override
-		public void restoreState(Tuple2<String, Long> state) throws Exception {
-			restoredState = state;
-		}
 	}
 
-	private static class CheckingRestoringFlatMapWithKeyedStateInOperator extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
-			implements CheckpointedRestoring<Tuple2<String, Long>> {
+	private static class CheckingRestoringFlatMapWithKeyedStateInOperator
+		extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -503,14 +451,8 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 			}
 
 			assertEquals(value.f1, state.value());
-			assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState);
 			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
 		}
-
-		@Override
-		public void restoreState(Tuple2<String, Long> state) throws Exception {
-			restoredState = state;
-		}
 	}
 
 	private static class KeyedStateSettingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
@@ -578,17 +520,6 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		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();
-		}
 	}
 
 	private static class CheckingRestoringUdfOperator
@@ -615,8 +546,6 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		@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);
 		}
 
@@ -624,15 +553,6 @@ public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigratio
 		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();
-		}
 	}
 
 	private static class TimelyStatefulOperator

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
deleted file mode 100644
index 1431d96..0000000
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/LegacyCheckpointedStreamingProgram.java
+++ /dev/null
@@ -1,143 +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.classloading.jar;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-
-/**
- * This test is the same as the {@link CheckpointedStreamingProgram} but using the
- * old and deprecated {@link Checkpointed} interface. It stays here in order to
- * guarantee that although deprecated, the old Checkpointed interface is still supported.
- * This is necessary to not break user code.
- * */
-public class LegacyCheckpointedStreamingProgram {
-
-	private static final int CHECKPOINT_INTERVALL = 100;
-
-	public static void main(String[] args) throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().disableSysoutLogging();
-		env.enableCheckpointing(CHECKPOINT_INTERVALL);
-		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 10000));
-		env.disableOperatorChaining();
-
-		DataStream<String> text = env.addSource(new SimpleStringGenerator());
-		text.map(new StatefulMapper()).addSink(new NoOpSink());
-		env.setParallelism(1);
-		env.execute("Checkpointed Streaming Program");
-	}
-
-	// with Checkpointing
-	private static class SimpleStringGenerator implements SourceFunction<String>, Checkpointed<Integer> {
-
-		private static final long serialVersionUID = 3700033137820808611L;
-
-		public boolean running = true;
-
-		@Override
-		public void run(SourceContext<String> ctx) throws Exception {
-			while (running) {
-				Thread.sleep(1);
-				ctx.collect("someString");
-			}
-		}
-
-		@Override
-		public void cancel() {
-			running = false;
-		}
-
-		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return null;
-		}
-
-		@Override
-		public void restoreState(Integer state) {
-
-		}
-	}
-
-	private static class StatefulMapper implements MapFunction<String, String>, Checkpointed<StatefulMapper>, CheckpointListener {
-
-		private static final long serialVersionUID = 2703630582894634440L;
-
-		private String someState;
-		private boolean atLeastOneSnapshotComplete = false;
-		private boolean restored = false;
-
-		@Override
-		public StatefulMapper snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return this;
-		}
-
-		@Override
-		public void restoreState(StatefulMapper state) {
-			restored = true;
-			this.someState = state.someState;
-			this.atLeastOneSnapshotComplete = state.atLeastOneSnapshotComplete;
-		}
-
-		@Override
-		public String map(String value) throws Exception {
-			if (!atLeastOneSnapshotComplete) {
-				// throttle consumption by the checkpoint interval until we have one snapshot.
-				Thread.sleep(CHECKPOINT_INTERVALL);
-			}
-			if (atLeastOneSnapshotComplete && !restored) {
-				throw new RuntimeException("Intended failure, to trigger restore");
-			}
-			if (restored) {
-				throw new SuccessException();
-				//throw new RuntimeException("All good");
-			}
-			someState = value; // update our state
-			return value;
-		}
-
-		@Override
-		public void notifyCheckpointComplete(long checkpointId) throws Exception {
-			atLeastOneSnapshotComplete = true;
-		}
-	}
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * We intentionally use a user specified failure exception.
-	 */
-	private static class SuccessException extends Exception {
-
-		private static final long serialVersionUID = 7073311460437532086L;
-	}
-
-	private static class NoOpSink implements SinkFunction<String> {
-		private static final long serialVersionUID = 2381410324190818620L;
-
-		@Override
-		public void invoke(String value) throws Exception {
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
index 3d78242..00d0b2c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java
@@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.akka.ListeningBehaviour;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointSerializers;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
@@ -89,6 +90,11 @@ public abstract class AbstractOperatorRestoreTestBase extends TestLogger {
 	private static final FiniteDuration timeout = new FiniteDuration(30L, TimeUnit.SECONDS);
 
 	@BeforeClass
+	public static void beforeClass() {
+		SavepointSerializers.setFailWhenLegacyStateDetected(false);
+	}
+
+	@BeforeClass
 	public static void setupCluster() throws Exception {
 		final Configuration configuration = new Configuration();
 


[08/11] flink git commit: [FLINK-7461] Remove Backwards compatibility with <= Flink 1.1

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/SerializedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/SerializedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/SerializedStateHandle.java
deleted file mode 100644
index 49d772e..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/SerializedStateHandle.java
+++ /dev/null
@@ -1,93 +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.migration.runtime.state.memory;
-
-import org.apache.flink.migration.runtime.state.AbstractCloseableHandle;
-import org.apache.flink.migration.runtime.state.StateHandle;
-import org.apache.flink.migration.util.MigrationInstantiationUtil;
-import org.apache.flink.util.InstantiationUtil;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * A state handle that represents its state in serialized form as bytes.
- *
- * @param <T> The type of state represented by this state handle.
- */
-@SuppressWarnings("deprecation")
-public class SerializedStateHandle<T extends Serializable> extends AbstractCloseableHandle implements StateHandle<T> {
-	
-	private static final long serialVersionUID = 4145685722538475769L;
-
-	/** The serialized data */
-	private final byte[] serializedData;
-	
-	/**
-	 * Creates a new serialized state handle, eagerly serializing the given state object.
-	 * 
-	 * @param value The state object.
-	 * @throws IOException Thrown, if the serialization fails.
-	 */
-	public SerializedStateHandle(T value) throws IOException {
-		this.serializedData = value == null ? null : InstantiationUtil.serializeObject(value);
-	}
-
-	/**
-	 * Creates a new serialized state handle, based in the given already serialized data.
-	 * 
-	 * @param serializedData The serialized data.
-	 */
-	public SerializedStateHandle(byte[] serializedData) {
-		this.serializedData = serializedData;
-	}
-	
-	@Override
-	public T getState(ClassLoader classLoader) throws Exception {
-		if (classLoader == null) {
-			throw new NullPointerException();
-		}
-
-		ensureNotClosed();
-		return serializedData == null ? null : MigrationInstantiationUtil.<T>deserializeObject(serializedData, classLoader);
-	}
-
-	/**
-	 * Gets the size of the serialized state.
-	 * @return The size of the serialized state.
-	 */
-	public int getSizeOfSerializedState() {
-		return serializedData.length;
-	}
-
-	/**
-	 * Discarding heap-memory backed state is a no-op, so this method does nothing.
-	 */
-	@Override
-	public void discardState() {}
-
-	@Override
-	public long getStateSize() {
-		return serializedData.length;
-	}
-
-	public byte[] getSerializedData() {
-		return serializedData;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationKeyGroupStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationKeyGroupStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationKeyGroupStateHandle.java
deleted file mode 100644
index 3f1ff55..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationKeyGroupStateHandle.java
+++ /dev/null
@@ -1,46 +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.migration.state;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
-import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.util.Migration;
-
-/**
- * This class is just a KeyGroupsStateHandle that is tagged as migration, to figure out which restore logic to apply,
- * e.g. when restoring backend data from a state handle.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Internal
-@Deprecated
-public class MigrationKeyGroupStateHandle extends KeyGroupsStateHandle implements Migration {
-
-	private static final long serialVersionUID = -8554427169776881697L;
-
-	/**
-	 * @param groupRangeOffsets range of key-group ids that in the state of this handle
-	 * @param streamStateHandle handle to the actual state of the key-groups
-	 */
-	public MigrationKeyGroupStateHandle(KeyGroupRangeOffsets groupRangeOffsets, StreamStateHandle streamStateHandle) {
-		super(groupRangeOffsets, streamStateHandle);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationStreamStateHandle.java
deleted file mode 100644
index 2201916..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/state/MigrationStreamStateHandle.java
+++ /dev/null
@@ -1,67 +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.migration.state;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FSDataInputStreamWrapper;
-import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.util.Migration;
-
-import java.io.IOException;
-
-/**
- * This class is just a StreamStateHandle that is tagged as migration, to figure out which restore logic to apply, e.g.
- * when restoring backend data from a state handle.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Internal
-@Deprecated
-public class MigrationStreamStateHandle implements StreamStateHandle, Migration {
-
-	private static final long serialVersionUID = -2332113722532150112L;
-	private final StreamStateHandle delegate;
-
-	public MigrationStreamStateHandle(StreamStateHandle delegate) {
-		this.delegate = delegate;
-	}
-
-	@Override
-	public FSDataInputStream openInputStream() throws IOException {
-		return new MigrationFSInputStream(delegate.openInputStream());
-	}
-
-	@Override
-	public void discardState() throws Exception {
-		delegate.discardState();
-	}
-
-	@Override
-	public long getStateSize() {
-		return delegate.getStateSize();
-	}
-
-	static class MigrationFSInputStream extends FSDataInputStreamWrapper implements Migration {
-
-		public MigrationFSInputStream(FSDataInputStream inputStream) {
-			super(inputStream);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskState.java b/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskState.java
deleted file mode 100644
index b044ffb..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskState.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.streaming.runtime.tasks;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.migration.runtime.state.KvStateSnapshot;
-import org.apache.flink.migration.runtime.state.StateHandle;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.HashMap;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@Internal
-@SuppressWarnings("deprecation")
-public class StreamTaskState implements Serializable, Closeable {
-
-	private static final long serialVersionUID = 1L;
-
-	private StateHandle<?> operatorState;
-
-	private StateHandle<Serializable> functionState;
-
-	private HashMap<String, KvStateSnapshot<?, ?, ?, ?>> kvStates;
-
-	// ------------------------------------------------------------------------
-
-	public StateHandle<?> getOperatorState() {
-		return operatorState;
-	}
-
-	public void setOperatorState(StateHandle<?> operatorState) {
-		this.operatorState = operatorState;
-	}
-
-	public StateHandle<Serializable> getFunctionState() {
-		return functionState;
-	}
-
-	public void setFunctionState(StateHandle<Serializable> functionState) {
-		this.functionState = functionState;
-	}
-
-	public HashMap<String, KvStateSnapshot<?, ?, ?, ?>> getKvStates() {
-		return kvStates;
-	}
-
-	public void setKvStates(HashMap<String, KvStateSnapshot<?, ?, ?, ?>> kvStates) {
-		this.kvStates = kvStates;
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Checks if this state object actually contains any state, or if all of the state
-	 * fields are null.
-	 *
-	 * @return True, if all state is null, false if at least one state is not null.
-	 */
-	public boolean isEmpty() {
-		return operatorState == null & functionState == null & kvStates == null;
-	}
-
-	@Override
-	public void close() throws IOException {
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskStateList.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskStateList.java b/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskStateList.java
deleted file mode 100644
index 7643039..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/streaming/runtime/tasks/StreamTaskStateList.java
+++ /dev/null
@@ -1,100 +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.migration.streaming.runtime.tasks;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.migration.runtime.state.KvStateSnapshot;
-import org.apache.flink.migration.runtime.state.StateHandle;
-
-import java.io.IOException;
-import java.util.HashMap;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@Internal
-@SuppressWarnings("deprecation")
-public class StreamTaskStateList implements StateHandle<StreamTaskState[]> {
-
-	private static final long serialVersionUID = 1L;
-
-	/** The states for all operator. */
-	private final StreamTaskState[] states;
-
-	public StreamTaskStateList(StreamTaskState[] states) throws Exception {
-		this.states = states;
-	}
-
-	public boolean isEmpty() {
-		for (StreamTaskState state : states) {
-			if (state != null) {
-				return false;
-			}
-		}
-		return true;
-	}
-
-	@Override
-	public StreamTaskState[] getState(ClassLoader userCodeClassLoader) {
-		return states;
-	}
-
-	@Override
-	public void discardState() throws Exception {
-	}
-
-	@Override
-	public long getStateSize() throws Exception {
-		long sumStateSize = 0;
-
-		if (states != null) {
-			for (StreamTaskState state : states) {
-				if (state != null) {
-					StateHandle<?> operatorState = state.getOperatorState();
-					StateHandle<?> functionState = state.getFunctionState();
-					HashMap<String, KvStateSnapshot<?, ?, ?, ?>> kvStates = state.getKvStates();
-
-					if (operatorState != null) {
-						sumStateSize += operatorState.getStateSize();
-					}
-
-					if (functionState != null) {
-						sumStateSize += functionState.getStateSize();
-					}
-
-					if (kvStates != null) {
-						for (KvStateSnapshot<?, ?, ?, ?> kvState : kvStates.values()) {
-							if (kvState != null) {
-								sumStateSize += kvState.getStateSize();
-							}
-						}
-					}
-				}
-			}
-		}
-
-		// State size as sum of all state sizes
-		return sumStateSize;
-	}
-
-	@Override
-	public void close() throws IOException {
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
index 145ff6a..a5f908d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
@@ -30,7 +30,7 @@ import java.util.Map;
 import java.util.Objects;
 
 /**
- * Simple container class which contains the raw/managed/legacy operator state and key-group state handles from all sub
+ * Simple container class which contains the raw/managed operator state and key-group state handles from all sub
  * tasks of an operator and therefore represents the complete state of a logical operator.
  */
 public class OperatorState implements CompositeStateHandle {
@@ -102,15 +102,6 @@ public class OperatorState implements CompositeStateHandle {
 		return maxParallelism;
 	}
 
-	public boolean hasNonPartitionedState() {
-		for (OperatorSubtaskState sts : operatorSubtaskStates.values()) {
-			if (sts != null && sts.getLegacyOperatorState() != null) {
-				return true;
-			}
-		}
-		return false;
-	}
-
 	@Override
 	public void discardState() throws Exception {
 		for (OperatorSubtaskState operatorSubtaskState : operatorSubtaskStates.values()) {

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
index 296b5ab..3df9c4f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
@@ -18,21 +18,18 @@
 
 package org.apache.flink.runtime.checkpoint;
 
-import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.state.CompositeStateHandle;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.SharedStateRegistry;
 import org.apache.flink.runtime.state.StateObject;
 import org.apache.flink.runtime.state.StateUtil;
-import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.util.Preconditions;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -63,16 +60,6 @@ public class OperatorSubtaskState implements CompositeStateHandle {
 	private static final long serialVersionUID = -2394696997971923995L;
 
 	/**
-	 * Legacy (non-repartitionable) operator state.
-	 *
-	 * @deprecated Non-repartitionable operator state that has been deprecated.
-	 * Can be removed when we remove the APIs for non-repartitionable operator state.
-	 */
-	@Deprecated
-	@Nullable
-	private final StreamStateHandle legacyOperatorState;
-
-	/**
 	 * Snapshot from the {@link org.apache.flink.runtime.state.OperatorStateBackend}.
 	 */
 	@Nonnull
@@ -103,39 +90,30 @@ public class OperatorSubtaskState implements CompositeStateHandle {
 	 */
 	private final long stateSize;
 
-	@VisibleForTesting
-	public OperatorSubtaskState(StreamStateHandle legacyOperatorState) {
-
-		this(legacyOperatorState,
-			Collections.<OperatorStateHandle>emptyList(),
-			Collections.<OperatorStateHandle>emptyList(),
-			Collections.<KeyedStateHandle>emptyList(),
-			Collections.<KeyedStateHandle>emptyList());
-	}
-
 	/**
 	 * Empty state.
 	 */
 	public OperatorSubtaskState() {
-		this(null);
+		this(
+			Collections.emptyList(),
+			Collections.emptyList(),
+			Collections.emptyList(),
+			Collections.emptyList());
 	}
 
 	public OperatorSubtaskState(
-		StreamStateHandle legacyOperatorState,
 		Collection<OperatorStateHandle> managedOperatorState,
 		Collection<OperatorStateHandle> rawOperatorState,
 		Collection<KeyedStateHandle> managedKeyedState,
 		Collection<KeyedStateHandle> rawKeyedState) {
 
-		this.legacyOperatorState = legacyOperatorState;
 		this.managedOperatorState = Preconditions.checkNotNull(managedOperatorState);
 		this.rawOperatorState = Preconditions.checkNotNull(rawOperatorState);
 		this.managedKeyedState = Preconditions.checkNotNull(managedKeyedState);
 		this.rawKeyedState = Preconditions.checkNotNull(rawKeyedState);
 
 		try {
-			long calculateStateSize = getSizeNullSafe(legacyOperatorState);
-			calculateStateSize += sumAllSizes(managedOperatorState);
+			long calculateStateSize = sumAllSizes(managedOperatorState);
 			calculateStateSize += sumAllSizes(rawOperatorState);
 			calculateStateSize += sumAllSizes(managedKeyedState);
 			calculateStateSize += sumAllSizes(rawKeyedState);
@@ -150,13 +128,12 @@ public class OperatorSubtaskState implements CompositeStateHandle {
 	 * Collections (except for legacy state).
 	 */
 	public OperatorSubtaskState(
-		StreamStateHandle legacyOperatorState,
 		OperatorStateHandle managedOperatorState,
 		OperatorStateHandle rawOperatorState,
 		KeyedStateHandle managedKeyedState,
 		KeyedStateHandle rawKeyedState) {
 
-		this(legacyOperatorState,
+		this(
 			singletonOrEmptyOnNull(managedOperatorState),
 			singletonOrEmptyOnNull(rawOperatorState),
 			singletonOrEmptyOnNull(managedKeyedState),
@@ -183,16 +160,6 @@ public class OperatorSubtaskState implements CompositeStateHandle {
 	// --------------------------------------------------------------------------------------------
 
 	/**
-	 * @deprecated Non-repartitionable operator state that has been deprecated.
-	 * Can be removed when we remove the APIs for non-repartitionable operator state.
-	 */
-	@Deprecated
-	@Nullable
-	public StreamStateHandle getLegacyOperatorState() {
-		return legacyOperatorState;
-	}
-
-	/**
 	 * Returns a handle to the managed operator state.
 	 */
 	@Nonnull
@@ -228,12 +195,11 @@ public class OperatorSubtaskState implements CompositeStateHandle {
 	public void discardState() {
 		try {
 			List<StateObject> toDispose =
-				new ArrayList<>(1 +
-					managedOperatorState.size() +
-					rawOperatorState.size() +
-					managedKeyedState.size() +
-					rawKeyedState.size());
-			toDispose.add(legacyOperatorState);
+				new ArrayList<>(
+						managedOperatorState.size() +
+						rawOperatorState.size() +
+						managedKeyedState.size() +
+						rawKeyedState.size());
 			toDispose.addAll(managedOperatorState);
 			toDispose.addAll(rawOperatorState);
 			toDispose.addAll(managedKeyedState);
@@ -281,9 +247,6 @@ public class OperatorSubtaskState implements CompositeStateHandle {
 		if (getStateSize() != that.getStateSize()) {
 			return false;
 		}
-		if (getLegacyOperatorState() != null ? !getLegacyOperatorState().equals(that.getLegacyOperatorState()) : that.getLegacyOperatorState() != null) {
-			return false;
-		}
 		if (!getManagedOperatorState().equals(that.getManagedOperatorState())) {
 			return false;
 		}
@@ -298,8 +261,7 @@ public class OperatorSubtaskState implements CompositeStateHandle {
 
 	@Override
 	public int hashCode() {
-		int result = getLegacyOperatorState() != null ? getLegacyOperatorState().hashCode() : 0;
-		result = 31 * result + getManagedOperatorState().hashCode();
+		int result = getManagedOperatorState().hashCode();
 		result = 31 * result + getRawOperatorState().hashCode();
 		result = 31 * result + getManagedKeyedState().hashCode();
 		result = 31 * result + getRawKeyedState().hashCode();
@@ -310,8 +272,7 @@ public class OperatorSubtaskState implements CompositeStateHandle {
 	@Override
 	public String toString() {
 		return "SubtaskState{" +
-			"legacyState=" + legacyOperatorState +
-			", operatorStateFromBackend=" + managedOperatorState +
+			"operatorStateFromBackend=" + managedOperatorState +
 			", operatorStateFromStream=" + rawOperatorState +
 			", keyedStateFromBackend=" + managedKeyedState +
 			", keyedStateFromStream=" + rawKeyedState +
@@ -320,8 +281,7 @@ public class OperatorSubtaskState implements CompositeStateHandle {
 	}
 
 	public boolean hasState() {
-		return legacyOperatorState != null
-			|| hasState(managedOperatorState)
+		return hasState(managedOperatorState)
 			|| hasState(rawOperatorState)
 			|| hasState(managedKeyedState)
 			|| hasState(rawKeyedState);

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
index b69285e..cc9f9cd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java
@@ -28,7 +28,6 @@ import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 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.util.Preconditions;
 
 import org.slf4j.Logger;
@@ -162,8 +161,6 @@ public class StateAssignmentOperation {
 			Execution currentExecutionAttempt = executionJobVertex.getTaskVertices()[subTaskIndex]
 				.getCurrentExecutionAttempt();
 
-			List<StreamStateHandle> subNonPartitionableState = new ArrayList<>();
-
 			Tuple2<Collection<KeyedStateHandle>, Collection<KeyedStateHandle>> subKeyedState = null;
 
 			List<Collection<OperatorStateHandle>> subManagedOperatorState = new ArrayList<>();
@@ -174,15 +171,6 @@ public class StateAssignmentOperation {
 				OperatorState operatorState = operatorStates.get(operatorIndex);
 				int oldParallelism = operatorState.getParallelism();
 
-				// NonPartitioned State
-
-				reAssignSubNonPartitionedStates(
-					operatorState,
-					subTaskIndex,
-					newParallelism,
-					oldParallelism,
-					subNonPartitionableState);
-
 				// PartitionedState
 				reAssignSubPartitionableState(
 					newManagedOperatorStates,
@@ -204,8 +192,7 @@ public class StateAssignmentOperation {
 			}
 
 			// check if a stateless task
-			if (!allElementsAreNull(subNonPartitionableState) ||
-				!allElementsAreNull(subManagedOperatorState) ||
+			if (!allElementsAreNull(subManagedOperatorState) ||
 				!allElementsAreNull(subRawOperatorState) ||
 				subKeyedState != null) {
 
@@ -226,7 +213,6 @@ public class StateAssignmentOperation {
 
 					OperatorSubtaskState operatorSubtaskState =
 						new OperatorSubtaskState(
-							subNonPartitionableState.get(i),
 							subManagedOperatorState.get(i),
 							subRawOperatorState.get(i),
 							managedKeyed,
@@ -314,24 +300,6 @@ public class StateAssignmentOperation {
 		return true;
 	}
 
-
-	private void reAssignSubNonPartitionedStates(
-			OperatorState operatorState,
-			int subTaskIndex,
-			int newParallelism,
-			int oldParallelism,
-		List<StreamStateHandle> subNonPartitionableState) {
-		if (oldParallelism == newParallelism) {
-			if (operatorState.getState(subTaskIndex) != null) {
-				subNonPartitionableState.add(operatorState.getState(subTaskIndex).getLegacyOperatorState());
-			} else {
-				subNonPartitionableState.add(null);
-			}
-		} else {
-			subNonPartitionableState.add(null);
-		}
-	}
-
 	private void reDistributePartitionableStates(
 			List<OperatorState> operatorStates, int newParallelism,
 			List<List<Collection<OperatorStateHandle>>> newManagedOperatorStates,
@@ -524,19 +492,6 @@ public class StateAssignmentOperation {
 					"is currently not supported.");
 			}
 		}
-
-		//----------------------------------------parallelism preconditions-----------------------------------------
-
-		final int oldParallelism = operatorState.getParallelism();
-		final int newParallelism = executionJobVertex.getParallelism();
-
-		if (operatorState.hasNonPartitionedState() && (oldParallelism != newParallelism)) {
-			throw new IllegalStateException("Cannot restore the latest checkpoint because " +
-				"the operator " + executionJobVertex.getJobVertexId() + " has non-partitioned " +
-				"state and its parallelism changed. The operator " + executionJobVertex.getJobVertexId() +
-				" has parallelism " + newParallelism + " whereas the corresponding " +
-				"state object has a parallelism of " + oldParallelism);
-		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java
index 20d675b..281693b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java
@@ -25,14 +25,12 @@ import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.SharedStateRegistry;
 import org.apache.flink.runtime.state.StateObject;
 import org.apache.flink.runtime.state.StateUtil;
-import org.apache.flink.runtime.state.StreamStateHandle;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.Arrays;
 
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
 /**
  * Container for the chained state of one parallel subtask of an operator/task. This is part of the
  * {@link TaskState}.
@@ -44,15 +42,6 @@ public class SubtaskState implements CompositeStateHandle {
 	private static final long serialVersionUID = -2394696997971923995L;
 
 	/**
-	 * Legacy (non-repartitionable) operator state.
-	 *
-	 * @deprecated Non-repartitionable operator state that has been deprecated.
-	 * Can be removed when we remove the APIs for non-repartitionable operator state.
-	 */
-	@Deprecated
-	private final ChainedStateHandle<StreamStateHandle> legacyOperatorState;
-
-	/**
 	 * Snapshot from the {@link org.apache.flink.runtime.state.OperatorStateBackend}.
 	 */
 	private final ChainedStateHandle<OperatorStateHandle> managedOperatorState;
@@ -80,21 +69,18 @@ public class SubtaskState implements CompositeStateHandle {
 	private final long stateSize;
 
 	public SubtaskState(
-			ChainedStateHandle<StreamStateHandle> legacyOperatorState,
 			ChainedStateHandle<OperatorStateHandle> managedOperatorState,
 			ChainedStateHandle<OperatorStateHandle> rawOperatorState,
 			KeyedStateHandle managedKeyedState,
 			KeyedStateHandle rawKeyedState) {
 
-		this.legacyOperatorState = checkNotNull(legacyOperatorState, "State");
 		this.managedOperatorState = managedOperatorState;
 		this.rawOperatorState = rawOperatorState;
 		this.managedKeyedState = managedKeyedState;
 		this.rawKeyedState = rawKeyedState;
 
 		try {
-			long calculateStateSize = getSizeNullSafe(legacyOperatorState);
-			calculateStateSize += getSizeNullSafe(managedOperatorState);
+			long calculateStateSize = getSizeNullSafe(managedOperatorState);
 			calculateStateSize += getSizeNullSafe(rawOperatorState);
 			calculateStateSize += getSizeNullSafe(managedKeyedState);
 			calculateStateSize += getSizeNullSafe(rawKeyedState);
@@ -110,15 +96,6 @@ public class SubtaskState implements CompositeStateHandle {
 
 	// --------------------------------------------------------------------------------------------
 
-	/**
-	 * @deprecated Non-repartitionable operator state that has been deprecated.
-	 * Can be removed when we remove the APIs for non-repartitionable operator state.
-	 */
-	@Deprecated
-	public ChainedStateHandle<StreamStateHandle> getLegacyOperatorState() {
-		return legacyOperatorState;
-	}
-
 	public ChainedStateHandle<OperatorStateHandle> getManagedOperatorState() {
 		return managedOperatorState;
 	}
@@ -140,7 +117,6 @@ public class SubtaskState implements CompositeStateHandle {
 		try {
 			StateUtil.bestEffortDiscardAllStateObjects(
 				Arrays.asList(
-					legacyOperatorState,
 					managedOperatorState,
 					rawOperatorState,
 					managedKeyedState,
@@ -183,11 +159,6 @@ public class SubtaskState implements CompositeStateHandle {
 			return false;
 		}
 
-		if (legacyOperatorState != null ?
-				!legacyOperatorState.equals(that.legacyOperatorState)
-				: that.legacyOperatorState != null) {
-			return false;
-		}
 		if (managedOperatorState != null ?
 				!managedOperatorState.equals(that.managedOperatorState)
 				: that.managedOperatorState != null) {
@@ -211,8 +182,7 @@ public class SubtaskState implements CompositeStateHandle {
 
 	@Override
 	public int hashCode() {
-		int result = legacyOperatorState != null ? legacyOperatorState.hashCode() : 0;
-		result = 31 * result + (managedOperatorState != null ? managedOperatorState.hashCode() : 0);
+		int result = (managedOperatorState != null ? managedOperatorState.hashCode() : 0);
 		result = 31 * result + (rawOperatorState != null ? rawOperatorState.hashCode() : 0);
 		result = 31 * result + (managedKeyedState != null ? managedKeyedState.hashCode() : 0);
 		result = 31 * result + (rawKeyedState != null ? rawKeyedState.hashCode() : 0);
@@ -223,8 +193,7 @@ public class SubtaskState implements CompositeStateHandle {
 	@Override
 	public String toString() {
 		return "SubtaskState{" +
-				"chainedStateHandle=" + legacyOperatorState +
-				", operatorStateFromBackend=" + managedOperatorState +
+				"operatorStateFromBackend=" + managedOperatorState +
 				", operatorStateFromStream=" + rawOperatorState +
 				", keyedStateFromBackend=" + managedKeyedState +
 				", keyedStateFromStream=" + rawKeyedState +

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java
index ed847a4..0f3bedb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java
@@ -48,7 +48,6 @@ public class TaskState implements CompositeStateHandle {
 	/** handles to non-partitioned states, subtaskindex -> subtaskstate */
 	private final Map<Integer, SubtaskState> subtaskStates;
 
-
 	/** parallelism of the operator when it was checkpointed */
 	private final int parallelism;
 
@@ -117,15 +116,6 @@ public class TaskState implements CompositeStateHandle {
 		return chainLength;
 	}
 
-	public boolean hasNonPartitionedState() {
-		for(SubtaskState sts : subtaskStates.values()) {
-			if (sts != null && !sts.getLegacyOperatorState().isEmpty()) {
-				return true;
-			}
-		}
-		return false;
-	}
-
 	@Override
 	public void discardState() throws Exception {
 		for (SubtaskState subtaskState : subtaskStates.values()) {

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointSerializers.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointSerializers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointSerializers.java
index c1fcf4f..12e9c5b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointSerializers.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointSerializers.java
@@ -18,8 +18,7 @@
 
 package org.apache.flink.runtime.checkpoint.savepoint;
 
-import org.apache.flink.migration.runtime.checkpoint.savepoint.SavepointV0;
-import org.apache.flink.migration.runtime.checkpoint.savepoint.SavepointV0Serializer;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.util.Preconditions;
 
 import java.util.HashMap;
@@ -30,15 +29,20 @@ import java.util.Map;
  */
 public class SavepointSerializers {
 
+	/** If this flag is true, restoring a savepoint fails if it contains legacy state (<= Flink 1.1 format) */
+	static boolean FAIL_WHEN_LEGACY_STATE_DETECTED = true;
 
 	private static final Map<Integer, SavepointSerializer<?>> SERIALIZERS = new HashMap<>(2);
 
 	static {
-		SERIALIZERS.put(SavepointV0.VERSION, SavepointV0Serializer.INSTANCE);
 		SERIALIZERS.put(SavepointV1.VERSION, SavepointV1Serializer.INSTANCE);
 		SERIALIZERS.put(SavepointV2.VERSION, SavepointV2Serializer.INSTANCE);
 	}
 
+	private SavepointSerializers() {
+		throw new AssertionError();
+	}
+
 	// ------------------------------------------------------------------------
 
 	/**
@@ -77,4 +81,12 @@ public class SavepointSerializers {
 		}
 	}
 
+	/**
+	 * This is only visible as a temporary solution to keep the stateful job migration it cases working from binary
+	 * savepoints that still contain legacy state (<= Flink 1.1).
+	 */
+	@VisibleForTesting
+	public static void setFailWhenLegacyStateDetected(boolean fail) {
+		FAIL_WHEN_LEGACY_STATE_DETECTED = fail;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
index 7beb1b8..586df57 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointStore.java
@@ -36,6 +36,7 @@ import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
+
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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 f67d54c..c26c983 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
@@ -33,6 +33,7 @@ import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
 import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+import org.apache.flink.util.Preconditions;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -59,7 +60,6 @@ public class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 	private static final byte KEY_GROUPS_HANDLE = 3;
 	private static final byte PARTITIONABLE_OPERATOR_STATE_HANDLE = 4;
 
-
 	public static final SavepointV1Serializer INSTANCE = new SavepointV1Serializer();
 
 	private SavepointV1Serializer() {
@@ -130,20 +130,15 @@ public class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 
 	private static void serializeSubtaskState(SubtaskState subtaskState, DataOutputStream dos) throws IOException {
 
-		dos.writeLong(-1);
-
-		ChainedStateHandle<StreamStateHandle> nonPartitionableState = subtaskState.getLegacyOperatorState();
+		//backwards compatibility, do not remove
+		dos.writeLong(-1L);
 
-		int len = nonPartitionableState != null ? nonPartitionableState.getLength() : 0;
-		dos.writeInt(len);
-		for (int i = 0; i < len; ++i) {
-			StreamStateHandle stateHandle = nonPartitionableState.get(i);
-			serializeStreamStateHandle(stateHandle, dos);
-		}
+		//backwards compatibility (number of legacy state handles), do not remove
+		dos.writeInt(0);
 
 		ChainedStateHandle<OperatorStateHandle> operatorStateBackend = subtaskState.getManagedOperatorState();
 
-		len = operatorStateBackend != null ? operatorStateBackend.getLength() : 0;
+		int len = operatorStateBackend != null ? operatorStateBackend.getLength() : 0;
 		dos.writeInt(len);
 		for (int i = 0; i < len; ++i) {
 			OperatorStateHandle stateHandle = operatorStateBackend.get(i);
@@ -171,12 +166,19 @@ public class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 		long ignoredDuration = dis.readLong();
 
 		int len = dis.readInt();
-		List<StreamStateHandle> nonPartitionableState = new ArrayList<>(len);
-		for (int i = 0; i < len; ++i) {
-			StreamStateHandle streamStateHandle = deserializeStreamStateHandle(dis);
-			nonPartitionableState.add(streamStateHandle);
-		}
 
+		if (SavepointSerializers.FAIL_WHEN_LEGACY_STATE_DETECTED) {
+			Preconditions.checkState(len == 0,
+				"Legacy state (from Flink <= 1.1, created through the 'Checkpointed' interface) is " +
+					"no longer supported starting from Flink 1.4. Please rewrite your job to use " +
+					"'CheckpointedFunction' instead!");
+
+		} else {
+			for (int i = 0; i < len; ++i) {
+				// absorb bytes from stream and ignore result
+				deserializeStreamStateHandle(dis);
+			}
+		}
 
 		len = dis.readInt();
 		List<OperatorStateHandle> operatorStateBackend = new ArrayList<>(len);
@@ -196,9 +198,6 @@ public class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 
 		KeyedStateHandle keyedStateStream = deserializeKeyedStateHandle(dis);
 
-		ChainedStateHandle<StreamStateHandle> nonPartitionableStateChain =
-				new ChainedStateHandle<>(nonPartitionableState);
-
 		ChainedStateHandle<OperatorStateHandle> operatorStateBackendChain =
 				new ChainedStateHandle<>(operatorStateBackend);
 
@@ -206,7 +205,6 @@ public class SavepointV1Serializer implements SavepointSerializer<SavepointV2> {
 				new ChainedStateHandle<>(operatorStateStream);
 
 		return new SubtaskState(
-				nonPartitionableStateChain,
 				operatorStateBackendChain,
 				operatorStateStreamChain,
 				keyedStateBackend,

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2.java
index bd364a2..9e406df 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2.java
@@ -29,7 +29,6 @@ import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.state.ChainedStateHandle;
 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.util.Preconditions;
 
 import java.util.Collection;
@@ -207,9 +206,6 @@ public class SavepointV2 implements Savepoint {
 					continue;
 				}
 
-				@SuppressWarnings("deprecation")
-				ChainedStateHandle<StreamStateHandle> nonPartitionedState =
-					subtaskState.getLegacyOperatorState();
 				ChainedStateHandle<OperatorStateHandle> partitioneableState =
 					subtaskState.getManagedOperatorState();
 				ChainedStateHandle<OperatorStateHandle> rawOperatorState =
@@ -240,7 +236,6 @@ public class SavepointV2 implements Savepoint {
 						}
 
 						OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(
-							nonPartitionedState != null ? nonPartitionedState.get(operatorIndex) : null,
 							partitioneableState != null ? partitioneableState.get(operatorIndex) : null,
 							rawOperatorState != null ? rawOperatorState.get(operatorIndex) : null,
 							managedKeyedState,

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java
index 15628a0..5636a52 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java
@@ -33,6 +33,7 @@ import org.apache.flink.runtime.state.StateHandleID;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
 import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+import org.apache.flink.util.Preconditions;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -256,13 +257,8 @@ class SavepointV2Serializer implements SavepointSerializer<SavepointV2> {
 
 		dos.writeLong(-1);
 
-		StreamStateHandle nonPartitionableState = subtaskState.getLegacyOperatorState();
-
-		int len = nonPartitionableState != null ? 1 : 0;
+		int len = 0;
 		dos.writeInt(len);
-		if (len == 1) {
-			serializeStreamStateHandle(nonPartitionableState, dos);
-		}
 
 		OperatorStateHandle operatorStateBackend = extractSingleton(subtaskState.getManagedOperatorState());
 
@@ -288,11 +284,23 @@ class SavepointV2Serializer implements SavepointSerializer<SavepointV2> {
 	}
 
 	private static OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) throws IOException {
-		// Duration field has been removed from SubtaskState
+		// Duration field has been removed from SubtaskState, do not remove
 		long ignoredDuration = dis.readLong();
 
+		// for compatibility, do not remove
 		int len = dis.readInt();
-		StreamStateHandle nonPartitionableState = len == 0 ? null : deserializeStreamStateHandle(dis);
+
+		if (SavepointSerializers.FAIL_WHEN_LEGACY_STATE_DETECTED) {
+			Preconditions.checkState(len == 0,
+				"Legacy state (from Flink <= 1.1, created through the 'Checkpointed' interface) is " +
+					"no longer supported starting from Flink 1.4. Please rewrite your job to use " +
+					"'CheckpointedFunction' instead!");
+		} else {
+			for (int i = 0; i < len; ++i) {
+				// absorb bytes from stream and ignore result
+				deserializeStreamStateHandle(dis);
+			}
+		}
 
 		len = dis.readInt();
 		OperatorStateHandle operatorStateBackend = len == 0 ? null : deserializeOperatorStateHandle(dis);
@@ -305,7 +313,6 @@ class SavepointV2Serializer implements SavepointSerializer<SavepointV2> {
 		KeyedStateHandle keyedStateStream = deserializeKeyedStateHandle(dis);
 
 		return new OperatorSubtaskState(
-				nonPartitionableState,
 				operatorStateBackend,
 				operatorStateStream,
 				keyedStateBackend,

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespaceSerializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespaceSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespaceSerializer.java
index 2800899..8b58891 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespaceSerializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespaceSerializer.java
@@ -21,7 +21,6 @@ package org.apache.flink.runtime.state;
 import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.migration.MigrationNamespaceSerializerProxy;
 
 import java.io.IOException;
 
@@ -90,11 +89,4 @@ public final class VoidNamespaceSerializer extends TypeSerializerSingleton<VoidN
 	public boolean canEqual(Object obj) {
 		return obj instanceof VoidNamespaceSerializer;
 	}
-
-	@Override
-	protected boolean isCompatibleSerializationFormatIdentifier(String identifier) {
-		// we might be replacing a migration namespace serializer, in which case we just assume compatibility
-		return super.isCompatibleSerializationFormatIdentifier(identifier)
-			|| identifier.equals(MigrationNamespaceSerializerProxy.class.getCanonicalName());
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
index d1c0466..e235b96 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
@@ -35,11 +35,6 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.migration.MigrationNamespaceSerializerProxy;
-import org.apache.flink.migration.MigrationUtil;
-import org.apache.flink.migration.runtime.state.KvStateSnapshot;
-import org.apache.flink.migration.runtime.state.memory.MigrationRestoreSnapshot;
-import org.apache.flink.migration.state.MigrationKeyGroupStateHandle;
 import org.apache.flink.runtime.checkpoint.AbstractAsyncSnapshotIOCallable;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.io.async.AsyncStoppableTaskWithCallback;
@@ -65,7 +60,6 @@ import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.runtime.state.internal.InternalMapState;
 import org.apache.flink.runtime.state.internal.InternalReducingState;
 import org.apache.flink.runtime.state.internal.InternalValueState;
-import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.StateMigrationException;
 
@@ -190,7 +184,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			// check compatibility results to determine if state migration is required
 			CompatibilityResult<N> namespaceCompatibility = CompatibilityUtil.resolveCompatibilityResult(
 					restoredMetaInfo.getNamespaceSerializer(),
-					MigrationNamespaceSerializerProxy.class,
+					null,
 					restoredMetaInfo.getNamespaceSerializerConfigSnapshot(),
 					newMetaInfo.getNamespaceSerializer());
 
@@ -405,11 +399,7 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			LOG.debug("Restoring snapshot from state handles: {}.", restoredState);
 		}
 
-		if (MigrationUtil.isOldSavepointKeyedState(restoredState)) {
-			restoreOldSavepointKeyedState(restoredState);
-		} else {
-			restorePartitionedState(restoredState);
-		}
+		restorePartitionedState(restoredState);
 	}
 
 	@SuppressWarnings({"unchecked"})
@@ -560,55 +550,6 @@ public class HeapKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	/**
-	 * @deprecated Used for backwards compatibility with previous savepoint versions.
-	 */
-	@SuppressWarnings({"unchecked", "rawtypes", "DeprecatedIsStillUsed"})
-	@Deprecated
-	private void restoreOldSavepointKeyedState(
-			Collection<KeyedStateHandle> stateHandles) throws IOException, ClassNotFoundException {
-
-		if (stateHandles.isEmpty()) {
-			return;
-		}
-
-		Preconditions.checkState(1 == stateHandles.size(), "Only one element expected here.");
-
-		KeyedStateHandle keyedStateHandle = stateHandles.iterator().next();
-		if (!(keyedStateHandle instanceof MigrationKeyGroupStateHandle)) {
-			throw new IllegalStateException("Unexpected state handle type, " +
-					"expected: " + MigrationKeyGroupStateHandle.class +
-					", but found " + keyedStateHandle.getClass());
-		}
-
-		MigrationKeyGroupStateHandle keyGroupStateHandle = (MigrationKeyGroupStateHandle) keyedStateHandle;
-
-		HashMap<String, KvStateSnapshot<K, ?, ?, ?>> namedStates;
-		try (FSDataInputStream inputStream = keyGroupStateHandle.openInputStream()) {
-			namedStates = InstantiationUtil.deserializeObject(inputStream, userCodeClassLoader);
-		}
-
-		for (Map.Entry<String, KvStateSnapshot<K, ?, ?, ?>> nameToState : namedStates.entrySet()) {
-
-			final String stateName = nameToState.getKey();
-			final KvStateSnapshot<K, ?, ?, ?> genericSnapshot = nameToState.getValue();
-
-			if (genericSnapshot instanceof MigrationRestoreSnapshot) {
-				MigrationRestoreSnapshot<K, ?, ?> stateSnapshot = (MigrationRestoreSnapshot<K, ?, ?>) genericSnapshot;
-				final StateTable rawResultMap =
-						stateSnapshot.deserialize(stateName, this);
-
-				// mimic a restored kv state meta info
-				restoredKvStateMetaInfos.put(stateName, rawResultMap.getMetaInfo().snapshot());
-
-				// add named state to the backend
-				stateTables.put(stateName, rawResultMap);
-			} else {
-				throw new IllegalStateException("Unknown state: " + genericSnapshot);
-			}
-		}
-	}
-
-	/**
 	 * Returns the total number of state entries across all keys/namespaces.
 	 */
 	@VisibleForTesting

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java
index 26db772..7c95a34 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java
@@ -29,7 +29,6 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.SharedStateRegistry;
-import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.Test;
@@ -92,15 +91,12 @@ public class CheckpointCoordinatorFailureTest extends TestLogger {
 
 		final long checkpointId = coord.getPendingCheckpoints().keySet().iterator().next();
 
-
-		StreamStateHandle legacyHandle = mock(StreamStateHandle.class);
 		KeyedStateHandle managedKeyedHandle = mock(KeyedStateHandle.class);
 		KeyedStateHandle rawKeyedHandle = mock(KeyedStateHandle.class);
 		OperatorStateHandle managedOpHandle = mock(OperatorStateHandle.class);
 		OperatorStateHandle rawOpHandle = mock(OperatorStateHandle.class);
 
 		final OperatorSubtaskState operatorSubtaskState = spy(new OperatorSubtaskState(
-			legacyHandle,
 			managedOpHandle,
 			rawOpHandle,
 			managedKeyedHandle,
@@ -126,7 +122,6 @@ public class CheckpointCoordinatorFailureTest extends TestLogger {
 
 		// make sure that the subtask state has been discarded after we could not complete it.
 		verify(operatorSubtaskState).discardState();
-		verify(operatorSubtaskState.getLegacyOperatorState()).discardState();
 		verify(operatorSubtaskState.getManagedOperatorState().iterator().next()).discardState();
 		verify(operatorSubtaskState.getRawOperatorState().iterator().next()).discardState();
 		verify(operatorSubtaskState.getManagedKeyedState().iterator().next()).discardState();

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
index 45cbbc3..4193c2c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java
@@ -90,7 +90,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -2149,15 +2148,13 @@ public class CheckpointCoordinatorTest extends TestLogger {
 
 		assertTrue(coord.getPendingCheckpoints().keySet().size() == 1);
 		long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet());
-		CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, 0L);
 
 		List<KeyGroupRange> keyGroupPartitions1 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1);
 		List<KeyGroupRange> keyGroupPartitions2 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2);
 
 		for (int index = 0; index < jobVertex1.getParallelism(); index++) {
-			StreamStateHandle valueSizeTuple = generateStateForVertex(jobVertexID1, index);
 			KeyGroupsStateHandle keyGroupState = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), false);
-			OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(valueSizeTuple, null, null, keyGroupState, null);
+			OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(null, null, keyGroupState, null);
 			TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot();
 			taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID1), operatorSubtaskState);
 			AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint(
@@ -2172,9 +2169,8 @@ public class CheckpointCoordinatorTest extends TestLogger {
 
 
 		for (int index = 0; index < jobVertex2.getParallelism(); index++) {
-			StreamStateHandle valueSizeTuple = generateStateForVertex(jobVertexID2, index);
 			KeyGroupsStateHandle keyGroupState = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), false);
-			OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(valueSizeTuple, null, null, keyGroupState, null);
+			OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(null, null, keyGroupState, null);
 			TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot();
 			taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID2), operatorSubtaskState);
 			AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint(
@@ -2214,137 +2210,6 @@ public class CheckpointCoordinatorTest extends TestLogger {
 		fail("The restoration should have failed because the max parallelism changed.");
 	}
 
-	/**
-	 * Tests that the checkpoint restoration fails if the parallelism of a job vertices with
-	 * non-partitioned state has changed.
-	 *
-	 * @throws Exception
-	 */
-	@Test(expected=IllegalStateException.class)
-	public void testRestoreLatestCheckpointFailureWhenParallelismChanges() throws Exception {
-		final JobID jid = new JobID();
-		final long timestamp = System.currentTimeMillis();
-
-		final JobVertexID jobVertexID1 = new JobVertexID();
-		final JobVertexID jobVertexID2 = new JobVertexID();
-		int parallelism1 = 3;
-		int parallelism2 = 2;
-		int maxParallelism1 = 42;
-		int maxParallelism2 = 13;
-
-		final ExecutionJobVertex jobVertex1 = mockExecutionJobVertex(
-			jobVertexID1,
-			parallelism1,
-			maxParallelism1);
-		final ExecutionJobVertex jobVertex2 = mockExecutionJobVertex(
-			jobVertexID2,
-			parallelism2,
-			maxParallelism2);
-
-		List<ExecutionVertex> allExecutionVertices = new ArrayList<>(parallelism1 + parallelism2);
-
-		allExecutionVertices.addAll(Arrays.asList(jobVertex1.getTaskVertices()));
-		allExecutionVertices.addAll(Arrays.asList(jobVertex2.getTaskVertices()));
-
-		ExecutionVertex[] arrayExecutionVertices =
-				allExecutionVertices.toArray(new ExecutionVertex[allExecutionVertices.size()]);
-
-		// set up the coordinator and validate the initial state
-		CheckpointCoordinator coord = new CheckpointCoordinator(
-			jid,
-			600000,
-			600000,
-			0,
-			Integer.MAX_VALUE,
-			ExternalizedCheckpointSettings.none(),
-			arrayExecutionVertices,
-			arrayExecutionVertices,
-			arrayExecutionVertices,
-			new StandaloneCheckpointIDCounter(),
-			new StandaloneCompletedCheckpointStore(1),
-			null,
-			Executors.directExecutor(),
-			SharedStateRegistry.DEFAULT_FACTORY);
-
-		// trigger the checkpoint
-		coord.triggerCheckpoint(timestamp, false);
-
-		assertTrue(coord.getPendingCheckpoints().keySet().size() == 1);
-		long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet());
-		CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, 0L);
-
-		List<KeyGroupRange> keyGroupPartitions1 =
-				StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1);
-		List<KeyGroupRange> keyGroupPartitions2 =
-				StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2);
-
-		for (int index = 0; index < jobVertex1.getParallelism(); index++) {
-			StreamStateHandle valueSizeTuple = generateStateForVertex(jobVertexID1, index);
-			KeyGroupsStateHandle keyGroupState = generateKeyGroupState(
-					jobVertexID1, keyGroupPartitions1.get(index), false);
-
-			OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(valueSizeTuple, null, null, keyGroupState, null);
-			TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot();
-			taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID1), operatorSubtaskState);
-
-			AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint(
-					jid,
-					jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(),
-					checkpointId,
-					new CheckpointMetrics(),
-					taskOperatorSubtaskStates);
-
-			coord.receiveAcknowledgeMessage(acknowledgeCheckpoint);
-		}
-
-
-		for (int index = 0; index < jobVertex2.getParallelism(); index++) {
-
-			StreamStateHandle state = generateStateForVertex(jobVertexID2, index);
-			KeyGroupsStateHandle keyGroupState = generateKeyGroupState(
-					jobVertexID2, keyGroupPartitions2.get(index), false);
-
-			OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(state, null, null, keyGroupState, null);
-			TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot();
-			taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID2), operatorSubtaskState);
-			AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint(
-					jid,
-					jobVertex2.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(),
-					checkpointId,
-					new CheckpointMetrics(),
-					taskOperatorSubtaskStates);
-
-			coord.receiveAcknowledgeMessage(acknowledgeCheckpoint);
-		}
-
-		List<CompletedCheckpoint> completedCheckpoints = coord.getSuccessfulCheckpoints();
-
-		assertEquals(1, completedCheckpoints.size());
-
-		Map<JobVertexID, ExecutionJobVertex> tasks = new HashMap<>();
-
-		int newParallelism1 = 4;
-		int newParallelism2 = 3;
-
-		final ExecutionJobVertex newJobVertex1 = mockExecutionJobVertex(
-			jobVertexID1,
-			newParallelism1,
-			maxParallelism1);
-
-		final ExecutionJobVertex newJobVertex2 = mockExecutionJobVertex(
-			jobVertexID2,
-			newParallelism2,
-			maxParallelism2);
-
-		tasks.put(jobVertexID1, newJobVertex1);
-		tasks.put(jobVertexID2, newJobVertex2);
-
-		coord.restoreLatestCheckpointedState(tasks, true, false);
-
-		fail("The restoration should have failed because the parallelism of an vertex with " +
-			"non-partitioned state changed.");
-	}
-
 	@Test
 	public void testRestoreLatestCheckpointedStateScaleIn() throws Exception {
 		testRestoreLatestCheckpointedStateWithChangingParallelism(false);
@@ -2439,12 +2304,10 @@ public class CheckpointCoordinatorTest extends TestLogger {
 
 		//vertex 1
 		for (int index = 0; index < jobVertex1.getParallelism(); index++) {
-			StreamStateHandle valueSizeTuple = generateStateForVertex(jobVertexID1, index);
 			OperatorStateHandle opStateBackend = generatePartitionableStateHandle(jobVertexID1, index, 2, 8, false);
 			KeyGroupsStateHandle keyedStateBackend = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), false);
 			KeyGroupsStateHandle keyedStateRaw = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), true);
-
-			OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(valueSizeTuple, opStateBackend, null, keyedStateBackend, keyedStateRaw);
+			OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(opStateBackend, null, keyedStateBackend, keyedStateRaw);
 			TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot();
 			taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID1), operatorSubtaskState);
 
@@ -2469,7 +2332,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			expectedOpStatesBackend.add(new ChainedStateHandle<>(Collections.singletonList(opStateBackend)));
 			expectedOpStatesRaw.add(new ChainedStateHandle<>(Collections.singletonList(opStateRaw)));
 
-			OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(null, opStateBackend, opStateRaw, keyedStateBackend, keyedStateRaw);
+			OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(opStateBackend, opStateRaw, keyedStateBackend, keyedStateRaw);
 			TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot();
 			taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID2), operatorSubtaskState);
 
@@ -2527,7 +2390,6 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			for (int idx = 0; idx < operatorIDs.size(); ++idx) {
 				OperatorID operatorID = operatorIDs.get(idx);
 				OperatorSubtaskState opState = taskStateHandles.getSubtaskStateByOperatorID(operatorID);
-				Assert.assertNull(opState.getLegacyOperatorState());
 				Collection<OperatorStateHandle> opStateBackend = opState.getManagedOperatorState();
 				Collection<OperatorStateHandle> opStateRaw = opState.getRawOperatorState();
 				allParallelManagedOpStates.add(opStateBackend);
@@ -2593,14 +2455,11 @@ public class CheckpointCoordinatorTest extends TestLogger {
 			OperatorState taskState = new OperatorState(id.f1, parallelism1, maxParallelism1);
 			operatorStates.put(id.f1, taskState);
 			for (int index = 0; index < taskState.getParallelism(); index++) {
-				StreamStateHandle subNonPartitionedState = 
-					generateStateForVertex(id.f0, index);
 				OperatorStateHandle subManagedOperatorState =
 					generatePartitionableStateHandle(id.f0, index, 2, 8, false);
 				OperatorStateHandle subRawOperatorState =
 					generatePartitionableStateHandle(id.f0, index, 2, 8, true);
-
-				OperatorSubtaskState subtaskState = new OperatorSubtaskState(subNonPartitionedState,
+				OperatorSubtaskState subtaskState = new OperatorSubtaskState(
 					subManagedOperatorState,
 					subRawOperatorState,
 					null,
@@ -2638,7 +2497,6 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				expectedRawOperatorState.add(ChainedStateHandle.wrapSingleHandle(subRawOperatorState));
 
 				OperatorSubtaskState subtaskState = new OperatorSubtaskState(
-					null,
 					subManagedOperatorState,
 					subRawOperatorState,
 					subManagedKeyedState,
@@ -2735,7 +2593,6 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				OperatorSubtaskState opState =
 					stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain));
 
-				assertNull(opState.getLegacyOperatorState());
 				assertTrue(opState.getManagedOperatorState().isEmpty());
 				assertTrue(opState.getRawOperatorState().isEmpty());
 			}
@@ -2745,16 +2602,11 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				OperatorSubtaskState opState =
 					stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain));
 
-				StreamStateHandle expectSubNonPartitionedState = generateStateForVertex(id1.f0, i);
 				OperatorStateHandle expectedManagedOpState = generatePartitionableStateHandle(
 					id1.f0, i, 2, 8, false);
 				OperatorStateHandle expectedRawOpState = generatePartitionableStateHandle(
 					id1.f0, i, 2, 8, true);
 
-				assertTrue(CommonTestUtils.isSteamContentEqual(
-					expectSubNonPartitionedState.openInputStream(),
-					opState.getLegacyOperatorState().openInputStream()));
-
 				Collection<OperatorStateHandle> managedOperatorState = opState.getManagedOperatorState();
 				assertEquals(1, managedOperatorState.size());
 				assertTrue(CommonTestUtils.isSteamContentEqual(expectedManagedOpState.openInputStream(),
@@ -2771,16 +2623,11 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				OperatorSubtaskState opState =
 					stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain));
 
-				StreamStateHandle expectSubNonPartitionedState = generateStateForVertex(id2.f0, i);
 				OperatorStateHandle expectedManagedOpState = generatePartitionableStateHandle(
 					id2.f0, i, 2, 8, false);
 				OperatorStateHandle expectedRawOpState = generatePartitionableStateHandle(
 					id2.f0, i, 2, 8, true);
 
-				assertTrue(CommonTestUtils.isSteamContentEqual(
-					expectSubNonPartitionedState.openInputStream(),
-					opState.getLegacyOperatorState().openInputStream()));
-
 				Collection<OperatorStateHandle> managedOperatorState = opState.getManagedOperatorState();
 				assertEquals(1, managedOperatorState.size());
 				assertTrue(CommonTestUtils.isSteamContentEqual(expectedManagedOpState.openInputStream(),
@@ -2816,8 +2663,6 @@ public class CheckpointCoordinatorTest extends TestLogger {
 
 				actualManagedOperatorStates.add(actualSubManagedOperatorState);
 				actualRawOperatorStates.add(actualSubRawOperatorState);
-
-				assertNull(opState.getLegacyOperatorState());
 			}
 
 			// operator 6
@@ -2825,7 +2670,6 @@ public class CheckpointCoordinatorTest extends TestLogger {
 				int operatorIndexInChain = 0;
 				OperatorSubtaskState opState =
 					stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain));
-				assertNull(opState.getLegacyOperatorState());
 				assertTrue(opState.getManagedOperatorState().isEmpty());
 				assertTrue(opState.getRawOperatorState().isEmpty());
 
@@ -3216,13 +3060,12 @@ public class CheckpointCoordinatorTest extends TestLogger {
 		int index,
 		KeyGroupRange keyGroupRange) throws IOException {
 
-		StreamStateHandle nonPartitionedState = generateStateForVertex(jobVertexID, index);
 		OperatorStateHandle partitionableState = generatePartitionableStateHandle(jobVertexID, index, 2, 8, false);
 		KeyGroupsStateHandle partitionedKeyGroupState = generateKeyGroupState(jobVertexID, keyGroupRange, false);
 
 		TaskStateSnapshot subtaskStates = spy(new TaskStateSnapshot());
 		OperatorSubtaskState subtaskState = spy(new OperatorSubtaskState(
-			nonPartitionedState, partitionableState, null, partitionedKeyGroupState, null)
+			partitionableState, null, partitionedKeyGroupState, null)
 		);
 
 		subtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID), subtaskState);
@@ -3236,17 +3079,10 @@ public class CheckpointCoordinatorTest extends TestLogger {
 
 		for (int i = 0; i < executionJobVertex.getParallelism(); i++) {
 
-			final List<OperatorID> operatorIds = executionJobVertex.getOperatorIDs();
-
 			TaskStateSnapshot stateSnapshot = executionJobVertex.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskStateSnapshot();
 
 			OperatorSubtaskState operatorState = stateSnapshot.getSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID));
 
-			StreamStateHandle expectNonPartitionedState = generateStateForVertex(jobVertexID, i);
-			assertTrue(CommonTestUtils.isSteamContentEqual(
-					expectNonPartitionedState.openInputStream(),
-				operatorState.getLegacyOperatorState().openInputStream()));
-
 			ChainedStateHandle<OperatorStateHandle> expectedOpStateBackend =
 					generateChainedPartitionableStateHandle(jobVertexID, i, 2, 8, false);
 
@@ -3926,7 +3762,7 @@ public class CheckpointCoordinatorTest extends TestLogger {
 					spy(new ByteStreamStateHandle("meta", new byte[]{'m'}))));
 
 			OperatorSubtaskState operatorSubtaskState =
-				spy(new OperatorSubtaskState(null,
+				spy(new OperatorSubtaskState(
 					Collections.<OperatorStateHandle>emptyList(),
 					Collections.<OperatorStateHandle>emptyList(),
 					Collections.<KeyedStateHandle>singletonList(managedState),

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
index 791bffa..1788434 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java
@@ -29,12 +29,10 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
-import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.SharedStateRegistry;
-import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.util.SerializableObject;
 
 import org.hamcrest.BaseMatcher;
@@ -67,7 +65,6 @@ public class CheckpointStateRestoreTest {
 	public void testSetState() {
 		try {
 
-			final ChainedStateHandle<StreamStateHandle> serializedState = CheckpointCoordinatorTest.generateChainedStateHandle(new SerializableObject());
 			KeyGroupRange keyGroupRange = KeyGroupRange.of(0,0);
 			List<SerializableObject> testStates = Collections.singletonList(new SerializableObject());
 			final KeyedStateHandle serializedKeyGroupStates = CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRange, testStates);
@@ -125,7 +122,6 @@ public class CheckpointStateRestoreTest {
 			subtaskStates.putSubtaskStateByOperatorID(
 				OperatorID.fromJobVertexID(statefulId),
 				new OperatorSubtaskState(
-					serializedState.get(0),
 					Collections.<OperatorStateHandle>emptyList(),
 					Collections.<OperatorStateHandle>emptyList(),
 					Collections.singletonList(serializedKeyGroupStates),
@@ -249,17 +245,13 @@ public class CheckpointStateRestoreTest {
 			Executors.directExecutor(),
 			SharedStateRegistry.DEFAULT_FACTORY);
 
-		StreamStateHandle serializedState = CheckpointCoordinatorTest
-				.generateChainedStateHandle(new SerializableObject())
-				.get(0);
-
 		// --- (2) Checkpoint misses state for a jobVertex (should work) ---
 		Map<OperatorID, OperatorState> checkpointTaskStates = new HashMap<>();
 		{
 			OperatorState taskState = new OperatorState(operatorId1, 3, 3);
-			taskState.putState(0, new OperatorSubtaskState(serializedState));
-			taskState.putState(1, new OperatorSubtaskState(serializedState));
-			taskState.putState(2, new OperatorSubtaskState(serializedState));
+			taskState.putState(0, new OperatorSubtaskState());
+			taskState.putState(1, new OperatorSubtaskState());
+			taskState.putState(2, new OperatorSubtaskState());
 
 			checkpointTaskStates.put(operatorId1, taskState);
 		}
@@ -286,7 +278,7 @@ public class CheckpointStateRestoreTest {
 		// There is no task for this
 		{
 			OperatorState taskState = new OperatorState(newOperatorID, 1, 1);
-			taskState.putState(0, new OperatorSubtaskState(serializedState));
+			taskState.putState(0, new OperatorSubtaskState());
 
 			checkpointTaskStates.put(newOperatorID, taskState);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java
index de1f599..acedb50 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java
@@ -77,7 +77,6 @@ public class CheckpointTestUtils {
 
 			OperatorState taskState = new OperatorState(new OperatorID(), numSubtasksPerTask, 128);
 
-			boolean hasNonPartitionableState = random.nextBoolean();
 			boolean hasOperatorStateBackend = random.nextBoolean();
 			boolean hasOperatorStateStream = random.nextBoolean();
 
@@ -87,7 +86,6 @@ public class CheckpointTestUtils {
 
 			for (int subtaskIdx = 0; subtaskIdx < numSubtasksPerTask; subtaskIdx++) {
 
-				StreamStateHandle nonPartitionableState = null;
 				StreamStateHandle operatorStateBackend =
 					new TestByteStreamStateHandleDeepCompare("b", ("Beautiful").getBytes(ConfigConstants.DEFAULT_CHARSET));
 				StreamStateHandle operatorStateStream =
@@ -101,11 +99,6 @@ public class CheckpointTestUtils {
 				offsetsMap.put("B", new OperatorStateHandle.StateMetaInfo(new long[]{30, 40, 50}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE));
 				offsetsMap.put("C", new OperatorStateHandle.StateMetaInfo(new long[]{60, 70, 80}, OperatorStateHandle.Mode.BROADCAST));
 
-				if (hasNonPartitionableState) {
-					nonPartitionableState =
-						new TestByteStreamStateHandleDeepCompare("a", ("Hi").getBytes(ConfigConstants.DEFAULT_CHARSET));
-				}
-
 				if (hasOperatorStateBackend) {
 					operatorStateHandleBackend = new OperatorStateHandle(offsetsMap, operatorStateBackend);
 				}
@@ -130,7 +123,6 @@ public class CheckpointTestUtils {
 				}
 
 				taskState.putState(subtaskIdx, new OperatorSubtaskState(
-						nonPartitionableState,
 						operatorStateHandleBackend,
 						operatorStateHandleStream,
 						keyedStateStream,
@@ -175,15 +167,11 @@ public class CheckpointTestUtils {
 
 			for (int subtaskIdx = 0; subtaskIdx < numSubtasksPerTask; subtaskIdx++) {
 
-				List<StreamStateHandle> nonPartitionableStates = new ArrayList<>(chainLength);
 				List<OperatorStateHandle> operatorStatesBackend = new ArrayList<>(chainLength);
 				List<OperatorStateHandle> operatorStatesStream = new ArrayList<>(chainLength);
 
 				for (int chainIdx = 0; chainIdx < chainLength; ++chainIdx) {
 
-					StreamStateHandle nonPartitionableState =
-							new TestByteStreamStateHandleDeepCompare("a-" + chainIdx, ("Hi-" + chainIdx).getBytes(
-								ConfigConstants.DEFAULT_CHARSET));
 					StreamStateHandle operatorStateBackend =
 							new TestByteStreamStateHandleDeepCompare("b-" + chainIdx, ("Beautiful-" + chainIdx).getBytes(ConfigConstants.DEFAULT_CHARSET));
 					StreamStateHandle operatorStateStream =
@@ -193,10 +181,6 @@ public class CheckpointTestUtils {
 					offsetsMap.put("B", new OperatorStateHandle.StateMetaInfo(new long[]{30, 40, 50}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE));
 					offsetsMap.put("C", new OperatorStateHandle.StateMetaInfo(new long[]{60, 70, 80}, OperatorStateHandle.Mode.BROADCAST));
 
-					if (chainIdx != noNonPartitionableStateAtIndex) {
-						nonPartitionableStates.add(nonPartitionableState);
-					}
-
 					if (chainIdx != noOperatorStateBackendAtIndex) {
 						OperatorStateHandle operatorStateHandleBackend =
 								new OperatorStateHandle(offsetsMap, operatorStateBackend);
@@ -222,7 +206,6 @@ public class CheckpointTestUtils {
 				}
 
 				taskState.putState(subtaskIdx, new SubtaskState(
-						new ChainedStateHandle<>(nonPartitionableStates),
 						new ChainedStateHandle<>(operatorStatesBackend),
 						new ChainedStateHandle<>(operatorStatesStream),
 						keyedStateStream,


[04/11] flink git commit: [FLINK-7461] Remove Backwards compatibility with <= Flink 1.1

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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
index 9f19064..d7df479 100644
--- 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
@@ -29,7 +29,6 @@ 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;
@@ -84,7 +83,7 @@ public class WindowOperatorMigrationTest {
 
 	@Parameterized.Parameters(name = "Migration Savepoint: {0}")
 	public static Collection<MigrationVersion> parameters () {
-		return Arrays.asList(MigrationVersion.v1_1, MigrationVersion.v1_2, MigrationVersion.v1_3);
+		return Arrays.asList(MigrationVersion.v1_2, MigrationVersion.v1_3);
 	}
 
 	/**
@@ -753,219 +752,6 @@ public class WindowOperatorMigrationTest {
 		testHarness.close();
 	}
 
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeAggregatingAlignedProcessingTimeWindowsSnapshot() throws Exception {
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		AggregatingProcessingTimeWindowOperator<String, Tuple2<String, Integer>> operator =
-			new AggregatingProcessingTimeWindowOperator<>(
-				new ReduceFunction<Tuple2<String, Integer>>() {
-					private static final long serialVersionUID = -8913160567151867987L;
-
-					@Override
-					public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) throws Exception {
-						return new Tuple2<>(value1.f0, value1.f1 + value2.f1);
-					}
-				},
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				inputType.createSerializer(new ExecutionConfig()),
-				3000,
-				3000);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.open();
-
-		testHarness.setProcessingTime(3);
-
-		// timestamp is ignored in processing time
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		// do a snapshot, close and restore again
-		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
-		OperatorSnapshotUtil.writeStateHandle(
-			snapshot,
-			"src/test/resources/win-op-migration-test-aggr-aligned-flink" + flinkGenerateSavepointVersion + "-snapshot");
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestoreAggregatingAlignedProcessingTimeWindows() throws Exception {
-		final int windowSize = 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(windowSize, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */,
-				LegacyWindowOperatorType.FAST_AGGREGATING);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.setup();
-
-		MigrationTestUtil.restoreFromSnapshot(
-			testHarness,
-			OperatorSnapshotUtil.getResourceFilename(
-				"win-op-migration-test-aggr-aligned-flink" + testMigrateVersion + "-snapshot"),
-			testMigrateVersion);
-
-		testHarness.open();
-
-		testHarness.setProcessingTime(5000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		testHarness.setProcessingTime(7000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.close();
-	}
-
-	/**
-	 * Manually run this to write binary snapshot data.
-	 */
-	@Ignore
-	@Test
-	public void writeAlignedProcessingTimeWindowsSnapshot() throws Exception {
-		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-
-		AccumulatingProcessingTimeWindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>> operator =
-			new AccumulatingProcessingTimeWindowOperator<>(
-					new InternalIterableWindowFunction<>(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, String, TimeWindow>() {
-
-						private static final long serialVersionUID = 6551516443265733803L;
-
-						@Override
-						public void apply(String s, TimeWindow window, Iterable<Tuple2<String, Integer>> input, Collector<Tuple2<String, Integer>> out) throws Exception {
-							int sum = 0;
-							for (Tuple2<String, Integer> anInput : input) {
-								sum += anInput.f1;
-							}
-							out.collect(new Tuple2<>(s, sum));
-						}
-					}),
-					new TupleKeySelector(),
-					BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-					inputType.createSerializer(new ExecutionConfig()),
-					3000,
-					3000);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		testHarness.open();
-
-		testHarness.setProcessingTime(3);
-
-		// timestamp is ignored in processing time
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), Long.MAX_VALUE));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), 7000));
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		// do a snapshot, close and restore again
-		OperatorStateHandles snapshot = testHarness.snapshot(0, 0);
-		OperatorSnapshotUtil.writeStateHandle(
-			snapshot,
-			"src/test/resources/win-op-migration-test-accum-aligned-flink" + flinkGenerateSavepointVersion + "-snapshot");
-		testHarness.close();
-	}
-
-	@Test
-	public void testRestoreAccumulatingAlignedProcessingTimeWindows() throws Exception {
-		final int windowSize = 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(windowSize, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				stateDesc,
-				new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
-				ProcessingTimeTrigger.create(),
-				0,
-				null /* late data output tag */,
-				LegacyWindowOperatorType.FAST_ACCUMULATING);
-
-		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.setup();
-
-		MigrationTestUtil.restoreFromSnapshot(
-			testHarness,
-			OperatorSnapshotUtil.getResourceFilename(
-				"win-op-migration-test-accum-aligned-flink" + testMigrateVersion + "-snapshot"),
-			testMigrateVersion);
-
-		testHarness.open();
-
-		testHarness.setProcessingTime(5000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), 2999));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), 2999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), 7000));
-
-		testHarness.setProcessingTime(7000);
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 3), 5999));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple2ResultSortComparator());
-
-		testHarness.close();
-	}
-
 	private static class TupleKeySelector implements KeySelector<Tuple2<String, Integer>, String> {
 		private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
index 8748ed4..821438e 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
@@ -63,7 +63,6 @@ import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
 import org.apache.flink.util.Collector;
 
 import org.junit.Assert;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.concurrent.TimeUnit;
@@ -336,32 +335,6 @@ public class WindowTranslationTest {
 		processElementAndEnsureOutput(winOperator, winOperator.getKeySelector(), BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1));
 	}
 
-
-	/**
-	 * Ignored because we currently don't have the fast processing-time window operator.
-	 */
-	@Test
-	@SuppressWarnings("rawtypes")
-	@Ignore
-	public void testReduceFastProcessingTime() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DataStream<Tuple2<String, Integer>> window = source
-				.keyBy(new TupleKeySelector())
-				.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.reduce(new DummyReducer());
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform =
-				(OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator = transform.getOperator();
-		Assert.assertTrue(operator instanceof AggregatingProcessingTimeWindowOperator);
-
-		processElementAndEnsureOutput(operator, null, BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1));
-	}
-
 	@Test
 	@SuppressWarnings("rawtypes")
 	public void testReduceWithWindowFunctionEventTime() throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
index 14ae733..f73499c 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
@@ -64,7 +64,6 @@ import org.apache.flink.runtime.taskmanager.TaskManagerActions;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
 import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.graph.StreamConfig;
@@ -75,7 +74,6 @@ import org.junit.Test;
 
 import java.io.EOFException;
 import java.io.IOException;
-import java.io.Serializable;
 import java.net.URL;
 import java.util.Collection;
 import java.util.Collections;
@@ -106,12 +104,6 @@ public class InterruptSensitiveRestoreTest {
 	private static final int OPERATOR_RAW = 1;
 	private static final int KEYED_MANAGED = 2;
 	private static final int KEYED_RAW = 3;
-	private static final int LEGACY = 4;
-
-	@Test
-	public void testRestoreWithInterruptLegacy() throws Exception {
-		testRestoreWithInterrupt(LEGACY);
-	}
 
 	@Test
 	public void testRestoreWithInterruptOperatorManaged() throws Exception {
@@ -137,18 +129,15 @@ public class InterruptSensitiveRestoreTest {
 
 		IN_RESTORE_LATCH.reset();
 		Configuration taskConfig = new Configuration();
-		StreamConfig streamConfig = new StreamConfig(taskConfig);
-		streamConfig.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+		StreamConfig cfg = new StreamConfig(taskConfig);
+		cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
 		switch (mode) {
 			case OPERATOR_MANAGED:
 			case OPERATOR_RAW:
 			case KEYED_MANAGED:
 			case KEYED_RAW:
-				streamConfig.setStateKeySerializer(IntSerializer.INSTANCE);
-				streamConfig.setStreamOperator(new StreamSource<>(new TestSource()));
-				break;
-			case LEGACY:
-				streamConfig.setStreamOperator(new StreamSource<>(new TestSourceLegacy()));
+				cfg.setStateKeySerializer(IntSerializer.INSTANCE);
+				cfg.setStreamOperator(new StreamSource<>(new TestSource(mode)));
 				break;
 			default:
 				throw new IllegalArgumentException();
@@ -156,7 +145,7 @@ public class InterruptSensitiveRestoreTest {
 
 		StreamStateHandle lockingHandle = new InterruptLockingStateHandle();
 
-		Task task = createTask(streamConfig, taskConfig, lockingHandle, mode);
+		Task task = createTask(cfg, taskConfig, lockingHandle, mode);
 
 		// start the task and wait until it is in "restore"
 		task.startTaskThread();
@@ -180,16 +169,15 @@ public class InterruptSensitiveRestoreTest {
 	// ------------------------------------------------------------------------
 
 	private static Task createTask(
-		StreamConfig streamConfig,
-		Configuration taskConfig,
-		StreamStateHandle state,
-		int mode) throws IOException {
+			StreamConfig streamConfig,
+			Configuration taskConfig,
+			StreamStateHandle state,
+			int mode) throws IOException {
 
 		NetworkEnvironment networkEnvironment = mock(NetworkEnvironment.class);
 		when(networkEnvironment.createKvStateTaskRegistry(any(JobID.class), any(JobVertexID.class)))
 				.thenReturn(mock(TaskKvStateRegistry.class));
 
-		StreamStateHandle operatorState = null;
 		Collection<KeyedStateHandle> keyedStateFromBackend = Collections.emptyList();
 		Collection<KeyedStateHandle> keyedStateFromStream = Collections.emptyList();
 		Collection<OperatorStateHandle> operatorStateBackend = Collections.emptyList();
@@ -206,7 +194,7 @@ public class InterruptSensitiveRestoreTest {
 				Collections.singletonList(new OperatorStateHandle(operatorStateMetadata, state));
 
 		List<KeyedStateHandle> keyedStateHandles =
-				Collections.<KeyedStateHandle>singletonList(new KeyGroupsStateHandle(keyGroupRangeOffsets, state));
+				Collections.singletonList(new KeyGroupsStateHandle(keyGroupRangeOffsets, state));
 
 		switch (mode) {
 			case OPERATOR_MANAGED:
@@ -221,15 +209,11 @@ public class InterruptSensitiveRestoreTest {
 			case KEYED_RAW:
 				keyedStateFromStream = keyedStateHandles;
 				break;
-			case LEGACY:
-				operatorState = state;
-				break;
 			default:
 				throw new IllegalArgumentException();
 		}
 
 		OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(
-			operatorState,
 			operatorStateBackend,
 			operatorStateStream,
 			keyedStateFromBackend,
@@ -238,14 +222,13 @@ public class InterruptSensitiveRestoreTest {
 		JobVertexID jobVertexID = new JobVertexID();
 		OperatorID operatorID = OperatorID.fromJobVertexID(jobVertexID);
 		streamConfig.setOperatorID(operatorID);
-
 		TaskStateSnapshot stateSnapshot = new TaskStateSnapshot();
 		stateSnapshot.putSubtaskStateByOperatorID(operatorID, operatorSubtaskState);
 		JobInformation jobInformation = new JobInformation(
 			new JobID(),
 			"test job name",
 			new SerializedValue<>(new ExecutionConfig()),
-			taskConfig,
+			new Configuration(),
 			Collections.<BlobKey>emptyList(),
 			Collections.<URL>emptyList());
 
@@ -302,11 +285,11 @@ public class InterruptSensitiveRestoreTest {
 			FSDataInputStream is = new FSDataInputStream() {
 
 				@Override
-				public void seek(long desired) throws IOException {
+				public void seek(long desired) {
 				}
 
 				@Override
-				public long getPos() throws IOException {
+				public long getPos() {
 					return 0;
 				}
 
@@ -358,33 +341,15 @@ public class InterruptSensitiveRestoreTest {
 
 	// ------------------------------------------------------------------------
 
-	private static class TestSourceLegacy implements SourceFunction<Object>, Checkpointed<Serializable> {
+	private static class TestSource implements SourceFunction<Object>, CheckpointedFunction {
 		private static final long serialVersionUID = 1L;
+		private final int testType;
 
-		@Override
-		public void run(SourceContext<Object> ctx) throws Exception {
-			fail("should never be called");
+		public TestSource(int testType) {
+			this.testType = testType;
 		}
 
 		@Override
-		public void cancel() {}
-
-		@Override
-		public Serializable snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			fail("should never be called");
-			return null;
-		}
-
-		@Override
-		public void restoreState(Serializable state) throws Exception {
-			fail("should never be called");
-		}
-	}
-
-	private static class TestSource implements SourceFunction<Object>, CheckpointedFunction {
-		private static final long serialVersionUID = 1L;
-
-		@Override
 		public void run(SourceContext<Object> ctx) throws Exception {
 			fail("should never be called");
 		}
@@ -399,6 +364,8 @@ public class InterruptSensitiveRestoreTest {
 
 		@Override
 		public void initializeState(FunctionInitializationContext context) throws Exception {
+			// raw keyed state is already read by timer service, all others to initialize the context...we only need to
+			// trigger this manually.
 			((StateInitializationContext) context).getRawOperatorStateInputs().iterator().next().getStream().read();
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java
index 3190620..8d80d66 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java
@@ -28,8 +28,6 @@ import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
@@ -47,21 +45,18 @@ import org.apache.flink.streaming.api.graph.StreamEdge;
 import org.apache.flink.streaming.api.graph.StreamNode;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator;
 import org.apache.flink.streaming.api.operators.StreamMap;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
 import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -69,7 +64,6 @@ import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
@@ -78,7 +72,6 @@ import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -93,7 +86,7 @@ import static org.junit.Assert.fail;
 public class OneInputStreamTaskTest extends TestLogger {
 
 	private static final ListStateDescriptor<Integer> TEST_DESCRIPTOR =
-			new ListStateDescriptor<>("test", new IntSerializer());
+		new ListStateDescriptor<>("test", new IntSerializer());
 
 	/**
 	 * This test verifies that open() and close() are correctly called. This test also verifies
@@ -129,8 +122,8 @@ public class OneInputStreamTaskTest extends TestLogger {
 		assertTrue("RichFunction methods where not called.", TestOpenCloseMapFunction.closeCalled);
 
 		TestHarnessUtil.assertOutputEquals("Output was not correct.",
-				expectedOutput,
-				testHarness.getOutput());
+			expectedOutput,
+			testHarness.getOutput());
 	}
 
 	/**
@@ -174,8 +167,8 @@ public class OneInputStreamTaskTest extends TestLogger {
 		testHarness.waitForInputProcessing();
 		expectedOutput.add(new Watermark(initialTime));
 		TestHarnessUtil.assertOutputEquals("Output was not correct.",
-				expectedOutput,
-				testHarness.getOutput());
+			expectedOutput,
+			testHarness.getOutput());
 
 		// contrary to checkpoint barriers these elements are not blocked by watermarks
 		testHarness.processElement(new StreamRecord<String>("Hello", initialTime));
@@ -215,7 +208,7 @@ public class OneInputStreamTaskTest extends TestLogger {
 		testHarness.processElement(new Watermark(initialTime + 6), 0, 0);
 		testHarness.processElement(new Watermark(initialTime + 5), 1, 1); // this watermark should be advanced first
 		testHarness.processElement(StreamStatus.IDLE, 1, 1); // once this is acknowledged,
-		                                                     // watermark (initial + 6) should be forwarded
+		// watermark (initial + 6) should be forwarded
 		testHarness.waitForInputProcessing();
 		expectedOutput.add(new Watermark(initialTime + 5));
 		expectedOutput.add(new Watermark(initialTime + 6));
@@ -263,21 +256,16 @@ public class OneInputStreamTaskTest extends TestLogger {
 		// ------------------ setup the chain ------------------
 
 		TriggerableFailOnWatermarkTestOperator headOperator = new TriggerableFailOnWatermarkTestOperator();
-		OperatorID headOperatorId = new OperatorID();
-
 		StreamConfig headOperatorConfig = testHarness.getStreamConfig();
 
 		WatermarkGeneratingTestOperator watermarkOperator = new WatermarkGeneratingTestOperator();
-		OperatorID watermarkOperatorId = new OperatorID();
-
 		StreamConfig watermarkOperatorConfig = new StreamConfig(new Configuration());
 
 		TriggerableFailOnWatermarkTestOperator tailOperator = new TriggerableFailOnWatermarkTestOperator();
-		OperatorID tailOperatorId = new OperatorID();
 		StreamConfig tailOperatorConfig = new StreamConfig(new Configuration());
 
 		headOperatorConfig.setStreamOperator(headOperator);
-		headOperatorConfig.setOperatorID(headOperatorId);
+		headOperatorConfig.setOperatorID(new OperatorID(42L, 42L));
 		headOperatorConfig.setChainStart();
 		headOperatorConfig.setChainIndex(0);
 		headOperatorConfig.setChainedOutputs(Collections.singletonList(new StreamEdge(
@@ -290,7 +278,7 @@ public class OneInputStreamTaskTest extends TestLogger {
 		)));
 
 		watermarkOperatorConfig.setStreamOperator(watermarkOperator);
-		watermarkOperatorConfig.setOperatorID(watermarkOperatorId);
+		watermarkOperatorConfig.setOperatorID(new OperatorID(4711L, 42L));
 		watermarkOperatorConfig.setTypeSerializerIn1(StringSerializer.INSTANCE);
 		watermarkOperatorConfig.setChainIndex(1);
 		watermarkOperatorConfig.setChainedOutputs(Collections.singletonList(new StreamEdge(
@@ -312,7 +300,7 @@ public class OneInputStreamTaskTest extends TestLogger {
 			null));
 
 		tailOperatorConfig.setStreamOperator(tailOperator);
-		tailOperatorConfig.setOperatorID(tailOperatorId);
+		tailOperatorConfig.setOperatorID(new OperatorID(123L, 123L));
 		tailOperatorConfig.setTypeSerializerIn1(StringSerializer.INSTANCE);
 		tailOperatorConfig.setBufferTimeout(0);
 		tailOperatorConfig.setChainIndex(2);
@@ -555,13 +543,11 @@ public class OneInputStreamTaskTest extends TestLogger {
 
 		long checkpointId = 1L;
 		long checkpointTimestamp = 1L;
-		long recoveryTimestamp = 3L;
-		long seed = 2L;
 		int numberChainedTasks = 11;
 
 		StreamConfig streamConfig = testHarness.getStreamConfig();
 
-		configureChainedTestingStreamOperator(streamConfig, numberChainedTasks, seed, recoveryTimestamp);
+		configureChainedTestingStreamOperator(streamConfig, numberChainedTasks);
 
 		AcknowledgeStreamMockEnvironment env = new AcknowledgeStreamMockEnvironment(
 			testHarness.jobConfig,
@@ -599,7 +585,7 @@ public class OneInputStreamTaskTest extends TestLogger {
 
 		StreamConfig restoredTaskStreamConfig = restoredTaskHarness.getStreamConfig();
 
-		configureChainedTestingStreamOperator(restoredTaskStreamConfig, numberChainedTasks, seed, recoveryTimestamp);
+		configureChainedTestingStreamOperator(restoredTaskStreamConfig, numberChainedTasks);
 
 		TaskStateSnapshot stateHandles = env.getCheckpointStateHandles();
 		Assert.assertEquals(numberChainedTasks, stateHandles.getSubtaskStateMappings().size());
@@ -625,16 +611,12 @@ public class OneInputStreamTaskTest extends TestLogger {
 
 	private void configureChainedTestingStreamOperator(
 		StreamConfig streamConfig,
-		int numberChainedTasks,
-		long seed,
-		long recoveryTimestamp) {
+		int numberChainedTasks) {
 
 		Preconditions.checkArgument(numberChainedTasks >= 1, "The operator chain must at least " +
 			"contain one operator.");
 
-		Random random = new Random(seed);
-
-		TestingStreamOperator<Integer, Integer> previousOperator = new TestingStreamOperator<>(random.nextLong(), recoveryTimestamp);
+		TestingStreamOperator<Integer, Integer> previousOperator = new TestingStreamOperator<>();
 		streamConfig.setStreamOperator(previousOperator);
 		streamConfig.setOperatorID(new OperatorID(0L, 0L));
 
@@ -643,7 +625,7 @@ public class OneInputStreamTaskTest extends TestLogger {
 		List<StreamEdge> outputEdges = new ArrayList<>(numberChainedTasks - 1);
 
 		for (int chainedIndex = 1; chainedIndex < numberChainedTasks; chainedIndex++) {
-			TestingStreamOperator<Integer, Integer> chainedOperator = new TestingStreamOperator<>(random.nextLong(), recoveryTimestamp);
+			TestingStreamOperator<Integer, Integer> chainedOperator = new TestingStreamOperator<>();
 			StreamConfig chainedConfig = new StreamConfig(new Configuration());
 			chainedConfig.setStreamOperator(chainedOperator);
 			chainedConfig.setOperatorID(new OperatorID(0L, chainedIndex));
@@ -702,17 +684,17 @@ public class OneInputStreamTaskTest extends TestLogger {
 		}
 
 		AcknowledgeStreamMockEnvironment(
-				Configuration jobConfig, Configuration taskConfig,
-				ExecutionConfig executionConfig, long memorySize,
-				MockInputSplitProvider inputSplitProvider, int bufferSize) {
+			Configuration jobConfig, Configuration taskConfig,
+			ExecutionConfig executionConfig, long memorySize,
+			MockInputSplitProvider inputSplitProvider, int bufferSize) {
 			super(jobConfig, taskConfig, executionConfig, memorySize, inputSplitProvider, bufferSize);
 		}
 
 		@Override
 		public void acknowledgeCheckpoint(
-				long checkpointId,
-				CheckpointMetrics checkpointMetrics,
-				TaskStateSnapshot checkpointStateHandles) {
+			long checkpointId,
+			CheckpointMetrics checkpointMetrics,
+			TaskStateSnapshot checkpointStateHandles) {
 
 			this.checkpointId = checkpointId;
 			this.checkpointStateHandles = checkpointStateHandles;
@@ -729,19 +711,14 @@ public class OneInputStreamTaskTest extends TestLogger {
 	}
 
 	private static class TestingStreamOperator<IN, OUT>
-			extends AbstractStreamOperator<OUT>
-			implements OneInputStreamOperator<IN, OUT>, StreamCheckpointedOperator {
+		extends AbstractStreamOperator<OUT>
+		implements OneInputStreamOperator<IN, OUT> {
 
 		private static final long serialVersionUID = 774614855940397174L;
 
 		public static int numberRestoreCalls = 0;
 		public static int numberSnapshotCalls = 0;
 
-		private final long seed;
-		private final long recoveryTimestamp;
-
-		private transient Random random;
-
 		@Override
 		public void open() throws Exception {
 			super.open();
@@ -767,7 +744,7 @@ public class OneInputStreamTaskTest extends TestLogger {
 		@Override
 		public void snapshotState(StateSnapshotContext context) throws Exception {
 			ListState<Integer> partitionableState =
-					getOperatorStateBackend().getListState(TEST_DESCRIPTOR);
+				getOperatorStateBackend().getListState(TEST_DESCRIPTOR);
 			partitionableState.clear();
 
 			partitionableState.add(42);
@@ -778,59 +755,14 @@ public class OneInputStreamTaskTest extends TestLogger {
 
 		@Override
 		public void initializeState(StateInitializationContext context) throws Exception {
-
-		}
-
-		TestingStreamOperator(long seed, long recoveryTimestamp) {
-			this.seed = seed;
-			this.recoveryTimestamp = recoveryTimestamp;
-		}
-
-		@Override
-		public void processElement(StreamRecord<IN> element) throws Exception {
-
-		}
-
-		@Override
-		public void snapshotState(FSDataOutputStream out, long checkpointId, long timestamp) throws Exception {
-			if (random == null) {
-				random = new Random(seed);
+			if (context.isRestored()) {
+				++numberRestoreCalls;
 			}
-
-			Serializable functionState = generateFunctionState();
-			Integer operatorState = generateOperatorState();
-
-			InstantiationUtil.serializeObject(out, functionState);
-			InstantiationUtil.serializeObject(out, operatorState);
 		}
 
 		@Override
-		public void restoreState(FSDataInputStream in) throws Exception {
-			numberRestoreCalls++;
-
-			if (random == null) {
-				random = new Random(seed);
-			}
-
-			assertEquals(this.recoveryTimestamp, recoveryTimestamp);
-
-			assertNotNull(in);
-
-			ClassLoader cl = Thread.currentThread().getContextClassLoader();
-
-			Serializable functionState = InstantiationUtil.deserializeObject(in, cl);
-			Integer operatorState = InstantiationUtil.deserializeObject(in, cl);
-
-			assertEquals(random.nextInt(), functionState);
-			assertEquals(random.nextInt(), (int) operatorState);
-		}
-
-		private Serializable generateFunctionState() {
-			return random.nextInt();
-		}
+		public void processElement(StreamRecord<IN> element) throws Exception {
 
-		private Integer generateOperatorState() {
-			return random.nextInt();
 		}
 	}
 
@@ -913,8 +845,8 @@ public class OneInputStreamTaskTest extends TestLogger {
 	 * <p>If it receives a watermark when it's not expecting one, it'll throw an exception and fail.
 	 */
 	private static class TriggerableFailOnWatermarkTestOperator
-			extends AbstractStreamOperator<String>
-			implements OneInputStreamOperator<String, String> {
+		extends AbstractStreamOperator<String>
+		implements OneInputStreamOperator<String, String> {
 
 		private static final long serialVersionUID = 2048954179291813243L;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
index 08c3207..a2dc6c4 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
@@ -81,7 +81,6 @@ import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.OperatorSnapshotResult;
 import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator;
 import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.operators.StreamSource;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -139,7 +138,6 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.withSettings;
 import static org.powermock.api.mockito.PowerMockito.whenNew;
 
 /**
@@ -307,9 +305,9 @@ public class StreamTaskTest extends TestLogger {
 		streamTask.setEnvironment(mockEnvironment);
 
 		// mock the operators
-		StreamOperator<?> streamOperator1 = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
-		StreamOperator<?> streamOperator2 = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
-		StreamOperator<?> streamOperator3 = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
+		StreamOperator<?> streamOperator1 = mock(StreamOperator.class);
+		StreamOperator<?> streamOperator2 = mock(StreamOperator.class);
+		StreamOperator<?> streamOperator3 = mock(StreamOperator.class);
 
 		// mock the returned snapshots
 		OperatorSnapshotResult operatorSnapshotResult1 = mock(OperatorSnapshotResult.class);
@@ -321,15 +319,6 @@ public class StreamTaskTest extends TestLogger {
 		when(streamOperator2.snapshotState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(operatorSnapshotResult2);
 		when(streamOperator3.snapshotState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenThrow(testException);
 
-		// mock the returned legacy snapshots
-		StreamStateHandle streamStateHandle1 = mock(StreamStateHandle.class);
-		StreamStateHandle streamStateHandle2 = mock(StreamStateHandle.class);
-		StreamStateHandle streamStateHandle3 = mock(StreamStateHandle.class);
-
-		when(streamOperator1.snapshotLegacyOperatorState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(streamStateHandle1);
-		when(streamOperator2.snapshotLegacyOperatorState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(streamStateHandle2);
-		when(streamOperator3.snapshotLegacyOperatorState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(streamStateHandle3);
-
 		OperatorID operatorID1 = new OperatorID();
 		OperatorID operatorID2 = new OperatorID();
 		OperatorID operatorID3 = new OperatorID();
@@ -359,10 +348,6 @@ public class StreamTaskTest extends TestLogger {
 
 		verify(operatorSnapshotResult1).cancel();
 		verify(operatorSnapshotResult2).cancel();
-
-		verify(streamStateHandle1).discardState();
-		verify(streamStateHandle2).discardState();
-		verify(streamStateHandle3).discardState();
 	}
 
 	/**
@@ -384,12 +369,12 @@ public class StreamTaskTest extends TestLogger {
 		CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp);
 		streamTask.setEnvironment(mockEnvironment);
 
-		StreamOperator<?> streamOperator1 = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
-		StreamOperator<?> streamOperator2 = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
-		StreamOperator<?> streamOperator3 = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
-
-		// mock the new state handles / futures
+		// mock the operators
+		StreamOperator<?> streamOperator1 = mock(StreamOperator.class);
+		StreamOperator<?> streamOperator2 = mock(StreamOperator.class);
+		StreamOperator<?> streamOperator3 = mock(StreamOperator.class);
 
+		// mock the new state operator snapshots
 		OperatorSnapshotResult operatorSnapshotResult1 = mock(OperatorSnapshotResult.class);
 		OperatorSnapshotResult operatorSnapshotResult2 = mock(OperatorSnapshotResult.class);
 		OperatorSnapshotResult operatorSnapshotResult3 = mock(OperatorSnapshotResult.class);
@@ -403,15 +388,6 @@ public class StreamTaskTest extends TestLogger {
 		when(streamOperator2.snapshotState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(operatorSnapshotResult2);
 		when(streamOperator3.snapshotState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(operatorSnapshotResult3);
 
-		// mock the legacy state snapshot
-		StreamStateHandle streamStateHandle1 = mock(StreamStateHandle.class);
-		StreamStateHandle streamStateHandle2 = mock(StreamStateHandle.class);
-		StreamStateHandle streamStateHandle3 = mock(StreamStateHandle.class);
-
-		when(streamOperator1.snapshotLegacyOperatorState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(streamStateHandle1);
-		when(streamOperator2.snapshotLegacyOperatorState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(streamStateHandle2);
-		when(streamOperator3.snapshotLegacyOperatorState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(streamStateHandle3);
-
 		OperatorID operatorID1 = new OperatorID();
 		OperatorID operatorID2 = new OperatorID();
 		OperatorID operatorID3 = new OperatorID();
@@ -438,10 +414,6 @@ public class StreamTaskTest extends TestLogger {
 		verify(operatorSnapshotResult1).cancel();
 		verify(operatorSnapshotResult2).cancel();
 		verify(operatorSnapshotResult3).cancel();
-
-		verify(streamStateHandle1).discardState();
-		verify(streamStateHandle2).discardState();
-		verify(streamStateHandle3).discardState();
 	}
 
 	/**
@@ -481,7 +453,7 @@ public class StreamTaskTest extends TestLogger {
 		CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp);
 		streamTask.setEnvironment(mockEnvironment);
 
-		StreamOperator<?> streamOperator = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
+		StreamOperator<?> streamOperator = mock(StreamOperator.class);
 
 		KeyedStateHandle managedKeyedStateHandle = mock(KeyedStateHandle.class);
 		KeyedStateHandle rawKeyedStateHandle = mock(KeyedStateHandle.class);
@@ -581,7 +553,6 @@ public class StreamTaskTest extends TestLogger {
 
 		whenNew(OperatorSubtaskState.class).
 			withArguments(
-				any(StreamStateHandle.class),
 				anyCollectionOf(OperatorStateHandle.class),
 				anyCollectionOf(OperatorStateHandle.class),
 				anyCollectionOf(KeyedStateHandle.class),
@@ -593,11 +564,10 @@ public class StreamTaskTest extends TestLogger {
 				completeSubtask.await();
 				Object[] arguments = invocation.getArguments();
 				return new OperatorSubtaskState(
-					(StreamStateHandle) arguments[0],
+					(OperatorStateHandle) arguments[0],
 					(OperatorStateHandle) arguments[1],
-					(OperatorStateHandle) arguments[2],
-					(KeyedStateHandle) arguments[3],
-					(KeyedStateHandle) arguments[4]
+					(KeyedStateHandle) arguments[2],
+					(KeyedStateHandle) arguments[3]
 				);
 			}
 		});
@@ -606,7 +576,7 @@ public class StreamTaskTest extends TestLogger {
 		CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp);
 		streamTask.setEnvironment(mockEnvironment);
 
-		final StreamOperator<?> streamOperator = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
+		final StreamOperator<?> streamOperator = mock(StreamOperator.class);
 		final OperatorID operatorID = new OperatorID();
 		when(streamOperator.getOperatorID()).thenReturn(operatorID);
 
@@ -717,7 +687,7 @@ public class StreamTaskTest extends TestLogger {
 
 		// mock the operators
 		StreamOperator<?> statelessOperator =
-				mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class));
+				mock(StreamOperator.class);
 
 		final OperatorID operatorID = new OperatorID();
 		when(statelessOperator.getOperatorID()).thenReturn(operatorID);

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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 720346a..9156f34 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
@@ -25,11 +25,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.CloseableRegistry;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FSDataOutputStream;
-import org.apache.flink.migration.runtime.checkpoint.savepoint.SavepointV0Serializer;
-import org.apache.flink.migration.streaming.runtime.tasks.StreamTaskState;
-import org.apache.flink.migration.util.MigrationInstantiationUtil;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.checkpoint.OperatorStateRepartitioner;
 import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
@@ -41,12 +36,10 @@ import org.apache.flink.runtime.operators.testutils.MockEnvironment;
 import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.graph.StreamConfig;
@@ -54,7 +47,6 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperatorTest;
 import org.apache.flink.streaming.api.operators.OperatorSnapshotResult;
 import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator;
 import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
@@ -72,7 +64,6 @@ import org.apache.flink.util.Preconditions;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.io.FileInputStream;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -308,36 +299,6 @@ public class AbstractStreamOperatorTestHarness<OUT> implements AutoCloseable {
 		setupCalled = true;
 	}
 
-	public void initializeStateFromLegacyCheckpoint(String checkpointFilename) throws Exception {
-
-		FileInputStream fin = new FileInputStream(checkpointFilename);
-		StreamTaskState state = MigrationInstantiationUtil.deserializeObject(fin, ClassLoader.getSystemClassLoader());
-		fin.close();
-
-		if (!setupCalled) {
-			setup();
-		}
-
-		StreamStateHandle stateHandle = SavepointV0Serializer.convertOperatorAndFunctionState(state);
-
-		List<KeyedStateHandle> keyGroupStatesList = new ArrayList<>();
-		if (state.getKvStates() != null) {
-			KeyGroupsStateHandle keyedStateHandle = SavepointV0Serializer.convertKeyedBackendState(
-					state.getKvStates(),
-					environment.getTaskInfo().getIndexOfThisSubtask(),
-					0);
-			keyGroupStatesList.add(keyedStateHandle);
-		}
-
-		// finally calling the initializeState() with the legacy operatorStateHandles
-		initializeState(new OperatorStateHandles(0,
-				stateHandle,
-				keyGroupStatesList,
-				Collections.<KeyedStateHandle>emptyList(),
-				Collections.<OperatorStateHandle>emptyList(),
-				Collections.<OperatorStateHandle>emptyList()));
-	}
-
 	/**
 	 * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#initializeState(OperatorSubtaskState)}.
 	 * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#setup(StreamTask, StreamConfig, Output)}
@@ -397,7 +358,6 @@ public class AbstractStreamOperatorTestHarness<OUT> implements AutoCloseable {
 					numSubtasks).get(subtaskIndex);
 
 			OperatorSubtaskState massagedOperatorStateHandles = new OperatorSubtaskState(
-				operatorStateHandles.getLegacyOperatorState(),
 				nullToEmptyCollection(localManagedOperatorState),
 				nullToEmptyCollection(localRawOperatorState),
 				nullToEmptyCollection(localManagedKeyGroupState),
@@ -473,7 +433,6 @@ public class AbstractStreamOperatorTestHarness<OUT> implements AutoCloseable {
 
 		return new OperatorStateHandles(
 			0,
-			null,
 			mergedManagedKeyedState,
 			mergedRawKeyedState,
 			mergedManagedOperatorState,
@@ -497,8 +456,6 @@ public class AbstractStreamOperatorTestHarness<OUT> implements AutoCloseable {
 	 */
 	public OperatorStateHandles snapshot(long checkpointId, long timestamp) throws Exception {
 
-		CheckpointStreamFactory streamFactory = stateBackend.createStreamFactory(new JobID(), "test_op");
-
 		OperatorSnapshotResult operatorStateResult = operator.snapshotState(
 			checkpointId,
 			timestamp,
@@ -510,21 +467,8 @@ public class AbstractStreamOperatorTestHarness<OUT> implements AutoCloseable {
 		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,
-			legacyStateHandle,
 			keyedManaged != null ? Collections.singletonList(keyedManaged) : null,
 			keyedRaw != null ? Collections.singletonList(keyedRaw) : null,
 			opManaged != null ? Collections.singletonList(opManaged) : null,
@@ -532,24 +476,6 @@ public class AbstractStreamOperatorTestHarness<OUT> implements AutoCloseable {
 	}
 
 	/**
-	 * Calls {@link StreamCheckpointedOperator#snapshotState(FSDataOutputStream, long, long)} if
-	 * the operator implements this interface.
-	 */
-	@Deprecated
-	public StreamStateHandle snapshotLegacy(long checkpointId, long timestamp) throws Exception {
-
-		CheckpointStreamFactory.CheckpointStateOutputStream outStream = stateBackend.createStreamFactory(
-				new JobID(),
-				"test_op").createCheckpointStateOutputStream(checkpointId, timestamp);
-		if (operator instanceof StreamCheckpointedOperator) {
-			((StreamCheckpointedOperator) operator).snapshotState(outStream, checkpointId, timestamp);
-			return outStream.closeAndGetHandle();
-		} else {
-			throw new RuntimeException("Operator is not StreamCheckpointedOperator");
-		}
-	}
-
-	/**
 	 * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#notifyOfCompletedCheckpoint(long)} ()}.
 	 */
 	public void notifyOfCompletedCheckpoint(long checkpointId) throws Exception {
@@ -557,22 +483,6 @@ public class AbstractStreamOperatorTestHarness<OUT> implements AutoCloseable {
 	}
 
 	/**
-	 * Calls {@link StreamCheckpointedOperator#restoreState(FSDataInputStream)} if
-	 * the operator implements this interface.
-	 */
-	@Deprecated
-	@SuppressWarnings("deprecation")
-	public void restore(StreamStateHandle snapshot) throws Exception {
-		if (operator instanceof StreamCheckpointedOperator) {
-			try (FSDataInputStream in = snapshot.openInputStream()) {
-				((StreamCheckpointedOperator) operator).restoreState(in);
-			}
-		} else {
-			throw new RuntimeException("Operator is not StreamCheckpointedOperator");
-		}
-	}
-
-	/**
 	 * Calls close and dispose on the operator.
 	 */
 	public void close() throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
index 0d42d9f..c2ec63a 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
@@ -23,33 +23,23 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.ClosureCleaner;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.checkpoint.StateAssignmentOperation;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
-import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateBackend;
 import org.apache.flink.runtime.state.KeyedStateHandle;
-import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator;
 import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
 import org.apache.flink.util.Migration;
 
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
-import java.util.concurrent.RunnableFuture;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.anyInt;
@@ -142,61 +132,6 @@ public class KeyedOneInputStreamOperatorTestHarness<K, IN, OUT>
 		}
 	}
 
-	/**
-	 *
-	 */
-	@Override
-	public StreamStateHandle snapshotLegacy(long checkpointId, long timestamp) throws Exception {
-		// simply use an in-memory handle
-		MemoryStateBackend backend = new MemoryStateBackend();
-
-		CheckpointStreamFactory streamFactory = backend.createStreamFactory(new JobID(), "test_op");
-		CheckpointStreamFactory.CheckpointStateOutputStream outStream =
-				streamFactory.createCheckpointStateOutputStream(checkpointId, timestamp);
-
-		if (operator instanceof StreamCheckpointedOperator) {
-			((StreamCheckpointedOperator) operator).snapshotState(outStream, checkpointId, timestamp);
-		}
-
-		if (keyedStateBackend != null) {
-			RunnableFuture<KeyedStateHandle> keyedSnapshotRunnable = keyedStateBackend.snapshot(
-					checkpointId,
-					timestamp,
-					streamFactory,
-					CheckpointOptions.forFullCheckpoint());
-			if (!keyedSnapshotRunnable.isDone()) {
-				Thread runner = new Thread(keyedSnapshotRunnable);
-				runner.start();
-			}
-			outStream.write(1);
-			ObjectOutputStream oos = new ObjectOutputStream(outStream);
-			oos.writeObject(keyedSnapshotRunnable.get());
-			oos.flush();
-		} else {
-			outStream.write(0);
-		}
-		return outStream.closeAndGetHandle();
-	}
-
-	/**
-	 *
-	 */
-	@Override
-	public void restore(StreamStateHandle snapshot) throws Exception {
-		try (FSDataInputStream inStream = snapshot.openInputStream()) {
-
-			if (operator instanceof StreamCheckpointedOperator) {
-				((StreamCheckpointedOperator) operator).restoreState(inStream);
-			}
-
-			byte keyedStatePresent = (byte) inStream.read();
-			if (keyedStatePresent == 1) {
-				ObjectInputStream ois = new ObjectInputStream(inStream);
-				this.restoredKeyedState = Collections.singletonList((KeyedStateHandle) ois.readObject());
-			}
-		}
-	}
-
 	private static boolean hasMigrationHandles(Collection<KeyedStateHandle> allKeyGroupsHandles) {
 		for (KeyedStateHandle handle : allKeyGroupsHandles) {
 			if (handle instanceof Migration) {

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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
index 7e32723..33f32e9 100644
--- 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
@@ -21,7 +21,6 @@ package org.apache.flink.streaming.util;
 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;
 
 import java.io.DataInputStream;
@@ -53,7 +52,8 @@ public class OperatorSnapshotUtil {
 
 			dos.writeInt(state.getOperatorChainIndex());
 
-			SavepointV1Serializer.serializeStreamStateHandle(state.getLegacyOperatorState(), dos);
+			// still required for compatibility
+			SavepointV1Serializer.serializeStreamStateHandle(null, dos);
 
 			Collection<OperatorStateHandle> rawOperatorState = state.getRawOperatorState();
 			if (rawOperatorState != null) {
@@ -108,7 +108,8 @@ public class OperatorSnapshotUtil {
 		try (DataInputStream dis = new DataInputStream(in)) {
 			int index = dis.readInt();
 
-			StreamStateHandle legacyState = SavepointV1Serializer.deserializeStreamStateHandle(dis);
+			// still required for compatibility to consume the bytes.
+			SavepointV1Serializer.deserializeStreamStateHandle(dis);
 
 			List<OperatorStateHandle> rawOperatorState = null;
 			int numRawOperatorStates = dis.readInt();
@@ -154,7 +155,12 @@ public class OperatorSnapshotUtil {
 				}
 			}
 
-			return new OperatorStateHandles(index, legacyState, managedKeyedState, rawKeyedState, managedOperatorState, rawOperatorState);
+			return new OperatorStateHandles(
+				index,
+				managedKeyedState,
+				rawKeyedState,
+				managedOperatorState,
+				rawOperatorState);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationTestUtil.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationTestUtil.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationTestUtil.java
index f723b34..1c95a04 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationTestUtil.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/migration/MigrationTestUtil.java
@@ -29,22 +29,16 @@ public class MigrationTestUtil {
 	/**
 	 * Restore from a snapshot taken with an older Flink version.
 	 *
-	 * @param testHarness the test harness to restore the snapshot to.
-	 * @param snapshotPath the absolute path to the snapshot.
+	 * @param testHarness          the test harness to restore the snapshot to.
+	 * @param snapshotPath         the absolute path to the snapshot.
 	 * @param snapshotFlinkVersion the Flink version of the snapshot.
-	 *
 	 * @throws Exception
 	 */
 	public static void restoreFromSnapshot(
-			AbstractStreamOperatorTestHarness<?> testHarness,
-			String snapshotPath,
-			MigrationVersion snapshotFlinkVersion) throws Exception {
+		AbstractStreamOperatorTestHarness<?> testHarness,
+		String snapshotPath,
+		MigrationVersion snapshotFlinkVersion) throws Exception {
 
-		if (snapshotFlinkVersion == MigrationVersion.v1_1) {
-			// Flink 1.1 snapshots should be read using the legacy restore method
-			testHarness.initializeStateFromLegacyCheckpoint(snapshotPath);
-		} else {
-			testHarness.initializeState(OperatorSnapshotUtil.readStateHandle(snapshotPath));
-		}
+		testHarness.initializeState(OperatorSnapshotUtil.readStateHandle(snapshotPath));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala
index 104400f..35a56d7 100644
--- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala
+++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/TimeWindowTranslationTest.scala
@@ -25,15 +25,15 @@ import org.apache.flink.api.java.tuple.Tuple
 import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.streaming.api.scala.function.WindowFunction
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.streaming.api.windowing.assigners.{SlidingAlignedProcessingTimeWindows, SlidingEventTimeWindows, TumblingAlignedProcessingTimeWindows}
+import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows
 import org.apache.flink.streaming.api.windowing.time.Time
 import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow
-import org.apache.flink.streaming.runtime.operators.windowing.{AccumulatingProcessingTimeWindowOperator, AggregatingProcessingTimeWindowOperator, WindowOperator}
+import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
 import org.apache.flink.util.Collector
 import org.junit.Assert._
-import org.junit.{Ignore, Test}
+import org.junit.Test
 
 /**
   * These tests verify that the api calls on [[WindowedStream]] that use the "time" shortcut
@@ -85,59 +85,6 @@ class TimeWindowTranslationTest extends StreamingMultipleProgramsTestBase {
     assertTrue(operator2.isInstanceOf[WindowOperator[_, _, _, _, _]])
   }
 
-  /**
-    * These tests ensure that the fast aligned time windows operator is used if the
-    * conditions are right.
-    */
-  @Test
-  def testReduceAlignedTimeWindows(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val source = env.fromElements(("hello", 1), ("hello", 2))
-    
-    val window1 = source
-      .keyBy(0)
-      .window(SlidingAlignedProcessingTimeWindows.of(Time.seconds(1), Time.milliseconds(100)))
-      .reduce(new DummyReducer())
-
-    val transform1 = window1.javaStream.getTransformation
-        .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-    
-    val operator1 = transform1.getOperator
-
-    assertTrue(operator1.isInstanceOf[AggregatingProcessingTimeWindowOperator[_, _]])
-  }
-
-  /**
-    * These tests ensure that the fast aligned time windows operator is used if the
-    * conditions are right.
-    */
-  @Test
-  def testApplyAlignedTimeWindows(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime)
-
-    val source = env.fromElements(("hello", 1), ("hello", 2))
-
-    val window1 = source
-      .keyBy(0)
-      .window(TumblingAlignedProcessingTimeWindows.of(Time.minutes(1)))
-      .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
-        def apply(
-                   key: Tuple,
-                   window: TimeWindow,
-                   values: Iterable[(String, Int)],
-                   out: Collector[(String, Int)]) { }
-      })
-
-    val transform1 = window1.javaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator1 = transform1.getOperator
-
-    assertTrue(operator1.isInstanceOf[AccumulatingProcessingTimeWindowOperator[_, _, _]])
-  }
-
   @Test
   def testReduceEventTimeWindows(): Unit = {
     val env = StreamExecutionEnvironment.getExecutionEnvironment

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
index cad6693..99fb6ef 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
@@ -44,7 +44,6 @@ import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -975,7 +974,7 @@ public class RescalingITCase extends TestLogger {
 		}
 	}
 
-	private static class PartitionedStateSource extends StateSourceBase implements CheckpointedFunction, CheckpointedRestoring<Integer> {
+	private static class PartitionedStateSource extends StateSourceBase implements CheckpointedFunction {
 
 		private static final long serialVersionUID = -359715965103593462L;
 		private static final int NUM_PARTITIONS = 7;
@@ -1030,10 +1029,5 @@ public class RescalingITCase extends TestLogger {
 				checkCorrectRestore[getRuntimeContext().getIndexOfThisSubtask()] = counter;
 			}
 		}
-
-		@Override
-		public void restoreState(Integer state) throws Exception {
-			counterPartitions.add(state);
-		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
index cc23545..1b7dafa 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
@@ -53,7 +53,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob;
 import org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess;
-import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
@@ -351,10 +351,6 @@ public class SavepointITCase extends TestLogger {
 					OperatorSubtaskState subtaskState = operatorState.getState(tdd.getSubtaskIndex());
 
 					assertNotNull(subtaskState);
-
-					errMsg = "Initial operator state mismatch.";
-					assertEquals(errMsg, subtaskState.getLegacyOperatorState(),
-						tdd.getTaskStateHandles().getSubtaskStateByOperatorID(operatorState.getOperatorID()).getLegacyOperatorState());
 				}
 			}
 
@@ -377,17 +373,18 @@ public class SavepointITCase extends TestLogger {
 			assertTrue(errMsg, resp.getClass() == getDisposeSavepointSuccess().getClass());
 
 			// - Verification START -------------------------------------------
-
 			// The checkpoint files
 			List<File> checkpointFiles = new ArrayList<>();
 
 			for (OperatorState stateForTaskGroup : savepoint.getOperatorStates()) {
 				for (OperatorSubtaskState subtaskState : stateForTaskGroup.getStates()) {
-					StreamStateHandle streamTaskState = subtaskState.getLegacyOperatorState();
+					Collection<OperatorStateHandle> streamTaskState = subtaskState.getManagedOperatorState();
 
-					if (streamTaskState != null) {
-						FileStateHandle fileStateHandle = (FileStateHandle) streamTaskState;
-						checkpointFiles.add(new File(fileStateHandle.getFilePath().toUri()));
+					if (streamTaskState != null && !streamTaskState.isEmpty()) {
+						for (OperatorStateHandle osh : streamTaskState) {
+							FileStateHandle fileStateHandle = (FileStateHandle) osh.getDelegateStateHandle();
+							checkpointFiles.add(new File(fileStateHandle.getFilePath().toUri()));
+						}
 					}
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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 21be7ba..eccc7e9 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,6 +25,7 @@ import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointSerializers;
 import org.apache.flink.runtime.client.JobListeningContext;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
@@ -39,6 +40,7 @@ import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.commons.io.FileUtils;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
@@ -64,6 +66,11 @@ import static junit.framework.Assert.fail;
  */
 public class SavepointMigrationTestBase extends TestBaseUtils {
 
+	@BeforeClass
+	public static void before() {
+		SavepointSerializers.setFailWhenLegacyStateDetected(false);
+	}
+
 	@Rule
 	public TemporaryFolder tempFolder = new TemporaryFolder();
 


[09/11] flink git commit: [FLINK-7461] Remove Backwards compatibility with <= Flink 1.1

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0Serializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0Serializer.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0Serializer.java
deleted file mode 100644
index d285906..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0Serializer.java
+++ /dev/null
@@ -1,425 +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.migration.runtime.checkpoint.savepoint;
-
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.migration.runtime.checkpoint.KeyGroupState;
-import org.apache.flink.migration.runtime.checkpoint.SubtaskState;
-import org.apache.flink.migration.runtime.checkpoint.TaskState;
-import org.apache.flink.migration.runtime.state.AbstractStateBackend;
-import org.apache.flink.migration.runtime.state.KvStateSnapshot;
-import org.apache.flink.migration.runtime.state.StateHandle;
-import org.apache.flink.migration.runtime.state.filesystem.AbstractFileStateHandle;
-import org.apache.flink.migration.runtime.state.memory.SerializedStateHandle;
-import org.apache.flink.migration.state.MigrationKeyGroupStateHandle;
-import org.apache.flink.migration.state.MigrationStreamStateHandle;
-import org.apache.flink.migration.streaming.runtime.tasks.StreamTaskState;
-import org.apache.flink.migration.streaming.runtime.tasks.StreamTaskStateList;
-import org.apache.flink.migration.util.SerializedValue;
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointSerializer;
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointV2;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.state.ChainedStateHandle;
-import org.apache.flink.runtime.state.CheckpointStreamFactory;
-import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
-import org.apache.flink.runtime.state.MultiStreamStateHandle;
-import org.apache.flink.runtime.state.OperatorStateHandle;
-import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.runtime.state.filesystem.FileStateHandle;
-import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
-import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory;
-import org.apache.flink.util.IOUtils;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.Preconditions;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * <p>
- * <p>In contrast to previous savepoint versions, this serializer makes sure
- * that no default Java serialization is used for serialization. Therefore, we
- * don't rely on any involved Java classes to stay the same.
- */
-@SuppressWarnings("deprecation")
-public class SavepointV0Serializer implements SavepointSerializer<SavepointV2> {
-
-	public static final SavepointV0Serializer INSTANCE = new SavepointV0Serializer();
-	private static final StreamStateHandle SIGNAL_0 = new ByteStreamStateHandle("SIGNAL_0", new byte[]{0});
-	private static final StreamStateHandle SIGNAL_1 = new ByteStreamStateHandle("SIGNAL_1", new byte[]{1});
-
-	private static final int MAX_SIZE = 4 * 1024 * 1024;
-
-	private SavepointV0Serializer() {
-	}
-
-
-	@Override
-	public void serialize(SavepointV2 savepoint, DataOutputStream dos) throws IOException {
-		throw new UnsupportedOperationException("This serializer is read-only and only exists for backwards compatibility");
-	}
-
-	@Override
-	public SavepointV2 deserialize(DataInputStream dis, ClassLoader userClassLoader) throws IOException {
-
-		long checkpointId = dis.readLong();
-
-		// Task states
-		int numTaskStates = dis.readInt();
-		List<TaskState> taskStates = new ArrayList<>(numTaskStates);
-
-		for (int i = 0; i < numTaskStates; i++) {
-			JobVertexID jobVertexId = new JobVertexID(dis.readLong(), dis.readLong());
-			int parallelism = dis.readInt();
-
-			// Add task state
-			TaskState taskState = new TaskState(jobVertexId, parallelism);
-			taskStates.add(taskState);
-
-			// Sub task states
-			int numSubTaskStates = dis.readInt();
-			for (int j = 0; j < numSubTaskStates; j++) {
-				int subtaskIndex = dis.readInt();
-
-				SerializedValue<StateHandle<?>> serializedValue = readSerializedValueStateHandle(dis);
-
-				long stateSize = dis.readLong();
-				long duration = dis.readLong();
-
-				SubtaskState subtaskState = new SubtaskState(
-						serializedValue,
-						stateSize,
-						duration);
-
-				taskState.putState(subtaskIndex, subtaskState);
-			}
-
-			// Key group states
-			int numKvStates = dis.readInt();
-			for (int j = 0; j < numKvStates; j++) {
-				int keyGroupIndex = dis.readInt();
-
-				SerializedValue<StateHandle<?>> serializedValue = readSerializedValueStateHandle(dis);
-
-				long stateSize = dis.readLong();
-				long duration = dis.readLong();
-
-				KeyGroupState keyGroupState = new KeyGroupState(
-						serializedValue,
-						stateSize,
-						duration);
-
-				taskState.putKvState(keyGroupIndex, keyGroupState);
-			}
-		}
-
-		try {
-
-			return convertSavepoint(taskStates, userClassLoader, checkpointId);
-		} catch (Exception e) {
-
-			throw new IOException(e);
-		}
-	}
-
-	private static SerializedValue<StateHandle<?>> readSerializedValueStateHandle(DataInputStream dis)
-			throws IOException {
-
-		int length = dis.readInt();
-
-		SerializedValue<StateHandle<?>> serializedValue;
-		if (length == -1) {
-			serializedValue = new SerializedValue<>(null);
-		} else {
-			byte[] serializedData = new byte[length];
-			dis.readFully(serializedData, 0, length);
-			serializedValue = SerializedValue.fromBytes(serializedData);
-		}
-
-		return serializedValue;
-	}
-
-	private SavepointV2 convertSavepoint(
-			List<TaskState> taskStates,
-			ClassLoader userClassLoader,
-			long checkpointID) throws Exception {
-
-		List<org.apache.flink.runtime.checkpoint.TaskState> newTaskStates = new ArrayList<>(taskStates.size());
-
-		for (TaskState taskState : taskStates) {
-			newTaskStates.add(convertTaskState(taskState, userClassLoader, checkpointID));
-		}
-
-		return new SavepointV2(checkpointID, newTaskStates);
-	}
-
-	private org.apache.flink.runtime.checkpoint.TaskState convertTaskState(
-			TaskState taskState,
-			ClassLoader userClassLoader,
-			long checkpointID) throws Exception {
-
-		JobVertexID jobVertexID = taskState.getJobVertexID();
-		int parallelism = taskState.getParallelism();
-		int chainLength = determineOperatorChainLength(taskState, userClassLoader);
-
-		org.apache.flink.runtime.checkpoint.TaskState newTaskState =
-				new org.apache.flink.runtime.checkpoint.TaskState(
-						jobVertexID,
-						parallelism,
-						parallelism,
-						chainLength);
-
-		if (chainLength > 0) {
-
-			Map<Integer, SubtaskState> subtaskStates = taskState.getSubtaskStatesById();
-
-			for (Map.Entry<Integer, SubtaskState> subtaskState : subtaskStates.entrySet()) {
-				int parallelInstanceIdx = subtaskState.getKey();
-				newTaskState.putState(parallelInstanceIdx, convertSubtaskState(
-						subtaskState.getValue(),
-						parallelInstanceIdx,
-						userClassLoader,
-						checkpointID));
-			}
-		}
-
-		return newTaskState;
-	}
-
-	private org.apache.flink.runtime.checkpoint.SubtaskState convertSubtaskState(
-			SubtaskState subtaskState,
-			int parallelInstanceIdx,
-			ClassLoader userClassLoader,
-			long checkpointID) throws Exception {
-
-		SerializedValue<StateHandle<?>> serializedValue = subtaskState.getState();
-
-		StreamTaskStateList stateList = (StreamTaskStateList) serializedValue.deserializeValue(userClassLoader);
-		StreamTaskState[] streamTaskStates = stateList.getState(userClassLoader);
-
-		List<StreamStateHandle> newChainStateList = Arrays.asList(new StreamStateHandle[streamTaskStates.length]);
-		KeyGroupsStateHandle newKeyedState = null;
-
-		for (int chainIdx = 0; chainIdx < streamTaskStates.length; ++chainIdx) {
-
-			StreamTaskState streamTaskState = streamTaskStates[chainIdx];
-			if (streamTaskState == null) {
-				continue;
-			}
-
-			newChainStateList.set(chainIdx, convertOperatorAndFunctionState(streamTaskState));
-			HashMap<String, KvStateSnapshot<?, ?, ?, ?>> oldKeyedState = streamTaskState.getKvStates();
-
-			if (null != oldKeyedState) {
-				Preconditions.checkState(null == newKeyedState, "Found more than one keyed state in chain");
-				newKeyedState = convertKeyedBackendState(oldKeyedState, parallelInstanceIdx, checkpointID);
-			}
-		}
-
-		ChainedStateHandle<StreamStateHandle> newChainedState = new ChainedStateHandle<>(newChainStateList);
-		ChainedStateHandle<OperatorStateHandle> nopChain =
-				new ChainedStateHandle<>(Arrays.asList(new OperatorStateHandle[newChainedState.getLength()]));
-
-		return new org.apache.flink.runtime.checkpoint.SubtaskState(
-				newChainedState,
-				nopChain,
-				nopChain,
-				newKeyedState,
-				null);
-	}
-
-	/**
-	 * This is public so that we can use it when restoring a legacy snapshot
-	 * in {@code AbstractStreamOperatorTestHarness}.
-	 */
-	public static StreamStateHandle convertOperatorAndFunctionState(StreamTaskState streamTaskState) throws Exception {
-
-		List<StreamStateHandle> mergeStateHandles = new ArrayList<>(4);
-
-		StateHandle<Serializable> functionState = streamTaskState.getFunctionState();
-		StateHandle<?> operatorState = streamTaskState.getOperatorState();
-
-		if (null != functionState) {
-			mergeStateHandles.add(SIGNAL_1);
-			mergeStateHandles.add(convertStateHandle(functionState));
-		} else {
-			mergeStateHandles.add(SIGNAL_0);
-		}
-
-		if (null != operatorState) {
-			mergeStateHandles.add(convertStateHandle(operatorState));
-		}
-
-		return new MigrationStreamStateHandle(new MultiStreamStateHandle(mergeStateHandles));
-	}
-
-	/**
-	 * This is public so that we can use it when restoring a legacy snapshot
-	 * in {@code AbstractStreamOperatorTestHarness}.
-	 */
-	public static KeyGroupsStateHandle convertKeyedBackendState(
-			HashMap<String, KvStateSnapshot<?, ?, ?, ?>> oldKeyedState,
-			int parallelInstanceIdx,
-			long checkpointID) throws Exception {
-
-		if (null != oldKeyedState) {
-
-			CheckpointStreamFactory checkpointStreamFactory = new MemCheckpointStreamFactory(MAX_SIZE);
-
-			CheckpointStreamFactory.CheckpointStateOutputStream keyedStateOut =
-					checkpointStreamFactory.createCheckpointStateOutputStream(checkpointID, 0L);
-
-			try {
-				final long offset = keyedStateOut.getPos();
-
-				InstantiationUtil.serializeObject(keyedStateOut, oldKeyedState);
-				StreamStateHandle streamStateHandle = keyedStateOut.closeAndGetHandle();
-				keyedStateOut = null; // makes IOUtils.closeQuietly(...) ignore this
-
-				if (null != streamStateHandle) {
-					KeyGroupRangeOffsets keyGroupRangeOffsets =
-							new KeyGroupRangeOffsets(parallelInstanceIdx, parallelInstanceIdx, new long[]{offset});
-
-					return new MigrationKeyGroupStateHandle(keyGroupRangeOffsets, streamStateHandle);
-				}
-			} finally {
-				IOUtils.closeQuietly(keyedStateOut);
-			}
-		}
-		return null;
-	}
-
-	private int determineOperatorChainLength(
-			TaskState taskState,
-			ClassLoader userClassLoader) throws IOException, ClassNotFoundException {
-
-		Collection<SubtaskState> subtaskStates = taskState.getStates();
-
-		if (subtaskStates == null || subtaskStates.isEmpty()) {
-			return 0;
-		}
-
-		SubtaskState firstSubtaskState = subtaskStates.iterator().next();
-		Object toCastTaskStateList = firstSubtaskState.getState().deserializeValue(userClassLoader);
-
-		if (toCastTaskStateList instanceof StreamTaskStateList) {
-			StreamTaskStateList taskStateList = (StreamTaskStateList) toCastTaskStateList;
-			StreamTaskState[] streamTaskStates = taskStateList.getState(userClassLoader);
-
-			return streamTaskStates.length;
-		}
-		return 0;
-	}
-
-	/**
-	 * This is public so that we can use it when restoring a legacy snapshot
-	 * in {@code AbstractStreamOperatorTestHarness}.
-	 */
-	public static StreamStateHandle convertStateHandle(StateHandle<?> oldStateHandle) throws Exception {
-		if (oldStateHandle instanceof AbstractFileStateHandle) {
-			Path path = ((AbstractFileStateHandle) oldStateHandle).getFilePath();
-			return new FileStateHandle(path, oldStateHandle.getStateSize());
-		} else if (oldStateHandle instanceof SerializedStateHandle) {
-			byte[] data = ((SerializedStateHandle<?>) oldStateHandle).getSerializedData();
-			return new ByteStreamStateHandle(String.valueOf(System.identityHashCode(data)), data);
-		} else if (oldStateHandle instanceof org.apache.flink.migration.runtime.state.memory.ByteStreamStateHandle) {
-			byte[] data =
-					((org.apache.flink.migration.runtime.state.memory.ByteStreamStateHandle) oldStateHandle).getData();
-			return new ByteStreamStateHandle(String.valueOf(System.identityHashCode(data)), data);
-		} else if (oldStateHandle instanceof AbstractStateBackend.DataInputViewHandle) {
-			return convertStateHandle(
-					((AbstractStateBackend.DataInputViewHandle) oldStateHandle).getStreamStateHandle());
-		}
-		throw new IllegalArgumentException("Unknown state handle type: " + oldStateHandle);
-	}
-
-	@VisibleForTesting
-	public void serializeOld(SavepointV0 savepoint, DataOutputStream dos) throws IOException {
-		dos.writeLong(savepoint.getCheckpointId());
-
-		Collection<org.apache.flink.migration.runtime.checkpoint.TaskState> taskStates = savepoint.getOldTaskStates();
-		dos.writeInt(taskStates.size());
-
-		for (org.apache.flink.migration.runtime.checkpoint.TaskState taskState : savepoint.getOldTaskStates()) {
-			// Vertex ID
-			dos.writeLong(taskState.getJobVertexID().getLowerPart());
-			dos.writeLong(taskState.getJobVertexID().getUpperPart());
-
-			// Parallelism
-			int parallelism = taskState.getParallelism();
-			dos.writeInt(parallelism);
-
-			// Sub task states
-			dos.writeInt(taskState.getNumberCollectedStates());
-
-			for (int i = 0; i < parallelism; i++) {
-				SubtaskState subtaskState = taskState.getState(i);
-
-				if (subtaskState != null) {
-					dos.writeInt(i);
-
-					SerializedValue<?> serializedValue = subtaskState.getState();
-					if (serializedValue == null) {
-						dos.writeInt(-1); // null
-					} else {
-						byte[] serialized = serializedValue.getByteArray();
-						dos.writeInt(serialized.length);
-						dos.write(serialized, 0, serialized.length);
-					}
-
-					dos.writeLong(subtaskState.getStateSize());
-					dos.writeLong(subtaskState.getDuration());
-				}
-			}
-
-			// Key group states
-			dos.writeInt(taskState.getNumberCollectedKvStates());
-
-			for (int i = 0; i < parallelism; i++) {
-				KeyGroupState keyGroupState = taskState.getKvState(i);
-
-				if (keyGroupState != null) {
-					dos.write(i);
-
-					SerializedValue<?> serializedValue = keyGroupState.getKeyGroupState();
-					if (serializedValue == null) {
-						dos.writeInt(-1); // null
-					} else {
-						byte[] serialized = serializedValue.getByteArray();
-						dos.writeInt(serialized.length);
-						dos.write(serialized, 0, serialized.length);
-					}
-
-					dos.writeLong(keyGroupState.getStateSize());
-					dos.writeLong(keyGroupState.getDuration());
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractCloseableHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractCloseableHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractCloseableHandle.java
deleted file mode 100644
index 775b304..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractCloseableHandle.java
+++ /dev/null
@@ -1,131 +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.migration.runtime.state;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
-
-/**
- * A simple base for closable handles.
- * 
- * Offers to register a stream (or other closable object) that close calls are delegated to if
- * the handle is closed or was already closed.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public abstract class AbstractCloseableHandle implements Closeable, Serializable {
-
-	/** Serial Version UID must be constant to maintain format compatibility */
-	private static final long serialVersionUID = 1L;
-
-	/** To atomically update the "closable" field without needing to add a member class like "AtomicBoolean */
-	private static final AtomicIntegerFieldUpdater<AbstractCloseableHandle> CLOSER = 
-			AtomicIntegerFieldUpdater.newUpdater(AbstractCloseableHandle.class, "isClosed");
-
-	// ------------------------------------------------------------------------
-
-	/** The closeable to close if this handle is closed late */ 
-	private transient volatile Closeable toClose;
-
-	/** Flag to remember if this handle was already closed */
-	@SuppressWarnings("unused") // this field is actually updated, but via the "CLOSER" updater
-	private transient volatile int isClosed;
-
-	// ------------------------------------------------------------------------
-
-	protected final void registerCloseable(Closeable toClose) throws IOException {
-		if (toClose == null) {
-			return;
-		}
-		
-		// NOTE: The order of operations matters here:
-		// (1) first setting the closeable
-		// (2) checking the flag.
-		// Because the order in the {@link #close()} method is the opposite, and
-		// both variables are volatile (reordering barriers), we can be sure that
-		// one of the methods always notices the effect of a concurrent call to the
-		// other method.
-
-		this.toClose = toClose;
-
-		// check if we were closed early
-		if (this.isClosed != 0) {
-			toClose.close();
-			throw new IOException("handle is closed");
-		}
-	}
-
-	/**
-	 * Closes the handle.
-	 * 
-	 * <p>If a "Closeable" has been registered via {@link #registerCloseable(Closeable)},
-	 * then this will be closes.
-	 * 
-	 * <p>If any "Closeable" will be registered via {@link #registerCloseable(Closeable)} in the future,
-	 * it will immediately be closed and that method will throw an exception.
-	 * 
-	 * @throws IOException Exceptions occurring while closing an already registered {@code Closeable}
-	 *                     are forwarded.
-	 * 
-	 * @see #registerCloseable(Closeable)
-	 */
-	@Override
-	public final void close() throws IOException {
-		// NOTE: The order of operations matters here:
-		// (1) first setting the closed flag
-		// (2) checking whether there is already a closeable
-		// Because the order in the {@link #registerCloseable(Closeable)} method is the opposite, and
-		// both variables are volatile (reordering barriers), we can be sure that
-		// one of the methods always notices the effect of a concurrent call to the
-		// other method.
-
-		if (CLOSER.compareAndSet(this, 0, 1)) {
-			final Closeable toClose = this.toClose;
-			if (toClose != null) {
-				this.toClose = null;
-				toClose.close();
-			}
-		}
-	}
-
-	/**
-	 * Checks whether this handle has been closed.
-	 * 
-	 * @return True is the handle is closed, false otherwise.
-	 */
-	public boolean isClosed() {
-		return isClosed != 0;
-	}
-
-	/**
-	 * This method checks whether the handle is closed and throws an exception if it is closed.
-	 * If the handle is not closed, this method does nothing.
-	 * 
-	 * @throws IOException Thrown, if the handle has been closed.
-	 */
-	public void ensureNotClosed() throws IOException {
-		if (isClosed != 0) {
-			throw new IOException("handle is closed");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractStateBackend.java
deleted file mode 100644
index 7c53c40..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/AbstractStateBackend.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * A state backend defines how state is stored and snapshotted during checkpoints.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public abstract class AbstractStateBackend implements Serializable {
-	
-	private static final long serialVersionUID = 4620413814639220247L;
-
-	/**
-	 * Simple state handle that resolved a {@link DataInputView} from a StreamStateHandle.
-	 */
-	public static final class DataInputViewHandle implements StateHandle<DataInputView> {
-
-		private static final long serialVersionUID = 2891559813513532079L;
-
-		private final StreamStateHandle stream;
-
-		private DataInputViewHandle(StreamStateHandle stream) {
-			this.stream = stream;
-		}
-
-		public StreamStateHandle getStreamStateHandle() {
-			return stream;
-		}
-
-		@Override
-		public DataInputView getState(ClassLoader userCodeClassLoader) throws Exception {
-			return new DataInputViewStreamWrapper(stream.getState(userCodeClassLoader));
-		}
-
-		@Override
-		public void discardState() throws Exception {
-			throw new UnsupportedOperationException();
-		}
-
-		@Override
-		public long getStateSize() throws Exception {
-			return stream.getStateSize();
-		}
-
-		@Override
-		public void close() throws IOException {
-			throw new UnsupportedOperationException();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/KvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/KvStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/KvStateSnapshot.java
deleted file mode 100644
index 687d415..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/KvStateSnapshot.java
+++ /dev/null
@@ -1,32 +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.migration.runtime.state;
-
-import org.apache.flink.api.common.state.State;
-import org.apache.flink.api.common.state.StateDescriptor;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public interface KvStateSnapshot<K, N, S extends State, SD extends StateDescriptor<S, ?>>
-		extends StateObject {
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateHandle.java
deleted file mode 100644
index fd3917f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateHandle.java
+++ /dev/null
@@ -1,41 +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.migration.runtime.state;
-
-/**
- * StateHandle is a general handle interface meant to abstract operator state fetching. 
- * A StateHandle implementation can for example include the state itself in cases where the state 
- * is lightweight or fetching it lazily from some external storage when the state is too large.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public interface StateHandle<T> extends StateObject {
-
-	/**
-	 * This retrieves and return the state represented by the handle.
-	 *
-	 * @param userCodeClassLoader Class loader for deserializing user code specific classes
-	 *
-	 * @return The state represented by the handle.
-	 * @throws Exception Thrown, if the state cannot be fetched.
-	 */
-	T getState(ClassLoader userCodeClassLoader) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateObject.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateObject.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateObject.java
deleted file mode 100644
index 59bc0ca..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StateObject.java
+++ /dev/null
@@ -1,58 +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.migration.runtime.state;
-
-/**
- * Base of all types that represent checkpointed state. Specializations are for
- * example {@link StateHandle StateHandles} (directly resolve to state) and 
- * {@link KvStateSnapshot key/value state snapshots}.
- * 
- * <p>State objects define how to:
- * <ul>
- *     <li><b>Discard State</b>: The {@link #discardState()} method defines how state is permanently
- *         disposed/deleted. After that method call, state may not be recoverable any more.</li>
- 
- *     <li><b>Close the current state access</b>: The {@link #close()} method defines how to
- *         stop the current access or recovery to the state. Called for example when an operation is
- *         canceled during recovery.</li>
- * </ul>
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public interface StateObject extends java.io.Closeable, java.io.Serializable {
-
-	/**
-	 * Discards the state referred to by this handle, to free up resources in
-	 * the persistent storage. This method is called when the handle will not be
-	 * used any more.
-	 */
-	void discardState() throws Exception;
-
-	/**
-	 * Returns the size of the state in bytes.
-	 *
-	 * <p>If the the size is not known, return {@code 0}.
-	 *
-	 * @return Size of the state in bytes.
-	 * @throws Exception If the operation fails during size retrieval.
-	 */
-	long getStateSize() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StreamStateHandle.java
deleted file mode 100644
index bfc57bc..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/StreamStateHandle.java
+++ /dev/null
@@ -1,40 +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.migration.runtime.state;
-
-import java.io.InputStream;
-import java.io.Serializable;
-
-/**
- * A state handle that produces an input stream when resolved.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public interface StreamStateHandle extends StateHandle<InputStream> {
-
-	/**
-	 * Converts this stream state handle into a state handle that de-serializes
-	 * the stream into an object using Java's serialization mechanism.
-	 *
-	 * @return The state handle that automatically de-serializes.
-	 */
-	<T extends Serializable> StateHandle<T> toSerializableHandle();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFileStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFileStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFileStateHandle.java
deleted file mode 100644
index a522a95..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFileStateHandle.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.filesystem;
-
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.migration.runtime.state.AbstractCloseableHandle;
-import org.apache.flink.migration.runtime.state.StateObject;
-import org.apache.flink.util.FileUtils;
-
-import java.io.IOException;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Base class for state that is stored in a file.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public abstract class AbstractFileStateHandle extends AbstractCloseableHandle implements StateObject {
-
-	private static final long serialVersionUID = 350284443258002355L;
-
-	/** The path to the file in the filesystem, fully describing the file system */
-	private final Path filePath;
-
-	/** Cached file system handle */
-	private transient FileSystem fs;
-
-	/**
-	 * Creates a new file state for the given file path.
-	 * 
-	 * @param filePath The path to the file that stores the state.
-	 */
-	protected AbstractFileStateHandle(Path filePath) {
-		this.filePath = checkNotNull(filePath);
-	}
-
-	/**
-	 * Gets the path where this handle's state is stored.
-	 * @return The path where this handle's state is stored.
-	 */
-	public Path getFilePath() {
-		return filePath;
-	}
-
-	/**
-	 * Discard the state by deleting the file that stores the state. If the parent directory
-	 * of the state is empty after deleting the state file, it is also deleted.
-	 * 
-	 * @throws Exception Thrown, if the file deletion (not the directory deletion) fails.
-	 */
-	@Override
-	public void discardState() throws Exception {
-		getFileSystem().delete(filePath, false);
-
-		try {
-			FileUtils.deletePathIfEmpty(getFileSystem(), filePath.getParent());
-		} catch (Exception ignored) {}
-	}
-
-	/**
-	 * Gets the file system that stores the file state.
-	 * @return The file system that stores the file state.
-	 * @throws IOException Thrown if the file system cannot be accessed.
-	 */
-	protected FileSystem getFileSystem() throws IOException {
-		if (fs == null) {
-			fs = FileSystem.get(filePath.toUri());
-		}
-		return fs;
-	}
-
-	/**
-	 * Returns the file size in bytes.
-	 *
-	 * @return The file size in bytes.
-	 * @throws IOException Thrown if the file system cannot be accessed.
-	 */
-	protected long getFileSize() throws IOException {
-		return getFileSystem().getFileStatus(filePath).getLen();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFsStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFsStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFsStateSnapshot.java
deleted file mode 100644
index 7099c61..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/AbstractFsStateSnapshot.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.filesystem;
-
-import org.apache.flink.api.common.state.State;
-import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
-import org.apache.flink.migration.runtime.state.KvStateSnapshot;
-import org.apache.flink.migration.runtime.state.memory.AbstractMigrationRestoreStrategy;
-import org.apache.flink.migration.runtime.state.memory.MigrationRestoreSnapshot;
-import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
-import org.apache.flink.runtime.state.heap.StateTable;
-
-import java.io.IOException;
-
-/**
- * A snapshot of a heap key/value state stored in a file.
- * 
- * @param <K> The type of the key in the snapshot state.
- * @param <N> The type of the namespace in the snapshot state.
- * @param <SV> The type of the state value.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public abstract class AbstractFsStateSnapshot<K, N, SV, S extends State, SD extends StateDescriptor<S, ?>> 
-		extends AbstractFileStateHandle implements KvStateSnapshot<K, N, S, SD>, MigrationRestoreSnapshot<K, N, SV> {
-
-	private static final long serialVersionUID = 1L;
-
-	/** Key Serializer */
-	protected final TypeSerializer<K> keySerializer;
-
-	/** Namespace Serializer */
-	protected final TypeSerializer<N> namespaceSerializer;
-
-	/** Serializer for the state value */
-	protected final TypeSerializer<SV> stateSerializer;
-
-	/** StateDescriptor, for sanity checks */
-	protected final SD stateDesc;
-
-	public AbstractFsStateSnapshot(TypeSerializer<K> keySerializer,
-		TypeSerializer<N> namespaceSerializer,
-		TypeSerializer<SV> stateSerializer,
-		SD stateDesc,
-		Path filePath) {
-		super(filePath);
-		this.stateDesc = stateDesc;
-		this.keySerializer = keySerializer;
-		this.stateSerializer = stateSerializer;
-		this.namespaceSerializer = namespaceSerializer;
-
-	}
-
-	@Override
-	public long getStateSize() throws IOException {
-		return getFileSize();
-	}
-
-	public TypeSerializer<K> getKeySerializer() {
-		return keySerializer;
-	}
-
-	public TypeSerializer<N> getNamespaceSerializer() {
-		return namespaceSerializer;
-	}
-
-	public TypeSerializer<SV> getStateSerializer() {
-		return stateSerializer;
-	}
-
-	public SD getStateDesc() {
-		return stateDesc;
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public StateTable<K, N, SV> deserialize(
-			String stateName,
-			HeapKeyedStateBackend<K> stateBackend) throws IOException {
-
-		final FileSystem fs = getFilePath().getFileSystem();
-		try (FSDataInputStream inStream = fs.open(getFilePath())) {
-			final DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(inStream);
-			AbstractMigrationRestoreStrategy<K, N, SV> restoreStrategy =
-					new AbstractMigrationRestoreStrategy<K, N, SV>(keySerializer, namespaceSerializer, stateSerializer) {
-						@Override
-						protected DataInputView openDataInputView() throws IOException {
-							return inView;
-						}
-					};
-			return restoreStrategy.deserialize(stateName, stateBackend);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileSerializableStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileSerializableStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileSerializableStateHandle.java
deleted file mode 100644
index b4a3a73..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileSerializableStateHandle.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.filesystem;
-
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.migration.runtime.state.StateHandle;
-import org.apache.flink.migration.util.MigrationInstantiationUtil;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.Serializable;
-
-/**
- * A state handle that points to state stored in a file via Java Serialization.
- * 
- * @param <T> The type of state pointed to by the state handle.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class FileSerializableStateHandle<T extends Serializable> extends AbstractFileStateHandle implements StateHandle<T> {
-
-	private static final long serialVersionUID = -657631394290213622L;
-
-	/**
-	 * Creates a new FileSerializableStateHandle pointing to state at the given file path.
-	 * 
-	 * @param filePath The path to the file containing the checkpointed state.
-	 */
-	public FileSerializableStateHandle(Path filePath) {
-		super(filePath);
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public T getState(ClassLoader classLoader) throws Exception {
-		ensureNotClosed();
-
-		try (FSDataInputStream inStream = getFileSystem().open(getFilePath())) {
-			// make sure any deserialization can be aborted
-			registerCloseable(inStream);
-
-			ObjectInputStream ois = new MigrationInstantiationUtil.ClassLoaderObjectInputStream(inStream, classLoader);
-			return (T) ois.readObject();
-		}
-	}
-
-	/**
-	 * Returns the file size in bytes.
-	 *
-	 * @return The file size in bytes.
-	 * @throws IOException Thrown if the file system cannot be accessed.
-	 */
-	@Override
-	public long getStateSize() throws IOException {
-		return getFileSize();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileStreamStateHandle.java
deleted file mode 100644
index 7444be1..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FileStreamStateHandle.java
+++ /dev/null
@@ -1,87 +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.migration.runtime.state.filesystem;
-
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.migration.runtime.state.StateHandle;
-import org.apache.flink.migration.runtime.state.StreamStateHandle;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Serializable;
-
-/**
- * A state handle that points to state in a file system, accessible as an input stream.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class FileStreamStateHandle extends AbstractFileStateHandle implements StreamStateHandle {
-
-	private static final long serialVersionUID = -6826990484549987311L;
-
-	/**
-	 * Creates a new FileStreamStateHandle pointing to state at the given file path.
-	 * 
-	 * @param filePath The path to the file containing the checkpointed state.
-	 */
-	public FileStreamStateHandle(Path filePath) {
-		super(filePath);
-	}
-
-	@Override
-	public InputStream getState(ClassLoader userCodeClassLoader) throws Exception {
-		ensureNotClosed();
-
-		InputStream inStream = getFileSystem().open(getFilePath());
-		// make sure the state handle is cancelable
-		registerCloseable(inStream);
-
-		return inStream; 
-	}
-
-	/**
-	 * Returns the file size in bytes.
-	 *
-	 * @return The file size in bytes.
-	 * @throws IOException Thrown if the file system cannot be accessed.
-	 */
-	@Override
-	public long getStateSize() throws IOException {
-		return getFileSize();
-	}
-
-	@Override
-	public <T extends Serializable> StateHandle<T> toSerializableHandle() {
-		FileSerializableStateHandle<T> handle = new FileSerializableStateHandle<>(getFilePath());
-
-		// forward closed status
-		if (isClosed()) {
-			try {
-				handle.close();
-			} catch (IOException e) {
-				// should not happen on a fresh handle, but forward anyways
-				throw new RuntimeException(e);
-			}
-		}
-
-		return handle;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsFoldingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsFoldingState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsFoldingState.java
deleted file mode 100644
index ec89ab8..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsFoldingState.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.filesystem;
-
-import org.apache.flink.api.common.state.FoldingState;
-import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.fs.Path;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class FsFoldingState<K, N, T, ACC> {
-	public static class Snapshot<K, N, T, ACC> extends AbstractFsStateSnapshot<K, N, ACC, FoldingState<T, ACC>, FoldingStateDescriptor<T, ACC>> {
-		private static final long serialVersionUID = 1L;
-
-		public Snapshot(TypeSerializer<K> keySerializer,
-			TypeSerializer<N> namespaceSerializer,
-			TypeSerializer<ACC> stateSerializer,
-			FoldingStateDescriptor<T, ACC> stateDescs,
-			Path filePath) {
-			super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, filePath);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsListState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsListState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsListState.java
deleted file mode 100644
index 71404ab..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsListState.java
+++ /dev/null
@@ -1,46 +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.migration.runtime.state.filesystem;
-
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.fs.Path;
-
-import java.util.ArrayList;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class FsListState<K, N, V> {
-
-	public static class Snapshot<K, N, V> extends AbstractFsStateSnapshot<K, N, ArrayList<V>, ListState<V>, ListStateDescriptor<V>> {
-		private static final long serialVersionUID = 1L;
-
-		public Snapshot(TypeSerializer<K> keySerializer,
-			TypeSerializer<N> namespaceSerializer,
-			TypeSerializer<ArrayList<V>> stateSerializer,
-			ListStateDescriptor<V> stateDescs,
-			Path filePath) {
-			super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, filePath);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsReducingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsReducingState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsReducingState.java
deleted file mode 100644
index 153f88c..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsReducingState.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.filesystem;
-
-import org.apache.flink.api.common.state.ReducingState;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.fs.Path;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class FsReducingState<K, N, V> {
-
-	public static class Snapshot<K, N, V> extends AbstractFsStateSnapshot<K, N, V, ReducingState<V>, ReducingStateDescriptor<V>> {
-		private static final long serialVersionUID = 1L;
-
-		public Snapshot(TypeSerializer<K> keySerializer,
-			TypeSerializer<N> namespaceSerializer,
-			TypeSerializer<V> stateSerializer,
-			ReducingStateDescriptor<V> stateDescs,
-			Path filePath) {
-			super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, filePath);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsStateBackend.java
deleted file mode 100644
index d177510..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsStateBackend.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.filesystem;
-
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.migration.runtime.state.AbstractStateBackend;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class FsStateBackend extends AbstractStateBackend {
-
-	private static final long serialVersionUID = -8191916350224044011L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(FsStateBackend.class);
-
-	/** By default, state smaller than 1024 bytes will not be written to files, but
-	 * will be stored directly with the metadata */
-	public static final int DEFAULT_FILE_STATE_THRESHOLD = 1024;
-
-	/** Maximum size of state that is stored with the metadata, rather than in files */
-	public static final int MAX_FILE_STATE_THRESHOLD = 1024 * 1024;
-	
-	/** Default size for the write buffer */
-	private static final int DEFAULT_WRITE_BUFFER_SIZE = 4096;
-	
-
-	/** The path to the directory for the checkpoint data, including the file system
-	 * description via scheme and optional authority */
-	private final Path basePath = null;
-
-	/** State below this size will be stored as part of the metadata, rather than in files */
-	private final int fileStateThreshold = 0;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsValueState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsValueState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsValueState.java
deleted file mode 100644
index d2ae48d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/filesystem/FsValueState.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.filesystem;
-
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.fs.Path;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class FsValueState<K, N, V> {
-
-	public static class Snapshot<K, N, V> extends AbstractFsStateSnapshot<K, N, V, ValueState<V>, ValueStateDescriptor<V>> {
-		private static final long serialVersionUID = 1L;
-
-		public Snapshot(TypeSerializer<K> keySerializer,
-			TypeSerializer<N> namespaceSerializer,
-			TypeSerializer<V> stateSerializer,
-			ValueStateDescriptor<V> stateDescs,
-			Path filePath) {
-			super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, filePath);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMemStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMemStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMemStateSnapshot.java
deleted file mode 100644
index aadfe4e..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMemStateSnapshot.java
+++ /dev/null
@@ -1,138 +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.migration.runtime.state.memory;
-
-import org.apache.flink.api.common.state.State;
-import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.migration.runtime.state.KvStateSnapshot;
-import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
-import org.apache.flink.runtime.state.heap.StateTable;
-import org.apache.flink.runtime.util.DataInputDeserializer;
-
-import java.io.IOException;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public abstract class AbstractMemStateSnapshot<K, N, SV, S extends State, SD extends StateDescriptor<S, ?>> 
-		implements KvStateSnapshot<K, N, S, SD>, MigrationRestoreSnapshot<K, N, SV> {
-
-	private static final long serialVersionUID = 1L;
-
-	/** Key Serializer */
-	protected final TypeSerializer<K> keySerializer;
-
-	/** Namespace Serializer */
-	protected final TypeSerializer<N> namespaceSerializer;
-
-	/** Serializer for the state value */
-	protected final TypeSerializer<SV> stateSerializer;
-
-	/** StateDescriptor, for sanity checks */
-	protected final SD stateDesc;
-
-	/** The serialized data of the state key/value pairs */
-	private final byte[] data;
-	
-	private transient boolean closed;
-
-	/**
-	 * Creates a new heap memory state snapshot.
-	 *
-	 * @param keySerializer The serializer for the keys.
-	 * @param namespaceSerializer The serializer for the namespace.
-	 * @param stateSerializer The serializer for the elements in the state HashMap
-	 * @param stateDesc The state identifier
-	 * @param data The serialized data of the state key/value pairs
-	 */
-	public AbstractMemStateSnapshot(TypeSerializer<K> keySerializer,
-		TypeSerializer<N> namespaceSerializer,
-		TypeSerializer<SV> stateSerializer,
-		SD stateDesc,
-		byte[] data) {
-		this.keySerializer = keySerializer;
-		this.namespaceSerializer = namespaceSerializer;
-		this.stateSerializer = stateSerializer;
-		this.stateDesc = stateDesc;
-		this.data = data;
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public StateTable<K, N, SV> deserialize(
-			String stateName,
-			HeapKeyedStateBackend<K> stateBackend) throws IOException {
-
-		final DataInputDeserializer inView = new DataInputDeserializer(data, 0, data.length);
-		AbstractMigrationRestoreStrategy<K, N, SV> restoreStrategy =
-				new AbstractMigrationRestoreStrategy<K, N, SV>(keySerializer, namespaceSerializer, stateSerializer) {
-					@Override
-					protected DataInputView openDataInputView() throws IOException {
-						return inView;
-					}
-				};
-		return restoreStrategy.deserialize(stateName, stateBackend);
-	}
-
-	/**
-	 * Discarding the heap state is a no-op.
-	 */
-	@Override
-	public void discardState() {}
-
-	@Override
-	public long getStateSize() {
-		return data.length;
-	}
-
-	@Override
-	public void close() {
-		closed = true;
-	}
-
-	public TypeSerializer<K> getKeySerializer() {
-		return keySerializer;
-	}
-
-	public TypeSerializer<N> getNamespaceSerializer() {
-		return namespaceSerializer;
-	}
-
-	public TypeSerializer<SV> getStateSerializer() {
-		return stateSerializer;
-	}
-
-	public byte[] getData() {
-		return data;
-	}
-
-	@Override
-	public String toString() {
-		return "AbstractMemStateSnapshot{" +
-				"keySerializer=" + keySerializer +
-				", namespaceSerializer=" + namespaceSerializer +
-				", stateSerializer=" + stateSerializer +
-				", stateDesc=" + stateDesc +
-				'}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMigrationRestoreStrategy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMigrationRestoreStrategy.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMigrationRestoreStrategy.java
deleted file mode 100644
index f58070e..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMigrationRestoreStrategy.java
+++ /dev/null
@@ -1,119 +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.migration.runtime.state.memory;
-
-import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.VoidSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo;
-import org.apache.flink.runtime.state.VoidNamespace;
-import org.apache.flink.runtime.state.VoidNamespaceSerializer;
-import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
-import org.apache.flink.runtime.state.heap.StateTable;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-
-/**
- * This class outlines the general strategy to restore from migration states.
- *
- * @param <K> type of key.
- * @param <N> type of namespace.
- * @param <S> type of state.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-public abstract class AbstractMigrationRestoreStrategy<K, N, S> implements MigrationRestoreSnapshot<K, N, S> {
-
-	/**
-	 * Key Serializer
-	 */
-	protected final TypeSerializer<K> keySerializer;
-
-	/**
-	 * Namespace Serializer
-	 */
-	protected final TypeSerializer<N> namespaceSerializer;
-
-	/**
-	 * Serializer for the state value
-	 */
-	protected final TypeSerializer<S> stateSerializer;
-
-	public AbstractMigrationRestoreStrategy(
-			TypeSerializer<K> keySerializer,
-			TypeSerializer<N> namespaceSerializer,
-			TypeSerializer<S> stateSerializer) {
-
-		this.keySerializer = Preconditions.checkNotNull(keySerializer);
-		this.namespaceSerializer = Preconditions.checkNotNull(namespaceSerializer);
-		this.stateSerializer = Preconditions.checkNotNull(stateSerializer);
-	}
-
-	@Override
-	public StateTable<K, N, S> deserialize(String stateName, HeapKeyedStateBackend<K> stateBackend) throws IOException {
-
-		Preconditions.checkNotNull(stateName, "State name is null. Cannot deserialize snapshot.");
-		Preconditions.checkNotNull(stateBackend, "State backend is null. Cannot deserialize snapshot.");
-
-		final KeyGroupRange keyGroupRange = stateBackend.getKeyGroupRange();
-		Preconditions.checkState(1 == keyGroupRange.getNumberOfKeyGroups(),
-				"Unexpected number of key-groups for restoring from Flink 1.1");
-
-		TypeSerializer<N> patchedNamespaceSerializer = this.namespaceSerializer;
-
-		if (patchedNamespaceSerializer instanceof VoidSerializer) {
-			patchedNamespaceSerializer = (TypeSerializer<N>) VoidNamespaceSerializer.INSTANCE;
-		}
-
-		RegisteredKeyedBackendStateMetaInfo<N, S> registeredKeyedBackendStateMetaInfo =
-				new RegisteredKeyedBackendStateMetaInfo<>(
-						StateDescriptor.Type.UNKNOWN,
-						stateName,
-						patchedNamespaceSerializer,
-						stateSerializer);
-
-		final StateTable<K, N, S> stateTable = stateBackend.newStateTable(registeredKeyedBackendStateMetaInfo);
-		final DataInputView inView = openDataInputView();
-		final int keyGroup = keyGroupRange.getStartKeyGroup();
-		final int numNamespaces = inView.readInt();
-
-		for (int i = 0; i < numNamespaces; i++) {
-			N namespace = namespaceSerializer.deserialize(inView);
-			if (null == namespace) {
-				namespace = (N) VoidNamespace.INSTANCE;
-			}
-			final int numKV = inView.readInt();
-			for (int j = 0; j < numKV; j++) {
-				K key = keySerializer.deserialize(inView);
-				S value = stateSerializer.deserialize(inView);
-				stateTable.put(key, keyGroup, namespace, value);
-			}
-		}
-		return stateTable;
-	}
-
-	/**
-	 * Different state handles require different code to end up with a {@link DataInputView}.
-	 */
-	protected abstract DataInputView openDataInputView() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/ByteStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/ByteStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/ByteStreamStateHandle.java
deleted file mode 100644
index c7fbab6..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/ByteStreamStateHandle.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.memory;
-
-import org.apache.flink.migration.runtime.state.AbstractCloseableHandle;
-import org.apache.flink.migration.runtime.state.StateHandle;
-import org.apache.flink.migration.runtime.state.StreamStateHandle;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Serializable;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public final class ByteStreamStateHandle extends AbstractCloseableHandle implements StreamStateHandle {
-
-	private static final long serialVersionUID = -5280226231200217594L;
-	
-	/** the state data */
-	private final byte[] data;
-
-	/**
-	 * Creates a new ByteStreamStateHandle containing the given data.
-	 * 
-	 * @param data The state data.
-	 */
-	public ByteStreamStateHandle(byte[] data) {
-		this.data = data;
-	}
-
-	@Override
-	public InputStream getState(ClassLoader userCodeClassLoader) throws Exception {
-		ensureNotClosed();
-
-		ByteArrayInputStream stream = new ByteArrayInputStream(data);
-		registerCloseable(stream);
-
-		return stream;
-	}
-
-	@Override
-	public void discardState() {}
-
-	@Override
-	public long getStateSize() {
-		return data.length;
-	}
-
-	@Override
-	public <T extends Serializable> StateHandle<T> toSerializableHandle() {
-		SerializedStateHandle<T> serializableHandle = new SerializedStateHandle<T>(data);
-
-		// forward the closed status
-		if (isClosed()) {
-			try {
-				serializableHandle.close();
-			} catch (IOException e) {
-				// should not happen on a fresh handle, but forward anyways
-				throw new RuntimeException(e);
-			}
-		}
-
-		return serializableHandle;
-	}
-
-	public byte[] getData() {
-		return data;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemFoldingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemFoldingState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemFoldingState.java
deleted file mode 100644
index ad820e4..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemFoldingState.java
+++ /dev/null
@@ -1,42 +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.migration.runtime.state.memory;
-
-import org.apache.flink.api.common.state.FoldingState;
-import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class MemFoldingState<K, N, T, ACC> {
-
-	public static class Snapshot<K, N, T, ACC> extends AbstractMemStateSnapshot<K, N, ACC, FoldingState<T, ACC>, FoldingStateDescriptor<T, ACC>> {
-		private static final long serialVersionUID = 1L;
-
-		public Snapshot(TypeSerializer<K> keySerializer,
-			TypeSerializer<N> namespaceSerializer,
-			TypeSerializer<ACC> stateSerializer,
-			FoldingStateDescriptor<T, ACC> stateDescs, byte[] data) {
-			super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, data);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemListState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemListState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemListState.java
deleted file mode 100644
index d76cda0..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemListState.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.memory;
-
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-import java.util.ArrayList;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class MemListState<K, N, V> {
-
-	public static class Snapshot<K, N, V> extends AbstractMemStateSnapshot<K, N, ArrayList<V>, ListState<V>, ListStateDescriptor<V>> {
-		private static final long serialVersionUID = 1L;
-
-		public Snapshot(TypeSerializer<K> keySerializer,
-			TypeSerializer<N> namespaceSerializer,
-			TypeSerializer<ArrayList<V>> stateSerializer,
-			ListStateDescriptor<V> stateDescs, byte[] data) {
-			super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, data);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemReducingState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemReducingState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemReducingState.java
deleted file mode 100644
index c39111c..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemReducingState.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.memory;
-
-import org.apache.flink.api.common.state.ReducingState;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-/**
- * Heap-backed partitioned {@link ReducingState} that is
- * snapshotted into a serialized memory copy.
- *
- * @param <K> The type of the key.
- * @param <N> The type of the namespace.
- * @param <V> The type of the values in the list state.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class MemReducingState<K, N, V> {
-
-	public static class Snapshot<K, N, V> extends AbstractMemStateSnapshot<K, N, V, ReducingState<V>, ReducingStateDescriptor<V>> {
-		private static final long serialVersionUID = 1L;
-
-		public Snapshot(TypeSerializer<K> keySerializer,
-			TypeSerializer<N> namespaceSerializer,
-			TypeSerializer<V> stateSerializer,
-			ReducingStateDescriptor<V> stateDescs, byte[] data) {
-			super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, data);
-		}
-	}}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemValueState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemValueState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemValueState.java
deleted file mode 100644
index 940d489..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MemValueState.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.runtime.state.memory;
-
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-/**
- * Heap-backed key/value state that is snapshotted into a serialized memory copy.
- *
- * @param <K> The type of the key.
- * @param <N> The type of the namespace.
- * @param <V> The type of the value.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class MemValueState<K, N, V> {
-
-	public static class Snapshot<K, N, V> extends AbstractMemStateSnapshot<K, N, V, ValueState<V>, ValueStateDescriptor<V>> {
-		private static final long serialVersionUID = 1L;
-
-		public Snapshot(TypeSerializer<K> keySerializer,
-			TypeSerializer<N> namespaceSerializer,
-			TypeSerializer<V> stateSerializer,
-			ValueStateDescriptor<V> stateDescs, byte[] data) {
-			super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, data);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MigrationRestoreSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MigrationRestoreSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MigrationRestoreSnapshot.java
deleted file mode 100644
index 4277b56..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/MigrationRestoreSnapshot.java
+++ /dev/null
@@ -1,35 +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.migration.runtime.state.memory;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend;
-import org.apache.flink.runtime.state.heap.StateTable;
-import org.apache.flink.util.Migration;
-
-import java.io.IOException;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@Internal
-public interface MigrationRestoreSnapshot<K, N, S> extends Migration {
-	StateTable<K, N, S> deserialize(String stateName, HeapKeyedStateBackend<K> stateBackend) throws IOException;
-}


[02/11] flink git commit: [FLINK-7505] Use lambdas in suppressed exception idiom

Posted by sr...@apache.org.
[FLINK-7505] Use lambdas in suppressed exception idiom


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

Branch: refs/heads/master
Commit: 5456cf9f8fc9156fd10e7542e8a2497a285cbff7
Parents: ca87bec
Author: Stefan Richter <s....@data-artisans.com>
Authored: Thu Aug 24 17:27:29 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Thu Aug 24 20:17:08 2017 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/util/LambdaUtil.java  | 63 ++++++++++++++++++++
 .../org/apache/flink/util/ThrowingConsumer.java | 37 ++++++++++++
 .../apache/flink/runtime/state/StateUtil.java   | 25 +-------
 3 files changed, 103 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5456cf9f/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java b/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java
new file mode 100644
index 0000000..8ac0f0e
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/util/LambdaUtil.java
@@ -0,0 +1,63 @@
+/*
+ * 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.util;
+
+/**
+ * This class offers utility functions for Java's lambda features.
+ */
+public final class LambdaUtil {
+
+	private LambdaUtil() {
+		throw new AssertionError();
+	}
+
+	/**
+	 * This method supplies all elements from the input to the consumer. Exceptions that happen on elements are
+	 * suppressed until all elements are processed. If exceptions happened for one or more of the inputs, they are
+	 * reported in a combining suppressed exception.
+	 *
+	 * @param inputs iterator for all inputs to the throwingConsumer.
+	 * @param throwingConsumer this consumer will be called for all elements delivered by the input iterator.
+	 * @param <T> the type of input.
+	 * @throws Exception collected exceptions that happened during the invocation of the consumer on the input elements.
+	 */
+	public static <T> void applyToAllWhileSuppressingExceptions(
+		Iterable<T> inputs,
+		ThrowingConsumer<T> throwingConsumer) throws Exception {
+
+		if (inputs != null && throwingConsumer != null) {
+			Exception exception = null;
+
+			for (T input : inputs) {
+
+				if (input != null) {
+					try {
+						throwingConsumer.accept(input);
+					} catch (Exception ex) {
+						exception = ExceptionUtils.firstOrSuppressed(ex, exception);
+					}
+				}
+			}
+
+			if (exception != null) {
+				throw exception;
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5456cf9f/flink-core/src/main/java/org/apache/flink/util/ThrowingConsumer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ThrowingConsumer.java b/flink-core/src/main/java/org/apache/flink/util/ThrowingConsumer.java
new file mode 100644
index 0000000..a180a12
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/util/ThrowingConsumer.java
@@ -0,0 +1,37 @@
+/*
+ * 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.util;
+
+/**
+ * This interface is basically Java's {@link java.util.function.Consumer} interface enhanced with the ability to throw
+ * an exception.
+ *
+ * @param <T> type of the consumed elements.
+ */
+@FunctionalInterface
+public interface ThrowingConsumer<T> {
+
+	/**
+	 * Performs this operation on the given argument.
+	 *
+	 * @param t the input argument
+	 * @throws Exception on errors during consumption
+	 */
+	void accept(T t) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5456cf9f/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
index 6f231e4..09d195a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.runtime.state;
 
-import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.FutureUtil;
+import org.apache.flink.util.LambdaUtil;
 
 import java.util.concurrent.RunnableFuture;
 
@@ -49,27 +49,8 @@ public class StateUtil {
 	 * @throws Exception exception that is a collection of all suppressed exceptions that were caught during iteration
 	 */
 	public static void bestEffortDiscardAllStateObjects(
-			Iterable<? extends StateObject> handlesToDiscard) throws Exception {
-
-		if (handlesToDiscard != null) {
-			Exception exception = null;
-
-			for (StateObject state : handlesToDiscard) {
-
-				if (state != null) {
-					try {
-						state.discardState();
-					}
-					catch (Exception ex) {
-						exception = ExceptionUtils.firstOrSuppressed(ex, exception);
-					}
-				}
-			}
-
-			if (exception != null) {
-				throw exception;
-			}
-		}
+		Iterable<? extends StateObject> handlesToDiscard) throws Exception {
+		LambdaUtil.applyToAllWhileSuppressingExceptions(handlesToDiscard, StateObject::discardState);
 	}
 
 	/**


[07/11] flink git commit: [FLINK-7461] Remove Backwards compatibility with <= Flink 1.1

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
deleted file mode 100644
index 16f3769..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/MigrationV0ToV1Test.java
+++ /dev/null
@@ -1,251 +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.runtime.checkpoint.savepoint;
-
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.core.fs.FSDataOutputStream;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.migration.runtime.checkpoint.savepoint.SavepointV0;
-import org.apache.flink.migration.runtime.checkpoint.savepoint.SavepointV0Serializer;
-import org.apache.flink.migration.runtime.state.KvStateSnapshot;
-import org.apache.flink.migration.runtime.state.memory.MemValueState;
-import org.apache.flink.migration.runtime.state.memory.SerializedStateHandle;
-import org.apache.flink.migration.streaming.runtime.tasks.StreamTaskState;
-import org.apache.flink.migration.streaming.runtime.tasks.StreamTaskStateList;
-import org.apache.flink.migration.util.MigrationInstantiationUtil;
-import org.apache.flink.runtime.checkpoint.SubtaskState;
-import org.apache.flink.runtime.checkpoint.TaskState;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.state.ChainedStateHandle;
-import org.apache.flink.runtime.state.KeyGroupsStateHandle;
-import org.apache.flink.runtime.state.KeyedStateHandle;
-import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.runtime.state.VoidNamespaceSerializer;
-import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
-import org.apache.flink.util.FileUtils;
-import org.apache.flink.util.InstantiationUtil;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-@SuppressWarnings("deprecation")
-public class MigrationV0ToV1Test {
-
-	@Rule
-	public TemporaryFolder tmp = new TemporaryFolder();
-
-	/**
-	 * Simple test of savepoint methods.
-	 */
-	@Test
-	public void testSavepointMigrationV0ToV1() throws Exception {
-
-		String target = tmp.getRoot().getAbsolutePath();
-
-		assertEquals(0, tmp.getRoot().listFiles().length);
-
-		long checkpointId = ThreadLocalRandom.current().nextLong(Integer.MAX_VALUE);
-		int numTaskStates = 4;
-		int numSubtaskStates = 16;
-
-		Collection<org.apache.flink.migration.runtime.checkpoint.TaskState> expected =
-				createTaskStatesOld(numTaskStates, numSubtaskStates);
-
-		SavepointV0 savepoint = new SavepointV0(checkpointId, expected);
-
-		assertEquals(SavepointV0.VERSION, savepoint.getVersion());
-		assertEquals(checkpointId, savepoint.getCheckpointId());
-		assertEquals(expected, savepoint.getOldTaskStates());
-
-		assertFalse(savepoint.getOldTaskStates().isEmpty());
-
-		Exception latestException = null;
-		Path path = null;
-		FSDataOutputStream fdos = null;
-
-		FileSystem fs = null;
-
-		try {
-
-			// Try to create a FS output stream
-			for (int attempt = 0; attempt < 10; attempt++) {
-				path = new Path(target, FileUtils.getRandomFilename("savepoint-"));
-
-				if (fs == null) {
-					fs = FileSystem.get(path.toUri());
-				}
-
-				try {
-					fdos = fs.create(path, FileSystem.WriteMode.NO_OVERWRITE);
-					break;
-				} catch (Exception e) {
-					latestException = e;
-				}
-			}
-
-			if (fdos == null) {
-				throw new IOException("Failed to create file output stream at " + path, latestException);
-			}
-
-			try (DataOutputStream dos = new DataOutputStream(fdos)) {
-				dos.writeInt(SavepointStore.MAGIC_NUMBER);
-				dos.writeInt(savepoint.getVersion());
-				SavepointV0Serializer.INSTANCE.serializeOld(savepoint, dos);
-			}
-
-			ClassLoader cl = Thread.currentThread().getContextClassLoader();
-
-			Savepoint sp = SavepointStore.loadSavepoint(path.toString(), cl);
-			int t = 0;
-			for (TaskState taskState : sp.getTaskStates()) {
-				for (int p = 0; p < taskState.getParallelism(); ++p) {
-					SubtaskState subtaskState = taskState.getState(p);
-					ChainedStateHandle<StreamStateHandle> legacyOperatorState = subtaskState.getLegacyOperatorState();
-					for (int c = 0; c < legacyOperatorState.getLength(); ++c) {
-						StreamStateHandle stateHandle = legacyOperatorState.get(c);
-						try (InputStream is = stateHandle.openInputStream()) {
-							Tuple4<Integer, Integer, Integer, Integer> expTestState = new Tuple4<>(0, t, p, c);
-							Tuple4<Integer, Integer, Integer, Integer> actTestState;
-							//check function state
-							if (p % 4 != 0) {
-								assertEquals(1, is.read());
-								actTestState = InstantiationUtil.deserializeObject(is, cl);
-								assertEquals(expTestState, actTestState);
-							} else {
-								assertEquals(0, is.read());
-							}
-
-							//check operator state
-							expTestState.f0 = 1;
-							actTestState = InstantiationUtil.deserializeObject(is, cl);
-							assertEquals(expTestState, actTestState);
-						}
-					}
-
-					//check keyed state
-					KeyedStateHandle keyedStateHandle = subtaskState.getManagedKeyedState();
-
-					if (t % 3 != 0) {
-
-						assertTrue(keyedStateHandle instanceof KeyGroupsStateHandle);
-
-						KeyGroupsStateHandle keyGroupsStateHandle = (KeyGroupsStateHandle) keyedStateHandle;
-
-						assertEquals(1, keyGroupsStateHandle.getKeyGroupRange().getNumberOfKeyGroups());
-						assertEquals(p, keyGroupsStateHandle.getGroupRangeOffsets().getKeyGroupRange().getStartKeyGroup());
-
-						ByteStreamStateHandle stateHandle =
-								(ByteStreamStateHandle) keyGroupsStateHandle.getDelegateStateHandle();
-						HashMap<String, KvStateSnapshot<?, ?, ?, ?>> testKeyedState =
-								MigrationInstantiationUtil.deserializeObject(stateHandle.getData(), cl);
-
-						assertEquals(2, testKeyedState.size());
-						for (KvStateSnapshot<?, ?, ?, ?> snapshot : testKeyedState.values()) {
-							MemValueState.Snapshot<?, ?, ?> castedSnapshot = (MemValueState.Snapshot<?, ?, ?>) snapshot;
-							byte[] data = castedSnapshot.getData();
-							assertEquals(t, data[0]);
-							assertEquals(p, data[1]);
-						}
-					} else {
-						assertEquals(null, keyedStateHandle);
-					}
-				}
-
-				++t;
-			}
-
-			savepoint.dispose();
-
-		} finally {
-			// Dispose
-			SavepointStore.removeSavepointFile(path.toString());
-		}
-	}
-
-	private static Collection<org.apache.flink.migration.runtime.checkpoint.TaskState> createTaskStatesOld(
-			int numTaskStates, int numSubtaskStates) throws Exception {
-
-		List<org.apache.flink.migration.runtime.checkpoint.TaskState> taskStates = new ArrayList<>(numTaskStates);
-
-		for (int i = 0; i < numTaskStates; i++) {
-			org.apache.flink.migration.runtime.checkpoint.TaskState taskState =
-					new org.apache.flink.migration.runtime.checkpoint.TaskState(new JobVertexID(), numSubtaskStates);
-			for (int j = 0; j < numSubtaskStates; j++) {
-
-				StreamTaskState[] streamTaskStates = new StreamTaskState[2];
-
-				for (int k = 0; k < streamTaskStates.length; k++) {
-					StreamTaskState state = new StreamTaskState();
-					Tuple4<Integer, Integer, Integer, Integer> testState = new Tuple4<>(0, i, j, k);
-					if (j % 4 != 0) {
-						state.setFunctionState(new SerializedStateHandle<Serializable>(testState));
-					}
-					testState = new Tuple4<>(1, i, j, k);
-					state.setOperatorState(new SerializedStateHandle<>(testState));
-
-					if ((0 == k) && (i % 3 != 0)) {
-						HashMap<String, KvStateSnapshot<?, ?, ?, ?>> testKeyedState = new HashMap<>(2);
-						for (int l = 0; l < 2; ++l) {
-							String name = "keyed-" + l;
-							KvStateSnapshot<?, ?, ?, ?> testKeyedSnapshot =
-									new MemValueState.Snapshot<>(
-											IntSerializer.INSTANCE,
-											VoidNamespaceSerializer.INSTANCE,
-											IntSerializer.INSTANCE,
-											new ValueStateDescriptor<>(name, Integer.class, 0),
-											new byte[]{(byte) i, (byte) j});
-							testKeyedState.put(name, testKeyedSnapshot);
-						}
-						state.setKvStates(testKeyedState);
-					}
-					streamTaskStates[k] = state;
-				}
-
-				StreamTaskStateList streamTaskStateList = new StreamTaskStateList(streamTaskStates);
-				org.apache.flink.migration.util.SerializedValue<
-						org.apache.flink.migration.runtime.state.StateHandle<?>> handle =
-						new org.apache.flink.migration.util.SerializedValue<
-								org.apache.flink.migration.runtime.state.StateHandle<?>>(streamTaskStateList);
-
-				taskState.putState(j, new org.apache.flink.migration.runtime.checkpoint.SubtaskState(handle, 0, 0));
-			}
-
-			taskStates.add(taskState);
-		}
-
-		return taskStates;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
index 933c7a0..173730a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
@@ -61,6 +61,7 @@ import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
 import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
 import org.apache.flink.runtime.taskmanager.TaskManager;
 import org.apache.flink.runtime.testingUtils.TestingJobManager;
@@ -72,6 +73,7 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
 import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore;
 import org.apache.flink.runtime.util.TestByteStreamStateHandleDeepCompare;
 import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.TestLogger;
 
 import akka.actor.ActorRef;
@@ -98,6 +100,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executor;
@@ -560,7 +563,8 @@ public class JobManagerHARecoveryTest extends TestLogger {
 			TaskStateSnapshot taskStateHandles) throws Exception {
 			int subtaskIndex = getIndexInSubtaskGroup();
 			if (subtaskIndex < recoveredStates.length) {
-				try (FSDataInputStream in = taskStateHandles.getSubtaskStateMappings().iterator().next().getValue().getLegacyOperatorState().openInputStream()) {
+				OperatorStateHandle operatorStateHandle = extractSingletonOperatorState(taskStateHandles);
+				try (FSDataInputStream in = operatorStateHandle.openInputStream()) {
 					recoveredStates[subtaskIndex] = InstantiationUtil.deserializeObject(in, getUserCodeClassLoader());
 				}
 			}
@@ -572,11 +576,21 @@ public class JobManagerHARecoveryTest extends TestLogger {
 					String.valueOf(UUID.randomUUID()),
 					InstantiationUtil.serializeObject(checkpointMetaData.getCheckpointId()));
 
+			Map<String, OperatorStateHandle.StateMetaInfo> stateNameToPartitionOffsets = new HashMap<>(1);
+			stateNameToPartitionOffsets.put(
+				"test-state",
+				new OperatorStateHandle.StateMetaInfo(new long[]{0L}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE));
+
+			OperatorStateHandle operatorStateHandle = new OperatorStateHandle(stateNameToPartitionOffsets, byteStreamStateHandle);
+
 			TaskStateSnapshot checkpointStateHandles = new TaskStateSnapshot();
 			checkpointStateHandles.putSubtaskStateByOperatorID(
 				OperatorID.fromJobVertexID(getEnvironment().getJobVertexId()),
-				new OperatorSubtaskState(byteStreamStateHandle)
-			);
+				new OperatorSubtaskState(
+					Collections.singletonList(operatorStateHandle),
+					Collections.emptyList(),
+					Collections.emptyList(),
+					Collections.emptyList()));
 
 			getEnvironment().acknowledgeCheckpoint(
 					checkpointMetaData.getCheckpointId(),
@@ -614,5 +628,17 @@ public class JobManagerHARecoveryTest extends TestLogger {
 		public static long[] getRecoveredStates() {
 			return recoveredStates;
 		}
+
+		private static OperatorStateHandle extractSingletonOperatorState(TaskStateSnapshot taskStateHandles) {
+			Set<Map.Entry<OperatorID, OperatorSubtaskState>> subtaskStateMappings = taskStateHandles.getSubtaskStateMappings();
+			Preconditions.checkNotNull(subtaskStateMappings);
+			Preconditions.checkState(subtaskStateMappings.size()  == 1);
+			OperatorSubtaskState subtaskState = subtaskStateMappings.iterator().next().getValue();
+			Collection<OperatorStateHandle> managedOperatorState =
+				Preconditions.checkNotNull(subtaskState).getManagedOperatorState();
+			Preconditions.checkNotNull(managedOperatorState);
+			Preconditions.checkState(managedOperatorState.size()  == 1);
+			return managedOperatorState.iterator().next();
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java
index d022cdc..b36ac86 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java
@@ -75,7 +75,6 @@ public class CheckpointMessagesTest {
 			checkpointStateHandles.putSubtaskStateByOperatorID(
 				new OperatorID(),
 				new OperatorSubtaskState(
-					CheckpointCoordinatorTest.generateStreamStateHandle(new MyHandle()),
 					CheckpointCoordinatorTest.generatePartitionableStateHandle(new JobVertexID(), 0, 2, 8, false),
 					null,
 					CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRange, Collections.singletonList(new MyHandle())),

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/api/graph/StreamGraphHasherV1.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/api/graph/StreamGraphHasherV1.java b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/api/graph/StreamGraphHasherV1.java
deleted file mode 100644
index dd6148c..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/api/graph/StreamGraphHasherV1.java
+++ /dev/null
@@ -1,282 +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.migration.streaming.api.graph;
-
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.graph.StreamGraphHasher;
-import org.apache.flink.streaming.api.graph.StreamNode;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
-
-import org.apache.flink.shaded.guava18.com.google.common.hash.HashFunction;
-import org.apache.flink.shaded.guava18.com.google.common.hash.Hasher;
-import org.apache.flink.shaded.guava18.com.google.common.hash.Hashing;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.charset.Charset;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-
-import static org.apache.flink.util.StringUtils.byteToHexString;
-
-/**
- * StreamGraphHasher from Flink 1.1. This contains duplicated code to ensure that the algorithm does not change with
- * future Flink versions.
- *
- * <p>DO NOT MODIFY THIS CLASS
- */
-public class StreamGraphHasherV1 implements StreamGraphHasher {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StreamGraphHasherV1.class);
-
-	@Override
-	public Map<Integer, byte[]> traverseStreamGraphAndGenerateHashes(StreamGraph streamGraph) {
-		// The hash function used to generate the hash
-		final HashFunction hashFunction = Hashing.murmur3_128(0);
-		final Map<Integer, byte[]> hashes = new HashMap<>();
-
-		Set<Integer> visited = new HashSet<>();
-		Queue<StreamNode> remaining = new ArrayDeque<>();
-
-		// We need to make the source order deterministic. The source IDs are
-		// not returned in the same order, which means that submitting the same
-		// program twice might result in different traversal, which breaks the
-		// deterministic hash assignment.
-		List<Integer> sources = new ArrayList<>();
-		for (Integer sourceNodeId : streamGraph.getSourceIDs()) {
-			sources.add(sourceNodeId);
-		}
-		Collections.sort(sources);
-
-		//
-		// Traverse the graph in a breadth-first manner. Keep in mind that
-		// the graph is not a tree and multiple paths to nodes can exist.
-		//
-
-		// Start with source nodes
-		for (Integer sourceNodeId : sources) {
-			remaining.add(streamGraph.getStreamNode(sourceNodeId));
-			visited.add(sourceNodeId);
-		}
-
-		StreamNode currentNode;
-		while ((currentNode = remaining.poll()) != null) {
-			// Generate the hash code. Because multiple path exist to each
-			// node, we might not have all required inputs available to
-			// generate the hash code.
-			if (generateNodeHash(currentNode, hashFunction, hashes, streamGraph.isChainingEnabled())) {
-				// Add the child nodes
-				for (StreamEdge outEdge : currentNode.getOutEdges()) {
-					StreamNode child = outEdge.getTargetVertex();
-
-					if (!visited.contains(child.getId())) {
-						remaining.add(child);
-						visited.add(child.getId());
-					}
-				}
-			} else {
-				// We will revisit this later.
-				visited.remove(currentNode.getId());
-			}
-		}
-
-		return hashes;
-	}
-
-	/**
-	 * Generates a hash for the node and returns whether the operation was
-	 * successful.
-	 *
-	 * @param node         The node to generate the hash for
-	 * @param hashFunction The hash function to use
-	 * @param hashes       The current state of generated hashes
-	 * @return <code>true</code> if the node hash has been generated.
-	 * <code>false</code>, otherwise. If the operation is not successful, the
-	 * hash needs be generated at a later point when all input is available.
-	 * @throws IllegalStateException If node has user-specified hash and is
-	 *                               intermediate node of a chain
-	 */
-	private boolean generateNodeHash(
-			StreamNode node,
-			HashFunction hashFunction,
-			Map<Integer, byte[]> hashes,
-			boolean isChainingEnabled) {
-
-		// Check for user-specified ID
-		String userSpecifiedHash = node.getTransformationUID();
-
-		if (userSpecifiedHash == null) {
-			// Check that all input nodes have their hashes computed
-			for (StreamEdge inEdge : node.getInEdges()) {
-				// If the input node has not been visited yet, the current
-				// node will be visited again at a later point when all input
-				// nodes have been visited and their hashes set.
-				if (!hashes.containsKey(inEdge.getSourceId())) {
-					return false;
-				}
-			}
-
-			Hasher hasher = hashFunction.newHasher();
-			byte[] hash = generateDeterministicHash(node, hasher, hashes, isChainingEnabled);
-
-			if (hashes.put(node.getId(), hash) != null) {
-				// Sanity check
-				throw new IllegalStateException("Unexpected state. Tried to add node hash " +
-						"twice. This is probably a bug in the JobGraph generator.");
-			}
-
-			return true;
-		} else {
-			Hasher hasher = hashFunction.newHasher();
-			byte[] hash = generateUserSpecifiedHash(node, hasher);
-
-			for (byte[] previousHash : hashes.values()) {
-				if (Arrays.equals(previousHash, hash)) {
-					throw new IllegalArgumentException("Hash collision on user-specified ID. " +
-							"Most likely cause is a non-unique ID. Please check that all IDs " +
-							"specified via `uid(String)` are unique.");
-				}
-			}
-
-			if (hashes.put(node.getId(), hash) != null) {
-				// Sanity check
-				throw new IllegalStateException("Unexpected state. Tried to add node hash " +
-						"twice. This is probably a bug in the JobGraph generator.");
-			}
-
-			return true;
-		}
-	}
-
-	/**
-	 * Generates a hash from a user-specified ID.
-	 */
-	private byte[] generateUserSpecifiedHash(StreamNode node, Hasher hasher) {
-		hasher.putString(node.getTransformationUID(), Charset.forName("UTF-8"));
-
-		return hasher.hash().asBytes();
-	}
-
-	/**
-	 * Generates a deterministic hash from node-local properties and input and
-	 * output edges.
-	 */
-	private byte[] generateDeterministicHash(
-			StreamNode node,
-			Hasher hasher,
-			Map<Integer, byte[]> hashes,
-			boolean isChainingEnabled) {
-
-		// Include stream node to hash. We use the current size of the computed
-		// hashes as the ID. We cannot use the node's ID, because it is
-		// assigned from a static counter. This will result in two identical
-		// programs having different hashes.
-		generateNodeLocalHash(node, hasher, hashes.size());
-
-		// Include chained nodes to hash
-		for (StreamEdge outEdge : node.getOutEdges()) {
-			if (isChainable(outEdge, isChainingEnabled)) {
-				StreamNode chainedNode = outEdge.getTargetVertex();
-
-				// Use the hash size again, because the nodes are chained to
-				// this node. This does not add a hash for the chained nodes.
-				generateNodeLocalHash(chainedNode, hasher, hashes.size());
-			}
-		}
-
-		byte[] hash = hasher.hash().asBytes();
-
-		// Make sure that all input nodes have their hash set before entering
-		// this loop (calling this method).
-		for (StreamEdge inEdge : node.getInEdges()) {
-			byte[] otherHash = hashes.get(inEdge.getSourceId());
-
-			// Sanity check
-			if (otherHash == null) {
-				throw new IllegalStateException("Missing hash for input node "
-						+ inEdge.getSourceVertex() + ". Cannot generate hash for "
-						+ node + ".");
-			}
-
-			for (int j = 0; j < hash.length; j++) {
-				hash[j] = (byte) (hash[j] * 37 ^ otherHash[j]);
-			}
-		}
-
-		if (LOG.isDebugEnabled()) {
-			String udfClassName = "";
-			if (node.getOperator() instanceof AbstractUdfStreamOperator) {
-				udfClassName = ((AbstractUdfStreamOperator<?, ?>) node.getOperator())
-						.getUserFunction().getClass().getName();
-			}
-
-			LOG.debug("Generated hash '" + byteToHexString(hash) + "' for node " +
-					"'" + node.toString() + "' {id: " + node.getId() + ", " +
-					"parallelism: " + node.getParallelism() + ", " +
-					"user function: " + udfClassName + "}");
-		}
-
-		return hash;
-	}
-
-	private boolean isChainable(StreamEdge edge, boolean isChainingEnabled) {
-		StreamNode upStreamVertex = edge.getSourceVertex();
-		StreamNode downStreamVertex = edge.getTargetVertex();
-
-		StreamOperator<?> headOperator = upStreamVertex.getOperator();
-		StreamOperator<?> outOperator = downStreamVertex.getOperator();
-
-		return downStreamVertex.getInEdges().size() == 1
-				&& outOperator != null
-				&& headOperator != null
-				&& upStreamVertex.isSameSlotSharingGroup(downStreamVertex)
-				&& outOperator.getChainingStrategy() == ChainingStrategy.ALWAYS
-				&& (headOperator.getChainingStrategy() == ChainingStrategy.HEAD ||
-				headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS)
-				&& (edge.getPartitioner() instanceof ForwardPartitioner)
-				&& upStreamVertex.getParallelism() == downStreamVertex.getParallelism()
-				&& isChainingEnabled;
-	}
-
-	private void generateNodeLocalHash(StreamNode node, Hasher hasher, int id) {
-		hasher.putInt(id);
-
-		hasher.putInt(node.getParallelism());
-
-		if (node.getOperator() instanceof AbstractUdfStreamOperator) {
-			String udfClassName = ((AbstractUdfStreamOperator<?, ?>) node.getOperator())
-					.getUserFunction().getClass().getName();
-
-			hasher.putString(udfClassName, Charset.forName("UTF-8"));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
deleted file mode 100644
index b1471b2..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
+++ /dev/null
@@ -1,293 +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.migration.streaming.runtime.streamrecord;
-
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-import java.io.IOException;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Legacy multiplexing {@link TypeSerializer} for stream records, watermarks and other stream
- * elements.
- */
-public class MultiplexingStreamRecordSerializer<T> extends TypeSerializer<StreamElement> {
-
-
-	private static final long serialVersionUID = 1L;
-
-	private static final int TAG_REC_WITH_TIMESTAMP = 0;
-	private static final int TAG_REC_WITHOUT_TIMESTAMP = 1;
-	private static final int TAG_WATERMARK = 2;
-
-
-	private final TypeSerializer<T> typeSerializer;
-
-	public MultiplexingStreamRecordSerializer(TypeSerializer<T> serializer) {
-		if (serializer instanceof MultiplexingStreamRecordSerializer || serializer instanceof StreamRecordSerializer) {
-			throw new RuntimeException("StreamRecordSerializer given to StreamRecordSerializer as value TypeSerializer: " + serializer);
-		}
-		this.typeSerializer = requireNonNull(serializer);
-	}
-
-	public TypeSerializer<T> getContainedTypeSerializer() {
-		return this.typeSerializer;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-
-	@Override
-	public MultiplexingStreamRecordSerializer<T> duplicate() {
-		TypeSerializer<T> copy = typeSerializer.duplicate();
-		return (copy == typeSerializer) ? this : new MultiplexingStreamRecordSerializer<T>(copy);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	@Override
-	public StreamRecord<T> createInstance() {
-		return new StreamRecord<T>(typeSerializer.createInstance());
-	}
-
-	@Override
-	public int getLength() {
-		return -1;
-	}
-
-	@Override
-	public StreamElement copy(StreamElement from) {
-		// we can reuse the timestamp since Instant is immutable
-		if (from.isRecord()) {
-			StreamRecord<T> fromRecord = from.asRecord();
-			return fromRecord.copy(typeSerializer.copy(fromRecord.getValue()));
-		}
-		else if (from.isWatermark()) {
-			// is immutable
-			return from;
-		}
-		else {
-			throw new RuntimeException();
-		}
-	}
-
-	@Override
-	public StreamElement copy(StreamElement from, StreamElement reuse) {
-		if (from.isRecord() && reuse.isRecord()) {
-			StreamRecord<T> fromRecord = from.asRecord();
-			StreamRecord<T> reuseRecord = reuse.asRecord();
-
-			T valueCopy = typeSerializer.copy(fromRecord.getValue(), reuseRecord.getValue());
-			fromRecord.copyTo(valueCopy, reuseRecord);
-			return reuse;
-		}
-		else if (from.isWatermark()) {
-			// is immutable
-			return from;
-		}
-		else {
-			throw new RuntimeException("Cannot copy " + from + " -> " + reuse);
-		}
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		int tag = source.readByte();
-		target.write(tag);
-
-		if (tag == TAG_REC_WITH_TIMESTAMP) {
-			// move timestamp
-			target.writeLong(source.readLong());
-			typeSerializer.copy(source, target);
-		}
-		else if (tag == TAG_REC_WITHOUT_TIMESTAMP) {
-			typeSerializer.copy(source, target);
-		}
-		else if (tag == TAG_WATERMARK) {
-			target.writeLong(source.readLong());
-		}
-		else {
-			throw new IOException("Corrupt stream, found tag: " + tag);
-		}
-	}
-
-	@Override
-	public void serialize(StreamElement value, DataOutputView target) throws IOException {
-		if (value.isRecord()) {
-			StreamRecord<T> record = value.asRecord();
-
-			if (record.hasTimestamp()) {
-				target.write(TAG_REC_WITH_TIMESTAMP);
-				target.writeLong(record.getTimestamp());
-			} else {
-				target.write(TAG_REC_WITHOUT_TIMESTAMP);
-			}
-			typeSerializer.serialize(record.getValue(), target);
-		}
-		else if (value.isWatermark()) {
-			target.write(TAG_WATERMARK);
-			target.writeLong(value.asWatermark().getTimestamp());
-		}
-		else {
-			throw new RuntimeException();
-		}
-	}
-
-	@Override
-	public StreamElement deserialize(DataInputView source) throws IOException {
-		int tag = source.readByte();
-		if (tag == TAG_REC_WITH_TIMESTAMP) {
-			long timestamp = source.readLong();
-			return new StreamRecord<T>(typeSerializer.deserialize(source), timestamp);
-		}
-		else if (tag == TAG_REC_WITHOUT_TIMESTAMP) {
-			return new StreamRecord<T>(typeSerializer.deserialize(source));
-		}
-		else if (tag == TAG_WATERMARK) {
-			return new Watermark(source.readLong());
-		}
-		else {
-			throw new IOException("Corrupt stream, found tag: " + tag);
-		}
-	}
-
-	@Override
-	public StreamElement deserialize(StreamElement reuse, DataInputView source) throws IOException {
-		int tag = source.readByte();
-		if (tag == TAG_REC_WITH_TIMESTAMP) {
-			long timestamp = source.readLong();
-			T value = typeSerializer.deserialize(source);
-			StreamRecord<T> reuseRecord = reuse.asRecord();
-			reuseRecord.replace(value, timestamp);
-			return reuseRecord;
-		}
-		else if (tag == TAG_REC_WITHOUT_TIMESTAMP) {
-			T value = typeSerializer.deserialize(source);
-			StreamRecord<T> reuseRecord = reuse.asRecord();
-			reuseRecord.replace(value);
-			return reuseRecord;
-		}
-		else if (tag == TAG_WATERMARK) {
-			return new Watermark(source.readLong());
-		}
-		else {
-			throw new IOException("Corrupt stream, found tag: " + tag);
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	// Serializer configuration snapshotting & compatibility
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public MultiplexingStreamRecordSerializerConfigSnapshot snapshotConfiguration() {
-		return new MultiplexingStreamRecordSerializerConfigSnapshot<>(typeSerializer);
-	}
-
-	@Override
-	public CompatibilityResult<StreamElement> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof MultiplexingStreamRecordSerializerConfigSnapshot) {
-			Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> previousTypeSerializerAndConfig =
-				((MultiplexingStreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
-
-			CompatibilityResult<T> compatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousTypeSerializerAndConfig.f0,
-					UnloadableDummyTypeSerializer.class,
-					previousTypeSerializerAndConfig.f1,
-					typeSerializer);
-
-			if (!compatResult.isRequiresMigration()) {
-				return CompatibilityResult.compatible();
-			} else if (compatResult.getConvertDeserializer() != null) {
-				return CompatibilityResult.requiresMigration(
-					new MultiplexingStreamRecordSerializer<>(
-						new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer())));
-			}
-		}
-
-		return CompatibilityResult.requiresMigration();
-	}
-
-	/**
-	 * Configuration snapshot specific to the {@link MultiplexingStreamRecordSerializer}.
-	 */
-	public static final class MultiplexingStreamRecordSerializerConfigSnapshot<T>
-			extends CompositeTypeSerializerConfigSnapshot {
-
-		private static final int VERSION = 1;
-
-		/** This empty nullary constructor is required for deserializing the configuration. */
-		public MultiplexingStreamRecordSerializerConfigSnapshot() {}
-
-		public MultiplexingStreamRecordSerializerConfigSnapshot(TypeSerializer<T> typeSerializer) {
-			super(typeSerializer);
-		}
-
-		@Override
-		public int getVersion() {
-			return VERSION;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof MultiplexingStreamRecordSerializer) {
-			MultiplexingStreamRecordSerializer<?> other = (MultiplexingStreamRecordSerializer<?>) obj;
-
-			return other.canEqual(this) && typeSerializer.equals(other.typeSerializer);
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return obj instanceof MultiplexingStreamRecordSerializer;
-	}
-
-	@Override
-	public int hashCode() {
-		return typeSerializer.hashCode();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java
deleted file mode 100644
index e018ba0..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java
+++ /dev/null
@@ -1,208 +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,
- * WITHOUStreamRecord<?>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.migration.streaming.runtime.streamrecord;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-
-/**
- * Serializer for {@link StreamRecord}. This version ignores timestamps and only deals with
- * the element.
- *
- * <p>{@link MultiplexingStreamRecordSerializer} is a version that deals with timestamps and also
- * multiplexes {@link org.apache.flink.streaming.api.watermark.Watermark Watermarks} in the same
- * stream with {@link StreamRecord StreamRecords}.
- *
- * @see MultiplexingStreamRecordSerializer
- *
- * @param <T> The type of value in the {@link StreamRecord}
- */
-@Internal
-public final class StreamRecordSerializer<T> extends TypeSerializer<StreamRecord<T>> {
-
-	private static final long serialVersionUID = 1L;
-
-	private final TypeSerializer<T> typeSerializer;
-
-	public StreamRecordSerializer(TypeSerializer<T> serializer) {
-		if (serializer instanceof StreamRecordSerializer) {
-			throw new RuntimeException("StreamRecordSerializer given to StreamRecordSerializer as value TypeSerializer: " + serializer);
-		}
-		this.typeSerializer = Preconditions.checkNotNull(serializer);
-	}
-
-	public TypeSerializer<T> getContainedTypeSerializer() {
-		return this.typeSerializer;
-	}
-
-	// ------------------------------------------------------------------------
-	//  General serializer and type utils
-	// ------------------------------------------------------------------------
-
-	@Override
-	public StreamRecordSerializer<T> duplicate() {
-		TypeSerializer<T> serializerCopy = typeSerializer.duplicate();
-		return serializerCopy == typeSerializer ? this : new StreamRecordSerializer<T>(serializerCopy);
-	}
-
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-
-	@Override
-	public int getLength() {
-		return typeSerializer.getLength();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Type serialization, copying, instantiation
-	// ------------------------------------------------------------------------
-
-	@Override
-	public StreamRecord<T> createInstance() {
-		try {
-			return new StreamRecord<T>(typeSerializer.createInstance());
-		} catch (Exception e) {
-			throw new RuntimeException("Cannot instantiate StreamRecord.", e);
-		}
-	}
-
-	@Override
-	public StreamRecord<T> copy(StreamRecord<T> from) {
-		return from.copy(typeSerializer.copy(from.getValue()));
-	}
-
-	@Override
-	public StreamRecord<T> copy(StreamRecord<T> from, StreamRecord<T> reuse) {
-		from.copyTo(typeSerializer.copy(from.getValue(), reuse.getValue()), reuse);
-		return reuse;
-	}
-
-	@Override
-	public void serialize(StreamRecord<T> value, DataOutputView target) throws IOException {
-		typeSerializer.serialize(value.getValue(), target);
-	}
-
-	@Override
-	public StreamRecord<T> deserialize(DataInputView source) throws IOException {
-		return new StreamRecord<T>(typeSerializer.deserialize(source));
-	}
-
-	@Override
-	public StreamRecord<T> deserialize(StreamRecord<T> reuse, DataInputView source) throws IOException {
-		T element = typeSerializer.deserialize(reuse.getValue(), source);
-		reuse.replace(element);
-		return reuse;
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		typeSerializer.copy(source, target);
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof StreamRecordSerializer) {
-			StreamRecordSerializer<?> other = (StreamRecordSerializer<?>) obj;
-
-			return other.canEqual(this) && typeSerializer.equals(other.typeSerializer);
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return obj instanceof StreamRecordSerializer;
-	}
-
-	@Override
-	public int hashCode() {
-		return typeSerializer.hashCode();
-	}
-
-	// --------------------------------------------------------------------------------------------
-	// Serializer configuration snapshotting & compatibility
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public StreamRecordSerializerConfigSnapshot snapshotConfiguration() {
-		return new StreamRecordSerializerConfigSnapshot<>(typeSerializer);
-	}
-
-	@Override
-	public CompatibilityResult<StreamRecord<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof StreamRecordSerializerConfigSnapshot) {
-			Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> previousTypeSerializerAndConfig =
-				((StreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
-
-			CompatibilityResult<T> compatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousTypeSerializerAndConfig.f0,
-					UnloadableDummyTypeSerializer.class,
-					previousTypeSerializerAndConfig.f1,
-					typeSerializer);
-
-			if (!compatResult.isRequiresMigration()) {
-				return CompatibilityResult.compatible();
-			} else if (compatResult.getConvertDeserializer() != null) {
-				return CompatibilityResult.requiresMigration(
-					new StreamRecordSerializer<>(
-						new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer())));
-			}
-		}
-
-		return CompatibilityResult.requiresMigration();
-	}
-
-	/**
-	 * Configuration snapshot specific to the {@link StreamRecordSerializer}.
-	 */
-	public static final class StreamRecordSerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot {
-
-		private static final int VERSION = 1;
-
-		/** This empty nullary constructor is required for deserializing the configuration. */
-		public StreamRecordSerializerConfigSnapshot() {}
-
-		public StreamRecordSerializerConfigSnapshot(TypeSerializer<T> typeSerializer) {
-			super(typeSerializer);
-		}
-
-		@Override
-		public int getVersion() {
-			return VERSION;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
deleted file mode 100644
index cb3c7cc..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.checkpoint;
-
-import org.apache.flink.annotation.PublicEvolving;
-
-import java.io.Serializable;
-
-/**
- * This method must be implemented by functions that have state that needs to be
- * checkpointed. The functions get a call whenever a checkpoint should take place
- * and return a snapshot of their state, which will be checkpointed.
- *
- * <h1>Deprecation and Replacement</h1>
- * The short cut replacement for this interface is via {@link ListCheckpointed} and works
- * as shown in the example below. The {@code ListCheckpointed} interface returns a list of
- * elements (
- *
- * <p><pre>{@code
- * public class ExampleFunction<T> implements MapFunction<T, T>, ListCheckpointed<Integer> {
- *
- *     private int count;
- *
- *     public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
- *         return Collections.singletonList(this.count);
- *     }
- *
- *     public void restoreState(List<Integer> state) throws Exception {
- *         this.value = state.isEmpty() ? 0 : state.get(0);
- *     }
- *
- *     public T map(T value) {
- *         count++;
- *         return value;
- *     }
- * }
- * }</pre>
- *
- * @param <T> The type of the operator state.
- *
- * @deprecated Please use {@link ListCheckpointed} as illustrated above, or
- *             {@link CheckpointedFunction} for more control over the checkpointing process.
- */
-@Deprecated
-@PublicEvolving
-public interface Checkpointed<T extends Serializable> extends CheckpointedRestoring<T> {
-
-	/**
-	 * Gets the current state of the function of operator. The state must reflect the result of all
-	 * prior invocations to this function.
-	 *
-	 * @param checkpointId The ID of the checkpoint.
-	 * @param checkpointTimestamp The timestamp of the checkpoint, as derived by
-	 *                            System.currentTimeMillis() on the JobManager.
-	 *
-	 * @return A snapshot of the operator state.
-	 *
-	 * @throws Exception Thrown if the creation of the state object failed. This causes the
-	 *                   checkpoint to fail. The system may decide to fail the operation (and trigger
-	 *                   recovery), or to discard this checkpoint attempt and to continue running
-	 *                   and to try again with the next checkpoint attempt.
-	 */
-	T snapshotState(long checkpointId, long checkpointTimestamp) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java
deleted file mode 100644
index 5138b49..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java
+++ /dev/null
@@ -1,61 +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.checkpoint;
-
-import org.apache.flink.annotation.PublicEvolving;
-
-import java.io.Serializable;
-
-/**
- * This interface marks a function/operator as checkpointed similar to the
- * {@link Checkpointed} interface, but gives the Flink framework the option to
- * perform the checkpoint asynchronously. Note that asynchronous checkpointing for
- * this interface has not been implemented.
- *
- * <h1>Deprecation and Replacement</h1>
- * The shortcut replacement for this interface is via {@link ListCheckpointed} and works
- * as shown in the example below. Please refer to the JavaDocs of {@link ListCheckpointed} for
- * a more detailed description of how to use the new interface.
- *
- * <p><pre>{@code
- * public class ExampleFunction<T> implements MapFunction<T, T>, ListCheckpointed<Integer> {
- *
- *     private int count;
- *
- *     public List<Integer> snapshotState(long checkpointId, long timestamp) throws Exception {
- *         return Collections.singletonList(this.count);
- *     }
- *
- *     public void restoreState(List<Integer> state) throws Exception {
- *         this.value = state.isEmpty() ? 0 : state.get(0);
- *     }
- *
- *     public T map(T value) {
- *         count++;
- *         return value;
- *     }
- * }
- * }</pre>
- *
- * @deprecated Please use {@link ListCheckpointed} and {@link CheckpointedFunction} instead,
- *             as illustrated in the example above.
- */
-@Deprecated
-@PublicEvolving
-public interface CheckpointedAsynchronously<T extends Serializable> extends Checkpointed<T> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedRestoring.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedRestoring.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedRestoring.java
deleted file mode 100644
index cfaa505..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedRestoring.java
+++ /dev/null
@@ -1,43 +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.checkpoint;
-
-import org.apache.flink.annotation.PublicEvolving;
-
-import java.io.Serializable;
-
-/**
- * This deprecated interface contains the methods for restoring from the legacy checkpointing mechanism of state.
- * @param <T> type of the restored state.
- *
- * @deprecated Please use {@link CheckpointedFunction} or {@link ListCheckpointed} after restoring your legacy state.
- */
-@Deprecated
-@PublicEvolving
-public interface CheckpointedRestoring<T extends Serializable> {
-	/**
-	 * Restores the state of the function or operator to that of a previous checkpoint.
-	 * This method is invoked when a function is executed as part of a recovery run.
-	 *
-	 * <p>Note that restoreState() is called before open().
-	 *
-	 * @param state The state to be restored.
-	 */
-	void restoreState(T state) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/LegacyWindowOperatorType.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/LegacyWindowOperatorType.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/LegacyWindowOperatorType.java
deleted file mode 100644
index bb6e4bc..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/LegacyWindowOperatorType.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-/**
- * For specifying what type of window operator was used to create the state
- * that a {@link org.apache.flink.streaming.runtime.operators.windowing.WindowOperator}
- * is restoring from. This is used to signal that state written using an aligned processing-time
- * window operator should be restored.
- */
-public enum LegacyWindowOperatorType {
-
-	FAST_ACCUMULATING(true, false),
-
-	FAST_AGGREGATING(false, true),
-
-	NONE(false, false);
-
-	// ------------------------------------------------------------------------
-
-	private final boolean fastAccumulating;
-	private final boolean fastAggregating;
-
-	LegacyWindowOperatorType(boolean fastAccumulating, boolean fastAggregating) {
-		this.fastAccumulating = fastAccumulating;
-		this.fastAggregating = fastAggregating;
-	}
-
-	public boolean isFastAccumulating() {
-		return fastAccumulating;
-	}
-
-	public boolean isFastAggregating() {
-		return fastAggregating;
-	}
-
-	@Override
-	public String toString() {
-		if (fastAccumulating) {
-			return "AccumulatingProcessingTimeWindowOperator";
-		} else if (fastAggregating) {
-			return "AggregatingProcessingTimeWindowOperator";
-		} else {
-			return "WindowOperator";
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index 348861f..f904a10 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -23,7 +23,6 @@ import org.apache.flink.annotation.Public;
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.functions.AggregateFunction;
 import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.functions.RichFunction;
 import org.apache.flink.api.common.state.AggregatingStateDescriptor;
@@ -50,19 +49,11 @@ import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.windowing.assigners.BaseAlignedWindowAssigner;
 import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingAlignedProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingAlignedProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
 import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
 import org.apache.flink.streaming.api.windowing.evictors.Evictor;
 import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
-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.AccumulatingProcessingTimeWindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.AggregatingProcessingTimeWindowOperator;
 import org.apache.flink.streaming.runtime.operators.windowing.EvictingWindowOperator;
 import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator;
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalAggregateProcessWindowFunction;
@@ -227,33 +218,7 @@ public class WindowedStream<T, K, W extends Window> {
 
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
-
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "WindowedStream." + callLocation;
-
-		SingleOutputStreamOperator<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
-		if (result != null) {
-			return result;
-		}
-
-		LegacyWindowOperatorType legacyOpType = getLegacyWindowType(function);
-		return reduce(function, new PassThroughWindowFunction<K, W, T>(), legacyOpType);
-	}
-
-	/**
-	 * Applies the given window function to each window. The window function is called for each
-	 * evaluation of the window for each key individually. The output of the window function is
-	 * interpreted as a regular non-windowed stream.
-	 *
-	 * <p>Arriving data is incrementally aggregated using the given reducer.
-	 *
-	 * @param reduceFunction The reduce function that is used for incremental aggregation.
-	 * @param function The window function.
-	 * @return The data stream that is the result of applying the window function to the window.
-	 */
-	@PublicEvolving
-	public <R> SingleOutputStreamOperator<R> reduce(ReduceFunction<T> reduceFunction, WindowFunction<T, R, K, W> function) {
-		return reduce(reduceFunction, function, LegacyWindowOperatorType.NONE);
+		return reduce(function, new PassThroughWindowFunction<K, W, T>());
 	}
 
 	/**
@@ -265,39 +230,15 @@ public class WindowedStream<T, K, W extends Window> {
 	 *
 	 * @param reduceFunction The reduce function that is used for incremental aggregation.
 	 * @param function The window function.
-	 * @param resultType Type information for the result type of the window function
 	 * @return The data stream that is the result of applying the window function to the window.
 	 */
-	@PublicEvolving
 	public <R> SingleOutputStreamOperator<R> reduce(
-		ReduceFunction<T> reduceFunction,
-		WindowFunction<T, R, K, W> function,
-		TypeInformation<R> resultType) {
-		return reduce(reduceFunction, function, resultType, LegacyWindowOperatorType.NONE);
-	}
-
-	/**
-	 * Applies the given window function to each window. The window function is called for each
-	 * evaluation of the window for each key individually. The output of the window function is
-	 * interpreted as a regular non-windowed stream.
-	 *
-	 * <p>Arriving data is incrementally aggregated using the given reducer.
-	 *
-	 * @param reduceFunction The reduce function that is used for incremental aggregation.
-	 * @param function The window function.
-	 * @param legacyWindowOpType When migrating from an older Flink version, this flag indicates
-	 *                           the type of the previous operator whose state we inherit.
-	 * @return The data stream that is the result of applying the window function to the window.
-	 */
-	private <R> SingleOutputStreamOperator<R> reduce(
 			ReduceFunction<T> reduceFunction,
-			WindowFunction<T, R, K, W> function,
-			LegacyWindowOperatorType legacyWindowOpType) {
+			WindowFunction<T, R, K, W> function) {
 
 		TypeInformation<T> inType = input.getType();
 		TypeInformation<R> resultType = getWindowFunctionReturnType(function, inType);
-
-		return reduce(reduceFunction, function, resultType, legacyWindowOpType);
+		return reduce(reduceFunction, function, resultType);
 	}
 
 	/**
@@ -310,15 +251,12 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param reduceFunction The reduce function that is used for incremental aggregation.
 	 * @param function The window function.
 	 * @param resultType Type information for the result type of the window function.
-	 * @param legacyWindowOpType When migrating from an older Flink version, this flag indicates
-	 *                           the type of the previous operator whose state we inherit.
 	 * @return The data stream that is the result of applying the window function to the window.
 	 */
-	private <R> SingleOutputStreamOperator<R> reduce(
+	public <R> SingleOutputStreamOperator<R> reduce(
 			ReduceFunction<T> reduceFunction,
 			WindowFunction<T, R, K, W> function,
-			TypeInformation<R> resultType,
-			LegacyWindowOperatorType legacyWindowOpType) {
+			TypeInformation<R> resultType) {
 
 		if (reduceFunction instanceof RichFunction) {
 			throw new UnsupportedOperationException("ReduceFunction of reduce can not be a RichFunction.");
@@ -374,8 +312,7 @@ public class WindowedStream<T, K, W extends Window> {
 					new InternalSingleValueWindowFunction<>(function),
 					trigger,
 					allowedLateness,
-					lateDataOutputTag,
-					legacyWindowOpType);
+					lateDataOutputTag);
 		}
 
 		return input.transform(opName, resultType, operator);
@@ -1183,12 +1120,6 @@ public class WindowedStream<T, K, W extends Window> {
 
 		String udfName = "WindowedStream." + callLocation;
 
-		SingleOutputStreamOperator<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
-		if (result != null) {
-			return result;
-		}
-
-		LegacyWindowOperatorType legacyWindowOpType = getLegacyWindowType(function);
 		String opName;
 		KeySelector<T, K> keySel = input.getKeySelector();
 
@@ -1231,8 +1162,7 @@ public class WindowedStream<T, K, W extends Window> {
 					function,
 					trigger,
 					allowedLateness,
-					lateDataOutputTag,
-					legacyWindowOpType);
+					lateDataOutputTag);
 		}
 
 		return input.transform(opName, resultType, operator);
@@ -1629,119 +1559,6 @@ public class WindowedStream<T, K, W extends Window> {
 		return reduce(aggregator);
 	}
 
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	private LegacyWindowOperatorType getLegacyWindowType(Function function) {
-		if (windowAssigner instanceof SlidingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
-			if (function instanceof ReduceFunction) {
-				return LegacyWindowOperatorType.FAST_AGGREGATING;
-			} else if (function instanceof WindowFunction) {
-				return LegacyWindowOperatorType.FAST_ACCUMULATING;
-			}
-		} else if (windowAssigner instanceof TumblingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
-			if (function instanceof ReduceFunction) {
-				return LegacyWindowOperatorType.FAST_AGGREGATING;
-			} else if (function instanceof WindowFunction) {
-				return LegacyWindowOperatorType.FAST_ACCUMULATING;
-			}
-		}
-		return LegacyWindowOperatorType.NONE;
-	}
-
-	private <R> SingleOutputStreamOperator<R> createFastTimeOperatorIfValid(
-			ReduceFunction<?> function,
-			TypeInformation<R> resultType,
-			String functionName) {
-
-		if (windowAssigner.getClass() == SlidingAlignedProcessingTimeWindows.class && trigger == null && evictor == null) {
-			SlidingAlignedProcessingTimeWindows timeWindows = (SlidingAlignedProcessingTimeWindows) windowAssigner;
-			final long windowLength = timeWindows.getSize();
-			final long windowSlide = timeWindows.getSlide();
-
-			String opName = "Fast " + timeWindows + " of " + functionName;
-
-			@SuppressWarnings("unchecked")
-			ReduceFunction<T> reducer = (ReduceFunction<T>) function;
-
-			@SuppressWarnings("unchecked")
-			OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
-					new AggregatingProcessingTimeWindowOperator<>(
-							reducer, input.getKeySelector(),
-							input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-							input.getType().createSerializer(getExecutionEnvironment().getConfig()),
-							windowLength, windowSlide);
-			return input.transform(opName, resultType, op);
-
-		} else if (windowAssigner.getClass() == TumblingAlignedProcessingTimeWindows.class && trigger == null && evictor == null) {
-			TumblingAlignedProcessingTimeWindows timeWindows = (TumblingAlignedProcessingTimeWindows) windowAssigner;
-			final long windowLength = timeWindows.getSize();
-			final long windowSlide = timeWindows.getSize();
-
-			String opName = "Fast " + timeWindows + " of " + functionName;
-
-			@SuppressWarnings("unchecked")
-			ReduceFunction<T> reducer = (ReduceFunction<T>) function;
-
-			@SuppressWarnings("unchecked")
-			OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
-					new AggregatingProcessingTimeWindowOperator<>(
-							reducer,
-							input.getKeySelector(),
-							input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-							input.getType().createSerializer(getExecutionEnvironment().getConfig()),
-							windowLength, windowSlide);
-			return input.transform(opName, resultType, op);
-		}
-
-		return null;
-	}
-
-	private <R> SingleOutputStreamOperator<R> createFastTimeOperatorIfValid(
-			InternalWindowFunction<Iterable<T>, R, K, W> function,
-			TypeInformation<R> resultType,
-			String functionName) {
-
-		if (windowAssigner.getClass() == SlidingAlignedProcessingTimeWindows.class && trigger == null && evictor == null) {
-			SlidingAlignedProcessingTimeWindows timeWindows = (SlidingAlignedProcessingTimeWindows) windowAssigner;
-			final long windowLength = timeWindows.getSize();
-			final long windowSlide = timeWindows.getSlide();
-
-			String opName = "Fast " + timeWindows + " of " + functionName;
-
-			@SuppressWarnings("unchecked")
-			InternalWindowFunction<Iterable<T>, R, K, TimeWindow> timeWindowFunction =
-					(InternalWindowFunction<Iterable<T>, R, K, TimeWindow>) function;
-
-			OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
-					timeWindowFunction, input.getKeySelector(),
-					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-					input.getType().createSerializer(getExecutionEnvironment().getConfig()),
-					windowLength, windowSlide);
-			return input.transform(opName, resultType, op);
-		} else if (windowAssigner.getClass() == TumblingAlignedProcessingTimeWindows.class && trigger == null && evictor == null) {
-			TumblingAlignedProcessingTimeWindows timeWindows = (TumblingAlignedProcessingTimeWindows) windowAssigner;
-			final long windowLength = timeWindows.getSize();
-			final long windowSlide = timeWindows.getSize();
-
-			String opName = "Fast " + timeWindows + " of " + functionName;
-
-			@SuppressWarnings("unchecked")
-			InternalWindowFunction<Iterable<T>, R, K, TimeWindow> timeWindowFunction =
-					(InternalWindowFunction<Iterable<T>, R, K, TimeWindow>) function;
-
-			OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
-					timeWindowFunction, input.getKeySelector(),
-					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-					input.getType().createSerializer(getExecutionEnvironment().getConfig()),
-					windowLength, windowSlide);
-			return input.transform(opName, resultType, op);
-		}
-
-		return null;
-	}
-
 	public StreamExecutionEnvironment getExecutionEnvironment() {
 		return input.getExecutionEnvironment();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java
index 3c4cfbd..fedd791 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java
@@ -32,7 +32,6 @@ import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
 import org.apache.flink.util.Preconditions;
 
 import org.slf4j.Logger;
@@ -66,7 +65,7 @@ import java.util.TreeMap;
  */
 @Internal
 public class ContinuousFileMonitoringFunction<OUT>
-	extends RichSourceFunction<TimestampedFileInputSplit> implements CheckpointedFunction, CheckpointedRestoring<Long> {
+	extends RichSourceFunction<TimestampedFileInputSplit> implements CheckpointedFunction {
 
 	private static final long serialVersionUID = 1L;
 
@@ -375,12 +374,4 @@ public class ContinuousFileMonitoringFunction<OUT>
 			LOG.debug("{} checkpointed {}.", getClass().getSimpleName(), globalModificationTime);
 		}
 	}
-
-	@Override
-	public void restoreState(Long state) throws Exception {
-		this.globalModificationTime = state;
-
-		LOG.info("{} (taskIdx={}) restored global modification time from an older Flink version: {}",
-			getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), globalModificationTime);
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java
index 3a9e8e1..e14cfda 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java
@@ -25,30 +25,23 @@ import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.metrics.Counter;
 import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.StateSnapshotContext;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.CheckpointedRestoringOperator;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
 import org.apache.flink.streaming.api.operators.StreamSourceContexts;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Preconditions;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.io.ObjectInputStream;
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.PriorityQueue;
 import java.util.Queue;
@@ -60,15 +53,15 @@ import static org.apache.flink.util.Preconditions.checkState;
  * The operator that reads the {@link TimestampedFileInputSplit splits} received from the preceding
  * {@link ContinuousFileMonitoringFunction}. Contrary to the {@link ContinuousFileMonitoringFunction}
  * which has a parallelism of 1, this operator can have DOP > 1.
- * <p/>
- * As soon as a split descriptor is received, it is put in a queue, and have another
+ *
+ * <p>As soon as a split descriptor is received, it is put in a queue, and have another
  * thread read the actual data of the split. This architecture allows the separation of the
  * reading thread from the one emitting the checkpoint barriers, thus removing any potential
  * back-pressure.
  */
 @Internal
 public class ContinuousFileReaderOperator<OUT> extends AbstractStreamOperator<OUT>
-	implements OneInputStreamOperator<TimestampedFileInputSplit, OUT>, OutputTypeConfigurable<OUT>, CheckpointedRestoringOperator {
+	implements OneInputStreamOperator<TimestampedFileInputSplit, OUT>, OutputTypeConfigurable<OUT> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -422,83 +415,4 @@ public class ContinuousFileReaderOperator<OUT> extends AbstractStreamOperator<OU
 				getClass().getSimpleName(), subtaskIdx, readerState.size(), readerState);
 		}
 	}
-
-	// ------------------------------------------------------------------------
-	//  Restoring / Migrating from an older Flink version.
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void restoreState(FSDataInputStream in) throws Exception {
-
-		LOG.info("{} (taskIdx={}) restoring state from an older Flink version.",
-			getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask());
-
-		// this is just to read the byte indicating if we have udf state or not
-		int hasUdfState = in.read();
-
-		Preconditions.checkArgument(hasUdfState == 0);
-
-		final ObjectInputStream ois = new ObjectInputStream(in);
-		final DataInputViewStreamWrapper div = new DataInputViewStreamWrapper(in);
-
-		// read the split that was being read
-		FileInputSplit currSplit = (FileInputSplit) ois.readObject();
-
-		// read the pending splits list
-		List<FileInputSplit> pendingSplits = new LinkedList<>();
-		int noOfSplits = div.readInt();
-		for (int i = 0; i < noOfSplits; i++) {
-			FileInputSplit split = (FileInputSplit) ois.readObject();
-			pendingSplits.add(split);
-		}
-
-		// read the state of the format
-		Serializable formatState = (Serializable) ois.readObject();
-
-		div.close();
-
-		if (restoredReaderState == null) {
-			restoredReaderState = new ArrayList<>();
-		}
-
-		// we do not know the modification time of the retrieved splits, so we assign them
-		// artificial ones, with the only constraint that they respect the relative order of the
-		// retrieved splits, because modification time is going to be used to sort the splits within
-		// the "pending splits" priority queue.
-
-		long now = getProcessingTimeService().getCurrentProcessingTime();
-		long runningModTime = Math.max(now, noOfSplits + 1);
-
-		TimestampedFileInputSplit currentSplit = createTimestampedFileSplit(currSplit, --runningModTime, formatState);
-		restoredReaderState.add(currentSplit);
-		for (FileInputSplit split : pendingSplits) {
-			TimestampedFileInputSplit timestampedSplit = createTimestampedFileSplit(split, --runningModTime);
-			restoredReaderState.add(timestampedSplit);
-		}
-
-		if (LOG.isDebugEnabled()) {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("{} (taskIdx={}) restored {} splits from legacy: {}.",
-					getClass().getSimpleName(),
-					getRuntimeContext().getIndexOfThisSubtask(),
-					restoredReaderState.size(),
-					restoredReaderState);
-			}
-		}
-	}
-
-	private TimestampedFileInputSplit createTimestampedFileSplit(FileInputSplit split, long modificationTime) {
-		return createTimestampedFileSplit(split, modificationTime, null);
-	}
-
-	private TimestampedFileInputSplit createTimestampedFileSplit(FileInputSplit split, long modificationTime, Serializable state) {
-		TimestampedFileInputSplit timestampedSplit = new TimestampedFileInputSplit(
-			modificationTime, split.getSplitNumber(), split.getPath(),
-			split.getStart(), split.getLength(), split.getHostnames());
-
-		if (state != null) {
-			timestampedSplit.setSplitState(state);
-		}
-		return timestampedSplit;
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
index abaa74e..884b899 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
@@ -26,7 +26,6 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.migration.streaming.api.graph.StreamGraphHasherV1;
 import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
@@ -116,7 +115,7 @@ public class StreamingJobGraphGenerator {
 	private StreamingJobGraphGenerator(StreamGraph streamGraph) {
 		this.streamGraph = streamGraph;
 		this.defaultStreamGraphHasher = new StreamGraphHasherV2();
-		this.legacyStreamGraphHashers = Arrays.asList(new StreamGraphHasherV1(), new StreamGraphUserHashHasher());
+		this.legacyStreamGraphHashers = Arrays.asList(new StreamGraphUserHashHasher());
 
 		this.jobVertices = new HashMap<>();
 		this.builtVertices = new HashSet<>();
@@ -241,14 +240,14 @@ public class StreamingJobGraphGenerator {
 				createChain(nonChainable.getTargetId(), nonChainable.getTargetId(), hashes, legacyHashes, 0, chainedOperatorHashes);
 			}
 
-			List<Tuple2<byte[], byte[]>> operatorHashes = chainedOperatorHashes.get(startNodeId);
-			if (operatorHashes == null) {
-				operatorHashes = new ArrayList<>();
-				chainedOperatorHashes.put(startNodeId, operatorHashes);
-			}
+			List<Tuple2<byte[], byte[]>> operatorHashes =
+				chainedOperatorHashes.computeIfAbsent(startNodeId, k -> new ArrayList<>());
 
 			byte[] primaryHashBytes = hashes.get(currentNodeId);
-			operatorHashes.add(new Tuple2<>(primaryHashBytes, legacyHashes.get(1).get(currentNodeId)));
+
+			for (Map<Integer, byte[]> legacyHash : legacyHashes) {
+				operatorHashes.add(new Tuple2<>(primaryHashBytes, legacyHash.get(currentNodeId)));
+			}
 
 			chainedNames.put(currentNodeId, createChainedName(currentNodeId, chainableOutputs));
 			chainedMinResources.put(currentNodeId, createChainedMinResources(currentNodeId, chainableOutputs));

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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 a72b9fe..a28fc30 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
@@ -28,7 +28,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.MetricOptions;
-import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.metrics.Counter;
@@ -55,7 +54,6 @@ import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.StateInitializationContextImpl;
 import org.apache.flink.runtime.state.StateSnapshotContext;
 import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
-import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.streaming.api.graph.StreamConfig;
@@ -251,42 +249,6 @@ public abstract class AbstractStreamOperator<OUT>
 				getContainingTask().getCancelables()); // access to register streams for canceling
 
 		initializeState(initializationContext);
-
-		if (restoring) {
-
-			// finally restore the legacy state in case we are
-			// migrating from a previous Flink version.
-
-			restoreStreamCheckpointed(stateHandles);
-		}
-	}
-
-	/**
-	 * @deprecated Non-repartitionable operator state that has been deprecated.
-	 * Can be removed when we remove the APIs for non-repartitionable operator state.
-	 */
-	@Deprecated
-	private void restoreStreamCheckpointed(OperatorSubtaskState stateHandles) throws Exception {
-		StreamStateHandle state = stateHandles.getLegacyOperatorState();
-		if (null != state) {
-			if (this instanceof CheckpointedRestoringOperator) {
-
-				LOG.debug("Restore state of task {} in operator with id ({}).",
-					getContainingTask().getName(), getOperatorID());
-
-				FSDataInputStream is = state.openInputStream();
-				try {
-					getContainingTask().getCancelables().registerClosable(is);
-					((CheckpointedRestoringOperator) this).restoreState(is);
-				} finally {
-					getContainingTask().getCancelables().unregisterClosable(is);
-					is.close();
-				}
-			} else {
-				throw new Exception(
-						"Found legacy operator state for operator that does not implement StreamCheckpointedOperator.");
-			}
-		}
 	}
 
 	/**
@@ -451,35 +413,6 @@ public abstract class AbstractStreamOperator<OUT>
 	}
 
 	/**
-	 * @deprecated Non-repartitionable operator state that has been deprecated.
-	 * Can be removed when we remove the APIs for non-repartitionable operator state.
-	 */
-	@SuppressWarnings("deprecation")
-	@Deprecated
-	@Override
-	public StreamStateHandle snapshotLegacyOperatorState(long checkpointId, long timestamp, CheckpointOptions checkpointOptions) throws Exception {
-		if (this instanceof StreamCheckpointedOperator) {
-			CheckpointStreamFactory factory = getCheckpointStreamFactory(checkpointOptions);
-
-			final CheckpointStreamFactory.CheckpointStateOutputStream outStream =
-				factory.createCheckpointStateOutputStream(checkpointId, timestamp);
-
-			getContainingTask().getCancelables().registerClosable(outStream);
-
-			try {
-				((StreamCheckpointedOperator) this).snapshotState(outStream, checkpointId, timestamp);
-				return outStream.closeAndGetHandle();
-			}
-			finally {
-				getContainingTask().getCancelables().unregisterClosable(outStream);
-				outStream.close();
-			}
-		} else {
-			return null;
-		}
-	}
-
-	/**
 	 * Stream operators with state which can be restored need to override this hook method.
 	 *
 	 * @param context context that allows to register different states.


[06/11] flink git commit: [FLINK-7461] Remove Backwards compatibility with <= Flink 1.1

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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 efbebf4..329ce18 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
@@ -24,23 +24,15 @@ import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.util.FunctionUtils;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.StateSnapshotContext;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
 import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
 import org.apache.flink.streaming.util.functions.StreamingFunctionUtils;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.Migration;
-
-import java.io.Serializable;
 
 import static java.util.Objects.requireNonNull;
 
@@ -57,8 +49,7 @@ import static java.util.Objects.requireNonNull;
 @PublicEvolving
 public abstract class AbstractUdfStreamOperator<OUT, F extends Function>
 		extends AbstractStreamOperator<OUT>
-		implements OutputTypeConfigurable<OUT>,
-		StreamCheckpointedOperator {
+		implements OutputTypeConfigurable<OUT> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -132,59 +123,6 @@ public abstract class AbstractUdfStreamOperator<OUT, F extends Function>
 	// ------------------------------------------------------------------------
 
 	@Override
-	public void snapshotState(FSDataOutputStream out, long checkpointId, long timestamp) throws Exception {
-		if (userFunction instanceof Checkpointed) {
-			@SuppressWarnings("unchecked")
-			Checkpointed<Serializable> chkFunction = (Checkpointed<Serializable>) userFunction;
-
-			Serializable udfState;
-			try {
-				udfState = chkFunction.snapshotState(checkpointId, timestamp);
-				if (udfState != null) {
-					out.write(1);
-					InstantiationUtil.serializeObject(out, udfState);
-				} else {
-					out.write(0);
-				}
-			} catch (Exception e) {
-				throw new Exception("Failed to draw state snapshot from function: " + e.getMessage(), e);
-			}
-		}
-	}
-
-	@Override
-	public void restoreState(FSDataInputStream in) throws Exception {
-		boolean haveReadUdfStateFlag = false;
-		if (userFunction instanceof Checkpointed ||
-				(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());
-				if (functionState != null) {
-					try {
-						chkFunction.restoreState(functionState);
-					} catch (Exception e) {
-						throw new Exception("Failed to restore state to function: " + e.getMessage(), e);
-					}
-				}
-			}
-		}
-
-		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) {
-				throw new Exception("Found UDF state but operator is not instance of CheckpointedRestoring");
-			}
-		}
-	}
-
-	@Override
 	public void notifyOfCompletedCheckpoint(long checkpointId) throws Exception {
 		super.notifyOfCompletedCheckpoint(checkpointId);
 
@@ -219,23 +157,11 @@ public abstract class AbstractUdfStreamOperator<OUT, F extends Function>
 
 	private void checkUdfCheckpointingPreconditions() {
 
-		boolean newCheckpointInferface = false;
-
-		if (userFunction instanceof CheckpointedFunction) {
-			newCheckpointInferface = true;
-		}
-
-		if (userFunction instanceof ListCheckpointed) {
-			if (newCheckpointInferface) {
-				throw new IllegalStateException("User functions are not allowed to implement " +
-						"CheckpointedFunction AND ListCheckpointed.");
-			}
-			newCheckpointInferface = true;
-		}
+		if (userFunction instanceof CheckpointedFunction
+			&& userFunction instanceof ListCheckpointed) {
 
-		if (newCheckpointInferface && userFunction instanceof Checkpointed) {
-			throw new IllegalStateException("User functions are not allowed to implement Checkpointed AND " +
-					"CheckpointedFunction/ListCheckpointed.");
+			throw new IllegalStateException("User functions are not allowed to implement " +
+				"CheckpointedFunction AND ListCheckpointed.");
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CheckpointedRestoringOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CheckpointedRestoringOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CheckpointedRestoringOperator.java
deleted file mode 100644
index 33304e4..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CheckpointedRestoringOperator.java
+++ /dev/null
@@ -1,50 +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 org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-
-/**
- * Interface for {@link StreamOperator StreamOperators} that can restore from a Flink 1.1
- * legacy snapshot that was done using the {@link StreamCheckpointedOperator} interface.
- *
- * @deprecated {@link Checkpointed} has been deprecated as well. This class can be
- * removed when we remove that interface.
- */
-@Deprecated
-public interface CheckpointedRestoringOperator {
-
-	/**
-	 * Restores the operator state, if this operator's execution is recovering from a checkpoint.
-	 * This method restores the operator state (if the operator is stateful) and the key/value state
-	 * (if it had been used and was initialized when the snapshot occurred).
-	 *
-	 * <p>This method is called after {@link StreamOperator#setup(StreamTask, StreamConfig, Output)}
-	 * and before {@link StreamOperator#open()}.
-	 *
-	 * @param in The stream from which we have to restore our state.
-	 *
-	 * @throws Exception Exceptions during state restore should be forwarded, so that the system can
-	 *                   properly react to failed state restore and fail the execution attempt.
-	 */
-	void restoreState(FSDataInputStream in) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamCheckpointedOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamCheckpointedOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamCheckpointedOperator.java
deleted file mode 100644
index 986e2b7..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamCheckpointedOperator.java
+++ /dev/null
@@ -1,43 +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 org.apache.flink.core.fs.FSDataOutputStream;
-
-/**
- * @deprecated This interface is deprecated without replacement.
- * All operators are now checkpointed.
- */
-@Deprecated
-public interface StreamCheckpointedOperator extends CheckpointedRestoringOperator {
-
-	/**
-	 * Called to draw a state snapshot from the operator. This method snapshots the operator state
-	 * (if the operator is stateful).
-	 *
-	 * @param out The stream to which we have to write our state.
-	 * @param checkpointId The ID of the checkpoint.
-	 * @param timestamp The timestamp of the checkpoint.
-	 *
-	 * @throws Exception Forwards exceptions that occur while drawing snapshots from the operator
-	 *                   and the key/value state.
-	 */
-	void snapshotState(FSDataOutputStream out, long checkpointId, long timestamp) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
index 9d5e02b..38b4aee 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
@@ -22,7 +22,6 @@ import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
 import org.apache.flink.runtime.jobgraph.OperatorID;
-import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
@@ -105,21 +104,6 @@ public interface StreamOperator<OUT> extends Serializable {
 		CheckpointOptions checkpointOptions) throws Exception;
 
 	/**
-	 * Takes a snapshot of the legacy operator state defined via {@link StreamCheckpointedOperator}.
-	 *
-	 * @return The handle to the legacy operator state, or null, if no state was snapshotted.
-	 * @throws Exception This method should forward any type of exception that happens during snapshotting.
-	 *
-	 * @deprecated This method will be removed as soon as no more operators use the legacy state code paths
-	 */
-	@SuppressWarnings("deprecation")
-	@Deprecated
-	StreamStateHandle snapshotLegacyOperatorState(
-		long checkpointId,
-		long timestamp,
-		CheckpointOptions checkpointOptions) throws Exception;
-
-	/**
 	 * Provides state handles to restore the operator state.
 	 *
 	 * @param stateHandles state handles to the operator state.

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingAlignedProcessingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingAlignedProcessingTimeWindows.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingAlignedProcessingTimeWindows.java
deleted file mode 100644
index 252f997..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingAlignedProcessingTimeWindows.java
+++ /dev/null
@@ -1,68 +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.windowing.assigners;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.streaming.api.windowing.time.Time;
-
-/**
- * This is a special window assigner used to tell the system to use the
- * <i>"Fast Aligned Processing Time Window Operator"</i> for windowing.
- *
- * <p>Prior Flink versions used that operator automatically for simple processing time
- * windows (tumbling and sliding) when no custom trigger and no evictor was specified.
- * In the current Flink version, that operator is only used when programs explicitly
- * specify this window assigner. This is only intended for special cases where programs relied on
- * the better performance of the fast aligned window operator, and are willing to accept the lack
- * of support for various features as indicated below:
- *
- * <ul>
- *     <li>No custom state backend can be selected, the operator always stores data on the Java heap.</li>
- *     <li>The operator does not support key groups, meaning it cannot change the parallelism.</li>
- *     <li>Future versions of Flink may not be able to resume from checkpoints/savepoints taken by this
- *         operator.</li>
- * </ul>
- *
- * <p>Future implementation plans: We plan to add some of the optimizations used by this operator to
- * the general window operator, so that future versions of Flink will not have the performance/functionality
- * trade-off any more.
- *
- * <p>Note on implementation: The concrete operator instantiated by this assigner is either the
- * {@link org.apache.flink.streaming.runtime.operators.windowing.AggregatingProcessingTimeWindowOperator}
- * or {@link org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator}.
- */
-@PublicEvolving
-public final class TumblingAlignedProcessingTimeWindows extends BaseAlignedWindowAssigner {
-
-	private static final long serialVersionUID = -6217477609512299842L;
-
-	public TumblingAlignedProcessingTimeWindows(long size) {
-		super(size);
-	}
-
-	/**
-	 * Creates a new {@code TumblingAlignedProcessingTimeWindows} {@link WindowAssigner} that assigns
-	 * elements to time windows based on the element timestamp.
-	 *
-	 * @param size The size of the generated windows.
-	 */
-	public static TumblingAlignedProcessingTimeWindows of(Time size) {
-		return new TumblingAlignedProcessingTimeWindows(size.toMilliseconds());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
deleted file mode 100644
index 83a7528..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
+++ /dev/null
@@ -1,331 +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.annotation.Internal;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-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.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.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
-import org.apache.flink.util.MathUtils;
-
-import org.apache.commons.math3.util.ArithmeticUtils;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Base class for special window operator implementation for windows that fire at the same time for
- * all keys.
- *
- * @deprecated Deprecated in favour of the generic {@link WindowOperator}. This was an
- * optimized implementation used for aligned windows.
- */
-@Internal
-@Deprecated
-public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT, STATE, F extends Function>
-		extends AbstractUdfStreamOperator<OUT, F>
-		implements OneInputStreamOperator<IN, OUT>, ProcessingTimeCallback {
-
-	private static final long serialVersionUID = 3245500864882459867L;
-
-	private static final long MIN_SLIDE_TIME = 50;
-
-	// ----- fields for operator parametrization -----
-
-	private final Function function;
-	private final KeySelector<IN, KEY> keySelector;
-
-	private final TypeSerializer<KEY> keySerializer;
-	private final TypeSerializer<STATE> stateTypeSerializer;
-
-	private final long windowSize;
-	private final long windowSlide;
-	private final long paneSize;
-	private final int numPanesPerWindow;
-
-	// ----- fields for operator functionality -----
-
-	private transient AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes;
-
-	private transient TimestampedCollector<OUT> out;
-
-	private transient RestoredState<IN, KEY, STATE, OUT> restoredState;
-
-	private transient long nextEvaluationTime;
-	private transient long nextSlideTime;
-
-	protected AbstractAlignedProcessingTimeWindowOperator(
-			F function,
-			KeySelector<IN, KEY> keySelector,
-			TypeSerializer<KEY> keySerializer,
-			TypeSerializer<STATE> stateTypeSerializer,
-			long windowLength,
-			long windowSlide) {
-		super(function);
-
-		if (windowLength < MIN_SLIDE_TIME) {
-			throw new IllegalArgumentException("Window length must be at least " + MIN_SLIDE_TIME + " msecs");
-		}
-		if (windowSlide < MIN_SLIDE_TIME) {
-			throw new IllegalArgumentException("Window slide must be at least " + MIN_SLIDE_TIME + " msecs");
-		}
-		if (windowLength < windowSlide) {
-			throw new IllegalArgumentException("The window size must be larger than the window slide");
-		}
-
-		final long paneSlide = ArithmeticUtils.gcd(windowLength, windowSlide);
-		if (paneSlide < MIN_SLIDE_TIME) {
-			throw new IllegalArgumentException(String.format(
-					"Cannot compute window of size %d msecs sliding by %d msecs. " +
-							"The unit of grouping is too small: %d msecs", windowLength, windowSlide, paneSlide));
-		}
-
-		this.function = requireNonNull(function);
-		this.keySelector = requireNonNull(keySelector);
-		this.keySerializer = requireNonNull(keySerializer);
-		this.stateTypeSerializer = requireNonNull(stateTypeSerializer);
-		this.windowSize = windowLength;
-		this.windowSlide = windowSlide;
-		this.paneSize = paneSlide;
-		this.numPanesPerWindow = MathUtils.checkedDownCast(windowLength / paneSlide);
-	}
-
-	protected abstract AbstractKeyedTimePanes<IN, KEY, STATE, OUT> createPanes(
-			KeySelector<IN, KEY> keySelector, Function function);
-
-	// ------------------------------------------------------------------------
-	//  startup and shutdown
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void open() throws Exception {
-		super.open();
-
-		out = new TimestampedCollector<>(output);
-
-		// decide when to first compute the window and when to slide it
-		// the values should align with the start of time (that is, the UNIX epoch, not the big bang)
-		final long now = getProcessingTimeService().getCurrentProcessingTime();
-		nextEvaluationTime = now + windowSlide - (now % windowSlide);
-		nextSlideTime = now + paneSize - (now % paneSize);
-
-		final long firstTriggerTime = Math.min(nextEvaluationTime, nextSlideTime);
-
-		// check if we restored state and if we need to fire some windows based on that restored state
-		if (restoredState == null) {
-			// initial empty state: create empty panes that gather the elements per slide
-			panes = createPanes(keySelector, function);
-		}
-		else {
-			// restored state
-			panes = restoredState.panes;
-
-			long nextPastEvaluationTime = restoredState.nextEvaluationTime;
-			long nextPastSlideTime = restoredState.nextSlideTime;
-			long nextPastTriggerTime = Math.min(nextPastEvaluationTime, nextPastSlideTime);
-			int numPanesRestored = panes.getNumPanes();
-
-			// fire windows from the past as long as there are more panes with data and as long
-			// as the missed trigger times have not caught up with the presence
-			while (numPanesRestored > 0 && nextPastTriggerTime < firstTriggerTime) {
-				// evaluate the window from the past
-				if (nextPastTriggerTime == nextPastEvaluationTime) {
-					computeWindow(nextPastTriggerTime);
-					nextPastEvaluationTime += windowSlide;
-				}
-
-				// evaluate slide from the past
-				if (nextPastTriggerTime == nextPastSlideTime) {
-					panes.slidePanes(numPanesPerWindow);
-					numPanesRestored--;
-					nextPastSlideTime += paneSize;
-				}
-
-				nextPastTriggerTime = Math.min(nextPastEvaluationTime, nextPastSlideTime);
-			}
-		}
-
-		// make sure the first window happens
-		getProcessingTimeService().registerTimer(firstTriggerTime, this);
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-
-		// early stop the triggering thread, so it does not attempt to return any more data
-		stopTriggers();
-	}
-
-	@Override
-	public void dispose() throws Exception {
-		super.dispose();
-
-		// acquire the lock during shutdown, to prevent trigger calls at the same time
-		// fail-safe stop of the triggering thread (in case of an error)
-		stopTriggers();
-
-		// release the panes. panes may still be null if dispose is called
-		// after open() failed
-		if (panes != null) {
-			panes.dispose();
-		}
-	}
-
-	private void stopTriggers() {
-		// reset the action timestamps. this makes sure any pending triggers will not evaluate
-		nextEvaluationTime = -1L;
-		nextSlideTime = -1L;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Receiving elements and triggers
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void processElement(StreamRecord<IN> element) throws Exception {
-		panes.addElementToLatestPane(element.getValue());
-	}
-
-	@Override
-	public void onProcessingTime(long timestamp) throws Exception {
-		// first we check if we actually trigger the window function
-		if (timestamp == nextEvaluationTime) {
-			// compute and output the results
-			computeWindow(timestamp);
-
-			nextEvaluationTime += windowSlide;
-		}
-
-		// check if we slide the panes by one. this may happen in addition to the
-		// window computation, or just by itself
-		if (timestamp == nextSlideTime) {
-			panes.slidePanes(numPanesPerWindow);
-			nextSlideTime += paneSize;
-		}
-
-		long nextTriggerTime = Math.min(nextEvaluationTime, nextSlideTime);
-		getProcessingTimeService().registerTimer(nextTriggerTime, this);
-	}
-
-	private void computeWindow(long timestamp) throws Exception {
-		out.setAbsoluteTimestamp(timestamp);
-		panes.truncatePanes(numPanesPerWindow);
-		panes.evaluateWindow(out, new TimeWindow(timestamp - windowSize, timestamp), this);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Checkpointing
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void snapshotState(FSDataOutputStream out, long checkpointId, long timestamp) throws Exception {
-		super.snapshotState(out, checkpointId, timestamp);
-
-		// we write the panes with the key/value maps into the stream, as well as when this state
-		// should have triggered and slided
-
-		DataOutputViewStreamWrapper outView = new DataOutputViewStreamWrapper(out);
-
-		outView.writeLong(nextEvaluationTime);
-		outView.writeLong(nextSlideTime);
-
-		panes.writeToOutput(outView, keySerializer, stateTypeSerializer);
-
-		outView.flush();
-	}
-
-	@Override
-	public void restoreState(FSDataInputStream in) throws Exception {
-		super.restoreState(in);
-
-		DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(in);
-
-		final long nextEvaluationTime = inView.readLong();
-		final long nextSlideTime = inView.readLong();
-
-		AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes = createPanes(keySelector, function);
-
-		panes.readFromInput(inView, keySerializer, stateTypeSerializer);
-
-		restoredState = new RestoredState<>(panes, nextEvaluationTime, nextSlideTime);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Property access (for testing)
-	// ------------------------------------------------------------------------
-
-	public long getWindowSize() {
-		return windowSize;
-	}
-
-	public long getWindowSlide() {
-		return windowSlide;
-	}
-
-	public long getPaneSize() {
-		return paneSize;
-	}
-
-	public int getNumPanesPerWindow() {
-		return numPanesPerWindow;
-	}
-
-	public long getNextEvaluationTime() {
-		return nextEvaluationTime;
-	}
-
-	public long getNextSlideTime() {
-		return nextSlideTime;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	@Override
-	public String toString() {
-		return "Window (processing time) (length=" + windowSize + ", slide=" + windowSlide + ')';
-	}
-
-	// ------------------------------------------------------------------------
-	// ------------------------------------------------------------------------
-
-	private static final class RestoredState<IN, KEY, STATE, OUT> {
-
-		final AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes;
-		final long nextEvaluationTime;
-		final long nextSlideTime;
-
-		RestoredState(AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes, long nextEvaluationTime, long nextSlideTime) {
-			this.panes = panes;
-			this.nextEvaluationTime = nextEvaluationTime;
-			this.nextSlideTime = nextSlideTime;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
deleted file mode 100644
index d67121a..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.runtime.state.ArrayListSerializer;
-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.InternalWindowFunction;
-
-import java.util.ArrayList;
-
-/**
- * Special window operator implementation for windows that fire at the same time for all keys with
- * accumulating windows.
- *
- * @deprecated Deprecated in favour of the generic {@link WindowOperator}. This was an
- * optimized implementation used for aligned windows.
- */
-@Internal
-@Deprecated
-public class AccumulatingProcessingTimeWindowOperator<KEY, IN, OUT>
-		extends AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT, ArrayList<IN>, InternalWindowFunction<Iterable<IN>, OUT, KEY, TimeWindow>> {
-
-	private static final long serialVersionUID = 7305948082830843475L;
-
-	public AccumulatingProcessingTimeWindowOperator(
-			InternalWindowFunction<Iterable<IN>, OUT, KEY, TimeWindow> function,
-			KeySelector<IN, KEY> keySelector,
-			TypeSerializer<KEY> keySerializer,
-			TypeSerializer<IN> valueSerializer,
-			long windowLength,
-			long windowSlide) {
-		super(function, keySelector, keySerializer,
-				new ArrayListSerializer<>(valueSerializer), windowLength, windowSlide);
-	}
-
-	@Override
-	protected AccumulatingKeyedTimePanes<IN, KEY, OUT> createPanes(KeySelector<IN, KEY> keySelector, Function function) {
-		@SuppressWarnings("unchecked")
-		InternalWindowFunction<Iterable<IN>, OUT, KEY, Window> windowFunction = (InternalWindowFunction<Iterable<IN>, OUT, KEY, Window>) function;
-
-		return new AccumulatingKeyedTimePanes<>(keySelector, windowFunction);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java
deleted file mode 100644
index 6747383..0000000
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java
+++ /dev/null
@@ -1,58 +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.annotation.Internal;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-
-/**
- * Special window operator implementation for windows that fire at the same time for all keys with
- * aggregating windows.
- *
- * @deprecated Deprecated in favour of the generic {@link WindowOperator}. This was an
- * optimized implementation used for aligned windows.
- */
-@Internal
-@Deprecated
-public class AggregatingProcessingTimeWindowOperator<KEY, IN>
-		extends AbstractAlignedProcessingTimeWindowOperator<KEY, IN, IN, IN, ReduceFunction<IN>> {
-
-	private static final long serialVersionUID = 7305948082830843475L;
-
-	public AggregatingProcessingTimeWindowOperator(
-			ReduceFunction<IN> function,
-			KeySelector<IN, KEY> keySelector,
-			TypeSerializer<KEY> keySerializer,
-			TypeSerializer<IN> aggregateSerializer,
-			long windowLength,
-			long windowSlide) {
-		super(function, keySelector, keySerializer, aggregateSerializer, windowLength, windowSlide);
-	}
-
-	@Override
-	protected AggregatingKeyedTimePanes<IN, KEY> createPanes(KeySelector<IN, KEY> keySelector, Function function) {
-		@SuppressWarnings("unchecked")
-		ReduceFunction<IN> windowFunction = (ReduceFunction<IN>) function;
-
-		return new AggregatingKeyedTimePanes<IN, KEY>(keySelector, windowFunction);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index 880907d..b14739f 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
@@ -41,17 +41,12 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.runtime.state.ArrayListSerializer;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.runtime.state.internal.InternalAppendingState;
 import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.runtime.state.internal.InternalMergingState;
-import org.apache.flink.streaming.api.datastream.LegacyWindowOperatorType;
 import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
 import org.apache.flink.streaming.api.operators.ChainingStrategy;
 import org.apache.flink.streaming.api.operators.InternalTimer;
@@ -61,8 +56,6 @@ import org.apache.flink.streaming.api.operators.TimestampedCollector;
 import org.apache.flink.streaming.api.operators.Triggerable;
 import org.apache.flink.streaming.api.windowing.assigners.BaseAlignedWindowAssigner;
 import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
 import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
 import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
@@ -70,16 +63,9 @@ import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.OutputTag;
-import org.apache.flink.util.Preconditions;
 
-import org.apache.commons.math3.util.ArithmeticUtils;
-
-import java.io.IOException;
 import java.io.Serializable;
 import java.util.Collection;
-import java.util.Comparator;
-import java.util.List;
-import java.util.PriorityQueue;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -180,34 +166,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 
 	protected transient InternalTimerService<W> internalTimerService;
 
-	// ------------------------------------------------------------------------
-	// State restored in case of migration from an older version (backwards compatibility)
-	// ------------------------------------------------------------------------
-
-	/**
-	 * A flag indicating if we are migrating from a regular {@link WindowOperator}
-	 * or one of the deprecated {@link AccumulatingProcessingTimeWindowOperator} and
-	 * {@link AggregatingProcessingTimeWindowOperator}.
-	 */
-	private final LegacyWindowOperatorType legacyWindowOperatorType;
-
-	/**
-	 * The elements restored when migrating from an older, deprecated
-	 * {@link AccumulatingProcessingTimeWindowOperator} or
-	 * {@link AggregatingProcessingTimeWindowOperator}. */
-	private transient PriorityQueue<StreamRecord<IN>> restoredFromLegacyAlignedOpRecords;
-
-	/**
-	 * The restored processing time timers when migrating from an
-	 * older version of the {@link WindowOperator}.
-	 */
-	private transient PriorityQueue<Timer<K, W>> restoredFromLegacyProcessingTimeTimers;
-
-	/** The restored event time timer when migrating from an
-	 * older version of the {@link WindowOperator}.
-	 */
-	private transient PriorityQueue<Timer<K, W>> restoredFromLegacyEventTimeTimers;
-
 	/**
 	 * Creates a new {@code WindowOperator} based on the given policies and user functions.
 	 */
@@ -222,25 +180,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 			long allowedLateness,
 			OutputTag<IN> lateDataOutputTag) {
 
-		this(windowAssigner, windowSerializer, keySelector, keySerializer,
-			windowStateDescriptor, windowFunction, trigger, allowedLateness, lateDataOutputTag, LegacyWindowOperatorType.NONE);
-	}
-
-	/**
-	 * Creates a new {@code WindowOperator} based on the given policies and user functions.
-	 */
-	public WindowOperator(
-			WindowAssigner<? super IN, W> windowAssigner,
-			TypeSerializer<W> windowSerializer,
-			KeySelector<IN, K> keySelector,
-			TypeSerializer<K> keySerializer,
-			StateDescriptor<? extends AppendingState<IN, ACC>, ?> windowStateDescriptor,
-			InternalWindowFunction<ACC, OUT, K, W> windowFunction,
-			Trigger<? super IN, ? super W> trigger,
-			long allowedLateness,
-			OutputTag<IN> lateDataOutputTag,
-			LegacyWindowOperatorType legacyWindowOperatorType) {
-
 		super(windowFunction);
 
 		checkArgument(!(windowAssigner instanceof BaseAlignedWindowAssigner),
@@ -261,7 +200,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		this.trigger = checkNotNull(trigger);
 		this.allowedLateness = allowedLateness;
 		this.lateDataOutputTag = lateDataOutputTag;
-		this.legacyWindowOperatorType = legacyWindowOperatorType;
 
 		setChainingStrategy(ChainingStrategy.ALWAYS);
 	}
@@ -321,8 +259,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 					getOrCreateKeyedState(VoidNamespaceSerializer.INSTANCE, mergingSetsStateDescriptor);
 			mergingSetsState.setCurrentNamespace(VoidNamespace.INSTANCE);
 		}
-
-		registerRestoredLegacyStateState();
 	}
 
 	@Override
@@ -1037,256 +973,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	}
 
 	// ------------------------------------------------------------------------
-	//  Restoring / Migrating from an older Flink version.
-	// ------------------------------------------------------------------------
-
-	private static final int BEGIN_OF_STATE_MAGIC_NUMBER = 0x0FF1CE42;
-
-	private static final int BEGIN_OF_PANE_MAGIC_NUMBER = 0xBADF00D5;
-
-	@Override
-	public void restoreState(FSDataInputStream in) throws Exception {
-		super.restoreState(in);
-
-		LOG.info("{} (taskIdx={}) restoring {} state from an older Flink version.",
-			getClass().getSimpleName(), legacyWindowOperatorType, getRuntimeContext().getIndexOfThisSubtask());
-
-		DataInputViewStreamWrapper streamWrapper = new DataInputViewStreamWrapper(in);
-
-		switch (legacyWindowOperatorType) {
-			case NONE:
-				restoreFromLegacyWindowOperator(streamWrapper);
-				break;
-			case FAST_ACCUMULATING:
-			case FAST_AGGREGATING:
-				restoreFromLegacyAlignedWindowOperator(streamWrapper);
-				break;
-		}
-	}
-
-	public void registerRestoredLegacyStateState() throws Exception {
-
-		switch (legacyWindowOperatorType) {
-			case NONE:
-				reregisterStateFromLegacyWindowOperator();
-				break;
-			case FAST_ACCUMULATING:
-			case FAST_AGGREGATING:
-				reregisterStateFromLegacyAlignedWindowOperator();
-				break;
-		}
-	}
-
-	private void restoreFromLegacyAlignedWindowOperator(DataInputViewStreamWrapper in) throws IOException {
-		Preconditions.checkArgument(legacyWindowOperatorType != LegacyWindowOperatorType.NONE);
-
-		final long nextEvaluationTime = in.readLong();
-		final long nextSlideTime = in.readLong();
-
-		validateMagicNumber(BEGIN_OF_STATE_MAGIC_NUMBER, in.readInt());
-
-		restoredFromLegacyAlignedOpRecords = new PriorityQueue<>(42,
-			new Comparator<StreamRecord<IN>>() {
-				@Override
-				public int compare(StreamRecord<IN> o1, StreamRecord<IN> o2) {
-					return Long.compare(o1.getTimestamp(), o2.getTimestamp());
-				}
-			}
-		);
-
-		switch (legacyWindowOperatorType) {
-			case FAST_ACCUMULATING:
-				restoreElementsFromLegacyAccumulatingAlignedWindowOperator(in, nextSlideTime);
-				break;
-			case FAST_AGGREGATING:
-				restoreElementsFromLegacyAggregatingAlignedWindowOperator(in, nextSlideTime);
-				break;
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("{} (taskIdx={}) restored {} events from legacy {}.",
-				getClass().getSimpleName(),
-				getRuntimeContext().getIndexOfThisSubtask(),
-				restoredFromLegacyAlignedOpRecords.size(),
-				legacyWindowOperatorType);
-		}
-	}
-
-	private void restoreElementsFromLegacyAccumulatingAlignedWindowOperator(DataInputView in, long nextSlideTime) throws IOException {
-		int numPanes = in.readInt();
-		final long paneSize = getPaneSize();
-		long nextElementTimestamp = nextSlideTime - (numPanes * paneSize);
-
-		@SuppressWarnings("unchecked")
-		ArrayListSerializer<IN> ser = new ArrayListSerializer<>((TypeSerializer<IN>) getStateDescriptor().getSerializer());
-
-		while (numPanes > 0) {
-			validateMagicNumber(BEGIN_OF_PANE_MAGIC_NUMBER, in.readInt());
-
-			nextElementTimestamp += paneSize - 1; // the -1 is so that the elements fall into the correct time-frame
-
-			final int numElementsInPane = in.readInt();
-			for (int i = numElementsInPane - 1; i >= 0; i--) {
-				K key = keySerializer.deserialize(in);
-
-				@SuppressWarnings("unchecked")
-				List<IN> valueList = ser.deserialize(in);
-				for (IN record: valueList) {
-					restoredFromLegacyAlignedOpRecords.add(new StreamRecord<>(record, nextElementTimestamp));
-				}
-			}
-			numPanes--;
-		}
-	}
-
-	private void restoreElementsFromLegacyAggregatingAlignedWindowOperator(DataInputView in, long nextSlideTime) throws IOException {
-		int numPanes = in.readInt();
-		final long paneSize = getPaneSize();
-		long nextElementTimestamp = nextSlideTime - (numPanes * paneSize);
-
-		while (numPanes > 0) {
-			validateMagicNumber(BEGIN_OF_PANE_MAGIC_NUMBER, in.readInt());
-
-			nextElementTimestamp += paneSize - 1; // the -1 is so that the elements fall into the correct time-frame
-
-			final int numElementsInPane = in.readInt();
-			for (int i = numElementsInPane - 1; i >= 0; i--) {
-				K key = keySerializer.deserialize(in);
-
-				@SuppressWarnings("unchecked")
-				IN value = (IN) getStateDescriptor().getSerializer().deserialize(in);
-				restoredFromLegacyAlignedOpRecords.add(new StreamRecord<>(value, nextElementTimestamp));
-			}
-			numPanes--;
-		}
-	}
-
-	private long getPaneSize() {
-		Preconditions.checkArgument(
-			legacyWindowOperatorType == LegacyWindowOperatorType.FAST_ACCUMULATING ||
-				legacyWindowOperatorType == LegacyWindowOperatorType.FAST_AGGREGATING);
-
-		final long paneSlide;
-		if (windowAssigner instanceof SlidingProcessingTimeWindows) {
-			SlidingProcessingTimeWindows timeWindows = (SlidingProcessingTimeWindows) windowAssigner;
-			paneSlide = ArithmeticUtils.gcd(timeWindows.getSize(), timeWindows.getSlide());
-		} else {
-			TumblingProcessingTimeWindows timeWindows = (TumblingProcessingTimeWindows) windowAssigner;
-			paneSlide = timeWindows.getSize(); // this is valid as windowLength == windowSlide == timeWindows.getSize
-		}
-		return paneSlide;
-	}
-
-	private static void validateMagicNumber(int expected, int found) throws IOException {
-		if (expected != found) {
-			throw new IOException("Corrupt state stream - wrong magic number. " +
-				"Expected '" + Integer.toHexString(expected) +
-				"', found '" + Integer.toHexString(found) + '\'');
-		}
-	}
-
-	private void restoreFromLegacyWindowOperator(DataInputViewStreamWrapper in) throws IOException {
-		Preconditions.checkArgument(legacyWindowOperatorType == LegacyWindowOperatorType.NONE);
-
-		int numWatermarkTimers = in.readInt();
-		this.restoredFromLegacyEventTimeTimers = new PriorityQueue<>(Math.max(numWatermarkTimers, 1));
-
-		for (int i = 0; i < numWatermarkTimers; i++) {
-			K key = keySerializer.deserialize(in);
-			W window = windowSerializer.deserialize(in);
-			long timestamp = in.readLong();
-
-			Timer<K, W> timer = new Timer<>(timestamp, key, window);
-			restoredFromLegacyEventTimeTimers.add(timer);
-		}
-
-		int numProcessingTimeTimers = in.readInt();
-		this.restoredFromLegacyProcessingTimeTimers = new PriorityQueue<>(Math.max(numProcessingTimeTimers, 1));
-
-		for (int i = 0; i < numProcessingTimeTimers; i++) {
-			K key = keySerializer.deserialize(in);
-			W window = windowSerializer.deserialize(in);
-			long timestamp = in.readLong();
-
-			Timer<K, W> timer = new Timer<>(timestamp, key, window);
-			restoredFromLegacyProcessingTimeTimers.add(timer);
-		}
-
-		// just to read all the rest, although we do not really use this information.
-		int numProcessingTimeTimerTimestamp = in.readInt();
-		for (int i = 0; i < numProcessingTimeTimerTimestamp; i++) {
-			in.readLong();
-			in.readInt();
-		}
-
-		if (LOG.isDebugEnabled()) {
-			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
-
-			if (restoredFromLegacyEventTimeTimers != null && !restoredFromLegacyEventTimeTimers.isEmpty()) {
-				LOG.debug("{} (taskIdx={}) restored {} event time timers from an older Flink version: {}",
-					getClass().getSimpleName(), subtaskIdx,
-					restoredFromLegacyEventTimeTimers.size(),
-					restoredFromLegacyEventTimeTimers);
-			}
-
-			if (restoredFromLegacyProcessingTimeTimers != null && !restoredFromLegacyProcessingTimeTimers.isEmpty()) {
-				LOG.debug("{} (taskIdx={}) restored {} processing time timers from an older Flink version: {}",
-					getClass().getSimpleName(), subtaskIdx,
-					restoredFromLegacyProcessingTimeTimers.size(),
-					restoredFromLegacyProcessingTimeTimers);
-			}
-		}
-	}
-
-	public void reregisterStateFromLegacyWindowOperator() {
-		// if we restore from an older version,
-		// we have to re-register the recovered state.
-
-		if (restoredFromLegacyEventTimeTimers != null && !restoredFromLegacyEventTimeTimers.isEmpty()) {
-
-			LOG.info("{} (taskIdx={}) re-registering event-time timers from an older Flink version.",
-				getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask());
-
-			for (Timer<K, W> timer : restoredFromLegacyEventTimeTimers) {
-				setCurrentKey(timer.key);
-				internalTimerService.registerEventTimeTimer(timer.window, timer.timestamp);
-			}
-		}
-
-		if (restoredFromLegacyProcessingTimeTimers != null && !restoredFromLegacyProcessingTimeTimers.isEmpty()) {
-
-			LOG.info("{} (taskIdx={}) re-registering processing-time timers from an older Flink version.",
-				getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask());
-
-			for (Timer<K, W> timer : restoredFromLegacyProcessingTimeTimers) {
-				setCurrentKey(timer.key);
-				internalTimerService.registerProcessingTimeTimer(timer.window, timer.timestamp);
-			}
-		}
-
-		// gc friendliness
-		restoredFromLegacyEventTimeTimers = null;
-		restoredFromLegacyProcessingTimeTimers = null;
-	}
-
-	public void reregisterStateFromLegacyAlignedWindowOperator() throws Exception {
-		if (restoredFromLegacyAlignedOpRecords != null && !restoredFromLegacyAlignedOpRecords.isEmpty()) {
-
-			LOG.info("{} (taskIdx={}) re-registering timers from legacy {} from an older Flink version.",
-				getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), legacyWindowOperatorType);
-
-			while (!restoredFromLegacyAlignedOpRecords.isEmpty()) {
-				StreamRecord<IN> record = restoredFromLegacyAlignedOpRecords.poll();
-				setCurrentKey(keySelector.getKey(record.getValue()));
-				processElement(record);
-			}
-		}
-
-		// gc friendliness
-		restoredFromLegacyAlignedOpRecords = null;
-	}
-
-	// ------------------------------------------------------------------------
 	// Getters for testing
 	// ------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
index 1dc0ee2..d0ab60a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
@@ -29,7 +29,6 @@ import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.migration.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
 
@@ -292,9 +291,6 @@ public final class StreamElementSerializer<T> extends TypeSerializer<StreamEleme
 		if (configSnapshot instanceof StreamElementSerializerConfigSnapshot) {
 			previousTypeSerializerAndConfig =
 				((StreamElementSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
-		} else if (configSnapshot instanceof MultiplexingStreamRecordSerializer.MultiplexingStreamRecordSerializerConfigSnapshot) {
-			previousTypeSerializerAndConfig =
-				((MultiplexingStreamRecordSerializer.MultiplexingStreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
 		} else {
 			return CompatibilityResult.requiresMigration();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java
index 4914075..0b03b79 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java
@@ -22,7 +22,6 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 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.util.CollectionUtil;
 
 import java.util.Collection;
@@ -37,8 +36,6 @@ public class OperatorStateHandles {
 
 	private final int operatorChainIndex;
 
-	private final StreamStateHandle legacyOperatorState;
-
 	private final Collection<KeyedStateHandle> managedKeyedState;
 	private final Collection<KeyedStateHandle> rawKeyedState;
 	private final Collection<OperatorStateHandle> managedOperatorState;
@@ -46,24 +43,18 @@ public class OperatorStateHandles {
 
 	public OperatorStateHandles(
 			int operatorChainIndex,
-			StreamStateHandle legacyOperatorState,
 			Collection<KeyedStateHandle> managedKeyedState,
 			Collection<KeyedStateHandle> rawKeyedState,
 			Collection<OperatorStateHandle> managedOperatorState,
 			Collection<OperatorStateHandle> rawOperatorState) {
 
 		this.operatorChainIndex = operatorChainIndex;
-		this.legacyOperatorState = legacyOperatorState;
 		this.managedKeyedState = managedKeyedState;
 		this.rawKeyedState = rawKeyedState;
 		this.managedOperatorState = managedOperatorState;
 		this.rawOperatorState = rawOperatorState;
 	}
 
-	public StreamStateHandle getLegacyOperatorState() {
-		return legacyOperatorState;
-	}
-
 	public Collection<KeyedStateHandle> getManagedKeyedState() {
 		return managedKeyedState;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 1ba5fb1..310df4d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -44,8 +44,6 @@ import org.apache.flink.runtime.state.KeyedStateHandle;
 import org.apache.flink.runtime.state.OperatorStateBackend;
 import org.apache.flink.runtime.state.OperatorStateHandle;
 import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.StateUtil;
-import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.graph.StreamConfig;
@@ -836,8 +834,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 
 		private final Map<OperatorID, OperatorSnapshotResult> operatorSnapshotsInProgress;
 
-		private Map<OperatorID, StreamStateHandle> nonPartitionedStateHandles;
-
 		private final CheckpointMetaData checkpointMetaData;
 		private final CheckpointMetrics checkpointMetrics;
 
@@ -848,7 +844,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 
 		AsyncCheckpointRunnable(
 				StreamTask<?, ?> owner,
-				Map<OperatorID, StreamStateHandle> nonPartitionedStateHandles,
 				Map<OperatorID, OperatorSnapshotResult> operatorSnapshotsInProgress,
 				CheckpointMetaData checkpointMetaData,
 				CheckpointMetrics checkpointMetrics,
@@ -858,7 +853,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 			this.operatorSnapshotsInProgress = Preconditions.checkNotNull(operatorSnapshotsInProgress);
 			this.checkpointMetaData = Preconditions.checkNotNull(checkpointMetaData);
 			this.checkpointMetrics = Preconditions.checkNotNull(checkpointMetrics);
-			this.nonPartitionedStateHandles = nonPartitionedStateHandles;
 			this.asyncStartNanos = asyncStartNanos;
 		}
 
@@ -876,7 +870,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 					OperatorSnapshotResult snapshotInProgress = entry.getValue();
 
 					OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(
-						nonPartitionedStateHandles.get(operatorID),
 						FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getOperatorStateManagedFuture()),
 						FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getOperatorStateRawFuture()),
 						FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getKeyedStateManagedFuture()),
@@ -968,13 +961,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 					}
 				}
 
-				// discard non partitioned state handles
-				try {
-					StateUtil.bestEffortDiscardAllStateObjects(nonPartitionedStateHandles.values());
-				} catch (Exception discardException) {
-					exception = ExceptionUtils.firstOrSuppressed(discardException, exception);
-				}
-
 				if (null != exception) {
 					throw exception;
 				}
@@ -1008,7 +994,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 
 		// ------------------------
 
-		private final Map<OperatorID, StreamStateHandle> nonPartitionedStates;
 		private final Map<OperatorID, OperatorSnapshotResult> operatorSnapshotsInProgress;
 
 		public CheckpointingOperation(
@@ -1022,7 +1007,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 			this.checkpointOptions = Preconditions.checkNotNull(checkpointOptions);
 			this.checkpointMetrics = Preconditions.checkNotNull(checkpointMetrics);
 			this.allOperators = owner.operatorChain.getAllOperators();
-			this.nonPartitionedStates = new HashMap<>(allOperators.length);
 			this.operatorSnapshotsInProgress = new HashMap<>(allOperators.length);
 		}
 
@@ -1068,18 +1052,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 						}
 					}
 
-					// Cleanup non partitioned state handles
-					for (StreamStateHandle nonPartitionedState : nonPartitionedStates.values()) {
-						if (nonPartitionedState != null) {
-							try {
-								nonPartitionedState.discardState();
-							} catch (Exception e) {
-								LOG.warn("Could not properly discard a non partitioned " +
-									"state. This might leave some orphaned files behind.", e);
-							}
-						}
-					}
-
 					if (LOG.isDebugEnabled()) {
 						LOG.debug("{} - did NOT finish synchronous part of checkpoint {}." +
 								"Alignment duration: {} ms, snapshot duration {} ms",
@@ -1094,20 +1066,12 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 		@SuppressWarnings("deprecation")
 		private void checkpointStreamOperator(StreamOperator<?> op) throws Exception {
 			if (null != op) {
-				// first call the legacy checkpoint code paths
-				StreamStateHandle legacyOperatorState = op.snapshotLegacyOperatorState(
-					checkpointMetaData.getCheckpointId(),
-					checkpointMetaData.getTimestamp(),
-					checkpointOptions);
-
-				OperatorID operatorID = op.getOperatorID();
-				nonPartitionedStates.put(operatorID, legacyOperatorState);
 
 				OperatorSnapshotResult snapshotInProgress = op.snapshotState(
 						checkpointMetaData.getCheckpointId(),
 						checkpointMetaData.getTimestamp(),
 						checkpointOptions);
-				operatorSnapshotsInProgress.put(operatorID, snapshotInProgress);
+				operatorSnapshotsInProgress.put(op.getOperatorID(), snapshotInProgress);
 			}
 		}
 
@@ -1115,7 +1079,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 
 			AsyncCheckpointRunnable asyncCheckpointRunnable = new AsyncCheckpointRunnable(
 					owner,
-					nonPartitionedStates,
 					operatorSnapshotsInProgress,
 					checkpointMetaData,
 					checkpointMetrics,

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java
index ff5f589..4ed689d 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java
@@ -28,7 +28,6 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.StateSnapshotContext;
-import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
@@ -65,7 +64,6 @@ public class AbstractUdfStreamOperatorLifecycleTest {
 			"UDF::open",
 			"OPERATOR::run",
 			"UDF::run",
-			"OPERATOR::snapshotLegacyOperatorState",
 			"OPERATOR::snapshotState",
 			"OPERATOR::close",
 			"UDF::close",
@@ -93,7 +91,6 @@ public class AbstractUdfStreamOperatorLifecycleTest {
 			"setup[class org.apache.flink.streaming.runtime.tasks.StreamTask, class " +
 			"org.apache.flink.streaming.api.graph.StreamConfig, interface " +
 			"org.apache.flink.streaming.api.operators.Output], " +
-			"snapshotLegacyOperatorState[long, long, class org.apache.flink.runtime.checkpoint.CheckpointOptions], " +
 			"snapshotState[long, long, class org.apache.flink.runtime.checkpoint.CheckpointOptions]]";
 
 	private static final String ALL_METHODS_RICH_FUNCTION = "[close[], getIterationRuntimeContext[], getRuntimeContext[]" +
@@ -207,7 +204,7 @@ public class AbstractUdfStreamOperatorLifecycleTest {
 	}
 
 	private static class LifecycleTrackingStreamSource<OUT, SRC extends SourceFunction<OUT>>
-			extends StreamSource<OUT, SRC> implements Serializable, StreamCheckpointedOperator {
+			extends StreamSource<OUT, SRC> implements Serializable {
 
 		private static final long serialVersionUID = 2431488948886850562L;
 		private transient Thread testCheckpointer;
@@ -266,12 +263,6 @@ public class AbstractUdfStreamOperatorLifecycleTest {
 		}
 
 		@Override
-		public StreamStateHandle snapshotLegacyOperatorState(long checkpointId, long timestamp, CheckpointOptions checkpointOptions) throws Exception {
-			ACTUAL_ORDER_TRACKING.add("OPERATOR::snapshotLegacyOperatorState");
-			return super.snapshotLegacyOperatorState(checkpointId, timestamp, checkpointOptions);
-		}
-
-		@Override
 		public void initializeState(StateInitializationContext context) throws Exception {
 			ACTUAL_ORDER_TRACKING.add("OPERATOR::initializeState");
 			super.initializeState(context);

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyProcessWindowFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyProcessWindowFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyProcessWindowFunctionTest.java
deleted file mode 100644
index 7dba4af..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyProcessWindowFunctionTest.java
+++ /dev/null
@@ -1,332 +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 org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.util.ListCollector;
-import org.apache.flink.api.common.state.FoldingState;
-import org.apache.flink.api.common.state.FoldingStateDescriptor;
-import org.apache.flink.api.common.state.KeyedStateStore;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.MapState;
-import org.apache.flink.api.common.state.MapStateDescriptor;
-import org.apache.flink.api.common.state.ReducingState;
-import org.apache.flink.api.common.state.ReducingStateDescriptor;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.base.ByteSerializer;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.windowing.FoldApplyProcessAllWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.FoldApplyProcessWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.transformations.SourceTransformation;
-import org.apache.flink.streaming.api.transformations.StreamTransformation;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableProcessAllWindowFunction;
-import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableProcessWindowFunction;
-import org.apache.flink.util.Collector;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Tests for {@link FoldApplyProcessWindowFunction}.
- */
-public class FoldApplyProcessWindowFunctionTest {
-
-	/**
-	 * Tests that the FoldWindowFunction gets the output type serializer set by the
-	 * StreamGraphGenerator and checks that the FoldWindowFunction computes the correct result.
-	 */
-	@Test
-	public void testFoldWindowFunctionOutputTypeConfigurable() throws Exception{
-		StreamExecutionEnvironment env = new DummyStreamExecutionEnvironment();
-
-		List<StreamTransformation<?>> transformations = new ArrayList<>();
-
-		int initValue = 1;
-
-		FoldApplyProcessWindowFunction<Integer, TimeWindow, Integer, Integer, Integer> foldWindowFunction = new FoldApplyProcessWindowFunction<>(
-			initValue,
-			new FoldFunction<Integer, Integer>() {
-				@Override
-				public Integer fold(Integer accumulator, Integer value) throws Exception {
-					return accumulator + value;
-				}
-
-			},
-			new ProcessWindowFunction<Integer, Integer, Integer, TimeWindow>() {
-				@Override
-				public void process(Integer integer,
-									Context context,
-									Iterable<Integer> input,
-									Collector<Integer> out) throws Exception {
-					for (Integer in: input) {
-						out.collect(in);
-					}
-				}
-			},
-			BasicTypeInfo.INT_TYPE_INFO
-		);
-
-		AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> windowOperator = new AccumulatingProcessingTimeWindowOperator<>(
-			new InternalIterableProcessWindowFunction<>(foldWindowFunction),
-			new KeySelector<Integer, Integer>() {
-				private static final long serialVersionUID = -7951310554369722809L;
-
-				@Override
-				public Integer getKey(Integer value) throws Exception {
-					return value;
-				}
-			},
-			IntSerializer.INSTANCE,
-			IntSerializer.INSTANCE,
-			3000,
-			3000
-		);
-
-		SourceFunction<Integer> sourceFunction = new SourceFunction<Integer>(){
-
-			private static final long serialVersionUID = 8297735565464653028L;
-
-			@Override
-			public void run(SourceContext<Integer> ctx) throws Exception {
-
-			}
-
-			@Override
-			public void cancel() {
-
-			}
-		};
-
-		SourceTransformation<Integer> source = new SourceTransformation<>("", new StreamSource<>(sourceFunction), BasicTypeInfo.INT_TYPE_INFO, 1);
-
-		transformations.add(new OneInputTransformation<>(source, "test", windowOperator, BasicTypeInfo.INT_TYPE_INFO, 1));
-
-		StreamGraph streamGraph = StreamGraphGenerator.generate(env, transformations);
-
-		List<Integer> result = new ArrayList<>();
-		List<Integer> input = new ArrayList<>();
-		List<Integer> expected = new ArrayList<>();
-
-		input.add(1);
-		input.add(2);
-		input.add(3);
-
-		for (int value : input) {
-			initValue += value;
-		}
-
-		expected.add(initValue);
-
-		FoldApplyProcessWindowFunction<Integer, TimeWindow, Integer, Integer, Integer>.Context ctx = foldWindowFunction.new Context() {
-			@Override
-			public TimeWindow window() {
-				return new TimeWindow(0, 1);
-			}
-
-			@Override
-			public long currentProcessingTime() {
-				return 0;
-			}
-
-			@Override
-			public long currentWatermark() {
-				return 0;
-			}
-
-			@Override
-			public KeyedStateStore windowState() {
-				return new DummyKeyedStateStore();
-			}
-
-			@Override
-			public KeyedStateStore globalState() {
-				return new DummyKeyedStateStore();
-			}
-		};
-
-		foldWindowFunction.open(new Configuration());
-
-		foldWindowFunction.process(0, ctx, input, new ListCollector<>(result));
-
-		Assert.assertEquals(expected, result);
-	}
-
-		/**
-	 * Tests that the FoldWindowFunction gets the output type serializer set by the
-	 * StreamGraphGenerator and checks that the FoldWindowFunction computes the correct result.
-	 */
-	@Test
-	public void testFoldAllWindowFunctionOutputTypeConfigurable() throws Exception{
-		StreamExecutionEnvironment env = new DummyStreamExecutionEnvironment();
-
-		List<StreamTransformation<?>> transformations = new ArrayList<>();
-
-		int initValue = 1;
-
-		FoldApplyProcessAllWindowFunction<TimeWindow, Integer, Integer, Integer> foldWindowFunction = new FoldApplyProcessAllWindowFunction<>(
-			initValue,
-			new FoldFunction<Integer, Integer>() {
-				@Override
-				public Integer fold(Integer accumulator, Integer value) throws Exception {
-					return accumulator + value;
-				}
-
-			},
-			new ProcessAllWindowFunction<Integer, Integer, TimeWindow>() {
-				@Override
-				public void process(Context context,
-									Iterable<Integer> input,
-									Collector<Integer> out) throws Exception {
-					for (Integer in: input) {
-						out.collect(in);
-					}
-				}
-			},
-			BasicTypeInfo.INT_TYPE_INFO
-		);
-
-		AccumulatingProcessingTimeWindowOperator<Byte, Integer, Integer> windowOperator = new AccumulatingProcessingTimeWindowOperator<>(
-			new InternalIterableProcessAllWindowFunction<>(foldWindowFunction),
-			new KeySelector<Integer, Byte>() {
-				private static final long serialVersionUID = -7951310554369722809L;
-
-				@Override
-				public Byte getKey(Integer value) throws Exception {
-					return 0;
-				}
-			},
-			ByteSerializer.INSTANCE,
-			IntSerializer.INSTANCE,
-			3000,
-			3000
-		);
-
-		SourceFunction<Integer> sourceFunction = new SourceFunction<Integer>(){
-
-			private static final long serialVersionUID = 8297735565464653028L;
-
-			@Override
-			public void run(SourceContext<Integer> ctx) throws Exception {
-
-			}
-
-			@Override
-			public void cancel() {
-
-			}
-		};
-
-		SourceTransformation<Integer> source = new SourceTransformation<>("", new StreamSource<>(sourceFunction), BasicTypeInfo.INT_TYPE_INFO, 1);
-
-		transformations.add(new OneInputTransformation<>(source, "test", windowOperator, BasicTypeInfo.INT_TYPE_INFO, 1));
-
-		StreamGraph streamGraph = StreamGraphGenerator.generate(env, transformations);
-
-		List<Integer> result = new ArrayList<>();
-		List<Integer> input = new ArrayList<>();
-		List<Integer> expected = new ArrayList<>();
-
-		input.add(1);
-		input.add(2);
-		input.add(3);
-
-		for (int value : input) {
-			initValue += value;
-		}
-
-		expected.add(initValue);
-
-		FoldApplyProcessAllWindowFunction<TimeWindow, Integer, Integer, Integer>.Context ctx = foldWindowFunction.new Context() {
-			@Override
-			public TimeWindow window() {
-				return new TimeWindow(0, 1);
-			}
-
-			@Override
-			public KeyedStateStore windowState() {
-				return new DummyKeyedStateStore();
-			}
-
-			@Override
-			public KeyedStateStore globalState() {
-				return new DummyKeyedStateStore();
-			}
-		};
-
-		foldWindowFunction.open(new Configuration());
-
-		foldWindowFunction.process(ctx, input, new ListCollector<>(result));
-
-		Assert.assertEquals(expected, result);
-	}
-
-	private static class DummyKeyedStateStore implements KeyedStateStore {
-
-		@Override
-		public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
-			return null;
-		}
-
-		@Override
-		public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
-			return null;
-		}
-
-		@Override
-		public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
-			return null;
-		}
-
-		@Override
-		public <T, ACC> FoldingState<T, ACC> getFoldingState(FoldingStateDescriptor<T, ACC> stateProperties) {
-			return null;
-		}
-
-		@Override
-		public <UK, UV> MapState<UK, UV> getMapState(MapStateDescriptor<UK, UV> stateProperties) {
-			return null;
-		}
-	}
-
-	private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment {
-
-		@Override
-		public JobExecutionResult execute(String jobName) throws Exception {
-			return null;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyWindowFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyWindowFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyWindowFunctionTest.java
deleted file mode 100644
index 7cf18dd..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/FoldApplyWindowFunctionTest.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.util.ListCollector;
-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.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.windowing.FoldApplyWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.transformations.SourceTransformation;
-import org.apache.flink.streaming.api.transformations.StreamTransformation;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableWindowFunction;
-import org.apache.flink.util.Collector;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Tests for {@link FoldApplyWindowFunction}.
- */
-public class FoldApplyWindowFunctionTest {
-
-	/**
-	 * Tests that the FoldWindowFunction gets the output type serializer set by the
-	 * StreamGraphGenerator and checks that the FoldWindowFunction computes the correct result.
-	 */
-	@Test
-	public void testFoldWindowFunctionOutputTypeConfigurable() throws Exception{
-		StreamExecutionEnvironment env = new DummyStreamExecutionEnvironment();
-
-		List<StreamTransformation<?>> transformations = new ArrayList<>();
-
-		int initValue = 1;
-
-		FoldApplyWindowFunction<Integer, TimeWindow, Integer, Integer, Integer> foldWindowFunction = new FoldApplyWindowFunction<>(
-			initValue,
-			new FoldFunction<Integer, Integer>() {
-				private static final long serialVersionUID = -4849549768529720587L;
-
-				@Override
-				public Integer fold(Integer accumulator, Integer value) throws Exception {
-					return accumulator + value;
-				}
-			},
-			new WindowFunction<Integer, Integer, Integer, TimeWindow>() {
-				@Override
-				public void apply(Integer integer,
-					TimeWindow window,
-					Iterable<Integer> input,
-					Collector<Integer> out) throws Exception {
-					for (Integer in: input) {
-						out.collect(in);
-					}
-				}
-			},
-			BasicTypeInfo.INT_TYPE_INFO
-		);
-
-		AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> windowOperator = new AccumulatingProcessingTimeWindowOperator<>(
-			new InternalIterableWindowFunction<>(
-					foldWindowFunction),
-				new KeySelector<Integer, Integer>() {
-					private static final long serialVersionUID = -7951310554369722809L;
-
-					@Override
-					public Integer getKey(Integer value) throws Exception {
-						return value;
-					}
-				},
-				IntSerializer.INSTANCE,
-				IntSerializer.INSTANCE,
-				3000,
-				3000
-		);
-
-		SourceFunction<Integer> sourceFunction = new SourceFunction<Integer>(){
-
-			private static final long serialVersionUID = 8297735565464653028L;
-
-			@Override
-			public void run(SourceContext<Integer> ctx) throws Exception {
-
-			}
-
-			@Override
-			public void cancel() {
-
-			}
-		};
-
-		SourceTransformation<Integer> source = new SourceTransformation<>("", new StreamSource<>(sourceFunction), BasicTypeInfo.INT_TYPE_INFO, 1);
-
-		transformations.add(new OneInputTransformation<>(source, "test", windowOperator, BasicTypeInfo.INT_TYPE_INFO, 1));
-
-		StreamGraph streamGraph = StreamGraphGenerator.generate(env, transformations);
-
-		List<Integer> result = new ArrayList<>();
-		List<Integer> input = new ArrayList<>();
-		List<Integer> expected = new ArrayList<>();
-
-		input.add(1);
-		input.add(2);
-		input.add(3);
-
-		for (int value : input) {
-			initValue += value;
-		}
-
-		expected.add(initValue);
-
-		foldWindowFunction.apply(0, new TimeWindow(0, 1), input, new ListCollector<Integer>(result));
-
-		Assert.assertEquals(expected, result);
-	}
-
-	private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment {
-
-		@Override
-		public JobExecutionResult execute(String jobName) throws Exception {
-			return null;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/graph/StreamingJobGraphGeneratorNodeHashTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/graph/StreamingJobGraphGeneratorNodeHashTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/graph/StreamingJobGraphGeneratorNodeHashTest.java
index 58898d8..6dd08f6 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/graph/StreamingJobGraphGeneratorNodeHashTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/graph/StreamingJobGraphGeneratorNodeHashTest.java
@@ -437,7 +437,8 @@ public class StreamingJobGraphGeneratorNodeHashTest extends TestLogger {
 		StreamGraph streamGraph = env.getStreamGraph();
 		int idx = 1;
 		for (JobVertex jobVertex : streamGraph.getJobGraph().getVertices()) {
-			Assert.assertEquals(jobVertex.getIdAlternatives().get(1).toString(), userHashes.get(idx));
+			List<JobVertexID> idAlternatives = jobVertex.getIdAlternatives();
+			Assert.assertEquals(idAlternatives.get(idAlternatives.size() - 1).toString(), userHashes.get(idx));
 			--idx;
 		}
 	}


[05/11] flink git commit: [FLINK-7461] Remove Backwards compatibility with <= Flink 1.1

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
deleted file mode 100644
index a57dcf1..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
+++ /dev/null
@@ -1,1116 +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.TaskInfo;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.CoreOptions;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup;
-import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
-import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo;
-import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableProcessWindowFunction;
-import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableWindowFunction;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.util.Collector;
-
-import org.junit.After;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for {@link AccumulatingProcessingTimeWindowOperator}.
- */
-@SuppressWarnings({"serial"})
-@PrepareForTest(InternalIterableWindowFunction.class)
-@RunWith(PowerMockRunner.class)
-public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
-
-	@SuppressWarnings("unchecked")
-	private final InternalIterableWindowFunction<String, String, String, TimeWindow> mockFunction = mock(InternalIterableWindowFunction.class);
-
-	@SuppressWarnings("unchecked")
-	private final KeySelector<String, String> mockKeySelector = mock(KeySelector.class);
-
-	private final KeySelector<Integer, Integer> identitySelector = new KeySelector<Integer, Integer>() {
-		@Override
-		public Integer getKey(Integer value) {
-			return value;
-		}
-	};
-
-	private final InternalIterableWindowFunction<Integer, Integer, Integer, TimeWindow> validatingIdentityFunction =
-			new InternalIterableWindowFunction<>(new WindowFunction<Integer, Integer, Integer, TimeWindow>() {
-				@Override
-				public void apply(Integer key, TimeWindow window, Iterable<Integer> values, Collector<Integer> out) throws Exception {
-					for (Integer val : values) {
-						assertEquals(key, val);
-						out.collect(val);
-					}
-				}
-			});
-
-	private final InternalIterableProcessWindowFunction<Integer, Integer, Integer, TimeWindow> validatingIdentityProcessFunction =
-			new InternalIterableProcessWindowFunction<>(new ProcessWindowFunction<Integer, Integer, Integer, TimeWindow>() {
-				@Override
-				public void process(Integer key, Context context, Iterable<Integer> values, Collector<Integer> out) throws Exception {
-					for (Integer val : values) {
-						assertEquals(key, val);
-						out.collect(val);
-					}
-				}
-			});
-
-	// ------------------------------------------------------------------------
-
-	public AccumulatingAlignedProcessingTimeWindowOperatorTest() {
-		ClosureCleaner.clean(identitySelector, false);
-		ClosureCleaner.clean(validatingIdentityFunction, false);
-		ClosureCleaner.clean(validatingIdentityProcessFunction, false);
-	}
-
-	// ------------------------------------------------------------------------
-
-	@After
-	public void checkNoTriggerThreadsRunning() {
-		// make sure that all the threads we trigger are shut down
-		long deadline = System.currentTimeMillis() + 5000;
-		while (StreamTask.TRIGGER_THREAD_GROUP.activeCount() > 0 && System.currentTimeMillis() < deadline) {
-			try {
-				Thread.sleep(10);
-			}
-			catch (InterruptedException ignored) {}
-		}
-
-		assertTrue("Not all trigger threads where properly shut down",
-				StreamTask.TRIGGER_THREAD_GROUP.activeCount() == 0);
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testInvalidParameters() {
-		try {
-			assertInvalidParameter(-1L, -1L);
-			assertInvalidParameter(10000L, -1L);
-			assertInvalidParameter(-1L, 1000L);
-			assertInvalidParameter(1000L, 2000L);
-
-			// actual internal slide is too low here:
-			assertInvalidParameter(1000L, 999L);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testWindowSizeAndSlide() {
-		try {
-			AccumulatingProcessingTimeWindowOperator<String, String, String> op;
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
-			assertEquals(5000, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(1000, op.getPaneSize());
-			assertEquals(5, op.getNumPanesPerWindow());
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
-			assertEquals(1000, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(1000, op.getPaneSize());
-			assertEquals(1, op.getNumPanesPerWindow());
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
-			assertEquals(1500, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(500, op.getPaneSize());
-			assertEquals(3, op.getNumPanesPerWindow());
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
-			assertEquals(1200, op.getWindowSize());
-			assertEquals(1100, op.getWindowSlide());
-			assertEquals(100, op.getPaneSize());
-			assertEquals(12, op.getNumPanesPerWindow());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testWindowTriggerTimeAlignment() throws Exception {
-
-		try {
-			AccumulatingProcessingTimeWindowOperator<String, String, String> op =
-					new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-							StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
-
-			KeyedOneInputStreamOperatorTestHarness<String, String, String> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, mockKeySelector, BasicTypeInfo.STRING_TYPE_INFO);
-
-			testHarness.open();
-
-			assertTrue(op.getNextSlideTime() % 1000 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			testHarness.close();
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
-
-			testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, mockKeySelector, BasicTypeInfo.STRING_TYPE_INFO);
-
-			testHarness.open();
-
-			assertTrue(op.getNextSlideTime() % 1000 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			testHarness.close();
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
-
-			testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, mockKeySelector, BasicTypeInfo.STRING_TYPE_INFO);
-
-			testHarness.open();
-
-			assertTrue(op.getNextSlideTime() % 500 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			testHarness.close();
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
-
-			testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, mockKeySelector, BasicTypeInfo.STRING_TYPE_INFO);
-
-			testHarness.open();
-
-			assertEquals(0, op.getNextSlideTime() % 100);
-			assertEquals(0, op.getNextEvaluationTime() % 1100);
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testTumblingWindow() throws Exception {
-		try {
-			final int windowSize = 50;
-
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSize);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Integer> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, identitySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.open();
-
-			final int numElements = 1000;
-
-			long currentTime = 0;
-
-			for (int i = 0; i < numElements; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-				currentTime = currentTime + 10;
-				testHarness.setProcessingTime(currentTime);
-			}
-
-			List<Integer> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-			assertEquals(numElements, result.size());
-
-			Collections.sort(result);
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(i, result.get(i).intValue());
-			}
-
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testTumblingWindowWithProcessFunction() throws Exception {
-		try {
-			final int windowSize = 50;
-
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-				new AccumulatingProcessingTimeWindowOperator<>(
-					validatingIdentityProcessFunction, identitySelector,
-					IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-					windowSize, windowSize);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Integer> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(op, identitySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.open();
-
-			final int numElements = 1000;
-
-			long currentTime = 0;
-
-			for (int i = 0; i < numElements; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-				currentTime = currentTime + 10;
-				testHarness.setProcessingTime(currentTime);
-			}
-
-			List<Integer> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-			assertEquals(numElements, result.size());
-
-			Collections.sort(result);
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(i, result.get(i).intValue());
-			}
-
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSlidingWindow() throws Exception {
-
-		// tumbling window that triggers every 20 milliseconds
-		AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-				new AccumulatingProcessingTimeWindowOperator<>(
-						validatingIdentityFunction, identitySelector,
-						IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50);
-
-		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Integer> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(op, identitySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-		testHarness.open();
-
-		final int numElements = 1000;
-
-		long currentTime = 0;
-
-		for (int i = 0; i < numElements; i++) {
-			testHarness.processElement(new StreamRecord<>(i));
-			currentTime = currentTime + 10;
-			testHarness.setProcessingTime(currentTime);
-		}
-
-		// get and verify the result
-		List<Integer> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-
-		// if we kept this running, each element would be in the result three times (for each slide).
-		// we are closing the window before the final panes are through three times, so we may have less
-		// elements.
-		if (result.size() < numElements || result.size() > 3 * numElements) {
-			fail("Wrong number of results: " + result.size());
-		}
-
-		Collections.sort(result);
-		int lastNum = -1;
-		int lastCount = -1;
-
-		for (int num : result) {
-			if (num == lastNum) {
-				lastCount++;
-				assertTrue(lastCount <= 3);
-			}
-			else {
-				lastNum = num;
-				lastCount = 1;
-			}
-		}
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testSlidingWindowWithProcessFunction() throws Exception {
-
-		// tumbling window that triggers every 20 milliseconds
-		AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-			new AccumulatingProcessingTimeWindowOperator<>(
-				validatingIdentityProcessFunction, identitySelector,
-				IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50);
-
-		KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Integer> testHarness =
-			new KeyedOneInputStreamOperatorTestHarness<>(op, identitySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-		testHarness.open();
-
-		final int numElements = 1000;
-
-		long currentTime = 0;
-
-		for (int i = 0; i < numElements; i++) {
-			testHarness.processElement(new StreamRecord<>(i));
-			currentTime = currentTime + 10;
-			testHarness.setProcessingTime(currentTime);
-		}
-
-		// get and verify the result
-		List<Integer> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-
-		// if we kept this running, each element would be in the result three times (for each slide).
-		// we are closing the window before the final panes are through three times, so we may have less
-		// elements.
-		if (result.size() < numElements || result.size() > 3 * numElements) {
-			fail("Wrong number of results: " + result.size());
-		}
-
-		Collections.sort(result);
-		int lastNum = -1;
-		int lastCount = -1;
-
-		for (int num : result) {
-			if (num == lastNum) {
-				lastCount++;
-				assertTrue(lastCount <= 3);
-			}
-			else {
-				lastNum = num;
-				lastCount = 1;
-			}
-		}
-
-		testHarness.close();
-	}
-
-	@Test
-	public void testTumblingWindowSingleElements() throws Exception {
-
-		try {
-
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE, 50, 50);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Integer> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, identitySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.open();
-
-			testHarness.setProcessingTime(0);
-
-			testHarness.processElement(new StreamRecord<>(1));
-			testHarness.processElement(new StreamRecord<>(2));
-
-			testHarness.setProcessingTime(50);
-
-			testHarness.processElement(new StreamRecord<>(3));
-			testHarness.processElement(new StreamRecord<>(4));
-			testHarness.processElement(new StreamRecord<>(5));
-
-			testHarness.setProcessingTime(100);
-
-			testHarness.processElement(new StreamRecord<>(6));
-
-			testHarness.setProcessingTime(200);
-
-			List<Integer> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-			assertEquals(6, result.size());
-
-			Collections.sort(result);
-			assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6), result);
-
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testTumblingWindowSingleElementsWithProcessFunction() throws Exception {
-
-		try {
-
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-				new AccumulatingProcessingTimeWindowOperator<>(
-					validatingIdentityProcessFunction, identitySelector,
-					IntSerializer.INSTANCE, IntSerializer.INSTANCE, 50, 50);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Integer> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(op, identitySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.open();
-
-			testHarness.setProcessingTime(0);
-
-			testHarness.processElement(new StreamRecord<>(1));
-			testHarness.processElement(new StreamRecord<>(2));
-
-			testHarness.setProcessingTime(50);
-
-			testHarness.processElement(new StreamRecord<>(3));
-			testHarness.processElement(new StreamRecord<>(4));
-			testHarness.processElement(new StreamRecord<>(5));
-
-			testHarness.setProcessingTime(100);
-
-			testHarness.processElement(new StreamRecord<>(6));
-
-			testHarness.setProcessingTime(200);
-
-			List<Integer> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-			assertEquals(6, result.size());
-
-			Collections.sort(result);
-			assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6), result);
-
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSlidingWindowSingleElements() throws Exception {
-		try {
-
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Integer> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, identitySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.setProcessingTime(0);
-
-			testHarness.open();
-
-			testHarness.processElement(new StreamRecord<>(1));
-			testHarness.processElement(new StreamRecord<>(2));
-
-			testHarness.setProcessingTime(50);
-			testHarness.setProcessingTime(100);
-			testHarness.setProcessingTime(150);
-
-			List<Integer> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-
-			assertEquals(6, result.size());
-
-			Collections.sort(result);
-			assertEquals(Arrays.asList(1, 1, 1, 2, 2, 2), result);
-
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSlidingWindowSingleElementsWithProcessFunction() throws Exception {
-		try {
-
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-				new AccumulatingProcessingTimeWindowOperator<>(
-					validatingIdentityProcessFunction, identitySelector,
-					IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Integer, Integer> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(op, identitySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.setProcessingTime(0);
-
-			testHarness.open();
-
-			testHarness.processElement(new StreamRecord<>(1));
-			testHarness.processElement(new StreamRecord<>(2));
-
-			testHarness.setProcessingTime(50);
-			testHarness.setProcessingTime(100);
-			testHarness.setProcessingTime(150);
-
-			List<Integer> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-
-			assertEquals(6, result.size());
-
-			Collections.sort(result);
-			assertEquals(Arrays.asList(1, 1, 1, 2, 2, 2), result);
-
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void checkpointRestoreWithPendingWindowTumblingWithProcessFunction() {
-		try {
-			final int windowSize = 200;
-
-			// tumbling window that triggers every 200 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-				new AccumulatingProcessingTimeWindowOperator<>(
-					validatingIdentityProcessFunction, identitySelector,
-					IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-					windowSize, windowSize);
-
-			OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
-				new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setup();
-			testHarness.open();
-
-			testHarness.setProcessingTime(0);
-
-			// inject some elements
-			final int numElementsFirst = 700;
-			final int numElements = 1000;
-			for (int i = 0; i < numElementsFirst; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-			}
-
-			// draw a snapshot and dispose the window
-			int beforeSnapShot = testHarness.getOutput().size();
-			StreamStateHandle state = testHarness.snapshotLegacy(1L, System.currentTimeMillis());
-			List<Integer> resultAtSnapshot = extractFromStreamRecords(testHarness.getOutput());
-			int afterSnapShot = testHarness.getOutput().size();
-			assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
-			assertTrue(afterSnapShot <= numElementsFirst);
-
-			// inject some random elements, which should not show up in the state
-			for (int i = 0; i < 300; i++) {
-				testHarness.processElement(new StreamRecord<>(i + numElementsFirst));
-			}
-
-			testHarness.close();
-			op.dispose();
-
-			// re-create the operator and restore the state
-			op = new AccumulatingProcessingTimeWindowOperator<>(
-				validatingIdentityProcessFunction, identitySelector,
-				IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-				windowSize, windowSize);
-
-			testHarness = new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setup();
-			testHarness.restore(state);
-			testHarness.open();
-
-			// inject some more elements
-			for (int i = numElementsFirst; i < numElements; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-			}
-
-			testHarness.setProcessingTime(400);
-
-			// get and verify the result
-			List<Integer> finalResult = new ArrayList<>();
-			finalResult.addAll(resultAtSnapshot);
-			List<Integer> finalPartialResult = extractFromStreamRecords(testHarness.getOutput());
-			finalResult.addAll(finalPartialResult);
-			assertEquals(numElements, finalResult.size());
-
-			Collections.sort(finalResult);
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(i, finalResult.get(i).intValue());
-			}
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void checkpointRestoreWithPendingWindowTumbling() {
-		try {
-			final int windowSize = 200;
-
-			// tumbling window that triggers every 200 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSize);
-
-			OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
-					new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setup();
-			testHarness.open();
-
-			testHarness.setProcessingTime(0);
-
-			// inject some elements
-			final int numElementsFirst = 700;
-			final int numElements = 1000;
-			for (int i = 0; i < numElementsFirst; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-			}
-
-			// draw a snapshot and dispose the window
-			int beforeSnapShot = testHarness.getOutput().size();
-			StreamStateHandle state = testHarness.snapshotLegacy(1L, System.currentTimeMillis());
-			List<Integer> resultAtSnapshot = extractFromStreamRecords(testHarness.getOutput());
-			int afterSnapShot = testHarness.getOutput().size();
-			assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
-			assertTrue(afterSnapShot <= numElementsFirst);
-
-			// inject some random elements, which should not show up in the state
-			for (int i = 0; i < 300; i++) {
-				testHarness.processElement(new StreamRecord<>(i + numElementsFirst));
-			}
-
-			testHarness.close();
-			op.dispose();
-
-			// re-create the operator and restore the state
-			op = new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSize);
-
-			testHarness = new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setup();
-			testHarness.restore(state);
-			testHarness.open();
-
-			// inject some more elements
-			for (int i = numElementsFirst; i < numElements; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-			}
-
-			testHarness.setProcessingTime(400);
-
-			// get and verify the result
-			List<Integer> finalResult = new ArrayList<>();
-			finalResult.addAll(resultAtSnapshot);
-			List<Integer> finalPartialResult = extractFromStreamRecords(testHarness.getOutput());
-			finalResult.addAll(finalPartialResult);
-			assertEquals(numElements, finalResult.size());
-
-			Collections.sort(finalResult);
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(i, finalResult.get(i).intValue());
-			}
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void checkpointRestoreWithPendingWindowSlidingWithProcessFunction() {
-		try {
-			final int factor = 4;
-			final int windowSlide = 50;
-			final int windowSize = factor * windowSlide;
-
-			// sliding window (200 msecs) every 50 msecs
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-				new AccumulatingProcessingTimeWindowOperator<>(
-					validatingIdentityProcessFunction, identitySelector,
-					IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-					windowSize, windowSlide);
-
-			OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
-				new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setProcessingTime(0);
-
-			testHarness.setup();
-			testHarness.open();
-
-			// inject some elements
-			final int numElements = 1000;
-			final int numElementsFirst = 700;
-
-			for (int i = 0; i < numElementsFirst; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-			}
-
-			// draw a snapshot
-			List<Integer> resultAtSnapshot = extractFromStreamRecords(testHarness.getOutput());
-			int beforeSnapShot = testHarness.getOutput().size();
-			StreamStateHandle state = testHarness.snapshotLegacy(1L, System.currentTimeMillis());
-			int afterSnapShot = testHarness.getOutput().size();
-			assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
-
-			assertTrue(resultAtSnapshot.size() <= factor * numElementsFirst);
-
-			// inject the remaining elements - these should not influence the snapshot
-			for (int i = numElementsFirst; i < numElements; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-			}
-
-			testHarness.close();
-
-			// re-create the operator and restore the state
-			op = new AccumulatingProcessingTimeWindowOperator<>(
-				validatingIdentityProcessFunction, identitySelector,
-				IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-				windowSize, windowSlide);
-
-			testHarness = new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setup();
-			testHarness.restore(state);
-			testHarness.open();
-
-			// inject again the remaining elements
-			for (int i = numElementsFirst; i < numElements; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-			}
-
-			testHarness.setProcessingTime(50);
-			testHarness.setProcessingTime(100);
-			testHarness.setProcessingTime(150);
-			testHarness.setProcessingTime(200);
-			testHarness.setProcessingTime(250);
-			testHarness.setProcessingTime(300);
-			testHarness.setProcessingTime(350);
-
-			// get and verify the result
-			List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
-			List<Integer> finalPartialResult = extractFromStreamRecords(testHarness.getOutput());
-			finalResult.addAll(finalPartialResult);
-			assertEquals(factor * numElements, finalResult.size());
-
-			Collections.sort(finalResult);
-			for (int i = 0; i < factor * numElements; i++) {
-				assertEquals(i / factor, finalResult.get(i).intValue());
-			}
-
-			testHarness.close();
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void checkpointRestoreWithPendingWindowSliding() {
-		try {
-			final int factor = 4;
-			final int windowSlide = 50;
-			final int windowSize = factor * windowSlide;
-
-			// sliding window (200 msecs) every 50 msecs
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSlide);
-
-			OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
-					new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setProcessingTime(0);
-
-			testHarness.setup();
-			testHarness.open();
-
-			// inject some elements
-			final int numElements = 1000;
-			final int numElementsFirst = 700;
-
-			for (int i = 0; i < numElementsFirst; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-			}
-
-			// draw a snapshot
-			List<Integer> resultAtSnapshot = extractFromStreamRecords(testHarness.getOutput());
-			int beforeSnapShot = testHarness.getOutput().size();
-			StreamStateHandle state = testHarness.snapshotLegacy(1L, System.currentTimeMillis());
-			int afterSnapShot = testHarness.getOutput().size();
-			assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
-
-			assertTrue(resultAtSnapshot.size() <= factor * numElementsFirst);
-
-			// inject the remaining elements - these should not influence the snapshot
-			for (int i = numElementsFirst; i < numElements; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-			}
-
-			testHarness.close();
-
-			// re-create the operator and restore the state
-			op = new AccumulatingProcessingTimeWindowOperator<>(
-					validatingIdentityFunction, identitySelector,
-					IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-					windowSize, windowSlide);
-
-			testHarness = new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setup();
-			testHarness.restore(state);
-			testHarness.open();
-
-			// inject again the remaining elements
-			for (int i = numElementsFirst; i < numElements; i++) {
-				testHarness.processElement(new StreamRecord<>(i));
-			}
-
-			testHarness.setProcessingTime(50);
-			testHarness.setProcessingTime(100);
-			testHarness.setProcessingTime(150);
-			testHarness.setProcessingTime(200);
-			testHarness.setProcessingTime(250);
-			testHarness.setProcessingTime(300);
-			testHarness.setProcessingTime(350);
-
-			// get and verify the result
-			List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
-			List<Integer> finalPartialResult = extractFromStreamRecords(testHarness.getOutput());
-			finalResult.addAll(finalPartialResult);
-			assertEquals(factor * numElements, finalResult.size());
-
-			Collections.sort(finalResult);
-			for (int i = 0; i < factor * numElements; i++) {
-				assertEquals(i / factor, finalResult.get(i).intValue());
-			}
-
-			testHarness.close();
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testKeyValueStateInWindowFunction() {
-		try {
-
-			StatefulFunction.globalCounts.clear();
-
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							new InternalIterableProcessWindowFunction<>(new StatefulFunction()),
-							identitySelector,
-							IntSerializer.INSTANCE,
-							IntSerializer.INSTANCE,
-							50,
-							50);
-
-			OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, identitySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.open();
-
-			testHarness.setProcessingTime(0);
-
-			testHarness.processElement(new StreamRecord<>(1));
-			testHarness.processElement(new StreamRecord<>(2));
-
-			op.processElement(new StreamRecord<>(1));
-			op.processElement(new StreamRecord<>(2));
-			op.processElement(new StreamRecord<>(1));
-			op.processElement(new StreamRecord<>(1));
-			op.processElement(new StreamRecord<>(2));
-			op.processElement(new StreamRecord<>(2));
-
-			testHarness.setProcessingTime(1000);
-
-			List<Integer> result = extractFromStreamRecords(testHarness.getOutput());
-			assertEquals(8, result.size());
-
-			Collections.sort(result);
-			assertEquals(Arrays.asList(1, 1, 1, 1, 2, 2, 2, 2), result);
-
-			assertEquals(4, StatefulFunction.globalCounts.get(1).intValue());
-			assertEquals(4, StatefulFunction.globalCounts.get(2).intValue());
-
-			testHarness.close();
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private void assertInvalidParameter(long windowSize, long windowSlide) {
-		try {
-			new AccumulatingProcessingTimeWindowOperator<String, String, String>(
-					mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE,
-					windowSize, windowSlide);
-			fail("This should fail with an IllegalArgumentException");
-		}
-		catch (IllegalArgumentException e) {
-			// expected
-		}
-		catch (Exception e) {
-			fail("Wrong exception. Expected IllegalArgumentException but found " + e.getClass().getSimpleName());
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static class StatefulFunction extends ProcessWindowFunction<Integer, Integer, Integer, TimeWindow> {
-
-		// we use a concurrent map here even though there is no concurrency, to
-		// get "volatile" style access to entries
-		private static final Map<Integer, Integer> globalCounts = new ConcurrentHashMap<>();
-
-		private ValueState<Integer> state;
-
-		@Override
-		public void open(Configuration parameters) {
-			assertNotNull(getRuntimeContext());
-			state = getRuntimeContext().getState(
-					new ValueStateDescriptor<>("totalCount", Integer.class, 0));
-		}
-
-		@Override
-		public void process(Integer key,
-						Context context,
-						Iterable<Integer> values,
-						Collector<Integer> out) throws Exception {
-			for (Integer i : values) {
-				// we need to update this state before emitting elements. Else, the test's main
-				// thread will have received all output elements before the state is updated and
-				// the checks may fail
-				state.update(state.value() + 1);
-				globalCounts.put(key, state.value());
-
-				out.collect(i);
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static StreamTask<?, ?> createMockTask() {
-		Configuration configuration = new Configuration();
-		configuration.setString(CoreOptions.STATE_BACKEND, "jobmanager");
-
-		StreamTask<?, ?> task = mock(StreamTask.class);
-		when(task.getAccumulatorMap()).thenReturn(new HashMap<String, Accumulator<?, ?>>());
-		when(task.getName()).thenReturn("Test task name");
-		when(task.getExecutionConfig()).thenReturn(new ExecutionConfig());
-
-		final TaskManagerRuntimeInfo mockTaskManagerRuntimeInfo = mock(TaskManagerRuntimeInfo.class);
-		when(mockTaskManagerRuntimeInfo.getConfiguration()).thenReturn(configuration);
-
-		final Environment env = mock(Environment.class);
-		when(env.getTaskInfo()).thenReturn(new TaskInfo("Test task name", 1, 0, 1, 0));
-		when(env.getUserClassLoader()).thenReturn(AggregatingAlignedProcessingTimeWindowOperatorTest.class.getClassLoader());
-		when(env.getMetricGroup()).thenReturn(new UnregisteredTaskMetricsGroup());
-		when(env.getTaskManagerInfo()).thenReturn(new TestingTaskManagerRuntimeInfo());
-
-		when(task.getEnvironment()).thenReturn(env);
-		return task;
-	}
-
-	private static StreamTask<?, ?> createMockTaskWithTimer(
-		final ProcessingTimeService timerService) {
-		StreamTask<?, ?> mockTask = createMockTask();
-		when(mockTask.getProcessingTimeService()).thenReturn(timerService);
-		return mockTask;
-	}
-
-	@SuppressWarnings({"unchecked", "rawtypes"})
-	private <T> List<T> extractFromStreamRecords(Iterable<?> input) {
-		List<T> result = new ArrayList<>();
-		for (Object in : input) {
-			if (in instanceof StreamRecord) {
-				result.add((T) ((StreamRecord) in).getValue());
-			}
-		}
-		return result;
-	}
-
-	private static void shutdownTimerServiceAndWait(ProcessingTimeService timers) throws Exception {
-		timers.shutdownService();
-
-		while (!timers.isTerminated()) {
-			Thread.sleep(2);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
deleted file mode 100644
index 62f4f0b..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
+++ /dev/null
@@ -1,863 +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.functions.RichReduceFunction;
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.state.StreamStateHandle;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-
-import org.junit.After;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for aligned {@link AggregatingProcessingTimeWindowOperator}.
- */
-@SuppressWarnings("serial")
-public class AggregatingAlignedProcessingTimeWindowOperatorTest {
-
-	@SuppressWarnings("unchecked")
-	private final ReduceFunction<String> mockFunction = mock(ReduceFunction.class);
-
-	@SuppressWarnings("unchecked")
-	private final KeySelector<String, String> mockKeySelector = mock(KeySelector.class);
-
-	private final KeySelector<Tuple2<Integer, Integer>, Integer> fieldOneSelector =
-			new KeySelector<Tuple2<Integer, Integer>, Integer>() {
-				@Override
-				public Integer getKey(Tuple2<Integer, Integer> value) {
-					return value.f0;
-				}
-	};
-
-	private final ReduceFunction<Tuple2<Integer, Integer>> sumFunction = new ReduceFunction<Tuple2<Integer, Integer>>() {
-		@Override
-		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> value1, Tuple2<Integer, Integer> value2) {
-			return new Tuple2<>(value1.f0, value1.f1 + value2.f1);
-		}
-	};
-
-	private final TypeSerializer<Tuple2<Integer, Integer>> tupleSerializer =
-			new TupleTypeInfo<Tuple2<Integer, Integer>>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO)
-					.createSerializer(new ExecutionConfig());
-
-	private final Comparator<Tuple2<Integer, Integer>> tupleComparator = new Comparator<Tuple2<Integer, Integer>>() {
-		@Override
-		public int compare(Tuple2<Integer, Integer> o1, Tuple2<Integer, Integer> o2) {
-			int diff0 = o1.f0 - o2.f0;
-			int diff1 = o1.f1 - o2.f1;
-			return diff0 != 0 ? diff0 : diff1;
-		}
-	};
-
-	// ------------------------------------------------------------------------
-
-	public AggregatingAlignedProcessingTimeWindowOperatorTest() {
-		ClosureCleaner.clean(fieldOneSelector, false);
-		ClosureCleaner.clean(sumFunction, false);
-	}
-
-	// ------------------------------------------------------------------------
-
-	@After
-	public void checkNoTriggerThreadsRunning() {
-		// make sure that all the threads we trigger are shut down
-		long deadline = System.currentTimeMillis() + 5000;
-		while (StreamTask.TRIGGER_THREAD_GROUP.activeCount() > 0 && System.currentTimeMillis() < deadline) {
-			try {
-				Thread.sleep(10);
-			}
-			catch (InterruptedException ignored) {}
-		}
-
-		assertTrue("Not all trigger threads where properly shut down",
-				StreamTask.TRIGGER_THREAD_GROUP.activeCount() == 0);
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testInvalidParameters() {
-		try {
-			assertInvalidParameter(-1L, -1L);
-			assertInvalidParameter(10000L, -1L);
-			assertInvalidParameter(-1L, 1000L);
-			assertInvalidParameter(1000L, 2000L);
-
-			// actual internal slide is too low here:
-			assertInvalidParameter(1000L, 999L);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testWindowSizeAndSlide() {
-		try {
-			AggregatingProcessingTimeWindowOperator<String, String> op;
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
-			assertEquals(5000, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(1000, op.getPaneSize());
-			assertEquals(5, op.getNumPanesPerWindow());
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
-			assertEquals(1000, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(1000, op.getPaneSize());
-			assertEquals(1, op.getNumPanesPerWindow());
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
-			assertEquals(1500, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(500, op.getPaneSize());
-			assertEquals(3, op.getNumPanesPerWindow());
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
-			assertEquals(1200, op.getWindowSize());
-			assertEquals(1100, op.getWindowSlide());
-			assertEquals(100, op.getPaneSize());
-			assertEquals(12, op.getNumPanesPerWindow());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testWindowTriggerTimeAlignment() throws Exception {
-		try {
-
-			AggregatingProcessingTimeWindowOperator<String, String> op =
-					new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-						StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
-
-			KeyedOneInputStreamOperatorTestHarness<String, String, String> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, mockKeySelector, BasicTypeInfo.STRING_TYPE_INFO);
-			testHarness.open();
-
-			assertTrue(op.getNextSlideTime() % 1000 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			testHarness.close();
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
-
-			testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, mockKeySelector, BasicTypeInfo.STRING_TYPE_INFO);
-			testHarness.open();
-
-			assertTrue(op.getNextSlideTime() % 1000 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			testHarness.close();
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
-
-			testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, mockKeySelector, BasicTypeInfo.STRING_TYPE_INFO);
-			testHarness.open();
-
-			assertTrue(op.getNextSlideTime() % 500 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			testHarness.close();
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
-
-			testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, mockKeySelector, BasicTypeInfo.STRING_TYPE_INFO);
-			testHarness.open();
-
-			assertTrue(op.getNextSlideTime() % 100 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1100 == 0);
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testTumblingWindowUniqueElements() throws Exception {
-
-		try {
-			final int windowSize = 50;
-
-			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, fieldOneSelector,
-							IntSerializer.INSTANCE, tupleSerializer,
-							windowSize, windowSize);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, fieldOneSelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.open();
-
-			final int numElements = 1000;
-
-			long currentTime = 0;
-
-			for (int i = 0; i < numElements; i++) {
-				StreamRecord<Tuple2<Integer, Integer>> next = new StreamRecord<>(new Tuple2<>(i, i));
-				testHarness.processElement(next);
-				currentTime = currentTime + 10;
-				testHarness.setProcessingTime(currentTime);
-			}
-
-			// get and verify the result
-			List<Tuple2<Integer, Integer>> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-			assertEquals(numElements, result.size());
-
-			testHarness.close();
-
-			Collections.sort(result, tupleComparator);
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(i, result.get(i).f0.intValue());
-				assertEquals(i, result.get(i).f1.intValue());
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testTumblingWindowDuplicateElements() throws Exception {
-		try {
-			final int windowSize = 50;
-
-			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, fieldOneSelector,
-							IntSerializer.INSTANCE, tupleSerializer,
-							windowSize, windowSize);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, fieldOneSelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.setProcessingTime(0);
-			testHarness.open();
-
-			final int numWindows = 10;
-
-			long previousNextTime = 0;
-			int window = 1;
-
-			long currentTime = 0;
-
-			while (window <= numWindows) {
-				long nextTime = op.getNextEvaluationTime();
-				int val = ((int) nextTime) ^ ((int) (nextTime >>> 32));
-
-				StreamRecord<Tuple2<Integer, Integer>> next =  new StreamRecord<>(new Tuple2<>(val, val));
-				testHarness.processElement(next);
-
-				if (nextTime != previousNextTime) {
-					window++;
-					previousNextTime = nextTime;
-				}
-				currentTime = currentTime + 1;
-				testHarness.setProcessingTime(currentTime);
-			}
-
-			testHarness.setProcessingTime(currentTime + 100);
-
-			List<Tuple2<Integer, Integer>> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-
-			testHarness.close();
-
-			// we have ideally one element per window. we may have more, when we emitted a value into the
-			// successive window (corner case), so we can have twice the number of elements, in the worst case.
-			assertTrue(result.size() >= numWindows && result.size() <= 2 * numWindows);
-
-			// deduplicate for more accurate checks
-			HashSet<Tuple2<Integer, Integer>> set = new HashSet<>(result);
-			assertTrue(set.size() == 10);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSlidingWindow() throws Exception {
-		try {
-			// tumbling window that triggers every 20 milliseconds
-			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, fieldOneSelector,
-							IntSerializer.INSTANCE, tupleSerializer,
-							150, 50);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, fieldOneSelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.open();
-
-			final int numElements = 1000;
-
-			long currentTime = 0;
-
-			for (int i = 0; i < numElements; i++) {
-				StreamRecord<Tuple2<Integer, Integer>> next = new StreamRecord<>(new Tuple2<>(i, i));
-				testHarness.processElement(next);
-				currentTime = currentTime + 1;
-				testHarness.setProcessingTime(currentTime);
-			}
-
-			// get and verify the result
-			List<Tuple2<Integer, Integer>> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-
-			testHarness.close();
-
-			// every element can occur between one and three times
-			if (result.size() < numElements || result.size() > 3 * numElements) {
-				System.out.println(result);
-				fail("Wrong number of results: " + result.size());
-			}
-
-			Collections.sort(result, tupleComparator);
-			int lastNum = -1;
-			int lastCount = -1;
-
-			for (Tuple2<Integer, Integer> val : result) {
-				assertEquals(val.f0, val.f1);
-
-				if (val.f0 == lastNum) {
-					lastCount++;
-					assertTrue(lastCount <= 3);
-				}
-				else {
-					lastNum = val.f0;
-					lastCount = 1;
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSlidingWindowSingleElements() throws Exception {
-		try {
-			// tumbling window that triggers every 20 milliseconds
-			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, fieldOneSelector,
-							IntSerializer.INSTANCE, tupleSerializer, 150, 50);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, fieldOneSelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.open();
-
-			testHarness.setProcessingTime(0);
-
-			StreamRecord<Tuple2<Integer, Integer>> next1 = new StreamRecord<>(new Tuple2<>(1, 1));
-			testHarness.processElement(next1);
-
-			StreamRecord<Tuple2<Integer, Integer>> next2 = new StreamRecord<>(new Tuple2<>(2, 2));
-			testHarness.processElement(next2);
-
-			testHarness.setProcessingTime(50);
-			testHarness.setProcessingTime(100);
-			testHarness.setProcessingTime(150);
-
-			List<Tuple2<Integer, Integer>> result = extractFromStreamRecords(testHarness.extractOutputStreamRecords());
-			assertEquals(6, result.size());
-
-			Collections.sort(result, tupleComparator);
-			assertEquals(Arrays.asList(
-					new Tuple2<>(1, 1),
-					new Tuple2<>(1, 1),
-					new Tuple2<>(1, 1),
-					new Tuple2<>(2, 2),
-					new Tuple2<>(2, 2),
-					new Tuple2<>(2, 2)
-			), result);
-
-			testHarness.close();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testPropagateExceptionsFromProcessElement() throws Exception {
-
-		try {
-			ReduceFunction<Tuple2<Integer, Integer>> failingFunction = new FailingFunction(100);
-
-			// the operator has a window time that is so long that it will not fire in this test
-			final long hundredYears = 100L * 365 * 24 * 60 * 60 * 1000;
-			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							failingFunction, fieldOneSelector,
-							IntSerializer.INSTANCE, tupleSerializer,
-							hundredYears, hundredYears);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testHarness =
-					new KeyedOneInputStreamOperatorTestHarness<>(op, fieldOneSelector, BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.open();
-
-			for (int i = 0; i < 100; i++) {
-				StreamRecord<Tuple2<Integer, Integer>> next = new StreamRecord<>(new Tuple2<>(1, 1));
-				testHarness.processElement(next);
-			}
-
-			try {
-				StreamRecord<Tuple2<Integer, Integer>> next = new StreamRecord<>(new Tuple2<>(1, 1));
-				testHarness.processElement(next);
-				fail("This fail with an exception");
-			}
-			catch (Exception e) {
-				assertTrue(e.getMessage().contains("Artificial Test Exception"));
-			}
-
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void checkpointRestoreWithPendingWindowTumbling() {
-		try {
-			final int windowSize = 200;
-
-			// tumbling window that triggers every 50 milliseconds
-			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, fieldOneSelector,
-							IntSerializer.INSTANCE, tupleSerializer,
-							windowSize, windowSize);
-
-			OneInputStreamOperatorTestHarness<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testHarness =
-					new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setProcessingTime(0);
-
-			testHarness.setup();
-			testHarness.open();
-
-			// inject some elements
-			final int numElementsFirst = 700;
-			final int numElements = 1000;
-
-			for (int i = 0; i < numElementsFirst; i++) {
-				StreamRecord<Tuple2<Integer, Integer>> next = new StreamRecord<>(new Tuple2<>(i, i));
-				testHarness.processElement(next);
-			}
-
-			// draw a snapshot
-			List<Tuple2<Integer, Integer>> resultAtSnapshot = extractFromStreamRecords(testHarness.getOutput());
-			int beforeSnapShot = resultAtSnapshot.size();
-			StreamStateHandle state = testHarness.snapshotLegacy(1L, System.currentTimeMillis());
-			int afterSnapShot = testHarness.getOutput().size();
-			assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
-
-			assertTrue(resultAtSnapshot.size() <= numElementsFirst);
-
-			// inject some random elements, which should not show up in the state
-			for (int i = numElementsFirst; i < numElements; i++) {
-				StreamRecord<Tuple2<Integer, Integer>> next = new StreamRecord<>(new Tuple2<>(i, i));
-				testHarness.processElement(next);
-			}
-
-			testHarness.close();
-			op.dispose();
-
-			// re-create the operator and restore the state
-			op = new AggregatingProcessingTimeWindowOperator<>(
-					sumFunction, fieldOneSelector,
-					IntSerializer.INSTANCE, tupleSerializer,
-					windowSize, windowSize);
-
-			testHarness = new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setup();
-			testHarness.restore(state);
-			testHarness.open();
-
-			// inject the remaining elements
-			for (int i = numElementsFirst; i < numElements; i++) {
-				StreamRecord<Tuple2<Integer, Integer>> next = new StreamRecord<>(new Tuple2<>(i, i));
-				testHarness.processElement(next);
-			}
-
-			testHarness.setProcessingTime(200);
-
-			// get and verify the result
-			List<Tuple2<Integer, Integer>> finalResult = new ArrayList<>(resultAtSnapshot);
-			List<Tuple2<Integer, Integer>> partialFinalResult = extractFromStreamRecords(testHarness.getOutput());
-			finalResult.addAll(partialFinalResult);
-			assertEquals(numElements, finalResult.size());
-
-			Collections.sort(finalResult, tupleComparator);
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(i, finalResult.get(i).f0.intValue());
-				assertEquals(i, finalResult.get(i).f1.intValue());
-			}
-
-			testHarness.close();
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void checkpointRestoreWithPendingWindowSliding() {
-		try {
-			final int factor = 4;
-			final int windowSlide = 50;
-			final int windowSize = factor * windowSlide;
-
-			// sliding window (200 msecs) every 50 msecs
-			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, fieldOneSelector,
-							IntSerializer.INSTANCE, tupleSerializer,
-							windowSize, windowSlide);
-
-			OneInputStreamOperatorTestHarness<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testHarness =
-					new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setProcessingTime(0);
-
-			testHarness.setup();
-			testHarness.open();
-
-			// inject some elements
-			final int numElements = 1000;
-			final int numElementsFirst = 700;
-
-			for (int i = 0; i < numElementsFirst; i++) {
-				StreamRecord<Tuple2<Integer, Integer>> next = new StreamRecord<>(new Tuple2<>(i, i));
-				testHarness.processElement(next);
-			}
-
-			// draw a snapshot
-			List<Tuple2<Integer, Integer>> resultAtSnapshot = extractFromStreamRecords(testHarness.getOutput());
-			int beforeSnapShot = resultAtSnapshot.size();
-			StreamStateHandle state = testHarness.snapshotLegacy(1L, System.currentTimeMillis());
-			int afterSnapShot = testHarness.getOutput().size();
-			assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
-
-			assertTrue(resultAtSnapshot.size() <= factor * numElementsFirst);
-
-			// inject the remaining elements - these should not influence the snapshot
-			for (int i = numElementsFirst; i < numElements; i++) {
-				StreamRecord<Tuple2<Integer, Integer>> next = new StreamRecord<>(new Tuple2<>(i, i));
-				testHarness.processElement(next);
-			}
-
-			testHarness.close();
-			op.dispose();
-
-			// re-create the operator and restore the state
-			op = new AggregatingProcessingTimeWindowOperator<>(
-					sumFunction, fieldOneSelector,
-					IntSerializer.INSTANCE, tupleSerializer,
-					windowSize, windowSlide);
-
-			testHarness = new OneInputStreamOperatorTestHarness<>(op);
-
-			testHarness.setup();
-			testHarness.restore(state);
-			testHarness.open();
-
-			// inject again the remaining elements
-			for (int i = numElementsFirst; i < numElements; i++) {
-				StreamRecord<Tuple2<Integer, Integer>> next = new StreamRecord<>(new Tuple2<>(i, i));
-				testHarness.processElement(next);
-			}
-
-			testHarness.setProcessingTime(50);
-			testHarness.setProcessingTime(100);
-			testHarness.setProcessingTime(150);
-			testHarness.setProcessingTime(200);
-			testHarness.setProcessingTime(250);
-			testHarness.setProcessingTime(300);
-			testHarness.setProcessingTime(350);
-			testHarness.setProcessingTime(400);
-
-			// get and verify the result
-			List<Tuple2<Integer, Integer>> finalResult = new ArrayList<>(resultAtSnapshot);
-			List<Tuple2<Integer, Integer>> partialFinalResult = extractFromStreamRecords(testHarness.getOutput());
-			finalResult.addAll(partialFinalResult);
-			assertEquals(numElements * factor, finalResult.size());
-
-			Collections.sort(finalResult, tupleComparator);
-			for (int i = 0; i < factor * numElements; i++) {
-				assertEquals(i / factor, finalResult.get(i).f0.intValue());
-				assertEquals(i / factor, finalResult.get(i).f1.intValue());
-			}
-
-			testHarness.close();
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testKeyValueStateInWindowFunctionTumbling() {
-		try {
-			final long twoSeconds = 2000;
-
-			StatefulFunction.globalCounts.clear();
-
-			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							new StatefulFunction(), fieldOneSelector,
-							IntSerializer.INSTANCE, tupleSerializer, twoSeconds, twoSeconds);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(
-					op,
-					fieldOneSelector,
-					BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.setProcessingTime(0);
-			testHarness.open();
-
-			// because the window interval is so large, everything should be in one window
-			// and aggregate into one value per key
-
-			for (int i = 0; i < 10; i++) {
-				StreamRecord<Tuple2<Integer, Integer>> next1 = new StreamRecord<>(new Tuple2<>(1, i));
-				testHarness.processElement(next1);
-
-				StreamRecord<Tuple2<Integer, Integer>> next2 = new StreamRecord<>(new Tuple2<>(2, i));
-				testHarness.processElement(next2);
-			}
-
-			testHarness.setProcessingTime(1000);
-
-			int count1 = StatefulFunction.globalCounts.get(1);
-			int count2 = StatefulFunction.globalCounts.get(2);
-
-			assertTrue(count1 >= 2 && count1 <= 2 * 10);
-			assertEquals(count1, count2);
-
-			testHarness.close();
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testKeyValueStateInWindowFunctionSliding() {
-		try {
-			final int factor = 2;
-			final int windowSlide = 50;
-			final int windowSize = factor * windowSlide;
-
-			StatefulFunction.globalCounts.clear();
-
-			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							new StatefulFunction(), fieldOneSelector,
-							IntSerializer.INSTANCE, tupleSerializer, windowSize, windowSlide);
-
-			KeyedOneInputStreamOperatorTestHarness<Integer, Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> testHarness = new KeyedOneInputStreamOperatorTestHarness<>(
-					op,
-					fieldOneSelector,
-					BasicTypeInfo.INT_TYPE_INFO);
-
-			testHarness.setProcessingTime(0);
-
-			testHarness.open();
-
-			// because the window interval is so large, everything should be in one window
-			// and aggregate into one value per key
-			final int numElements = 100;
-
-			// because we do not release the lock here, these elements
-			for (int i = 0; i < numElements; i++) {
-
-				StreamRecord<Tuple2<Integer, Integer>> next1 = new StreamRecord<>(new Tuple2<>(1, i));
-				StreamRecord<Tuple2<Integer, Integer>> next2 = new StreamRecord<>(new Tuple2<>(2, i));
-				StreamRecord<Tuple2<Integer, Integer>> next3 = new StreamRecord<>(new Tuple2<>(1, i));
-				StreamRecord<Tuple2<Integer, Integer>> next4 = new StreamRecord<>(new Tuple2<>(2, i));
-
-				testHarness.processElement(next1);
-				testHarness.processElement(next2);
-				testHarness.processElement(next3);
-				testHarness.processElement(next4);
-			}
-
-			testHarness.setProcessingTime(50);
-			testHarness.setProcessingTime(100);
-			testHarness.setProcessingTime(150);
-			testHarness.setProcessingTime(200);
-
-			int count1 = StatefulFunction.globalCounts.get(1);
-			int count2 = StatefulFunction.globalCounts.get(2);
-
-			assertTrue(count1 >= 2 && count1 <= 2 * numElements);
-			assertEquals(count1, count2);
-
-			testHarness.close();
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private void assertInvalidParameter(long windowSize, long windowSlide) {
-		try {
-			new AggregatingProcessingTimeWindowOperator<>(
-					mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE,
-					windowSize, windowSlide);
-			fail("This should fail with an IllegalArgumentException");
-		}
-		catch (IllegalArgumentException e) {
-			// expected
-		}
-		catch (Exception e) {
-			fail("Wrong exception. Expected IllegalArgumentException but found " + e.getClass().getSimpleName());
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static class FailingFunction implements ReduceFunction<Tuple2<Integer, Integer>> {
-
-		private final int failAfterElements;
-
-		private int numElements;
-
-		FailingFunction(int failAfterElements) {
-			this.failAfterElements = failAfterElements;
-		}
-
-		@Override
-		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> value1, Tuple2<Integer, Integer> value2) throws Exception {
-			numElements++;
-
-			if (numElements >= failAfterElements) {
-				throw new Exception("Artificial Test Exception");
-			}
-
-			return new Tuple2<>(value1.f0, value1.f1 + value2.f1);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static class StatefulFunction extends RichReduceFunction<Tuple2<Integer, Integer>> {
-
-		private static final Map<Integer, Integer> globalCounts = new ConcurrentHashMap<>();
-
-		private ValueState<Integer> state;
-
-		@Override
-		public void open(Configuration parameters) {
-			assertNotNull(getRuntimeContext());
-
-			// start with one, so the final count is correct and we test that we do not
-			// initialize with 0 always by default
-			state = getRuntimeContext().getState(new ValueStateDescriptor<>("totalCount", Integer.class, 1));
-		}
-
-		@Override
-		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> value1, Tuple2<Integer, Integer> value2) throws Exception {
-			state.update(state.value() + 1);
-			globalCounts.put(value1.f0, state.value());
-
-			return new Tuple2<>(value1.f0, value1.f1 + value2.f1);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	@SuppressWarnings({"unchecked", "rawtypes"})
-	private <T> List<T> extractFromStreamRecords(Iterable<?> input) {
-		List<T> result = new ArrayList<>();
-		for (Object in : input) {
-			if (in instanceof StreamRecord) {
-				result.add((T) ((StreamRecord) in).getValue());
-			}
-		}
-		return result;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
index a7c6f47..f967a5b 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
@@ -61,7 +61,6 @@ import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
 import org.apache.flink.util.Collector;
 
 import org.junit.Assert;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.concurrent.TimeUnit;
@@ -310,31 +309,6 @@ public class AllWindowTranslationTest {
 		processElementAndEnsureOutput(winOperator, winOperator.getKeySelector(), BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1));
 	}
 
-
-	/**
-	 * Ignored because we currently don't have the fast processing-time window operator.
-	 */
-	@Test
-	@SuppressWarnings("rawtypes")
-	@Ignore
-	public void testReduceFastProcessingTime() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DataStream<Tuple2<String, Integer>> window = source
-				.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.reduce(new DummyReducer());
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform =
-				(OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator = transform.getOperator();
-		Assert.assertTrue(operator instanceof AggregatingProcessingTimeWindowOperator);
-
-		processElementAndEnsureOutput(operator, null, BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1));
-	}
-
 	@Test
 	@SuppressWarnings("rawtypes")
 	public void testReduceWithWindowFunctionEventTime() throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
index dc0e21c..d525ba6 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
@@ -29,13 +29,10 @@ import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.WindowedStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingAlignedProcessingTimeWindows;
 import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingAlignedProcessingTimeWindows;
 import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
@@ -43,7 +40,6 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.util.Collector;
 
 import org.junit.Assert;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.concurrent.TimeUnit;
@@ -96,60 +92,6 @@ public class TimeWindowTranslationTest {
 		Assert.assertTrue(operator2 instanceof WindowOperator);
 	}
 
-	/**
-	 * These tests ensure that the fast aligned time windows operator is used if the
-	 * conditions are right.
-	 */
-	@Test
-	public void testReduceAlignedTimeWindows() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.keyBy(0)
-				.window(SlidingAlignedProcessingTimeWindows.of(Time.of(1000, TimeUnit.MILLISECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.reduce(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
-	}
-
-	/**
-	 * These tests ensure that the fast aligned time windows operator is used if the
-	 * conditions are right.
-	 */
-	@Test
-	public void testApplyAlignedTimeWindows() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.keyBy(0)
-				.window(TumblingAlignedProcessingTimeWindows.of(Time.of(1000, TimeUnit.MILLISECONDS)))
-				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(Tuple tuple,
-							TimeWindow window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof AccumulatingProcessingTimeWindowOperator);
-	}
-
 	@Test
 	@SuppressWarnings("rawtypes")
 	public void testReduceEventTimeWindows() throws Exception {
@@ -232,49 +174,6 @@ public class TimeWindowTranslationTest {
 		Assert.assertTrue(winOperator1.getStateDescriptor() instanceof ListStateDescriptor);
 	}
 
-	/**
-	 * These tests ensure that the fast aligned time windows operator is used if the
-	 * conditions are right.
-	 *
-	 * <p>TODO: update once the fast aligned time windows operator is in
-	 */
-	@Ignore
-	@Test
-	public void testNonParallelFastTimeWindows() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.timeWindowAll(Time.of(1000, TimeUnit.MILLISECONDS),
-						Time.of(100, TimeUnit.MILLISECONDS))
-				.reduce(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.timeWindowAll(Time.of(1000, TimeUnit.MILLISECONDS))
-				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(
-							TimeWindow window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
-		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
-	}
-
 	// ------------------------------------------------------------------------
 	//  UDFs
 	// ------------------------------------------------------------------------


[11/11] flink git commit: [FLINK-7461] Remove Backwards compatibility with <= Flink 1.1

Posted by sr...@apache.org.
[FLINK-7461] Remove Backwards compatibility with <= Flink 1.1


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

Branch: refs/heads/master
Commit: 6642768ad8f8c5d1856742a6d148f7724c20666c
Parents: 5456cf9
Author: Stefan Richter <s....@data-artisans.com>
Authored: Thu Aug 24 17:28:14 2017 +0200
Committer: Stefan Richter <s....@data-artisans.com>
Committed: Thu Aug 24 20:17:08 2017 +0200

----------------------------------------------------------------------
 .../streaming/connectors/fs/RollingSink.java    |   22 +-
 .../connectors/fs/bucketing/BucketingSink.java  |   23 +-
 .../fs/bucketing/RollingSinkMigrationTest.java  |  213 ----
 .../RollingToBucketingMigrationTest.java        |  174 ---
 .../kafka/FlinkKafkaConsumerBase.java           |   20 +-
 .../FlinkKafkaConsumerBaseMigrationTest.java    |    4 +-
 .../kinesis/FlinkKinesisConsumer.java           |   24 +-
 .../FlinkKinesisConsumerMigrationTest.java      |  149 ---
 .../kinesis/FlinkKinesisConsumerTest.java       |   32 -
 .../ExactlyOnceValidatingConsumerThread.java    |   20 +-
 .../state/RocksDBKeyedStateBackend.java         |  194 +--
 .../streaming/state/RocksDBStateBackend.java    |   89 --
 .../state/RocksDBAsyncSnapshotTest.java         |   17 +-
 .../util/MigrationInstantiationUtil.java        |   96 --
 .../flink/migration/util/SerializedValue.java   |   98 --
 .../ContinuousFileProcessingMigrationTest.java  |    1 -
 .../main/java/org/apache/flink/cep/nfa/NFA.java |  100 +-
 .../org/apache/flink/cep/nfa/SharedBuffer.java  |  114 --
 .../flink/cep/nfa/compiler/NFACompiler.java     |  114 --
 .../AbstractKeyedCEPPatternOperator.java        |  325 +----
 .../cep/operator/CEPMigration11to13Test.java    |  346 ------
 .../flink/cep/operator/CEPMigrationTest.java    |    2 +-
 .../MigrationNamespaceSerializerProxy.java      |  132 ---
 .../apache/flink/migration/MigrationUtil.java   |   38 -
 .../api/common/state/ListStateDescriptor.java   |  113 --
 .../runtime/checkpoint/KeyGroupState.java       |   87 --
 .../runtime/checkpoint/SubtaskState.java        |  108 --
 .../migration/runtime/checkpoint/TaskState.java |  164 ---
 .../checkpoint/savepoint/SavepointV0.java       |  115 --
 .../savepoint/SavepointV0Serializer.java        |  425 -------
 .../runtime/state/AbstractCloseableHandle.java  |  131 --
 .../runtime/state/AbstractStateBackend.java     |   75 --
 .../runtime/state/KvStateSnapshot.java          |   32 -
 .../migration/runtime/state/StateHandle.java    |   41 -
 .../migration/runtime/state/StateObject.java    |   58 -
 .../runtime/state/StreamStateHandle.java        |   40 -
 .../filesystem/AbstractFileStateHandle.java     |  101 --
 .../filesystem/AbstractFsStateSnapshot.java     |  118 --
 .../filesystem/FileSerializableStateHandle.java |   76 --
 .../state/filesystem/FileStreamStateHandle.java |   87 --
 .../state/filesystem/FsFoldingState.java        |   44 -
 .../runtime/state/filesystem/FsListState.java   |   46 -
 .../state/filesystem/FsReducingState.java       |   44 -
 .../state/filesystem/FsStateBackend.java        |   54 -
 .../runtime/state/filesystem/FsValueState.java  |   44 -
 .../state/memory/AbstractMemStateSnapshot.java  |  138 ---
 .../AbstractMigrationRestoreStrategy.java       |  119 --
 .../state/memory/ByteStreamStateHandle.java     |   89 --
 .../runtime/state/memory/MemFoldingState.java   |   42 -
 .../runtime/state/memory/MemListState.java      |   45 -
 .../runtime/state/memory/MemReducingState.java  |   48 -
 .../runtime/state/memory/MemValueState.java     |   48 -
 .../state/memory/MigrationRestoreSnapshot.java  |   35 -
 .../state/memory/SerializedStateHandle.java     |   93 --
 .../state/MigrationKeyGroupStateHandle.java     |   46 -
 .../state/MigrationStreamStateHandle.java       |   67 --
 .../runtime/tasks/StreamTaskState.java          |   88 --
 .../runtime/tasks/StreamTaskStateList.java      |  100 --
 .../flink/runtime/checkpoint/OperatorState.java |   11 +-
 .../checkpoint/OperatorSubtaskState.java        |   70 +-
 .../checkpoint/StateAssignmentOperation.java    |   47 +-
 .../flink/runtime/checkpoint/SubtaskState.java  |   39 +-
 .../flink/runtime/checkpoint/TaskState.java     |   10 -
 .../savepoint/SavepointSerializers.java         |   18 +-
 .../checkpoint/savepoint/SavepointStore.java    |    1 +
 .../savepoint/SavepointV1Serializer.java        |   38 +-
 .../checkpoint/savepoint/SavepointV2.java       |    5 -
 .../savepoint/SavepointV2Serializer.java        |   25 +-
 .../runtime/state/VoidNamespaceSerializer.java  |    8 -
 .../state/heap/HeapKeyedStateBackend.java       |   63 +-
 .../CheckpointCoordinatorFailureTest.java       |    5 -
 .../checkpoint/CheckpointCoordinatorTest.java   |  178 +--
 .../checkpoint/CheckpointStateRestoreTest.java  |   16 +-
 .../savepoint/CheckpointTestUtils.java          |   17 -
 .../savepoint/MigrationV0ToV1Test.java          |  251 ----
 .../jobmanager/JobManagerHARecoveryTest.java    |   32 +-
 .../messages/CheckpointMessagesTest.java        |    1 -
 .../api/graph/StreamGraphHasherV1.java          |  282 -----
 .../MultiplexingStreamRecordSerializer.java     |  293 -----
 .../streamrecord/StreamRecordSerializer.java    |  208 ----
 .../streaming/api/checkpoint/Checkpointed.java  |   80 --
 .../checkpoint/CheckpointedAsynchronously.java  |   61 -
 .../api/checkpoint/CheckpointedRestoring.java   |   43 -
 .../datastream/LegacyWindowOperatorType.java    |   63 -
 .../api/datastream/WindowedStream.java          |  197 +---
 .../ContinuousFileMonitoringFunction.java       |   11 +-
 .../source/ContinuousFileReaderOperator.java    |   92 +-
 .../api/graph/StreamingJobGraphGenerator.java   |   15 +-
 .../api/operators/AbstractStreamOperator.java   |   67 --
 .../operators/AbstractUdfStreamOperator.java    |   84 +-
 .../CheckpointedRestoringOperator.java          |   50 -
 .../operators/StreamCheckpointedOperator.java   |   43 -
 .../streaming/api/operators/StreamOperator.java |   16 -
 .../TumblingAlignedProcessingTimeWindows.java   |   68 --
 ...ractAlignedProcessingTimeWindowOperator.java |  331 ------
 ...ccumulatingProcessingTimeWindowOperator.java |   64 -
 ...AggregatingProcessingTimeWindowOperator.java |   58 -
 .../operators/windowing/WindowOperator.java     |  314 -----
 .../streamrecord/StreamElementSerializer.java   |    4 -
 .../runtime/tasks/OperatorStateHandles.java     |    9 -
 .../streaming/runtime/tasks/StreamTask.java     |   39 +-
 .../AbstractUdfStreamOperatorLifecycleTest.java |   11 +-
 .../FoldApplyProcessWindowFunctionTest.java     |  332 ------
 .../operators/FoldApplyWindowFunctionTest.java  |  152 ---
 .../StreamingJobGraphGeneratorNodeHashTest.java |    3 +-
 ...AlignedProcessingTimeWindowOperatorTest.java | 1116 ------------------
 ...AlignedProcessingTimeWindowOperatorTest.java |  863 --------------
 .../windowing/AllWindowTranslationTest.java     |   26 -
 .../windowing/TimeWindowTranslationTest.java    |  101 --
 .../windowing/WindowOperatorMigrationTest.java  |  216 +---
 .../windowing/WindowTranslationTest.java        |   27 -
 .../tasks/InterruptSensitiveRestoreTest.java    |   71 +-
 .../runtime/tasks/OneInputStreamTaskTest.java   |  124 +-
 .../streaming/runtime/tasks/StreamTaskTest.java |   58 +-
 .../util/AbstractStreamOperatorTestHarness.java |   90 --
 .../KeyedOneInputStreamOperatorTestHarness.java |   65 -
 .../streaming/util/OperatorSnapshotUtil.java    |   14 +-
 .../util/migration/MigrationTestUtil.java       |   18 +-
 .../api/scala/TimeWindowTranslationTest.scala   |   59 +-
 .../test/checkpointing/RescalingITCase.java     |    8 +-
 .../test/checkpointing/SavepointITCase.java     |   17 +-
 .../utils/SavepointMigrationTestBase.java       |    7 +
 ...atefulJobSavepointFrom11MigrationITCase.java |  562 ---------
 ...atefulJobSavepointFrom12MigrationITCase.java |  106 +-
 .../jar/LegacyCheckpointedStreamingProgram.java |  143 ---
 .../AbstractOperatorRestoreTestBase.java        |    6 +
 126 files changed, 318 insertions(+), 12512 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
index 3d3ea05..e5758e8 100644
--- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
@@ -29,7 +29,6 @@ import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink;
 import org.apache.flink.util.Preconditions;
@@ -132,7 +131,7 @@ import java.util.UUID;
 @Deprecated
 public class RollingSink<T> extends RichSinkFunction<T>
 		implements InputTypeConfigurable, CheckpointedFunction,
-					CheckpointListener, CheckpointedRestoring<RollingSink.BucketState> {
+					CheckpointListener {
 
 	private static final long serialVersionUID = 1L;
 
@@ -759,25 +758,6 @@ public class RollingSink<T> extends RichSinkFunction<T>
 	}
 
 	// --------------------------------------------------------------------------------------------
-	//  Backwards compatibility with Flink 1.1
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public void restoreState(BucketState state) throws Exception {
-		LOG.info("{} (taskIdx={}) restored bucket state from an older Flink version: {}",
-			getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), state);
-
-		try {
-			initFileSystem();
-		} catch (IOException e) {
-			LOG.error("Error while creating FileSystem when restoring the state of the RollingSink.", e);
-			throw new RuntimeException("Error while creating FileSystem when restoring the state of the RollingSink.", e);
-		}
-
-		handleRestoredBucketState(state);
-	}
-
-	// --------------------------------------------------------------------------------------------
 	//  Setters for User configuration values
 	// --------------------------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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 70168b5..cc924a4 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
@@ -30,7 +30,6 @@ import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 import org.apache.flink.streaming.connectors.fs.Clock;
@@ -154,8 +153,7 @@ import java.util.UUID;
  */
 public class BucketingSink<T>
 		extends RichSinkFunction<T>
-		implements InputTypeConfigurable, CheckpointedFunction, CheckpointListener,
-					CheckpointedRestoring<RollingSink.BucketState>, ProcessingTimeCallback {
+		implements InputTypeConfigurable, CheckpointedFunction, CheckpointListener, ProcessingTimeCallback {
 
 	private static final long serialVersionUID = 1L;
 
@@ -873,25 +871,6 @@ public class BucketingSink<T>
 	}
 
 	// --------------------------------------------------------------------------------------------
-	//  Backwards compatibility with Flink 1.1
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public void restoreState(RollingSink.BucketState state) throws Exception {
-		LOG.info("{} (taskIdx={}) restored bucket state from the RollingSink an older Flink version: {}",
-			getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), state);
-
-		try {
-			initFileSystem();
-		} catch (IOException e) {
-			LOG.error("Error while creating FileSystem when restoring the state of the BucketingSink.", e);
-			throw new RuntimeException("Error while creating FileSystem when restoring the state of the BucketingSink.", e);
-		}
-
-		handleRestoredRollingSinkState(state);
-	}
-
-	// --------------------------------------------------------------------------------------------
 	//  Setters for User configuration values
 	// --------------------------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingSinkMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingSinkMigrationTest.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingSinkMigrationTest.java
deleted file mode 100644
index e041379..0000000
--- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingSinkMigrationTest.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.fs.bucketing;
-
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.fs.RollingSink;
-import org.apache.flink.streaming.connectors.fs.StringWriter;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.util.OperatingSystem;
-
-import org.apache.commons.io.FileUtils;
-import org.junit.Assert;
-import org.junit.Assume;
-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;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Tests the migration from 1.1 snapshots.
- */
-@Deprecated
-public class RollingSinkMigrationTest {
-
-	@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";
-
-	@BeforeClass
-	public static void verifyOS() {
-		Assume.assumeTrue("HDFS cluster cannot be started on Windows without extensions.", !OperatingSystem.isWindows());
-	}
-
-	@Test
-	public void testMigration() throws Exception {
-
-		/*
-		* Code ran to get the snapshot:
-		*
-		* final File outDir = tempFolder.newFolder();
-
-		RollingSink<String> sink = new RollingSink<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> testHarness1 =
-			new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
-
-		testHarness1.setup();
-		testHarness1.open();
-
-		testHarness1.processElement(new StreamRecord<>("test1", 0L));
-		testHarness1.processElement(new StreamRecord<>("test2", 0L));
-
-		checkFs(outDir, 1, 1, 0, 0);
-
-		testHarness1.processElement(new StreamRecord<>("test3", 0L));
-		testHarness1.processElement(new StreamRecord<>("test4", 0L));
-		testHarness1.processElement(new StreamRecord<>("test5", 0L));
-
-		checkFs(outDir, 1, 4, 0, 0);
-
-		StreamTaskState taskState = testHarness1.snapshot(0, 0);
-		testHarness1.snaphotToFile(taskState, "src/test/resources/rolling-sink-migration-test-flink1.1-snapshot");
-		testHarness1.close();
-		* */
-
-		final File outDir = tempFolder.newFolder();
-
-		RollingSink<String> sink = new ValidatingRollingSink<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> testHarness1 = new OneInputStreamOperatorTestHarness<>(
-			new StreamSink<>(sink), 10, 1, 0);
-		testHarness1.setup();
-		testHarness1.initializeStateFromLegacyCheckpoint(getResourceFilename("rolling-sink-migration-test-flink1.1-snapshot"));
-		testHarness1.open();
-
-		testHarness1.processElement(new StreamRecord<>("test1", 0L));
-		testHarness1.processElement(new StreamRecord<>("test2", 0L));
-
-		checkFs(outDir, 1, 1, 0, 0);
-
-		testHarness1.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);
-	}
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = RollingSinkMigrationTest.class.getClassLoader();
-		URL resource = cl.getResource(filename);
-		return resource.getFile();
-	}
-
-	static class ValidatingRollingSink<T> extends RollingSink<T> {
-
-		private static final long serialVersionUID = -4263974081712009141L;
-
-		ValidatingRollingSink(String basePath) {
-			super(basePath);
-		}
-
-		@Override
-		public void restoreState(BucketState state) throws Exception {
-
-			/**
-			 * this validates that we read the state that was checkpointed by the previous version. We expect it to be:
-			 * In-progress=/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-4
-			 * 					validLength=6
-			 * pendingForNextCheckpoint=[]
-			 * pendingForPrevCheckpoints={0=[	/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-0,
-			 * 									/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-1,
-			 * 									/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-2,
-			 * 									/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-3]}
-			 * */
-
-			String current = state.currentFile;
-			long validLength = state.currentFileValidLength;
-
-			Assert.assertEquals("/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-4", current);
-			Assert.assertEquals(6, validLength);
-
-			List<String> pendingFiles = state.pendingFiles;
-			Assert.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/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-" + i,
-						files.get(i));
-				}
-			}
-			super.restoreState(state);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingToBucketingMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingToBucketingMigrationTest.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingToBucketingMigrationTest.java
deleted file mode 100644
index 8a8dbd6..0000000
--- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/RollingToBucketingMigrationTest.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.fs.bucketing;
-
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.connectors.fs.RollingSink;
-import org.apache.flink.streaming.connectors.fs.StringWriter;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.util.OperatingSystem;
-
-import org.apache.commons.io.FileUtils;
-import org.junit.Assert;
-import org.junit.Assume;
-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;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Tests the migration from {@link RollingSink} to {@link BucketingSink}.
- */
-public class RollingToBucketingMigrationTest {
-
-	@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";
-
-	@BeforeClass
-	public static void verifyOS() {
-		Assume.assumeTrue("HDFS cluster cannot be started on Windows without extensions.", !OperatingSystem.isWindows());
-	}
-
-	@Test
-	public void testMigration() throws Exception {
-		final File outDir = tempFolder.newFolder();
-
-		BucketingSink<String> sink = 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> testHarness1 = new OneInputStreamOperatorTestHarness<>(
-			new StreamSink<>(sink), 10, 1, 0);
-		testHarness1.setup();
-		testHarness1.initializeStateFromLegacyCheckpoint(getResourceFilename("rolling-sink-migration-test-flink1.1-snapshot"));
-		testHarness1.open();
-
-		testHarness1.processElement(new StreamRecord<>("test1", 0L));
-		testHarness1.processElement(new StreamRecord<>("test2", 0L));
-
-		checkFs(outDir, 1, 1, 0, 0);
-
-		testHarness1.close();
-	}
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = RollingToBucketingMigrationTest.class.getClassLoader();
-		URL resource = cl.getResource(filename);
-		return resource.getFile();
-	}
-
-	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;
-
-		ValidatingBucketingSink(String basePath) {
-			super(basePath);
-		}
-
-		@Override
-		public void restoreState(RollingSink.BucketState state) throws Exception {
-
-			/**
-			 * this validates that we read the state that was checkpointed by the previous version. We expect it to be:
-			 * In-progress=/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-4
-			 * 					validLength=6
-			 * pendingForNextCheckpoint=[]
-			 * pendingForPrevCheckpoints={0=[	/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-0,
-			 * 									/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-1,
-			 * 									/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-2,
-			 * 									/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-3]}
-			 * */
-
-			String current = state.currentFile;
-			long validLength = state.currentFileValidLength;
-
-			Assert.assertEquals("/var/folders/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-4", current);
-			Assert.assertEquals(6, validLength);
-
-			List<String> pendingFiles = state.pendingFiles;
-			Assert.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/z5/fxvg1j6s6mn94nsf8b1yc8s80000gn/T/junit2927527303216950257/junit5645682027227039270/2017-01-09--18/part-0-" + i,
-						files.get(i));
-				}
-			}
-
-			super.restoreState(state);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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 f3c9e5e..3088b15 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
@@ -33,7 +33,6 @@ import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
@@ -78,8 +77,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFunction<T> implements
 		CheckpointListener,
 		ResultTypeQueryable<T>,
-		CheckpointedFunction,
-		CheckpointedRestoring<HashMap<KafkaTopicPartition, Long>> {
+		CheckpointedFunction {
 
 	private static final long serialVersionUID = -6272159445203409112L;
 
@@ -767,22 +765,6 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 	}
 
 	@Override
-	public final void restoreState(HashMap<KafkaTopicPartition, Long> restoredOffsets) {
-		LOG.info("{} (taskIdx={}) restoring offsets from an older version: {}",
-			getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), restoredOffsets);
-
-		restoredFromOldState = true;
-
-		if (restoredOffsets.size() > 0 && discoveryIntervalMillis != PARTITION_DISCOVERY_DISABLED) {
-			throw new IllegalArgumentException(
-				"Topic / partition discovery cannot be enabled if the job is restored from a savepoint from Flink 1.1.x.");
-		}
-
-		restoredState = new TreeMap<>(new KafkaTopicPartition.Comparator());
-		restoredState.putAll(restoredOffsets);
-	}
-
-	@Override
 	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
 		if (!running) {
 			LOG.debug("notifyCheckpointComplete() called on closed source");

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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
index e3f337e..84f0e38 100644
--- 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
@@ -92,7 +92,7 @@ public class FlinkKafkaConsumerBaseMigrationTest {
 
 	@Parameterized.Parameters(name = "Migration Savepoint: {0}")
 	public static Collection<MigrationVersion> parameters () {
-		return Arrays.asList(MigrationVersion.v1_1, MigrationVersion.v1_2, MigrationVersion.v1_3);
+		return Arrays.asList(MigrationVersion.v1_2, MigrationVersion.v1_3);
 	}
 
 	public FlinkKafkaConsumerBaseMigrationTest(MigrationVersion testMigrateVersion) {
@@ -322,7 +322,7 @@ public class FlinkKafkaConsumerBaseMigrationTest {
 	 */
 	@Test
 	public void testRestoreFailsWithNonEmptyPreFlink13StatesIfDiscoveryEnabled() throws Exception {
-		assumeTrue(testMigrateVersion == MigrationVersion.v1_1 || testMigrateVersion == MigrationVersion.v1_2);
+		assumeTrue(testMigrateVersion == MigrationVersion.v1_3 || testMigrateVersion == MigrationVersion.v1_2);
 
 		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
index d127f2b..5689229 100644
--- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
@@ -28,13 +28,11 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
 import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.InitialPosition;
 import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
 import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
 import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
 import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
@@ -72,8 +70,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class FlinkKinesisConsumer<T> extends RichParallelSourceFunction<T> implements
 		ResultTypeQueryable<T>,
-		CheckpointedFunction,
-		CheckpointedRestoring<HashMap<KinesisStreamShard, SequenceNumber>> {
+		CheckpointedFunction {
 
 	private static final long serialVersionUID = 4724006128720664870L;
 
@@ -352,7 +349,7 @@ public class FlinkKinesisConsumer<T> extends RichParallelSourceFunction<T> imple
 
 				if (LOG.isDebugEnabled()) {
 					LOG.debug("Snapshotted state, last processed sequence numbers: {}, checkpoint id: {}, timestamp: {}",
-						lastStateSnapshot.toString(), context.getCheckpointId(), context.getCheckpointTimestamp());
+						lastStateSnapshot, context.getCheckpointId(), context.getCheckpointTimestamp());
 				}
 
 				for (Map.Entry<StreamShardMetadata, SequenceNumber> entry : lastStateSnapshot.entrySet()) {
@@ -362,23 +359,6 @@ public class FlinkKinesisConsumer<T> extends RichParallelSourceFunction<T> imple
 		}
 	}
 
-	@Override
-	public void restoreState(HashMap<KinesisStreamShard, SequenceNumber> restoredState) throws Exception {
-		LOG.info("Subtask {} restoring offsets from an older Flink version: {}",
-			getRuntimeContext().getIndexOfThisSubtask(), sequenceNumsToRestore);
-
-		if (restoredState.isEmpty()) {
-			sequenceNumsToRestore = null;
-		} else {
-			sequenceNumsToRestore = new HashMap<>();
-			for (Map.Entry<KinesisStreamShard, SequenceNumber> stateEntry : restoredState.entrySet()) {
-				sequenceNumsToRestore.put(
-						KinesisStreamShard.convertToStreamShardMetadata(stateEntry.getKey()),
-						stateEntry.getValue());
-			}
-		}
-	}
-
 	/** This method is exposed for tests that need to mock the KinesisDataFetcher in the consumer. */
 	protected KinesisDataFetcher<T> createFetcher(
 			List<String> streams,

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java
deleted file mode 100644
index af84420..0000000
--- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java
+++ /dev/null
@@ -1,149 +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.kinesis;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
-import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
-import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
-import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
-import org.apache.flink.streaming.connectors.kinesis.model.StreamShardMetadata;
-import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
-import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-
-import com.amazonaws.services.kinesis.model.Shard;
-import org.junit.Test;
-
-import java.net.URL;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Properties;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for checking whether {@link FlinkKinesisConsumer} can restore from snapshots that were
- * done using the Flink 1.1 {@code FlinkKinesisConsumer}.
- */
-public class FlinkKinesisConsumerMigrationTest {
-
-	@Test
-	public void testRestoreFromFlink11WithEmptyState() throws Exception {
-		Properties testConfig = new Properties();
-		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
-		testConfig.setProperty(ConsumerConfigConstants.AWS_CREDENTIALS_PROVIDER, "BASIC");
-		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
-		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(testConfig);
-
-		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("kinesis-consumer-migration-test-flink1.1-empty-snapshot"));
-		testHarness.open();
-
-		// assert that no state was restored
-		assertEquals(null, consumerFunction.getRestoredState());
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	@Test
-	public void testRestoreFromFlink11() throws Exception {
-		Properties testConfig = new Properties();
-		testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1");
-		testConfig.setProperty(ConsumerConfigConstants.AWS_CREDENTIALS_PROVIDER, "BASIC");
-		testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
-		testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(testConfig);
-
-		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("kinesis-consumer-migration-test-flink1.1-snapshot"));
-		testHarness.open();
-
-		// the expected state in "kafka-consumer-migration-test-flink1.1-snapshot"
-		final HashMap<StreamShardMetadata, SequenceNumber> expectedState = new HashMap<>();
-		expectedState.put(KinesisStreamShard.convertToStreamShardMetadata(new KinesisStreamShard("fakeStream1",
-				new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0)))),
-			new SequenceNumber("987654321"));
-
-		// assert that state is correctly restored from legacy checkpoint
-		assertNotEquals(null, consumerFunction.getRestoredState());
-		assertEquals(1, consumerFunction.getRestoredState().size());
-		assertEquals(expectedState, consumerFunction.getRestoredState());
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = FlinkKinesisConsumerMigrationTest.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 FlinkKinesisConsumer<T> {
-		private static final long serialVersionUID = 1L;
-
-		@SuppressWarnings("unchecked")
-		DummyFlinkKafkaConsumer(Properties properties) {
-			super("test", mock(KinesisDeserializationSchema.class), properties);
-		}
-
-		@Override
-		protected KinesisDataFetcher<T> createFetcher(
-				List<String> streams,
-				SourceFunction.SourceContext<T> sourceContext,
-				RuntimeContext runtimeContext,
-				Properties configProps,
-				KinesisDeserializationSchema<T> deserializationSchema) {
-			return mock(KinesisDataFetcher.class);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
index a26e758..69d30cd 100644
--- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java
@@ -712,38 +712,6 @@ public class FlinkKinesisConsumerTest {
 
 	@Test
 	@SuppressWarnings("unchecked")
-	public void testFetcherShouldBeCorrectlySeededIfRestoringFromLegacyCheckpoint() throws Exception {
-		HashMap<StreamShardHandle, SequenceNumber> fakeRestoredState = getFakeRestoredStore("all");
-		HashMap<KinesisStreamShard, SequenceNumber> legacyFakeRestoredState = new HashMap<>();
-		for (Map.Entry<StreamShardHandle, SequenceNumber> kv : fakeRestoredState.entrySet()) {
-			legacyFakeRestoredState.put(new KinesisStreamShard(kv.getKey().getStreamName(), kv.getKey().getShard()), kv.getValue());
-		}
-
-		KinesisDataFetcher mockedFetcher = Mockito.mock(KinesisDataFetcher.class);
-		List<StreamShardHandle> shards = new ArrayList<>();
-		shards.addAll(fakeRestoredState.keySet());
-		when(mockedFetcher.discoverNewShardsToSubscribe()).thenReturn(shards);
-		PowerMockito.whenNew(KinesisDataFetcher.class).withAnyArguments().thenReturn(mockedFetcher);
-
-		// assume the given config is correct
-		PowerMockito.mockStatic(KinesisConfigUtil.class);
-		PowerMockito.doNothing().when(KinesisConfigUtil.class);
-
-		TestableFlinkKinesisConsumer consumer = new TestableFlinkKinesisConsumer(
-			"fakeStream", new Properties(), 10, 2);
-		consumer.restoreState(legacyFakeRestoredState);
-		consumer.open(new Configuration());
-		consumer.run(Mockito.mock(SourceFunction.SourceContext.class));
-
-		for (Map.Entry<StreamShardHandle, SequenceNumber> restoredShard : fakeRestoredState.entrySet()) {
-			Mockito.verify(mockedFetcher).registerNewSubscribedShardState(
-				new KinesisStreamShardState(KinesisDataFetcher.convertToStreamShardMetadata(restoredShard.getKey()),
-					restoredShard.getKey(), restoredShard.getValue()));
-		}
-	}
-
-	@Test
-	@SuppressWarnings("unchecked")
 	public void testFetcherShouldBeCorrectlySeededIfRestoringFromCheckpoint() throws Exception {
 
 		// ----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java
index 75356ef..1336652 100644
--- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java
@@ -21,7 +21,7 @@ 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.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
@@ -29,11 +29,14 @@ import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConsta
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -95,7 +98,7 @@ public class ExactlyOnceValidatingConsumerThread {
 		return new Thread(exactlyOnceValidationConsumer);
 	}
 
-	private static class ExactlyOnceValidatingMapper implements FlatMapFunction<String, String>, Checkpointed<BitSet> {
+	private static class ExactlyOnceValidatingMapper implements FlatMapFunction<String, String>, ListCheckpointed<BitSet> {
 
 		private static final Logger LOG = LoggerFactory.getLogger(ExactlyOnceValidatingMapper.class);
 
@@ -126,13 +129,18 @@ public class ExactlyOnceValidatingConsumerThread {
 		}
 
 		@Override
-		public BitSet snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			return validator;
+		public List<BitSet> snapshotState(long checkpointId, long timestamp) throws Exception {
+			return Collections.singletonList(validator);
 		}
 
 		@Override
-		public void restoreState(BitSet state) throws Exception {
-			this.validator = state;
+		public void restoreState(List<BitSet> state) throws Exception {
+			// we expect either 1 or 0 elements
+			if (state.size() == 1) {
+				validator = state.get(0);
+			} else {
+				Preconditions.checkState(state.isEmpty());
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index b7f386d..dd5b852 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -31,7 +31,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.core.fs.CloseableRegistry;
 import org.apache.flink.core.fs.FSDataInputStream;
@@ -39,15 +38,10 @@ import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
 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.core.memory.DataOutputViewStreamWrapper;
-import org.apache.flink.migration.MigrationNamespaceSerializerProxy;
-import org.apache.flink.migration.MigrationUtil;
-import org.apache.flink.migration.contrib.streaming.state.RocksDBStateBackend;
-import org.apache.flink.migration.state.MigrationKeyGroupStateHandle;
 import org.apache.flink.runtime.checkpoint.CheckpointOptions;
 import org.apache.flink.runtime.io.async.AbstractAsyncIOCallable;
 import org.apache.flink.runtime.io.async.AsyncStoppableTaskWithCallback;
@@ -57,7 +51,6 @@ import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.DoneFuture;
 import org.apache.flink.runtime.state.IncrementalKeyedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupRange;
-import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.KeyGroupRangeOffsets;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.KeyedBackendSerializationProxy;
@@ -80,7 +73,6 @@ import org.apache.flink.runtime.state.internal.InternalValueState;
 import org.apache.flink.runtime.util.SerializableObject;
 import org.apache.flink.util.FileUtils;
 import org.apache.flink.util.IOUtils;
-import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.StateMigrationException;
 
@@ -97,11 +89,9 @@ import org.rocksdb.Snapshot;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.ObjectInputStream;
 import java.io.OutputStream;
 import java.nio.file.Files;
 import java.util.ArrayList;
@@ -110,7 +100,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -140,9 +129,13 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	/** The name of the merge operator in RocksDB. Do not change except you know exactly what you do. */
 	public static final String MERGE_OPERATOR_NAME = "stringappendtest";
 
+	/** File suffix of sstable files. */
+	private static final String SST_FILE_SUFFIX = ".sst";
+
 	/** Bytes for the name of the column decriptor for the default column family. */
 	public static final byte[] DEFAULT_COLUMN_FAMILY_NAME_BYTES = "default".getBytes(ConfigConstants.DEFAULT_CHARSET);
 
+	/** String that identifies the operator that owns this backend. */
 	private final String operatorIdentifier;
 
 	/** The column family options from the options factory. */
@@ -206,8 +199,6 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	/** Unique ID of this backend. */
 	private UUID backendUID;
 
-	private static final String SST_FILE_SUFFIX = ".sst";
-
 	public RocksDBKeyedStateBackend(
 		String operatorIdentifier,
 		ClassLoader userCodeClassLoader,
@@ -311,10 +302,6 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		return keyGroupPrefixBytes;
 	}
 
-	private boolean hasRegisteredState() {
-		return !kvStateInformation.isEmpty();
-	}
-
 	/**
 	 * Triggers an asynchronous snapshot of the keyed state backend from RocksDB. This snapshot can be canceled and
 	 * is also stopped when the backend is closed through {@link #dispose()}. For each backend, this method must always
@@ -359,7 +346,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				throw new IOException("RocksDB closed.");
 			}
 
-			if (!hasRegisteredState()) {
+			if (kvStateInformation.isEmpty()) {
 				if (LOG.isDebugEnabled()) {
 					LOG.debug("Asynchronous RocksDB snapshot performed on empty keyed state at " +
 						checkpointTimestamp + " . Returning null.");
@@ -404,7 +391,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 			if (db != null) {
 
-				if (!hasRegisteredState()) {
+				if (kvStateInformation.isEmpty()) {
 					if (LOG.isDebugEnabled()) {
 						LOG.debug("Asynchronous RocksDB snapshot performed on empty keyed state at " + timestamp +
 							" . Returning null.");
@@ -887,11 +874,17 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		void takeSnapshot() throws Exception {
 			assert (Thread.holdsLock(stateBackend.asyncSnapshotLock));
 
+			final long lastCompletedCheckpoint;
+
 			// use the last completed checkpoint as the comparison base.
 			synchronized (stateBackend.materializedSstFiles) {
-				baseSstFiles = stateBackend.materializedSstFiles.get(stateBackend.lastCompletedCheckpointId);
+				lastCompletedCheckpoint = stateBackend.lastCompletedCheckpointId;
+				baseSstFiles = stateBackend.materializedSstFiles.get(lastCompletedCheckpoint);
 			}
 
+			LOG.trace("Taking incremental snapshot for checkpoint {}. Snapshot is based on last completed checkpoint {} " +
+				"assuming the following (shared) files as base: {}.", checkpointId, lastCompletedCheckpoint, baseSstFiles);
+
 			// save meta data
 			for (Map.Entry<String, Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>>> stateMetaInfoEntry
 				: stateBackend.kvStateInformation.entrySet()) {
@@ -929,7 +922,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 					if (fileName.endsWith(SST_FILE_SUFFIX)) {
 						final boolean existsAlready =
-							baseSstFiles == null ? false : baseSstFiles.contains(stateHandleID);
+							baseSstFiles != null && baseSstFiles.contains(stateHandleID);
 
 						if (existsAlready) {
 							// we introduce a placeholder state handle, that is replaced with the
@@ -982,7 +975,8 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			}
 
 			if (canceled) {
-				List<StateObject> statesToDiscard = new ArrayList<>();
+				Collection<StateObject> statesToDiscard =
+					new ArrayList<>(1 + miscFiles.size() + sstFiles.size());
 
 				statesToDiscard.add(metaStateHandle);
 				statesToDiscard.addAll(miscFiles.values());
@@ -1012,9 +1006,6 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		try {
 			if (restoreState == null || restoreState.isEmpty()) {
 				createDB();
-			} else if (MigrationUtil.isOldSavepointKeyedState(restoreState)) {
-				LOG.info("Converting RocksDB state from old savepoint.");
-				restoreOldSavepointKeyedState(restoreState);
 			} else if (restoreState.iterator().next() instanceof IncrementalKeyedStateHandle) {
 				RocksDBIncrementalRestoreOperation<K> restoreOperation = new RocksDBIncrementalRestoreOperation<>(this);
 				restoreOperation.restore(restoreState);
@@ -1035,14 +1026,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				return;
 			}
 
-			Iterator<Long> materializedCheckpointIterator = materializedSstFiles.keySet().iterator();
-			while (materializedCheckpointIterator.hasNext()) {
-				long materializedCheckpointId = materializedCheckpointIterator.next();
-
-				if (materializedCheckpointId < completedCheckpointId) {
-					materializedCheckpointIterator.remove();
-				}
-			}
+			materializedSstFiles.keySet().removeIf(checkpointId -> checkpointId < completedCheckpointId);
 
 			lastCompletedCheckpointId = completedCheckpointId;
 		}
@@ -1067,10 +1051,10 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		// we add the required descriptor for the default CF in last position.
 		columnFamilyDescriptors.add(new ColumnFamilyDescriptor(DEFAULT_COLUMN_FAMILY_NAME_BYTES, columnOptions));
 
-		RocksDB db;
+		RocksDB dbRef;
 
 		try {
-			db = RocksDB.open(
+			dbRef = RocksDB.open(
 				Preconditions.checkNotNull(dbOptions),
 				Preconditions.checkNotNull(path),
 				columnFamilyDescriptors,
@@ -1083,7 +1067,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		Preconditions.checkState(1 + stateColumnFamilyDescriptors.size() == stateColumnFamilyHandles.size(),
 			"Not all requested column family handles have been created");
 
-		return db;
+		return dbRef;
 	}
 
 	/**
@@ -1117,12 +1101,9 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		 * Restores all key-groups data that is referenced by the passed state handles.
 		 *
 		 * @param keyedStateHandles List of all key groups state handles that shall be restored.
-		 * @throws IOException
-		 * @throws ClassNotFoundException
-		 * @throws RocksDBException
 		 */
 		public void doRestore(Collection<KeyedStateHandle> keyedStateHandles)
-			throws IOException, StateMigrationException, ClassNotFoundException, RocksDBException {
+			throws IOException, StateMigrationException, RocksDBException {
 
 			rocksDBKeyedStateBackend.createDB();
 
@@ -1142,13 +1123,9 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 
 		/**
 		 * Restore one key groups state handle.
-		 *
-		 * @throws IOException
-		 * @throws RocksDBException
-		 * @throws ClassNotFoundException
 		 */
 		private void restoreKeyGroupsInStateHandle()
-			throws IOException, StateMigrationException, RocksDBException, ClassNotFoundException {
+			throws IOException, StateMigrationException, RocksDBException {
 			try {
 				currentStateHandleInStream = currentKeyGroupsStateHandle.openInputStream();
 				rocksDBKeyedStateBackend.cancelStreamRegistry.registerClosable(currentStateHandleInStream);
@@ -1251,13 +1228,13 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				//not empty key-group?
 				if (0L != offset) {
 					currentStateHandleInStream.seek(offset);
-					boolean keyGroupHasMoreKeys = true;
 					try (InputStream compressedKgIn = keygroupStreamCompressionDecorator.decorateWithCompression(currentStateHandleInStream)) {
 						DataInputViewStreamWrapper compressedKgInputView = new DataInputViewStreamWrapper(compressedKgIn);
 						//TODO this could be aware of keyGroupPrefixBytes and write only one byte if possible
 						int kvStateId = compressedKgInputView.readShort();
 						ColumnFamilyHandle handle = currentStateHandleKVStateColumnFamilies.get(kvStateId);
 						//insert all k/v pairs into DB
+						boolean keyGroupHasMoreKeys = true;
 						while (keyGroupHasMoreKeys) {
 							byte[] key = BytePrimitiveArraySerializer.INSTANCE.deserialize(compressedKgInputView);
 							byte[] value = BytePrimitiveArraySerializer.INSTANCE.deserialize(compressedKgInputView);
@@ -1557,7 +1534,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		void restore(Collection<KeyedStateHandle> restoreStateHandles) throws Exception {
 
 			boolean hasExtraKeys = (restoreStateHandles.size() > 1 ||
-				!restoreStateHandles.iterator().next().getKeyGroupRange().equals(stateBackend.keyGroupRange));
+				!Objects.equals(restoreStateHandles.iterator().next().getKeyGroupRange(), stateBackend.keyGroupRange));
 
 			if (hasExtraKeys) {
 				stateBackend.createDB();
@@ -1611,16 +1588,16 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				(RegisteredKeyedBackendStateMetaInfo.Snapshot<N, S>) restoredKvStateMetaInfos.get(descriptor.getName());
 
 			Preconditions.checkState(
-				newMetaInfo.getName().equals(restoredMetaInfo.getName()),
+				Objects.equals(newMetaInfo.getName(), restoredMetaInfo.getName()),
 				"Incompatible state names. " +
 					"Was [" + restoredMetaInfo.getName() + "], " +
 					"registered with [" + newMetaInfo.getName() + "].");
 
-			if (!newMetaInfo.getStateType().equals(StateDescriptor.Type.UNKNOWN)
-				&& !restoredMetaInfo.getStateType().equals(StateDescriptor.Type.UNKNOWN)) {
+			if (!Objects.equals(newMetaInfo.getStateType(), StateDescriptor.Type.UNKNOWN)
+				&& !Objects.equals(restoredMetaInfo.getStateType(), StateDescriptor.Type.UNKNOWN)) {
 
 				Preconditions.checkState(
-					newMetaInfo.getStateType().equals(restoredMetaInfo.getStateType()),
+					newMetaInfo.getStateType() == restoredMetaInfo.getStateType(),
 					"Incompatible state types. " +
 						"Was [" + restoredMetaInfo.getStateType() + "], " +
 						"registered with [" + newMetaInfo.getStateType() + "].");
@@ -1629,7 +1606,7 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 			// check compatibility results to determine if state migration is required
 			CompatibilityResult<N> namespaceCompatibility = CompatibilityUtil.resolveCompatibilityResult(
 				restoredMetaInfo.getNamespaceSerializer(),
-				MigrationNamespaceSerializerProxy.class,
+				null,
 				restoredMetaInfo.getNamespaceSerializerConfigSnapshot(),
 				newMetaInfo.getNamespaceSerializer());
 
@@ -1639,12 +1616,12 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 				restoredMetaInfo.getStateSerializerConfigSnapshot(),
 				newMetaInfo.getStateSerializer());
 
-			if (!namespaceCompatibility.isRequiresMigration() && !stateCompatibility.isRequiresMigration()) {
-				stateInfo.f1 = newMetaInfo;
-				return stateInfo.f0;
-			} else {
+			if (namespaceCompatibility.isRequiresMigration() || stateCompatibility.isRequiresMigration()) {
 				// TODO state migration currently isn't possible.
 				throw new StateMigrationException("State migration isn't supported, yet.");
+			} else {
+				stateInfo.f1 = newMetaInfo;
+				return stateInfo.f0;
 			}
 		}
 
@@ -1719,8 +1696,10 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 	}
 
 	@Override
-	protected <N, UK, UV> InternalMapState<N, UK, UV> createMapState(TypeSerializer<N> namespaceSerializer,
-	                                                                 MapStateDescriptor<UK, UV> stateDesc) throws Exception {
+	protected <N, UK, UV> InternalMapState<N, UK, UV> createMapState(
+		TypeSerializer<N> namespaceSerializer,
+		MapStateDescriptor<UK, UV> stateDesc) throws Exception {
+
 		ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer);
 
 		return new RocksDBMapState<>(columnFamily, namespaceSerializer, stateDesc, this);
@@ -1963,105 +1942,6 @@ public class RocksDBKeyedStateBackend<K> extends AbstractKeyedStateBackend<K> {
 		return instanceBasePath;
 	}
 
-	/**
-	 * For backwards compatibility, remove again later!
-	 *
-	 * @deprecated Internal method used for backwards compatibility.
-	 */
-	@Deprecated
-	private void restoreOldSavepointKeyedState(Collection<KeyedStateHandle> restoreState) throws Exception {
-		createDB();
-
-		Preconditions.checkState(1 == restoreState.size(), "Only one element expected here.");
-
-		KeyedStateHandle keyedStateHandle = restoreState.iterator().next();
-		if (!(keyedStateHandle instanceof MigrationKeyGroupStateHandle)) {
-			throw new IllegalStateException("Unexpected state handle type, " +
-				"expected: " + MigrationKeyGroupStateHandle.class +
-				", but found: " + keyedStateHandle.getClass());
-		}
-
-		MigrationKeyGroupStateHandle keyGroupStateHandle = (MigrationKeyGroupStateHandle) keyedStateHandle;
-
-		HashMap<String, RocksDBStateBackend.FinalFullyAsyncSnapshot> namedStates;
-		try (FSDataInputStream inputStream = keyGroupStateHandle.openInputStream()) {
-			namedStates = InstantiationUtil.deserializeObject(inputStream, userCodeClassLoader);
-		}
-
-		Preconditions.checkState(1 == namedStates.size(), "Only one element expected here.");
-		DataInputView inputView = namedStates.values().iterator().next().stateHandle.getState(userCodeClassLoader);
-
-		// first get the column family mapping
-		int numColumns = inputView.readInt();
-		Map<Byte, StateDescriptor<?, ?>> columnFamilyMapping = new HashMap<>(numColumns);
-		for (int i = 0; i < numColumns; i++) {
-			byte mappingByte = inputView.readByte();
-
-			ObjectInputStream ooIn =
-				new InstantiationUtil.ClassLoaderObjectInputStream(
-					new DataInputViewStream(inputView), userCodeClassLoader);
-
-			StateDescriptor<?, ?> stateDescriptor = (StateDescriptor<?, ?>) ooIn.readObject();
-
-			columnFamilyMapping.put(mappingByte, stateDescriptor);
-
-			// mimic a restored kv state meta info
-			restoredKvStateMetaInfos.put(
-				stateDescriptor.getName(),
-				new RegisteredKeyedBackendStateMetaInfo<>(
-					stateDescriptor.getType(),
-					stateDescriptor.getName(),
-					MigrationNamespaceSerializerProxy.INSTANCE,
-					stateDescriptor.getSerializer()).snapshot());
-
-			// this will fill in the k/v state information
-			getColumnFamily(stateDescriptor, MigrationNamespaceSerializerProxy.INSTANCE);
-		}
-
-		// try and read until EOF
-		try {
-			// the EOFException will get us out of this...
-			while (true) {
-				byte mappingByte = inputView.readByte();
-				ColumnFamilyHandle handle = getColumnFamily(
-					columnFamilyMapping.get(mappingByte), MigrationNamespaceSerializerProxy.INSTANCE);
-
-				byte[] keyAndNamespace = BytePrimitiveArraySerializer.INSTANCE.deserialize(inputView);
-
-				ByteArrayInputStreamWithPos bis = new ByteArrayInputStreamWithPos(keyAndNamespace);
-
-				K reconstructedKey = keySerializer.deserialize(new DataInputViewStreamWrapper(bis));
-				int len = bis.getPosition();
-
-				int keyGroup = (byte) KeyGroupRangeAssignment.assignToKeyGroup(reconstructedKey, numberOfKeyGroups);
-
-				if (keyGroupPrefixBytes == 1) {
-					// copy and override one byte (42) between key and namespace
-					System.arraycopy(keyAndNamespace, 0, keyAndNamespace, 1, len);
-					keyAndNamespace[0] = (byte) keyGroup;
-				} else {
-					byte[] largerKey = new byte[1 + keyAndNamespace.length];
-
-					// write key-group
-					largerKey[0] = (byte) ((keyGroup >> 8) & 0xFF);
-					largerKey[1] = (byte) (keyGroup & 0xFF);
-
-					// write key
-					System.arraycopy(keyAndNamespace, 0, largerKey, 2, len);
-
-					//skip one byte (42), write namespace
-					System.arraycopy(keyAndNamespace, 1 + len, largerKey, 2 + len, keyAndNamespace.length - len - 1);
-					keyAndNamespace = largerKey;
-				}
-
-				byte[] value = BytePrimitiveArraySerializer.INSTANCE.deserialize(inputView);
-				db.put(handle, keyAndNamespace, value);
-			}
-		} catch (EOFException e) {
-			// expected
-		}
-	}
-
 	@Override
 	public boolean supportsAsynchronousSnapshots() {
 		return true;

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/migration/contrib/streaming/state/RocksDBStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/migration/contrib/streaming/state/RocksDBStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/migration/contrib/streaming/state/RocksDBStateBackend.java
deleted file mode 100644
index 024d12e..0000000
--- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/migration/contrib/streaming/state/RocksDBStateBackend.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.contrib.streaming.state;
-
-import org.apache.flink.api.common.state.ValueState;
-import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.migration.runtime.state.AbstractStateBackend;
-import org.apache.flink.migration.runtime.state.KvStateSnapshot;
-import org.apache.flink.migration.runtime.state.StateHandle;
-
-import java.io.IOException;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * @deprecated Internal class used for backwards compatibility.
- */
-@Deprecated
-public class RocksDBStateBackend extends AbstractStateBackend {
-	private static final long serialVersionUID = 1L;
-
-	/**
-	 * Dummy {@link KvStateSnapshot} that holds the state of our one RocksDB data base.
-	 */
-	public static class FinalFullyAsyncSnapshot implements KvStateSnapshot<Object, Object, ValueState<Object>, ValueStateDescriptor<Object>> {
-		private static final long serialVersionUID = 1L;
-
-		public final StateHandle<DataInputView> stateHandle;
-		final long checkpointId;
-
-		/**
-		 * Creates a new snapshot from the given state parameters.
-		 */
-		private FinalFullyAsyncSnapshot(StateHandle<DataInputView> stateHandle, long checkpointId) {
-			this.stateHandle = requireNonNull(stateHandle);
-			this.checkpointId = checkpointId;
-		}
-
-		@Override
-		public final void discardState() throws Exception {
-			stateHandle.discardState();
-		}
-
-		@Override
-		public final long getStateSize() throws Exception {
-			return stateHandle.getStateSize();
-		}
-
-		@Override
-		public void close() throws IOException {
-			stateHandle.close();
-		}
-	}
-
-	/**
-	 * This class exists to provide a good error message if a user attempts to restore from a semi async snapshot.
-	 *
-	 * <p>see FLINK-5468
-	 */
-	@Deprecated
-	public static class FinalSemiAsyncSnapshot {
-
-		static {
-			throwExceptionOnLoadingThisClass();
-		}
-
-		private static void throwExceptionOnLoadingThisClass() {
-			throw new RuntimeException("Attempt to migrate RocksDB state created with semi async snapshot mode failed. "
-					+ "Unfortunately, this is not supported. Please create a new savepoint for the job using fully "
-					+ "async mode in Flink 1.1 and run migration again with the new savepoint.");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
index c752e53..98208fd 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
@@ -26,8 +26,6 @@ import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.common.typeutils.base.VoidSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
 import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
@@ -50,7 +48,6 @@ import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.AsynchronousException;
 import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
@@ -451,7 +448,7 @@ public class RocksDBAsyncSnapshotTest {
 
 	private static class AsyncCheckpointOperator
 		extends AbstractStreamOperator<String>
-		implements OneInputStreamOperator<String, String>, StreamCheckpointedOperator {
+		implements OneInputStreamOperator<String, String> {
 
 		@Override
 		public void open() throws Exception {
@@ -477,17 +474,5 @@ public class RocksDBAsyncSnapshotTest {
 
 			state.update(element.getValue());
 		}
-
-		@Override
-		public void snapshotState(
-				FSDataOutputStream out, long checkpointId, long timestamp) throws Exception {
-			// do nothing so that we don't block
-		}
-
-		@Override
-		public void restoreState(FSDataInputStream in) throws Exception {
-			// do nothing so that we don't block
-		}
-
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java
deleted file mode 100644
index 69e4e6d..0000000
--- a/flink-core/src/main/java/org/apache/flink/migration/util/MigrationInstantiationUtil.java
+++ /dev/null
@@ -1,96 +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.migration.util;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.util.InstantiationUtil;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.ObjectInputStream;
-import java.io.ObjectStreamClass;
-
-/**
- * Utility class to deserialize legacy classes for migration.
- */
-@PublicEvolving
-public final class MigrationInstantiationUtil {
-
-	public static class ClassLoaderObjectInputStream extends InstantiationUtil.ClassLoaderObjectInputStream {
-
-		private static final String ARRAY_PREFIX = "[L";
-		private static final String FLINK_BASE_PACKAGE = "org.apache.flink.";
-		private static final String FLINK_MIGRATION_PACKAGE = "org.apache.flink.migration.";
-
-		public ClassLoaderObjectInputStream(InputStream in, ClassLoader classLoader) throws IOException {
-			super(in, classLoader);
-		}
-
-		@Override
-		protected Class<?> resolveClass(ObjectStreamClass desc) throws IOException, ClassNotFoundException {
-			final String className = desc.getName();
-
-			// the flink package may be at position 0 (regular class) or position 2 (array)
-			final int flinkPackagePos;
-			if ((flinkPackagePos = className.indexOf(FLINK_BASE_PACKAGE)) == 0 ||
-					(flinkPackagePos == 2 && className.startsWith(ARRAY_PREFIX)))
-			{
-				final String modClassName = flinkPackagePos == 0 ?
-						FLINK_MIGRATION_PACKAGE + className.substring(FLINK_BASE_PACKAGE.length()) :
-						ARRAY_PREFIX + FLINK_MIGRATION_PACKAGE + className.substring(2 + FLINK_BASE_PACKAGE.length());
-
-				try {
-					return classLoader != null ?
-							Class.forName(modClassName, false, classLoader) :
-							Class.forName(modClassName);
-				}
-				catch (ClassNotFoundException ignored) {}
-			}
-
-			// either a non-Flink class, or not located in the migration package
-			return super.resolveClass(desc);
-		}
-	}
-
-	public static <T> T deserializeObject(byte[] bytes, ClassLoader cl) throws IOException, ClassNotFoundException {
-		return deserializeObject(new ByteArrayInputStream(bytes), cl);
-	}
-
-	@SuppressWarnings("unchecked")
-	public static <T> T deserializeObject(InputStream in, ClassLoader cl) throws IOException, ClassNotFoundException {
-		final ClassLoader old = Thread.currentThread().getContextClassLoader();
-		try (ObjectInputStream oois = new ClassLoaderObjectInputStream(in, cl)) {
-			Thread.currentThread().setContextClassLoader(cl);
-			return (T) oois.readObject();
-		} finally {
-			Thread.currentThread().setContextClassLoader(old);
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Private constructor to prevent instantiation.
-	 */
-	private MigrationInstantiationUtil() {
-		throw new IllegalAccessError();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java b/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java
deleted file mode 100644
index 6fa29d3..0000000
--- a/flink-core/src/main/java/org/apache/flink/migration/util/SerializedValue.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration.util;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.util.InstantiationUtil;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-/**
- * This class is used to transfer (via serialization) objects whose classes are not available
- * in the system class loader. When those objects are deserialized without access to their
- * special class loader, the deserialization fails with a {@code ClassNotFoundException}.
- *
- * To work around that issue, the SerializedValue serialized data immediately into a byte array.
- * When send through RPC or another service that uses serialization, only the byte array is
- * transferred. The object is deserialized later (upon access) and requires the accessor to
- * provide the corresponding class loader.
- *
- * @param <T> The type of the value held.
- * @deprecated Only used internally when migrating from previous savepoint versions.
- */
-@Deprecated
-@PublicEvolving
-public class SerializedValue<T> implements java.io.Serializable {
-
-	private static final long serialVersionUID = -3564011643393683761L;
-
-	/** The serialized data */
-	private final byte[] serializedData;
-
-	private SerializedValue(byte[] serializedData) {
-		this.serializedData = serializedData;
-	}
-
-	public SerializedValue(T value) throws IOException {
-		this.serializedData = value == null ? null : InstantiationUtil.serializeObject(value);
-	}
-
-	@SuppressWarnings("unchecked")
-	public T deserializeValue(ClassLoader loader) throws IOException, ClassNotFoundException {
-		return serializedData == null ? null : (T) MigrationInstantiationUtil.deserializeObject(serializedData, loader);
-	}
-
-	/**
-	 * Returns the serialized value or <code>null</code> if no value is set.
-	 *
-	 * @return Serialized data.
-	 */
-	public byte[] getByteArray() {
-		return serializedData;
-	}
-
-	public static <T> SerializedValue<T> fromBytes(byte[] serializedData) {
-		return new SerializedValue<T>(serializedData);
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public int hashCode() {
-		return serializedData == null ? 0 : Arrays.hashCode(serializedData);
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof SerializedValue) {
-			SerializedValue<?> other = (SerializedValue<?>) obj;
-			return this.serializedData == null ? other.serializedData == null :
-					(other.serializedData != null && Arrays.equals(this.serializedData, other.serializedData));
-		}
-		else {
-			return false;
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "SerializedValue";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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
index 78c57ed..602ad3e 100644
--- 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
@@ -76,7 +76,6 @@ public class ContinuousFileProcessingMigrationTest {
 	@Parameterized.Parameters(name = "Migration Savepoint / Mod Time: {0}")
 	public static Collection<Tuple2<MigrationVersion, Long>> parameters () {
 		return Arrays.asList(
-			Tuple2.of(MigrationVersion.v1_1, 1482144479339L),
 			Tuple2.of(MigrationVersion.v1_2, 1493116191000L),
 			Tuple2.of(MigrationVersion.v1_3, 1496532000000L));
 	}


[10/11] flink git commit: [FLINK-7461] Remove Backwards compatibility with <= Flink 1.1

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
index 11f14b9..78ac39c 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
@@ -43,11 +43,6 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.util.Preconditions;
 
-import org.apache.flink.shaded.guava18.com.google.common.base.Predicate;
-import org.apache.flink.shaded.guava18.com.google.common.collect.Iterators;
-
-import javax.annotation.Nullable;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -55,7 +50,6 @@ import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.OptionalDataException;
 import java.io.Serializable;
-import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -716,9 +710,7 @@ public class NFA<T> implements Serializable {
 		return result;
 	}
 
-	//////////////////////			Fault-Tolerance / Migration			//////////////////////
-
-	private static final String BEGINNING_STATE_NAME = "$beginningState$";
+	//////////////////////			Fault-Tolerance			//////////////////////
 
 	private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
 		ois.defaultReadObject();
@@ -729,103 +721,15 @@ public class NFA<T> implements Serializable {
 
 		final List<ComputationState<T>> readComputationStates = new ArrayList<>(numberComputationStates);
 
-		boolean afterMigration = false;
 		for (int i = 0; i < numberComputationStates; i++) {
 			ComputationState<T> computationState = readComputationState(ois);
-			if (computationState.getState().getName().equals(BEGINNING_STATE_NAME)) {
-				afterMigration = true;
-			}
-
 			readComputationStates.add(computationState);
 		}
 
-		if (afterMigration && !readComputationStates.isEmpty()) {
-			try {
-				//Backwards compatibility
-				this.computationStates.addAll(migrateNFA(readComputationStates));
-				final Field newSharedBufferField = NFA.class.getDeclaredField("eventSharedBuffer");
-				final Field sharedBufferField = NFA.class.getDeclaredField("sharedBuffer");
-				sharedBufferField.setAccessible(true);
-				newSharedBufferField.setAccessible(true);
-				newSharedBufferField.set(this, SharedBuffer.migrateSharedBuffer(this.sharedBuffer));
-				sharedBufferField.set(this, null);
-				sharedBufferField.setAccessible(false);
-				newSharedBufferField.setAccessible(false);
-			} catch (Exception e) {
-				throw new IllegalStateException("Could not migrate from earlier version", e);
-			}
-		} else {
-			this.computationStates.addAll(readComputationStates);
-		}
-
+		this.computationStates.addAll(readComputationStates);
 		nonDuplicatingTypeSerializer.clearReferences();
 	}
 
-	/**
-	 * Needed for backward compatibility. First migrates the {@link State} graph see {@link NFACompiler#migrateGraph(State)}.
-	 * Than recreates the {@link ComputationState}s with the new {@link State} graph.
-	 * @param readStates computation states read from snapshot
-	 * @return collection of migrated computation states
-	 */
-	private Collection<ComputationState<T>> migrateNFA(Collection<ComputationState<T>> readStates) {
-		final ArrayList<ComputationState<T>> computationStates = new ArrayList<>();
-
-		final State<T> startState = Iterators.find(
-			readStates.iterator(),
-			new Predicate<ComputationState<T>>() {
-				@Override
-				public boolean apply(@Nullable ComputationState<T> input) {
-					return input != null && input.getState().getName().equals(BEGINNING_STATE_NAME);
-				}
-			}).getState();
-
-		final Map<String, State<T>> convertedStates = NFACompiler.migrateGraph(startState);
-
-		for (ComputationState<T> readState : readStates) {
-			if (!readState.isStartState()) {
-				final String previousName = readState.getState().getName();
-				final String currentName = Iterators.find(
-					readState.getState().getStateTransitions().iterator(),
-					new Predicate<StateTransition<T>>() {
-						@Override
-						public boolean apply(@Nullable StateTransition<T> input) {
-							return input != null && input.getAction() == StateTransitionAction.TAKE;
-						}
-					}).getTargetState().getName();
-
-				final State<T> previousState = convertedStates.get(previousName);
-
-				computationStates.add(ComputationState.createState(
-					this,
-					convertedStates.get(currentName),
-					previousState,
-					readState.getEvent(),
-					0,
-					readState.getTimestamp(),
-					readState.getVersion(),
-					readState.getStartTimestamp()
-				));
-			}
-		}
-
-		final String startName = Iterators.find(convertedStates.values().iterator(), new Predicate<State<T>>() {
-			@Override
-			public boolean apply(@Nullable State<T> input) {
-				return input != null && input.isStart();
-			}
-		}).getName();
-
-		computationStates.add(ComputationState.createStartState(
-			this,
-			convertedStates.get(startName),
-			new DeweyNumber(this.startEventCounter)));
-
-		this.states.clear();
-		this.states.addAll(convertedStates.values());
-
-		return computationStates;
-	}
-
 	@SuppressWarnings("unchecked")
 	private ComputationState<T> readComputationState(ObjectInputStream ois) throws IOException, ClassNotFoundException {
 		final State<T> state = (State<T>) ois.readObject();

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
index c6f69b9..c36e7df 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.cep.nfa;
 
-import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.CompatibilityUtil;
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
@@ -335,47 +334,6 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 		this.pages = pages;
 	}
 
-	/**
-	 * For backward compatibility only. Previously the key in {@link SharedBuffer} was {@link State}.
-	 * Now it is {@link String}.
-	 */
-	@Internal
-	static <T> SharedBuffer<String, T> migrateSharedBuffer(SharedBuffer<State<T>, T> buffer) {
-
-		final Map<String, SharedBufferPage<String, T>> pageMap = new HashMap<>();
-		final Map<SharedBufferEntry<State<T>, T>, SharedBufferEntry<String, T>> entries = new HashMap<>();
-
-		for (Map.Entry<State<T>, SharedBufferPage<State<T>, T>> page : buffer.pages.entrySet()) {
-			final SharedBufferPage<String, T> newPage = new SharedBufferPage<>(page.getKey().getName());
-			pageMap.put(newPage.getKey(), newPage);
-
-			for (Map.Entry<ValueTimeWrapper<T>, SharedBufferEntry<State<T>, T>> pageEntry : page.getValue().entries.entrySet()) {
-				final SharedBufferEntry<String, T> newSharedBufferEntry = new SharedBufferEntry<>(
-					pageEntry.getKey(),
-					newPage);
-				newSharedBufferEntry.referenceCounter = pageEntry.getValue().referenceCounter;
-				entries.put(pageEntry.getValue(), newSharedBufferEntry);
-				newPage.entries.put(pageEntry.getKey(), newSharedBufferEntry);
-			}
-		}
-
-		for (Map.Entry<State<T>, SharedBufferPage<State<T>, T>> page : buffer.pages.entrySet()) {
-			for (Map.Entry<ValueTimeWrapper<T>, SharedBufferEntry<State<T>, T>> pageEntry : page.getValue().entries.entrySet()) {
-				final SharedBufferEntry<String, T> newEntry = entries.get(pageEntry.getValue());
-				for (SharedBufferEdge<State<T>, T> edge : pageEntry.getValue().edges) {
-					final SharedBufferEntry<String, T> targetNewEntry = entries.get(edge.getTarget());
-
-					final SharedBufferEdge<String, T> newEdge = new SharedBufferEdge<>(
-						targetNewEntry,
-						edge.getVersion());
-					newEntry.edges.add(newEdge);
-				}
-			}
-		}
-
-		return new SharedBuffer<>(buffer.valueSerializer, pageMap);
-	}
-
 	private SharedBufferEntry<K, V> get(
 			final K key,
 			final V value,
@@ -1177,76 +1135,4 @@ public class SharedBuffer<K extends Serializable, V> implements Serializable {
 			return CompatibilityResult.requiresMigration();
 		}
 	}
-
-	//////////////////			Java Serialization methods for backwards compatibility			//////////////////
-
-	private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
-		DataInputViewStreamWrapper source = new DataInputViewStreamWrapper(ois);
-		ArrayList<SharedBufferEntry<K, V>> entryList = new ArrayList<>();
-		ois.defaultReadObject();
-
-		this.pages = new HashMap<>();
-
-		int numberPages = ois.readInt();
-
-		for (int i = 0; i < numberPages; i++) {
-			// key of the page
-			@SuppressWarnings("unchecked")
-			K key = (K) ois.readObject();
-
-			SharedBufferPage<K, V> page = new SharedBufferPage<>(key);
-
-			pages.put(key, page);
-
-			int numberEntries = ois.readInt();
-
-			for (int j = 0; j < numberEntries; j++) {
-				// restore the SharedBufferEntries for the given page
-				V value = valueSerializer.deserialize(source);
-				long timestamp = ois.readLong();
-
-				ValueTimeWrapper<V> valueTimeWrapper = new ValueTimeWrapper<>(value, timestamp, 0);
-				SharedBufferEntry<K, V> sharedBufferEntry = new SharedBufferEntry<K, V>(valueTimeWrapper, page);
-
-				sharedBufferEntry.referenceCounter = ois.readInt();
-
-				page.entries.put(valueTimeWrapper, sharedBufferEntry);
-
-				entryList.add(sharedBufferEntry);
-			}
-		}
-
-		// read the edges of the shared buffer entries
-		int numberEdges = ois.readInt();
-
-		for (int j = 0; j < numberEdges; j++) {
-			int sourceIndex = ois.readInt();
-			int targetIndex = ois.readInt();
-
-			if (sourceIndex >= entryList.size() || sourceIndex < 0) {
-				throw new RuntimeException("Could not find source entry with index " + sourceIndex +
-						". This indicates a corrupted state.");
-			} else {
-				// We've already deserialized the shared buffer entry. Simply read its ID and
-				// retrieve the buffer entry from the list of entries
-				SharedBufferEntry<K, V> sourceEntry = entryList.get(sourceIndex);
-
-				final DeweyNumber version = (DeweyNumber) ois.readObject();
-				final SharedBufferEntry<K, V> target;
-
-				if (targetIndex >= 0) {
-					if (targetIndex >= entryList.size()) {
-						throw new RuntimeException("Could not find target entry with index " + targetIndex +
-								". This indicates a corrupted state.");
-					} else {
-						target = entryList.get(targetIndex);
-					}
-				} else {
-					target = null;
-				}
-
-				sourceEntry.edges.add(new SharedBufferEdge<K, V>(target, version));
-			}
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
index 593c94f..5698de6 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.cep.nfa.compiler;
 
-import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.nfa.NFA;
@@ -36,11 +35,6 @@ import org.apache.flink.cep.pattern.conditions.IterativeCondition;
 import org.apache.flink.cep.pattern.conditions.NotCondition;
 import org.apache.flink.streaming.api.windowing.time.Time;
 
-import org.apache.flink.shaded.guava18.com.google.common.base.Predicate;
-import org.apache.flink.shaded.guava18.com.google.common.collect.Iterators;
-
-import javax.annotation.Nullable;
-
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -858,114 +852,6 @@ public class NFACompiler {
 	}
 
 	/**
-	 * Used for migrating CEP graphs prior to 1.3. It removes the dummy start, adds the dummy end, and translates all
-	 * states to consuming ones by moving all TAKEs and IGNOREs to the next state. This method assumes each state
-	 * has at most one TAKE and one IGNORE and name of each state is unique. No PROCEED transition is allowed!
-	 *
-	 * @param oldStartState dummy start state of old graph
-	 * @param <T>           type of events
-	 * @return map of new states, where key is the name of a state and value is the state itself
-	 */
-	@Internal
-	public static <T> Map<String, State<T>> migrateGraph(State<T> oldStartState) {
-		State<T> oldFirst = oldStartState;
-		State<T> oldSecond = oldStartState.getStateTransitions().iterator().next().getTargetState();
-
-		StateTransition<T> oldFirstToSecondTake = Iterators.find(
-			oldFirst.getStateTransitions().iterator(),
-			new Predicate<StateTransition<T>>() {
-				@Override
-				public boolean apply(@Nullable StateTransition<T> input) {
-					return input != null && input.getAction() == StateTransitionAction.TAKE;
-				}
-
-			});
-
-		StateTransition<T> oldFirstIgnore = Iterators.find(
-			oldFirst.getStateTransitions().iterator(),
-			new Predicate<StateTransition<T>>() {
-				@Override
-				public boolean apply(@Nullable StateTransition<T> input) {
-					return input != null && input.getAction() == StateTransitionAction.IGNORE;
-				}
-
-			}, null);
-
-		StateTransition<T> oldSecondToThirdTake = Iterators.find(
-			oldSecond.getStateTransitions().iterator(),
-			new Predicate<StateTransition<T>>() {
-				@Override
-				public boolean apply(@Nullable StateTransition<T> input) {
-					return input != null && input.getAction() == StateTransitionAction.TAKE;
-				}
-
-			}, null);
-
-		final Map<String, State<T>> convertedStates = new HashMap<>();
-		State<T> newSecond;
-		State<T> newFirst = new State<>(oldSecond.getName(), State.StateType.Start);
-		convertedStates.put(newFirst.getName(), newFirst);
-		while (oldSecondToThirdTake != null) {
-
-			newSecond = new State<T>(oldSecondToThirdTake.getTargetState().getName(), State.StateType.Normal);
-			convertedStates.put(newSecond.getName(), newSecond);
-			newFirst.addTake(newSecond, oldFirstToSecondTake.getCondition());
-
-			if (oldFirstIgnore != null) {
-				newFirst.addIgnore(oldFirstIgnore.getCondition());
-			}
-
-			oldFirst = oldSecond;
-
-			oldFirstToSecondTake = Iterators.find(
-				oldFirst.getStateTransitions().iterator(),
-				new Predicate<StateTransition<T>>() {
-					@Override
-					public boolean apply(@Nullable StateTransition<T> input) {
-						return input != null && input.getAction() == StateTransitionAction.TAKE;
-					}
-
-				});
-
-			oldFirstIgnore = Iterators.find(
-				oldFirst.getStateTransitions().iterator(),
-				new Predicate<StateTransition<T>>() {
-					@Override
-					public boolean apply(@Nullable StateTransition<T> input) {
-						return input != null && input.getAction() == StateTransitionAction.IGNORE;
-					}
-
-				}, null);
-
-			oldSecond = oldSecondToThirdTake.getTargetState();
-
-			oldSecondToThirdTake = Iterators.find(
-				oldSecond.getStateTransitions().iterator(),
-				new Predicate<StateTransition<T>>() {
-					@Override
-					public boolean apply(@Nullable StateTransition<T> input) {
-						return input != null && input.getAction() == StateTransitionAction.TAKE;
-					}
-
-				}, null);
-
-			newFirst = newSecond;
-		}
-
-		final State<T> endingState = new State<>(ENDING_STATE_NAME, State.StateType.Final);
-
-		newFirst.addTake(endingState, oldFirstToSecondTake.getCondition());
-
-		if (oldFirstIgnore != null) {
-			newFirst.addIgnore(oldFirstIgnore.getCondition());
-		}
-
-		convertedStates.put(endingState.getName(), endingState);
-
-		return convertedStates;
-	}
-
-	/**
 	 * Factory interface for {@link NFA}.
 	 *
 	 * @param <T> Type of the input events which are processed by the NFA

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/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 7556d9f..257d3e7 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
@@ -24,48 +24,29 @@ import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.common.typeutils.base.CollectionSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.base.ListSerializer;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.EventComparator;
 import org.apache.flink.cep.nfa.NFA;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
-import org.apache.flink.core.fs.FSDataInputStream;
-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.runtime.state.StateInitializationContext;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
 import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.CheckpointedRestoringOperator;
 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.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.Preconditions;
 
 import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.PriorityQueue;
 import java.util.stream.Stream;
 import java.util.stream.StreamSupport;
@@ -80,23 +61,17 @@ import java.util.stream.StreamSupport;
  * @param <IN> Type of the input elements
  * @param <KEY> Type of the key on which the input stream is keyed
  * @param <OUT> Type of the output elements
- * @param <F> user function that can be applied to matching sequences or timed out sequences
  */
 public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Function>
 	extends AbstractUdfStreamOperator<OUT, F>
-	implements OneInputStreamOperator<IN, OUT>, Triggerable<KEY, VoidNamespace>, CheckpointedRestoringOperator {
+	implements OneInputStreamOperator<IN, OUT>, Triggerable<KEY, VoidNamespace> {
 
 	private static final long serialVersionUID = -4166778210774160757L;
 
-	private static final int INITIAL_PRIORITY_QUEUE_CAPACITY = 11;
-
 	private final boolean isProcessingTime;
 
 	private final TypeSerializer<IN> inputSerializer;
 
-	// necessary to serialize the set of seen keys
-	private final TypeSerializer<KEY> keySerializer;
-
 	///////////////			State			//////////////
 
 	private static final String NFA_OPERATOR_STATE_NAME = "nfaOperatorStateName";
@@ -115,12 +90,6 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 	 */
 	private long lastWatermark;
 
-	/**
-	 * A flag used in the case of migration that indicates if
-	 * we are restoring from an old keyed or non-keyed operator.
-	 */
-	private final boolean migratingFromOldKeyedOperator;
-
 	private final EventComparator<IN> comparator;
 
 	public AbstractKeyedCEPPatternOperator(
@@ -135,10 +104,7 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 
 		this.inputSerializer = Preconditions.checkNotNull(inputSerializer);
 		this.isProcessingTime = Preconditions.checkNotNull(isProcessingTime);
-		this.keySerializer = Preconditions.checkNotNull(keySerializer);
 		this.nfaFactory = Preconditions.checkNotNull(nfaFactory);
-
-		this.migratingFromOldKeyedOperator = migratingFromOldKeyedOperator;
 		this.comparator = comparator;
 	}
 
@@ -384,295 +350,6 @@ public abstract class AbstractKeyedCEPPatternOperator<IN, KEY, OUT, F extends Fu
 			long timestamp) throws Exception {
 	}
 
-	//////////////////////			Backwards Compatibility			//////////////////////
-
-	@Override
-	public void restoreState(FSDataInputStream in) throws Exception {
-		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(
-				"watermark-callbacks",
-				VoidNamespaceSerializer.INSTANCE,
-				this);
-
-		// this is with the old serializer so that we can read the state.
-		ValueState<NFA<IN>> oldNfaOperatorState = getRuntimeContext().getState(
-				new ValueStateDescriptor<>("nfaOperatorState", new NFA.Serializer<IN>()));
-
-		ValueState<PriorityQueue<StreamRecord<IN>>> oldPriorityQueueOperatorState =
-				getRuntimeContext().getState(
-					new ValueStateDescriptor<>(
-							"priorityQueueStateName",
-							new PriorityQueueSerializer<>(
-									((TypeSerializer) new StreamElementSerializer<>(inputSerializer)),
-									new PriorityQueueStreamRecordFactory<IN>()
-							)
-					)
-			);
-
-		if (migratingFromOldKeyedOperator) {
-			int numberEntries = inputView.readInt();
-			for (int i = 0; i < numberEntries; i++) {
-				KEY key = keySerializer.deserialize(inputView);
-				setCurrentKey(key);
-				saveRegisterWatermarkTimer();
-
-				NFA<IN> nfa = oldNfaOperatorState.value();
-				oldNfaOperatorState.clear();
-				nfaOperatorState.update(nfa);
-
-				PriorityQueue<StreamRecord<IN>> priorityQueue = oldPriorityQueueOperatorState.value();
-				if (priorityQueue != null && !priorityQueue.isEmpty()) {
-					Map<Long, List<IN>> elementMap = new HashMap<>();
-					for (StreamRecord<IN> record: priorityQueue) {
-						long timestamp = record.getTimestamp();
-						IN element = record.getValue();
-
-						List<IN> elements = elementMap.get(timestamp);
-						if (elements == null) {
-							elements = new ArrayList<>();
-							elementMap.put(timestamp, elements);
-						}
-						elements.add(element);
-					}
-
-					// write the old state into the new one.
-					for (Map.Entry<Long, List<IN>> entry: elementMap.entrySet()) {
-						elementQueueState.put(entry.getKey(), entry.getValue());
-					}
-
-					// clear the old state
-					oldPriorityQueueOperatorState.clear();
-				}
-			}
-		} else {
-
-			final ObjectInputStream ois = new ObjectInputStream(in);
-
-			// retrieve the NFA
-			@SuppressWarnings("unchecked")
-			NFA<IN> nfa = (NFA<IN>) ois.readObject();
-
-			// retrieve the elements that were pending in the priority queue
-			MultiplexingStreamRecordSerializer<IN> recordSerializer = new MultiplexingStreamRecordSerializer<>(inputSerializer);
-
-			Map<Long, List<IN>> elementMap = new HashMap<>();
-			int entries = ois.readInt();
-			for (int i = 0; i < entries; i++) {
-				StreamElement streamElement = recordSerializer.deserialize(inputView);
-				StreamRecord<IN> record = streamElement.<IN>asRecord();
-
-				long timestamp = record.getTimestamp();
-				IN element = record.getValue();
-
-				List<IN> elements = elementMap.get(timestamp);
-				if (elements == null) {
-					elements = new ArrayList<>();
-					elementMap.put(timestamp, elements);
-				}
-				elements.add(element);
-			}
-
-			// finally register the retrieved state with the new keyed state.
-			setCurrentKey((byte) 0);
-			nfaOperatorState.update(nfa);
-
-			// write the priority queue to the new map state.
-			for (Map.Entry<Long, List<IN>> entry: elementMap.entrySet()) {
-				elementQueueState.put(entry.getKey(), entry.getValue());
-			}
-
-			if (!isProcessingTime) {
-				// this is relevant only for event/ingestion time
-				setCurrentKey((byte) 0);
-				saveRegisterWatermarkTimer();
-			}
-			ois.close();
-		}
-	}
-
-	//////////////////////			Utility Classes			//////////////////////
-
-	/**
-	 * Custom type serializer implementation to serialize priority queues.
-	 *
-	 * @param <T> Type of the priority queue's elements
-	 */
-	private static class PriorityQueueSerializer<T> extends TypeSerializer<PriorityQueue<T>> {
-
-		private static final long serialVersionUID = -231980397616187715L;
-
-		private final TypeSerializer<T> elementSerializer;
-		private final PriorityQueueFactory<T> factory;
-
-		PriorityQueueSerializer(final TypeSerializer<T> elementSerializer, final PriorityQueueFactory<T> factory) {
-			this.elementSerializer = elementSerializer;
-			this.factory = factory;
-		}
-
-		@Override
-		public boolean isImmutableType() {
-			return false;
-		}
-
-		@Override
-		public TypeSerializer<PriorityQueue<T>> duplicate() {
-			return new PriorityQueueSerializer<>(elementSerializer.duplicate(), factory);
-		}
-
-		@Override
-		public PriorityQueue<T> createInstance() {
-			return factory.createPriorityQueue();
-		}
-
-		@Override
-		public PriorityQueue<T> copy(PriorityQueue<T> from) {
-			PriorityQueue<T> result = factory.createPriorityQueue();
-
-			for (T element: from) {
-				result.offer(elementSerializer.copy(element));
-			}
-
-			return result;
-		}
-
-		@Override
-		public PriorityQueue<T> copy(PriorityQueue<T> from, PriorityQueue<T> reuse) {
-			reuse.clear();
-
-			for (T element: from) {
-				reuse.offer(elementSerializer.copy(element));
-			}
-
-			return reuse;
-		}
-
-		@Override
-		public int getLength() {
-			return 0;
-		}
-
-		@Override
-		public void serialize(PriorityQueue<T> record, DataOutputView target) throws IOException {
-			target.writeInt(record.size());
-
-			for (T element: record) {
-				elementSerializer.serialize(element, target);
-			}
-		}
-
-		@Override
-		public PriorityQueue<T> deserialize(DataInputView source) throws IOException {
-			PriorityQueue<T> result = factory.createPriorityQueue();
-
-			return deserialize(result, source);
-		}
-
-		@Override
-		public PriorityQueue<T> deserialize(PriorityQueue<T> reuse, DataInputView source) throws IOException {
-			reuse.clear();
-
-			int numberEntries = source.readInt();
-
-			for (int i = 0; i < numberEntries; i++) {
-				reuse.offer(elementSerializer.deserialize(source));
-			}
-
-			return reuse;
-		}
-
-		@Override
-		public void copy(DataInputView source, DataOutputView target) throws IOException {
-
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			if (obj instanceof PriorityQueueSerializer) {
-				@SuppressWarnings("unchecked")
-				PriorityQueueSerializer<T> other = (PriorityQueueSerializer<T>) obj;
-
-				return factory.equals(other.factory) && elementSerializer.equals(other.elementSerializer);
-			} else {
-				return false;
-			}
-		}
-
-		@Override
-		public boolean canEqual(Object obj) {
-			return obj instanceof PriorityQueueSerializer;
-		}
-
-		@Override
-		public int hashCode() {
-			return Objects.hash(factory, elementSerializer);
-		}
-
-		// --------------------------------------------------------------------------------------------
-		// Serializer configuration snapshotting & compatibility
-		// --------------------------------------------------------------------------------------------
-
-		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
-			return new CollectionSerializerConfigSnapshot<>(elementSerializer);
-		}
-
-		@Override
-		public CompatibilityResult<PriorityQueue<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-			if (configSnapshot instanceof CollectionSerializerConfigSnapshot) {
-				Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> previousElemSerializerAndConfig =
-					((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
-
-				CompatibilityResult<T> compatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousElemSerializerAndConfig.f0,
-					UnloadableDummyTypeSerializer.class,
-					previousElemSerializerAndConfig.f1,
-					elementSerializer);
-
-				if (!compatResult.isRequiresMigration()) {
-					return CompatibilityResult.compatible();
-				} else if (compatResult.getConvertDeserializer() != null) {
-					return CompatibilityResult.requiresMigration(
-						new PriorityQueueSerializer<>(
-							new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer()), factory));
-				}
-			}
-
-			return CompatibilityResult.requiresMigration();
-		}
-	}
-
-	private interface PriorityQueueFactory<T> extends Serializable {
-		PriorityQueue<T> createPriorityQueue();
-	}
-
-	private static class PriorityQueueStreamRecordFactory<T> implements PriorityQueueFactory<StreamRecord<T>> {
-
-		private static final long serialVersionUID = 1254766984454616593L;
-
-		@Override
-		public PriorityQueue<StreamRecord<T>> createPriorityQueue() {
-			return new PriorityQueue<StreamRecord<T>>(INITIAL_PRIORITY_QUEUE_CAPACITY, new StreamRecordComparator<T>());
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			return obj instanceof PriorityQueueStreamRecordFactory;
-		}
-
-		@Override
-		public int hashCode() {
-			return getClass().hashCode();
-		}
-	}
-
 	//////////////////////			Testing Methods			//////////////////////
 
 	@VisibleForTesting

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java
deleted file mode 100644
index 843d668..0000000
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigration11to13Test.java
+++ /dev/null
@@ -1,346 +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.ByteSerializer;
-import org.apache.flink.api.java.functions.NullByteKeySelector;
-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.junit.Test;
-
-import java.net.URL;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests for migration from 1.1.x to 1.3.x.
- */
-public class CEPMigration11to13Test {
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = CEPMigration11to13Test.class.getClassLoader();
-		URL resource = cl.getResource(filename);
-		if (resource == null) {
-			throw new NullPointerException("Missing snapshot resource.");
-		}
-		return resource.getFile();
-	}
-
-	@Test
-	public void testKeyedCEPOperatorMigratation() throws Exception {
-
-		final Event startEvent = new Event(42, "start", 1.0);
-		final SubEvent middleEvent = new SubEvent(42, "foo", 1.0, 10.0);
-		final Event endEvent = new Event(42, "end", 1.0);
-
-		// uncomment these lines for regenerating the snapshot on Flink 1.1
-		/*
-		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness = new OneInputStreamOperatorTestHarness<>(
-				new KeyedCepOperator<>(
-						Event.createTypeSerializer(),
-						false,
-						keySelector,
-						IntSerializer.INSTANCE,
-						new NFAFactory()));
-		harness.configureForKeyedStream(keySelector, BasicTypeInfo.INT_TYPE_INFO);
-		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.processWatermark(new Watermark(2));
-
-		harness.processElement(new StreamRecord<Event>(middleEvent, 3));
-
-		// simulate snapshot/restore with empty element queue but NFA state
-		StreamTaskState snapshot = harness.snapshot(1, 1);
-		FileOutputStream out = new FileOutputStream(
-				"src/test/resources/cep-keyed-1_1-snapshot");
-		ObjectOutputStream oos = new ObjectOutputStream(out);
-		oos.writeObject(snapshot);
-		out.close();
-		harness.close();
-		*/
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness = CepOperatorTestUtilities.getCepTestHarness(
-			CepOperatorTestUtilities.getKeyedCepOpearator(false, new NFAFactory()));
-
-		try {
-			harness.setup();
-			harness
-				.initializeStateFromLegacyCheckpoint(getResourceFilename("cep-keyed-1_1-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 the result
-			assertEquals(2, result.size());
-
-			Object resultObject = result.poll();
-			assertTrue(resultObject instanceof StreamRecord);
-			StreamRecord<?> resultRecord = (StreamRecord<?>) resultObject;
-			assertTrue(resultRecord.getValue() instanceof Map);
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap =
-				(Map<String, List<Event>>) resultRecord.getValue();
-
-			assertEquals(startEvent, patternMap.get("start").get(0));
-			assertEquals(middleEvent, patternMap.get("middle").get(0));
-			assertEquals(endEvent, patternMap.get("end").get(0));
-
-			// and now go for a checkpoint with the new serializers
-
-			final Event startEvent1 = new Event(42, "start", 2.0);
-			final SubEvent middleEvent1 = new SubEvent(42, "foo", 1.0, 11.0);
-			final Event endEvent1 = new Event(42, "end", 2.0);
-
-			harness.processElement(new StreamRecord<Event>(startEvent1, 21));
-			harness.processElement(new StreamRecord<Event>(middleEvent1, 23));
-
-			// simulate snapshot/restore with some elements in internal sorting queue
-			OperatorStateHandles snapshot = harness.snapshot(1L, 1L);
-			harness.close();
-
-			harness = CepOperatorTestUtilities.getCepTestHarness(CepOperatorTestUtilities.getKeyedCepOpearator(
-				false,
-				new NFAFactory()));
-
-			harness.setup();
-			harness.initializeState(snapshot);
-			harness.open();
-
-			harness.processElement(new StreamRecord<>(endEvent1, 25));
-
-			harness.processWatermark(new Watermark(50));
-
-			result = harness.getOutput();
-
-			// watermark and the result
-			assertEquals(2, result.size());
-
-			Object resultObject1 = result.poll();
-			assertTrue(resultObject1 instanceof StreamRecord);
-			StreamRecord<?> resultRecord1 = (StreamRecord<?>) resultObject1;
-			assertTrue(resultRecord1.getValue() instanceof Map);
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap1 =
-				(Map<String, List<Event>>) resultRecord1.getValue();
-
-			assertEquals(startEvent1, patternMap1.get("start").get(0));
-			assertEquals(middleEvent1, patternMap1.get("middle").get(0));
-			assertEquals(endEvent1, patternMap1.get("end").get(0));
-		} finally {
-			harness.close();
-		}
-	}
-
-	@Test
-	public void testNonKeyedCEPFunctionMigration() throws Exception {
-
-		final Event startEvent = new Event(42, "start", 1.0);
-		final SubEvent middleEvent = new SubEvent(42, "foo", 1.0, 10.0);
-		final Event endEvent = new Event(42, "end", 1.0);
-
-		// uncomment these lines for regenerating the snapshot on Flink 1.1
-		/*
-		OneInputStreamOperatorTestHarness<Event, Map<String, Event>> harness = new OneInputStreamOperatorTestHarness<>(
-				new CEPPatternOperator<>(
-						Event.createTypeSerializer(),
-						false,
-						new NFAFactory()));
-		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.processWatermark(new Watermark(2));
-
-		harness.processElement(new StreamRecord<Event>(middleEvent, 3));
-
-		// simulate snapshot/restore with empty element queue but NFA state
-		StreamTaskState snapshot = harness.snapshot(1, 1);
-		FileOutputStream out = new FileOutputStream(
-				"src/test/resources/cep-non-keyed-1.1-snapshot");
-		ObjectOutputStream oos = new ObjectOutputStream(out);
-		oos.writeObject(snapshot);
-		out.close();
-		harness.close();
-		*/
-
-		NullByteKeySelector keySelector = new NullByteKeySelector();
-
-		OneInputStreamOperatorTestHarness<Event, Map<String, List<Event>>> harness =
-			new KeyedOneInputStreamOperatorTestHarness<Byte, Event, Map<String, List<Event>>>(
-				CepOperatorTestUtilities.getKeyedCepOpearator(false, new NFAFactory(), ByteSerializer.INSTANCE, false, null),
-				keySelector,
-				BasicTypeInfo.BYTE_TYPE_INFO);
-
-		try {
-			harness.setup();
-			harness.initializeStateFromLegacyCheckpoint(
-				getResourceFilename("cep-non-keyed-1.1-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 the result
-			assertEquals(2, result.size());
-
-			Object resultObject = result.poll();
-			assertTrue(resultObject instanceof StreamRecord);
-			StreamRecord<?> resultRecord = (StreamRecord<?>) resultObject;
-			assertTrue(resultRecord.getValue() instanceof Map);
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap =
-				(Map<String, List<Event>>) resultRecord.getValue();
-
-			assertEquals(startEvent, patternMap.get("start").get(0));
-			assertEquals(middleEvent, patternMap.get("middle").get(0));
-			assertEquals(endEvent, patternMap.get("end").get(0));
-
-			// and now go for a checkpoint with the new serializers
-
-			final Event startEvent1 = new Event(42, "start", 2.0);
-			final SubEvent middleEvent1 = new SubEvent(42, "foo", 1.0, 11.0);
-			final Event endEvent1 = new Event(42, "end", 2.0);
-
-			harness.processElement(new StreamRecord<Event>(startEvent1, 21));
-			harness.processElement(new StreamRecord<Event>(middleEvent1, 23));
-
-			// simulate snapshot/restore with some elements in internal sorting queue
-			OperatorStateHandles snapshot = harness.snapshot(1L, 1L);
-			harness.close();
-
-			harness = new KeyedOneInputStreamOperatorTestHarness<Byte, Event, Map<String, List<Event>>>(
-				CepOperatorTestUtilities.getKeyedCepOpearator(false, new NFAFactory(), ByteSerializer.INSTANCE),
-				keySelector,
-				BasicTypeInfo.BYTE_TYPE_INFO);
-
-			harness.setup();
-			harness.initializeState(snapshot);
-			harness.open();
-
-			harness.processElement(new StreamRecord<>(endEvent1, 25));
-
-			harness.processWatermark(new Watermark(50));
-
-			result = harness.getOutput();
-
-			// watermark and the result
-			assertEquals(2, result.size());
-
-			Object resultObject1 = result.poll();
-			assertTrue(resultObject1 instanceof StreamRecord);
-			StreamRecord<?> resultRecord1 = (StreamRecord<?>) resultObject1;
-			assertTrue(resultRecord1.getValue() instanceof Map);
-
-			@SuppressWarnings("unchecked")
-			Map<String, List<Event>> patternMap1 =
-				(Map<String, List<Event>>) resultRecord1.getValue();
-
-			assertEquals(startEvent1, patternMap1.get("start").get(0));
-			assertEquals(middleEvent1, patternMap1.get("middle").get(0));
-			assertEquals(endEvent1, patternMap1.get("end").get(0));
-		} finally {
-			harness.close();
-		}
-	}
-
-	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/6642768a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
index cf3c921..ed28f25 100644
--- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
+++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java
@@ -71,7 +71,7 @@ public class CEPMigrationTest {
 
 	@Parameterized.Parameters(name = "Migration Savepoint: {0}")
 	public static Collection<MigrationVersion> parameters () {
-		return Arrays.asList(MigrationVersion.v1_2, MigrationVersion.v1_3);
+		return Arrays.asList(MigrationVersion.v1_3);
 	}
 
 	public CEPMigrationTest(MigrationVersion migrateVersion) {

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java
deleted file mode 100644
index c4e23ca..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.migration;
-
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.ParameterlessTypeSerializerConfig;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * The purpose of this class is the be filled in as a placeholder for the namespace serializer when migrating from
- * Flink 1.1 savepoint (which did not include the namespace serializer) to Flink 1.2 (which always must include a
- * (non-null) namespace serializer. This is then replaced as soon as the user is re-registering her state again for
- * the first run under Flink 1.2 and provides again the real namespace serializer.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class MigrationNamespaceSerializerProxy extends TypeSerializer<Serializable> {
-
-	public static final MigrationNamespaceSerializerProxy INSTANCE = new MigrationNamespaceSerializerProxy();
-
-	private static final long serialVersionUID = -707800010807094491L;
-
-	private MigrationNamespaceSerializerProxy() {
-	}
-
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-
-	@Override
-	public TypeSerializer<Serializable> duplicate() {
-		return this;
-	}
-
-	@Override
-	public Serializable createInstance() {
-		throw new UnsupportedOperationException(
-				"This is just a proxy used during migration until the real type serializer is provided by the user.");
-	}
-
-	@Override
-	public Serializable copy(Serializable from) {
-		throw new UnsupportedOperationException(
-				"This is just a proxy used during migration until the real type serializer is provided by the user.");
-	}
-
-	@Override
-	public Serializable copy(Serializable from, Serializable reuse) {
-		throw new UnsupportedOperationException(
-				"This is just a proxy used during migration until the real type serializer is provided by the user.");
-	}
-
-	@Override
-	public int getLength() {
-		return -1;
-	}
-
-	@Override
-	public void serialize(Serializable record, DataOutputView target) throws IOException {
-		throw new UnsupportedOperationException(
-				"This is just a proxy used during migration until the real type serializer is provided by the user.");
-	}
-
-	@Override
-	public Serializable deserialize(DataInputView source) throws IOException {
-		throw new UnsupportedOperationException(
-				"This is just a proxy used during migration until the real type serializer is provided by the user.");
-	}
-
-	@Override
-	public Serializable deserialize(Serializable reuse, DataInputView source) throws IOException {
-		throw new UnsupportedOperationException(
-				"This is just a proxy used during migration until the real type serializer is provided by the user.");
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		throw new UnsupportedOperationException(
-				"This is just a proxy used during migration until the real type serializer is provided by the user.");
-	}
-
-	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
-		return new ParameterlessTypeSerializerConfig(getClass().getCanonicalName());
-	}
-
-	@Override
-	public CompatibilityResult<Serializable> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		// always assume compatibility since we're just a proxy for migration
-		return CompatibilityResult.compatible();
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		return obj instanceof MigrationNamespaceSerializerProxy;
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return true;
-	}
-
-	@Override
-	public int hashCode() {
-		return 42;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
deleted file mode 100644
index a6055a8..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationUtil.java
+++ /dev/null
@@ -1,38 +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.migration;
-
-import org.apache.flink.migration.state.MigrationKeyGroupStateHandle;
-import org.apache.flink.runtime.state.KeyedStateHandle;
-
-import java.util.Collection;
-
-/**
- * Utility functions for migration.
- */
-public class MigrationUtil {
-
-	@SuppressWarnings("deprecation")
-	public static boolean isOldSavepointKeyedState(Collection<KeyedStateHandle> keyedStateHandles) {
-		return (keyedStateHandles != null)
-				&& (keyedStateHandles.size() == 1)
-				&& (keyedStateHandles.iterator().next() instanceof MigrationKeyGroupStateHandle);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/api/common/state/ListStateDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/api/common/state/ListStateDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/migration/api/common/state/ListStateDescriptor.java
deleted file mode 100644
index 5196d2d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/api/common/state/ListStateDescriptor.java
+++ /dev/null
@@ -1,113 +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.migration.api.common.state;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.StateBinder;
-import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-/**
- * The old version of the {@link org.apache.flink.api.common.state.ListStateDescriptor}, retained for
- * serialization backwards compatibility.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Internal
-@Deprecated
-@SuppressWarnings("deprecation")
-public class ListStateDescriptor<T> extends StateDescriptor<ListState<T>, T> {
-	private static final long serialVersionUID = 1L;
-
-	/**
-	 * Creates a new {@code ListStateDescriptor} with the given name and list element type.
-	 *
-	 * <p>If this constructor fails (because it is not possible to describe the type via a class),
-	 * consider using the {@link #ListStateDescriptor(String, TypeInformation)} constructor.
-	 *
-	 * @param name The (unique) name for the state.
-	 * @param typeClass The type of the values in the state.
-	 */
-	public ListStateDescriptor(String name, Class<T> typeClass) {
-		super(name, typeClass, null);
-	}
-
-	/**
-	 * Creates a new {@code ListStateDescriptor} with the given name and list element type.
-	 *
-	 * @param name The (unique) name for the state.
-	 * @param typeInfo The type of the values in the state.
-	 */
-	public ListStateDescriptor(String name, TypeInformation<T> typeInfo) {
-		super(name, typeInfo, null);
-	}
-
-	/**
-	 * Creates a new {@code ListStateDescriptor} with the given name and list element type.
-	 *
-	 * @param name The (unique) name for the state.
-	 * @param typeSerializer The type serializer for the list values.
-	 */
-	public ListStateDescriptor(String name, TypeSerializer<T> typeSerializer) {
-		super(name, typeSerializer, null);
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public ListState<T> bind(StateBinder stateBinder) throws Exception {
-		throw new IllegalStateException("Cannot bind states with a legacy state descriptor.");
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		ListStateDescriptor<?> that = (ListStateDescriptor<?>) o;
-
-		return serializer.equals(that.serializer) && name.equals(that.name);
-
-	}
-
-	@Override
-	public int hashCode() {
-		int result = serializer.hashCode();
-		result = 31 * result + name.hashCode();
-		return result;
-	}
-
-	@Override
-	public String toString() {
-		return "ListStateDescriptor{" +
-				"serializer=" + serializer +
-				'}';
-	}
-
-	@Override
-	public org.apache.flink.api.common.state.StateDescriptor.Type getType() {
-		return org.apache.flink.api.common.state.StateDescriptor.Type.LIST;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/KeyGroupState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/KeyGroupState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/KeyGroupState.java
deleted file mode 100644
index 0b25e08..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/KeyGroupState.java
+++ /dev/null
@@ -1,87 +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.migration.runtime.checkpoint;
-
-import org.apache.flink.migration.runtime.state.StateHandle;
-import org.apache.flink.migration.util.SerializedValue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-
-/**
- * Simple container class which contains the serialized state handle for a key group.
- *
- * The key group state handle is kept in serialized form because it can contain user code classes
- * which might not be available on the JobManager.
- *
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class KeyGroupState implements Serializable {
-	private static final long serialVersionUID = -5926696455438467634L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(KeyGroupState.class);
-
-	private final SerializedValue<StateHandle<?>> keyGroupState;
-
-	private final long stateSize;
-
-	private final long duration;
-
-	public KeyGroupState(SerializedValue<StateHandle<?>> keyGroupState, long stateSize, long duration) {
-		this.keyGroupState = keyGroupState;
-
-		this.stateSize = stateSize;
-
-		this.duration = duration;
-	}
-
-	public SerializedValue<StateHandle<?>> getKeyGroupState() {
-		return keyGroupState;
-	}
-
-	public long getDuration() {
-		return duration;
-	}
-
-	public long getStateSize() {
-		return stateSize;
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof KeyGroupState) {
-			KeyGroupState other = (KeyGroupState) obj;
-
-			return keyGroupState.equals(other.keyGroupState) && stateSize == other.stateSize &&
-				duration == other.duration;
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public int hashCode() {
-		return (int) (this.stateSize ^ this.stateSize >>> 32) +
-			31 * ((int) (this.duration ^ this.duration >>> 32) +
-				31 * keyGroupState.hashCode());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/SubtaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/SubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/SubtaskState.java
deleted file mode 100644
index d42d146..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/SubtaskState.java
+++ /dev/null
@@ -1,108 +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.migration.runtime.checkpoint;
-
-import org.apache.flink.migration.runtime.state.StateHandle;
-import org.apache.flink.migration.util.SerializedValue;
-
-import java.io.Serializable;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class SubtaskState implements Serializable {
-
-	private static final long serialVersionUID = -2394696997971923995L;
-
-	/** The state of the parallel operator */
-	private final SerializedValue<StateHandle<?>> state;
-
-	/**
-	 * The state size. This is also part of the deserialized state handle.
-	 * We store it here in order to not deserialize the state handle when
-	 * gathering stats.
-	 */
-	private final long stateSize;
-
-	/** The duration of the acknowledged (ack timestamp - trigger timestamp). */
-	private final long duration;
-	
-	public SubtaskState(
-			SerializedValue<StateHandle<?>> state,
-			long stateSize,
-			long duration) {
-
-		this.state = checkNotNull(state, "State");
-		// Sanity check and don't fail checkpoint because of this.
-		this.stateSize = stateSize >= 0 ? stateSize : 0;
-
-		this.duration = duration;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	
-	public SerializedValue<StateHandle<?>> getState() {
-		return state;
-	}
-
-	public long getStateSize() {
-		return stateSize;
-	}
-
-	public long getDuration() {
-		return duration;
-	}
-
-	public void discard(ClassLoader userClassLoader) throws Exception {
-
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		else if (o instanceof SubtaskState) {
-			SubtaskState that = (SubtaskState) o;
-			return this.state.equals(that.state) && stateSize == that.stateSize &&
-				duration == that.duration;
-		}
-		else {
-			return false;
-		}
-	}
-
-	@Override
-	public int hashCode() {
-		return (int) (this.stateSize ^ this.stateSize >>> 32) +
-			31 * ((int) (this.duration ^ this.duration >>> 32) +
-				31 * state.hashCode());
-	}
-
-	@Override
-	public String toString() {
-		return String.format("SubtaskState(Size: %d, Duration: %d, State: %s)", stateSize, duration, state);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/TaskState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/TaskState.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/TaskState.java
deleted file mode 100644
index c0a7b2d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/TaskState.java
+++ /dev/null
@@ -1,164 +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.migration.runtime.checkpoint;
-
-import org.apache.flink.migration.runtime.state.StateHandle;
-import org.apache.flink.migration.util.SerializedValue;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-
-/**
- * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes.
- */
-@Deprecated
-@SuppressWarnings("deprecation")
-public class TaskState implements Serializable {
-
-	private static final long serialVersionUID = -4845578005863201810L;
-
-	private final JobVertexID jobVertexID;
-
-	/** Map of task states which can be accessed by their sub task index */
-	private final Map<Integer, SubtaskState> subtaskStates;
-
-	/** Map of key-value states which can be accessed by their key group index */
-	private final Map<Integer, KeyGroupState> kvStates;
-
-	/** Parallelism of the operator when it was checkpointed */
-	private final int parallelism;
-
-	public TaskState(JobVertexID jobVertexID, int parallelism) {
-		this.jobVertexID = jobVertexID;
-
-		this.subtaskStates = new HashMap<>(parallelism);
-
-		this.kvStates = new HashMap<>();
-
-		this.parallelism = parallelism;
-	}
-
-	public JobVertexID getJobVertexID() {
-		return jobVertexID;
-	}
-
-	public void putState(int subtaskIndex, SubtaskState subtaskState) {
-		if (subtaskIndex < 0 || subtaskIndex >= parallelism) {
-			throw new IndexOutOfBoundsException("The given sub task index " + subtaskIndex +
-				" exceeds the maximum number of sub tasks " + subtaskStates.size());
-		} else {
-			subtaskStates.put(subtaskIndex, subtaskState);
-		}
-	}
-
-	public SubtaskState getState(int subtaskIndex) {
-		if (subtaskIndex < 0 || subtaskIndex >= parallelism) {
-			throw new IndexOutOfBoundsException("The given sub task index " + subtaskIndex +
-				" exceeds the maximum number of sub tasks " + subtaskStates.size());
-		} else {
-			return subtaskStates.get(subtaskIndex);
-		}
-	}
-
-	public Collection<SubtaskState> getStates() {
-		return subtaskStates.values();
-	}
-
-	public Map<Integer, SubtaskState> getSubtaskStatesById() {
-		return subtaskStates;
-	}
-
-	public long getStateSize() {
-		long result = 0L;
-
-		for (SubtaskState subtaskState : subtaskStates.values()) {
-			result += subtaskState.getStateSize();
-		}
-
-		for (KeyGroupState keyGroupState : kvStates.values()) {
-			result += keyGroupState.getStateSize();
-		}
-
-		return result;
-	}
-
-	public int getNumberCollectedStates() {
-		return subtaskStates.size();
-	}
-
-	public int getParallelism() {
-		return parallelism;
-	}
-
-	public void putKvState(int keyGroupId, KeyGroupState keyGroupState) {
-		kvStates.put(keyGroupId, keyGroupState);
-	}
-
-	public KeyGroupState getKvState(int keyGroupId) {
-		return kvStates.get(keyGroupId);
-	}
-
-	/**
-	 * Retrieve the set of key-value state key groups specified by the given key group partition set.
-	 * The key groups are returned as a map where the key group index maps to the serialized state
-	 * handle of the key group.
-	 *
-	 * @param keyGroupPartition Set of key group indices
-	 * @return Map of serialized key group state handles indexed by their key group index.
-	 */
-	public Map<Integer, SerializedValue<StateHandle<?>>> getUnwrappedKvStates(Set<Integer> keyGroupPartition) {
-		HashMap<Integer, SerializedValue<StateHandle<?>>> result = new HashMap<>(keyGroupPartition.size());
-
-		for (Integer keyGroupId : keyGroupPartition) {
-			KeyGroupState keyGroupState = kvStates.get(keyGroupId);
-
-			if (keyGroupState != null) {
-				result.put(keyGroupId, kvStates.get(keyGroupId).getKeyGroupState());
-			}
-		}
-
-		return result;
-	}
-
-	public int getNumberCollectedKvStates() {
-		return kvStates.size();
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof TaskState) {
-			TaskState other = (TaskState) obj;
-
-			return jobVertexID.equals(other.jobVertexID) && parallelism == other.parallelism &&
-				subtaskStates.equals(other.subtaskStates) && kvStates.equals(other.kvStates);
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public int hashCode() {
-		return parallelism + 31 * Objects.hash(jobVertexID, subtaskStates, kvStates);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6642768a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0.java
deleted file mode 100644
index 7888d2f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/checkpoint/savepoint/SavepointV0.java
+++ /dev/null
@@ -1,115 +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.migration.runtime.checkpoint.savepoint;
-
-import org.apache.flink.migration.runtime.checkpoint.TaskState;
-import org.apache.flink.runtime.checkpoint.MasterState;
-import org.apache.flink.runtime.checkpoint.OperatorState;
-import org.apache.flink.runtime.checkpoint.savepoint.Savepoint;
-import org.apache.flink.util.Preconditions;
-
-import java.util.Collection;
-
-/**
- * Savepoint version 0.
- *
- * <p>This format was introduced with Flink 1.1.0.
- */
-@SuppressWarnings("deprecation")
-public class SavepointV0 implements Savepoint {
-
-	/** The savepoint version. */
-	public static final int VERSION = 0;
-
-	/** The checkpoint ID */
-	private final long checkpointId;
-
-	/** The task states */
-	private final Collection<TaskState> taskStates;
-
-	public SavepointV0(long checkpointId, Collection<TaskState> taskStates) {
-		this.checkpointId = checkpointId;
-		this.taskStates = Preconditions.checkNotNull(taskStates, "Task States");
-	}
-
-	@Override
-	public int getVersion() {
-		return VERSION;
-	}
-
-	@Override
-	public long getCheckpointId() {
-		return checkpointId;
-	}
-
-	@Override
-	public Collection<org.apache.flink.runtime.checkpoint.TaskState> getTaskStates() {
-		// since checkpoints are never deserialized into this format,
-		// this method should never be called
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public Collection<MasterState> getMasterStates() {
-		// since checkpoints are never deserialized into this format,
-		// this method should never be called
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public Collection<OperatorState> getOperatorStates() {
-		return null;
-	}
-
-	@Override
-	public void dispose() throws Exception {
-		//NOP
-	}
-
-
-	public Collection<TaskState> getOldTaskStates() {
-		return taskStates;
-	}
-
-	@Override
-	public String toString() {
-		return "Savepoint(version=" + VERSION + ")";
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		SavepointV0 that = (SavepointV0) o;
-		return checkpointId == that.checkpointId && getTaskStates().equals(that.getTaskStates());
-	}
-
-	@Override
-	public int hashCode() {
-		int result = (int) (checkpointId ^ (checkpointId >>> 32));
-		result = 31 * result + taskStates.hashCode();
-		return result;
-	}
-}