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;
- }
-}