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 3d3ea05cf3093..e5758e8920fdb 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.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 @@ @Deprecated public class RollingSink extends RichSinkFunction implements InputTypeConfigurable, CheckpointedFunction, - CheckpointListener, CheckpointedRestoring { + CheckpointListener { private static final long serialVersionUID = 1L; @@ -758,25 +757,6 @@ private void handleRestoredBucketState(BucketState bucketState) { } } - // -------------------------------------------------------------------------------------------- - // 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 // -------------------------------------------------------------------------------------------- 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 70168b55164bb..cc924a4a2056f 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.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 @@ */ public class BucketingSink extends RichSinkFunction - implements InputTypeConfigurable, CheckpointedFunction, CheckpointListener, - CheckpointedRestoring, ProcessingTimeCallback { + implements InputTypeConfigurable, CheckpointedFunction, CheckpointListener, ProcessingTimeCallback { private static final long serialVersionUID = 1L; @@ -872,25 +870,6 @@ private void handlePendingFilesForPreviousCheckpoints(Map> pe } } - // -------------------------------------------------------------------------------------------- - // 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 // -------------------------------------------------------------------------------------------- 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 e0413795b2aa7..0000000000000 --- 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 sink = new RollingSink(outDir.getAbsolutePath()) - .setWriter(new StringWriter()) - .setBatchSize(5) - .setPartPrefix(PART_PREFIX) - .setInProgressPrefix("") - .setPendingPrefix("") - .setValidLengthPrefix("") - .setInProgressSuffix(IN_PROGRESS_SUFFIX) - .setPendingSuffix(PENDING_SUFFIX) - .setValidLengthSuffix(VALID_LENGTH_SUFFIX); - - OneInputStreamOperatorTestHarness 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 sink = new ValidatingRollingSink(outDir.getAbsolutePath()) - .setWriter(new StringWriter()) - .setBatchSize(5) - .setPartPrefix(PART_PREFIX) - .setInProgressPrefix("") - .setPendingPrefix("") - .setValidLengthPrefix("") - .setInProgressSuffix(IN_PROGRESS_SUFFIX) - .setPendingSuffix(PENDING_SUFFIX) - .setValidLengthSuffix(VALID_LENGTH_SUFFIX); - - OneInputStreamOperatorTestHarness 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 extends RollingSink { - - 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 pendingFiles = state.pendingFiles; - Assert.assertTrue(pendingFiles.isEmpty()); - - final Map> pendingFilesPerCheckpoint = state.pendingFilesPerCheckpoint; - Assert.assertEquals(1, pendingFilesPerCheckpoint.size()); - - for (Map.Entry> entry: pendingFilesPerCheckpoint.entrySet()) { - long checkpoint = entry.getKey(); - List 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); - } - } -} 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 8a8dbd6bc9301..0000000000000 --- 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 sink = new ValidatingBucketingSink(outDir.getAbsolutePath()) - .setWriter(new StringWriter()) - .setBatchSize(5) - .setPartPrefix(PART_PREFIX) - .setInProgressPrefix("") - .setPendingPrefix("") - .setValidLengthPrefix("") - .setInProgressSuffix(IN_PROGRESS_SUFFIX) - .setPendingSuffix(PENDING_SUFFIX) - .setValidLengthSuffix(VALID_LENGTH_SUFFIX); - - OneInputStreamOperatorTestHarness 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 extends BucketingSink { - - 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 pendingFiles = state.pendingFiles; - Assert.assertTrue(pendingFiles.isEmpty()); - - final Map> pendingFilesPerCheckpoint = state.pendingFilesPerCheckpoint; - Assert.assertEquals(1, pendingFilesPerCheckpoint.size()); - - for (Map.Entry> entry: pendingFilesPerCheckpoint.entrySet()) { - long checkpoint = entry.getKey(); - List 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); - } - } -} 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 f3c9e5e342044..3088b1552a65d 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.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 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFunction implements CheckpointListener, ResultTypeQueryable, - CheckpointedFunction, - CheckpointedRestoring> { + CheckpointedFunction { private static final long serialVersionUID = -6272159445203409112L; @@ -766,22 +764,6 @@ public final void snapshotState(FunctionSnapshotContext context) throws Exceptio } } - @Override - public final void restoreState(HashMap 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) { 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 e3f337ec671dd..84f0e388e1969 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 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 void testRestore() throws Exception { */ @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 partitions = new ArrayList<>(PARTITION_STATE.keySet()); 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 d127f2b6fb710..5689229f0604a 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.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 @@ */ public class FlinkKinesisConsumer extends RichParallelSourceFunction implements ResultTypeQueryable, - CheckpointedFunction, - CheckpointedRestoring> { + CheckpointedFunction { private static final long serialVersionUID = 4724006128720664870L; @@ -352,7 +349,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { 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 entry : lastStateSnapshot.entrySet()) { @@ -362,23 +359,6 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } } - @Override - public void restoreState(HashMap 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 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 createFetcher( List streams, 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 af84420e6e3eb..0000000000000 --- 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 consumerFunction = new DummyFlinkKafkaConsumer<>(testConfig); - - StreamSource> consumerOperator = new StreamSource<>(consumerFunction); - - final AbstractStreamOperatorTestHarness 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 consumerFunction = new DummyFlinkKafkaConsumer<>(testConfig); - - StreamSource> consumerOperator = - new StreamSource<>(consumerFunction); - - final AbstractStreamOperatorTestHarness 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 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 extends FlinkKinesisConsumer { - private static final long serialVersionUID = 1L; - - @SuppressWarnings("unchecked") - DummyFlinkKafkaConsumer(Properties properties) { - super("test", mock(KinesisDeserializationSchema.class), properties); - } - - @Override - protected KinesisDataFetcher createFetcher( - List streams, - SourceFunction.SourceContext sourceContext, - RuntimeContext runtimeContext, - Properties configProps, - KinesisDeserializationSchema deserializationSchema) { - return mock(KinesisDataFetcher.class); - } - } -} 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 a26e758e72687..69d30cd220f64 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 @@ -710,38 +710,6 @@ public void testFetcherShouldNotBeRestoringFromFailureIfNotRestoringFromCheckpoi consumer.run(Mockito.mock(SourceFunction.SourceContext.class)); } - @Test - @SuppressWarnings("unchecked") - public void testFetcherShouldBeCorrectlySeededIfRestoringFromLegacyCheckpoint() throws Exception { - HashMap fakeRestoredState = getFakeRestoredStore("all"); - HashMap legacyFakeRestoredState = new HashMap<>(); - for (Map.Entry kv : fakeRestoredState.entrySet()) { - legacyFakeRestoredState.put(new KinesisStreamShard(kv.getKey().getStreamName(), kv.getKey().getShard()), kv.getValue()); - } - - KinesisDataFetcher mockedFetcher = Mockito.mock(KinesisDataFetcher.class); - List 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 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 { 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 75356efca8428..1336652226f13 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.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.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 void run() { return new Thread(exactlyOnceValidationConsumer); } - private static class ExactlyOnceValidatingMapper implements FlatMapFunction, Checkpointed { + private static class ExactlyOnceValidatingMapper implements FlatMapFunction, ListCheckpointed { private static final Logger LOG = LoggerFactory.getLogger(ExactlyOnceValidatingMapper.class); @@ -126,13 +129,18 @@ public void flatMap(String value, Collector out) throws Exception { } @Override - public BitSet snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { - return validator; + public List 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 state) throws Exception { + // we expect either 1 or 0 elements + if (state.size() == 1) { + validator = state.get(0); + } else { + Preconditions.checkState(state.isEmpty()); + } } } 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 b7f386d47b7e7..dd5b8522ea23e 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.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.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.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.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.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.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 extends AbstractKeyedStateBackend { /** 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 extends AbstractKeyedStateBackend { /** 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 int getKeyGroupPrefixBytes() { 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 @@ private RunnableFuture snapshotIncrementally( 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 @@ private RunnableFuture snapshotFully( 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 @@ private StreamStateHandle materializeMetaData() throws Exception { 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>> stateMetaInfoEntry : stateBackend.kvStateInformation.entrySet()) { @@ -929,7 +922,7 @@ KeyedStateHandle materializeSnapshot() throws Exception { 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 @@ void releaseResources(boolean canceled) { } if (canceled) { - List statesToDiscard = new ArrayList<>(); + Collection statesToDiscard = + new ArrayList<>(1 + miscFiles.size() + sstFiles.size()); statesToDiscard.add(metaStateHandle); statesToDiscard.addAll(miscFiles.values()); @@ -1012,9 +1006,6 @@ public void restore(Collection restoreState) throws Exception 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 restoreOperation = new RocksDBIncrementalRestoreOperation<>(this); restoreOperation.restore(restoreState); @@ -1035,14 +1026,7 @@ public void notifyCheckpointComplete(long completedCheckpointId) { return; } - Iterator 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 @@ private RocksDB openDB( // 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 @@ private RocksDB openDB( 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 RocksDBFullRestoreOperation(RocksDBKeyedStateBackend rocksDBKeyedState * 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 keyedStateHandles) - throws IOException, StateMigrationException, ClassNotFoundException, RocksDBException { + throws IOException, StateMigrationException, RocksDBException { rocksDBKeyedStateBackend.createDB(); @@ -1142,13 +1123,9 @@ public void doRestore(Collection keyedStateHandles) /** * 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 @@ private void restoreKVStateData() throws IOException, RocksDBException { //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 @@ private void createFileHardLinksInRestorePath( void restore(Collection 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 @@ protected ColumnFamilyHandle getColumnFamily( (RegisteredKeyedBackendStateMetaInfo.Snapshot) 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 @@ protected ColumnFamilyHandle getColumnFamily( // check compatibility results to determine if state migration is required CompatibilityResult namespaceCompatibility = CompatibilityUtil.resolveCompatibilityResult( restoredMetaInfo.getNamespaceSerializer(), - MigrationNamespaceSerializerProxy.class, + null, restoredMetaInfo.getNamespaceSerializerConfigSnapshot(), newMetaInfo.getNamespaceSerializer()); @@ -1639,12 +1616,12 @@ protected ColumnFamilyHandle getColumnFamily( 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 @@ protected InternalFoldingState createFoldingState( } @Override - protected InternalMapState createMapState(TypeSerializer namespaceSerializer, - MapStateDescriptor stateDesc) throws Exception { + protected InternalMapState createMapState( + TypeSerializer namespaceSerializer, + MapStateDescriptor stateDesc) throws Exception { + ColumnFamilyHandle columnFamily = getColumnFamily(stateDesc, namespaceSerializer); return new RocksDBMapState<>(columnFamily, namespaceSerializer, stateDesc, this); @@ -1963,105 +1942,6 @@ public File getInstanceBasePath() { return instanceBasePath; } - /** - * For backwards compatibility, remove again later! - * - * @deprecated Internal method used for backwards compatibility. - */ - @Deprecated - private void restoreOldSavepointKeyedState(Collection 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 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> 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; 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 024d12ea1be97..0000000000000 --- 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, ValueStateDescriptor> { - private static final long serialVersionUID = 1L; - - public final StateHandle stateHandle; - final long checkpointId; - - /** - * Creates a new snapshot from the given state parameters. - */ - private FinalFullyAsyncSnapshot(StateHandle 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. - * - *

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."); - } - } -} 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 c752e53ccd17c..98208fd6a75a6 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.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.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 void close() { private static class AsyncCheckpointOperator extends AbstractStreamOperator - implements OneInputStreamOperator, StreamCheckpointedOperator { + implements OneInputStreamOperator { @Override public void open() throws Exception { @@ -477,17 +474,5 @@ public void processElement(StreamRecord element) throws Exception { 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 - } - } } 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 69e4e6daa668e..0000000000000 --- 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 deserializeObject(byte[] bytes, ClassLoader cl) throws IOException, ClassNotFoundException { - return deserializeObject(new ByteArrayInputStream(bytes), cl); - } - - @SuppressWarnings("unchecked") - public static 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(); - } - -} 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 6fa29d3c554c7..0000000000000 --- 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 The type of the value held. - * @deprecated Only used internally when migrating from previous savepoint versions. - */ -@Deprecated -@PublicEvolving -public class SerializedValue 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 null if no value is set. - * - * @return Serialized data. - */ - public byte[] getByteArray() { - return serializedData; - } - - public static SerializedValue fromBytes(byte[] serializedData) { - return new SerializedValue(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"; - } -} 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 78c57edabf953..602ad3e1bb0ff 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> parameters () { return Arrays.asList( - Tuple2.of(MigrationVersion.v1_1, 1482144479339L), Tuple2.of(MigrationVersion.v1_2, 1493116191000L), Tuple2.of(MigrationVersion.v1_3, 1496532000000L)); } 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 11f14b9e83329..78ac39cac2ebd 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.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.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 @@ Map> extractCurrentMatches(final ComputationState computation 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 @@ private void readObject(ObjectInputStream ois) throws IOException, ClassNotFound final List> readComputationStates = new ArrayList<>(numberComputationStates); - boolean afterMigration = false; for (int i = 0; i < numberComputationStates; i++) { ComputationState 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> migrateNFA(Collection> readStates) { - final ArrayList> computationStates = new ArrayList<>(); - - final State startState = Iterators.find( - readStates.iterator(), - new Predicate>() { - @Override - public boolean apply(@Nullable ComputationState input) { - return input != null && input.getState().getName().equals(BEGINNING_STATE_NAME); - } - }).getState(); - - final Map> convertedStates = NFACompiler.migrateGraph(startState); - - for (ComputationState readState : readStates) { - if (!readState.isStartState()) { - final String previousName = readState.getState().getName(); - final String currentName = Iterators.find( - readState.getState().getStateTransitions().iterator(), - new Predicate>() { - @Override - public boolean apply(@Nullable StateTransition input) { - return input != null && input.getAction() == StateTransitionAction.TAKE; - } - }).getTargetState().getName(); - - final State 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>() { - @Override - public boolean apply(@Nullable State 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 readComputationState(ObjectInputStream ois) throws IOException, ClassNotFoundException { final State state = (State) ois.readObject(); 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 c6f69b9f10995..c36e7df866ce1 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 @@ private SharedBuffer( this.pages = pages; } - /** - * For backward compatibility only. Previously the key in {@link SharedBuffer} was {@link State}. - * Now it is {@link String}. - */ - @Internal - static SharedBuffer migrateSharedBuffer(SharedBuffer, T> buffer) { - - final Map> pageMap = new HashMap<>(); - final Map, T>, SharedBufferEntry> entries = new HashMap<>(); - - for (Map.Entry, SharedBufferPage, T>> page : buffer.pages.entrySet()) { - final SharedBufferPage newPage = new SharedBufferPage<>(page.getKey().getName()); - pageMap.put(newPage.getKey(), newPage); - - for (Map.Entry, SharedBufferEntry, T>> pageEntry : page.getValue().entries.entrySet()) { - final SharedBufferEntry 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, SharedBufferPage, T>> page : buffer.pages.entrySet()) { - for (Map.Entry, SharedBufferEntry, T>> pageEntry : page.getValue().entries.entrySet()) { - final SharedBufferEntry newEntry = entries.get(pageEntry.getValue()); - for (SharedBufferEdge, T> edge : pageEntry.getValue().edges) { - final SharedBufferEntry targetNewEntry = entries.get(edge.getTarget()); - - final SharedBufferEdge newEdge = new SharedBufferEdge<>( - targetNewEntry, - edge.getVersion()); - newEntry.edges.add(newEdge); - } - } - } - - return new SharedBuffer<>(buffer.valueSerializer, pageMap); - } - private SharedBufferEntry get( final K key, final V value, @@ -1177,76 +1135,4 @@ public CompatibilityResult> ensureCompatibility(TypeSerialize return CompatibilityResult.requiresMigration(); } } - - ////////////////// Java Serialization methods for backwards compatibility ////////////////// - - private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { - DataInputViewStreamWrapper source = new DataInputViewStreamWrapper(ois); - ArrayList> 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 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 valueTimeWrapper = new ValueTimeWrapper<>(value, timestamp, 0); - SharedBufferEntry sharedBufferEntry = new SharedBufferEntry(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 sourceEntry = entryList.get(sourceIndex); - - final DeweyNumber version = (DeweyNumber) ois.readObject(); - final SharedBufferEntry 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(target, version)); - } - } - } } 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 593c94f41e434..5698de62d8b8d 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.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; @@ -857,114 +851,6 @@ private void updateWithGreedyCondition( } } - /** - * 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 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 Map> migrateGraph(State oldStartState) { - State oldFirst = oldStartState; - State oldSecond = oldStartState.getStateTransitions().iterator().next().getTargetState(); - - StateTransition oldFirstToSecondTake = Iterators.find( - oldFirst.getStateTransitions().iterator(), - new Predicate>() { - @Override - public boolean apply(@Nullable StateTransition input) { - return input != null && input.getAction() == StateTransitionAction.TAKE; - } - - }); - - StateTransition oldFirstIgnore = Iterators.find( - oldFirst.getStateTransitions().iterator(), - new Predicate>() { - @Override - public boolean apply(@Nullable StateTransition input) { - return input != null && input.getAction() == StateTransitionAction.IGNORE; - } - - }, null); - - StateTransition oldSecondToThirdTake = Iterators.find( - oldSecond.getStateTransitions().iterator(), - new Predicate>() { - @Override - public boolean apply(@Nullable StateTransition input) { - return input != null && input.getAction() == StateTransitionAction.TAKE; - } - - }, null); - - final Map> convertedStates = new HashMap<>(); - State newSecond; - State newFirst = new State<>(oldSecond.getName(), State.StateType.Start); - convertedStates.put(newFirst.getName(), newFirst); - while (oldSecondToThirdTake != null) { - - newSecond = new State(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>() { - @Override - public boolean apply(@Nullable StateTransition input) { - return input != null && input.getAction() == StateTransitionAction.TAKE; - } - - }); - - oldFirstIgnore = Iterators.find( - oldFirst.getStateTransitions().iterator(), - new Predicate>() { - @Override - public boolean apply(@Nullable StateTransition input) { - return input != null && input.getAction() == StateTransitionAction.IGNORE; - } - - }, null); - - oldSecond = oldSecondToThirdTake.getTargetState(); - - oldSecondToThirdTake = Iterators.find( - oldSecond.getStateTransitions().iterator(), - new Predicate>() { - @Override - public boolean apply(@Nullable StateTransition input) { - return input != null && input.getAction() == StateTransitionAction.TAKE; - } - - }, null); - - newFirst = newSecond; - } - - final State 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}. * 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 7556d9f22bd83..257d3e70e4569 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.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 @@ * @param Type of the input elements * @param Type of the key on which the input stream is keyed * @param Type of the output elements - * @param user function that can be applied to matching sequences or timed out sequences */ public abstract class AbstractKeyedCEPPatternOperator extends AbstractUdfStreamOperator - implements OneInputStreamOperator, Triggerable, CheckpointedRestoringOperator { + implements OneInputStreamOperator, Triggerable { private static final long serialVersionUID = -4166778210774160757L; - private static final int INITIAL_PRIORITY_QUEUE_CAPACITY = 11; - private final boolean isProcessingTime; private final TypeSerializer inputSerializer; - // necessary to serialize the set of seen keys - private final TypeSerializer keySerializer; - /////////////// State ////////////// private static final String NFA_OPERATOR_STATE_NAME = "nfaOperatorStateName"; @@ -115,12 +90,6 @@ public abstract class AbstractKeyedCEPPatternOperator comparator; public AbstractKeyedCEPPatternOperator( @@ -135,10 +104,7 @@ public AbstractKeyedCEPPatternOperator( 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 @@ protected void processTimedOutSequences( 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> oldNfaOperatorState = getRuntimeContext().getState( - new ValueStateDescriptor<>("nfaOperatorState", new NFA.Serializer())); - - ValueState>> oldPriorityQueueOperatorState = - getRuntimeContext().getState( - new ValueStateDescriptor<>( - "priorityQueueStateName", - new PriorityQueueSerializer<>( - ((TypeSerializer) new StreamElementSerializer<>(inputSerializer)), - new PriorityQueueStreamRecordFactory() - ) - ) - ); - - if (migratingFromOldKeyedOperator) { - int numberEntries = inputView.readInt(); - for (int i = 0; i < numberEntries; i++) { - KEY key = keySerializer.deserialize(inputView); - setCurrentKey(key); - saveRegisterWatermarkTimer(); - - NFA nfa = oldNfaOperatorState.value(); - oldNfaOperatorState.clear(); - nfaOperatorState.update(nfa); - - PriorityQueue> priorityQueue = oldPriorityQueueOperatorState.value(); - if (priorityQueue != null && !priorityQueue.isEmpty()) { - Map> elementMap = new HashMap<>(); - for (StreamRecord record: priorityQueue) { - long timestamp = record.getTimestamp(); - IN element = record.getValue(); - - List 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> 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 nfa = (NFA) ois.readObject(); - - // retrieve the elements that were pending in the priority queue - MultiplexingStreamRecordSerializer recordSerializer = new MultiplexingStreamRecordSerializer<>(inputSerializer); - - Map> elementMap = new HashMap<>(); - int entries = ois.readInt(); - for (int i = 0; i < entries; i++) { - StreamElement streamElement = recordSerializer.deserialize(inputView); - StreamRecord record = streamElement.asRecord(); - - long timestamp = record.getTimestamp(); - IN element = record.getValue(); - - List 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> 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 Type of the priority queue's elements - */ - private static class PriorityQueueSerializer extends TypeSerializer> { - - private static final long serialVersionUID = -231980397616187715L; - - private final TypeSerializer elementSerializer; - private final PriorityQueueFactory factory; - - PriorityQueueSerializer(final TypeSerializer elementSerializer, final PriorityQueueFactory factory) { - this.elementSerializer = elementSerializer; - this.factory = factory; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public TypeSerializer> duplicate() { - return new PriorityQueueSerializer<>(elementSerializer.duplicate(), factory); - } - - @Override - public PriorityQueue createInstance() { - return factory.createPriorityQueue(); - } - - @Override - public PriorityQueue copy(PriorityQueue from) { - PriorityQueue result = factory.createPriorityQueue(); - - for (T element: from) { - result.offer(elementSerializer.copy(element)); - } - - return result; - } - - @Override - public PriorityQueue copy(PriorityQueue from, PriorityQueue 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 record, DataOutputView target) throws IOException { - target.writeInt(record.size()); - - for (T element: record) { - elementSerializer.serialize(element, target); - } - } - - @Override - public PriorityQueue deserialize(DataInputView source) throws IOException { - PriorityQueue result = factory.createPriorityQueue(); - - return deserialize(result, source); - } - - @Override - public PriorityQueue deserialize(PriorityQueue 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 other = (PriorityQueueSerializer) 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> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { - if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { - Tuple2, TypeSerializerConfigSnapshot> previousElemSerializerAndConfig = - ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); - - CompatibilityResult 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 extends Serializable { - PriorityQueue createPriorityQueue(); - } - - private static class PriorityQueueStreamRecordFactory implements PriorityQueueFactory> { - - private static final long serialVersionUID = 1254766984454616593L; - - @Override - public PriorityQueue> createPriorityQueue() { - return new PriorityQueue>(INITIAL_PRIORITY_QUEUE_CAPACITY, new StreamRecordComparator()); - } - - @Override - public boolean equals(Object obj) { - return obj instanceof PriorityQueueStreamRecordFactory; - } - - @Override - public int hashCode() { - return getClass().hashCode(); - } - } - ////////////////////// Testing Methods ////////////////////// @VisibleForTesting 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 843d6682d10d9..0000000000000 --- 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> 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(startEvent, 1)); - harness.processElement(new StreamRecord(new Event(42, "foobar", 1.0), 2)); - harness.processElement(new StreamRecord(new SubEvent(42, "barfoo", 1.0, 5.0), 3)); - harness.processWatermark(new Watermark(2)); - - harness.processElement(new StreamRecord(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>> 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 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> patternMap = - (Map>) 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(startEvent1, 21)); - harness.processElement(new StreamRecord(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> patternMap1 = - (Map>) 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> harness = new OneInputStreamOperatorTestHarness<>( - new CEPPatternOperator<>( - Event.createTypeSerializer(), - false, - new NFAFactory())); - harness.open(); - harness.processElement(new StreamRecord(startEvent, 1)); - harness.processElement(new StreamRecord(new Event(42, "foobar", 1.0), 2)); - harness.processElement(new StreamRecord(new SubEvent(42, "barfoo", 1.0, 5.0), 3)); - harness.processWatermark(new Watermark(2)); - - harness.processElement(new StreamRecord(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>> harness = - new KeyedOneInputStreamOperatorTestHarness>>( - 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 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> patternMap = - (Map>) 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(startEvent1, 21)); - harness.processElement(new StreamRecord(middleEvent1, 23)); - - // simulate snapshot/restore with some elements in internal sorting queue - OperatorStateHandles snapshot = harness.snapshot(1L, 1L); - harness.close(); - - harness = new KeyedOneInputStreamOperatorTestHarness>>( - 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> patternMap1 = - (Map>) 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 { - - private static final long serialVersionUID = 1173020762472766713L; - - private final boolean handleTimeout; - - private NFAFactory() { - this(false); - } - - private NFAFactory(boolean handleTimeout) { - this.handleTimeout = handleTimeout; - } - - @Override - public NFA createNFA() { - - Pattern pattern = Pattern.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 { - 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 { - 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 { - private static final long serialVersionUID = 7056763917392056548L; - - @Override - public boolean filter(Event value) throws Exception { - return value.getName().equals("end"); - } - } -} 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 cf3c9215fbe6b..ed28f254a43f6 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 parameters () { - return Arrays.asList(MigrationVersion.v1_2, MigrationVersion.v1_3); + return Arrays.asList(MigrationVersion.v1_3); } public CEPMigrationTest(MigrationVersion migrateVersion) { 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 c4e23ca27ffe4..0000000000000 --- 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 { - - public static final MigrationNamespaceSerializerProxy INSTANCE = new MigrationNamespaceSerializerProxy(); - - private static final long serialVersionUID = -707800010807094491L; - - private MigrationNamespaceSerializerProxy() { - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public TypeSerializer 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 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; - } -} 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 a6055a83ac64f..0000000000000 --- 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 keyedStateHandles) { - return (keyedStateHandles != null) - && (keyedStateHandles.size() == 1) - && (keyedStateHandles.iterator().next() instanceof MigrationKeyGroupStateHandle); - } - -} 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 5196d2dad6b96..0000000000000 --- 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 extends StateDescriptor, T> { - private static final long serialVersionUID = 1L; - - /** - * Creates a new {@code ListStateDescriptor} with the given name and list element type. - * - *

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 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 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 typeSerializer) { - super(name, typeSerializer, null); - } - - // ------------------------------------------------------------------------ - - @Override - public ListState 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; - } -} 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 0b25e0855a563..0000000000000 --- 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> keyGroupState; - - private final long stateSize; - - private final long duration; - - public KeyGroupState(SerializedValue> keyGroupState, long stateSize, long duration) { - this.keyGroupState = keyGroupState; - - this.stateSize = stateSize; - - this.duration = duration; - } - - public SerializedValue> 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()); - } -} 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 d42d1467c705c..0000000000000 --- 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> 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> 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> 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); - } -} 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 c0a7b2d39733b..0000000000000 --- 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 subtaskStates; - - /** Map of key-value states which can be accessed by their key group index */ - private final Map 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 getStates() { - return subtaskStates.values(); - } - - public Map 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>> getUnwrappedKvStates(Set keyGroupPartition) { - HashMap>> 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); - } -} 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 7888d2fe6774d..0000000000000 --- 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. - * - *

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 taskStates; - - public SavepointV0(long checkpointId, Collection 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 getTaskStates() { - // since checkpoints are never deserialized into this format, - // this method should never be called - throw new UnsupportedOperationException(); - } - - @Override - public Collection getMasterStates() { - // since checkpoints are never deserialized into this format, - // this method should never be called - throw new UnsupportedOperationException(); - } - - @Override - public Collection getOperatorStates() { - return null; - } - - @Override - public void dispose() throws Exception { - //NOP - } - - - public Collection 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; - } -} 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 d285906262565..0000000000000 --- 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; - -/** - *

- *

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 { - - 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 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> 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> 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> readSerializedValueStateHandle(DataInputStream dis) - throws IOException { - - int length = dis.readInt(); - - SerializedValue> 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 taskStates, - ClassLoader userClassLoader, - long checkpointID) throws Exception { - - List 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 subtaskStates = taskState.getSubtaskStatesById(); - - for (Map.Entry 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> serializedValue = subtaskState.getState(); - - StreamTaskStateList stateList = (StreamTaskStateList) serializedValue.deserializeValue(userClassLoader); - StreamTaskState[] streamTaskStates = stateList.getState(userClassLoader); - - List 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> oldKeyedState = streamTaskState.getKvStates(); - - if (null != oldKeyedState) { - Preconditions.checkState(null == newKeyedState, "Found more than one keyed state in chain"); - newKeyedState = convertKeyedBackendState(oldKeyedState, parallelInstanceIdx, checkpointID); - } - } - - ChainedStateHandle newChainedState = new ChainedStateHandle<>(newChainStateList); - ChainedStateHandle 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 mergeStateHandles = new ArrayList<>(4); - - StateHandle 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> 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 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 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()); - } - } - } - } -} 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 775b304748007..0000000000000 --- 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 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. - * - *

If a "Closeable" has been registered via {@link #registerCloseable(Closeable)}, - * then this will be closes. - * - *

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"); - } - } -} 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 7c53c406270c2..0000000000000 --- 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 { - - 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(); - } - } -} 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 687d41536a82e..0000000000000 --- 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> - extends StateObject { - -} 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 fd3917f2b735f..0000000000000 --- 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 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; -} 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 59bc0ca948e16..0000000000000 --- 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}. - * - *

State objects define how to: - *

    - *
  • Discard State: The {@link #discardState()} method defines how state is permanently - * disposed/deleted. After that method call, state may not be recoverable any more.
  • - - *
  • Close the current state access: 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.
  • - *
- * - * @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. - * - *

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; -} 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 bfc57bca97921..0000000000000 --- 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 { - - /** - * 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. - */ - StateHandle toSerializableHandle(); -} 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 a522a95b2c552..0000000000000 --- 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(); - } -} 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 7099c617df905..0000000000000 --- 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 The type of the key in the snapshot state. - * @param The type of the namespace in the snapshot state. - * @param 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> - extends AbstractFileStateHandle implements KvStateSnapshot, MigrationRestoreSnapshot { - - private static final long serialVersionUID = 1L; - - /** Key Serializer */ - protected final TypeSerializer keySerializer; - - /** Namespace Serializer */ - protected final TypeSerializer namespaceSerializer; - - /** Serializer for the state value */ - protected final TypeSerializer stateSerializer; - - /** StateDescriptor, for sanity checks */ - protected final SD stateDesc; - - public AbstractFsStateSnapshot(TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer 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 getKeySerializer() { - return keySerializer; - } - - public TypeSerializer getNamespaceSerializer() { - return namespaceSerializer; - } - - public TypeSerializer getStateSerializer() { - return stateSerializer; - } - - public SD getStateDesc() { - return stateDesc; - } - - @Override - @SuppressWarnings("unchecked") - public StateTable deserialize( - String stateName, - HeapKeyedStateBackend stateBackend) throws IOException { - - final FileSystem fs = getFilePath().getFileSystem(); - try (FSDataInputStream inStream = fs.open(getFilePath())) { - final DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(inStream); - AbstractMigrationRestoreStrategy restoreStrategy = - new AbstractMigrationRestoreStrategy(keySerializer, namespaceSerializer, stateSerializer) { - @Override - protected DataInputView openDataInputView() throws IOException { - return inView; - } - }; - return restoreStrategy.deserialize(stateName, stateBackend); - } - } -} 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 b4a3a730097d1..0000000000000 --- 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 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 extends AbstractFileStateHandle implements StateHandle { - - 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(); - } -} 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 7444be1c9e754..0000000000000 --- 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 StateHandle toSerializableHandle() { - FileSerializableStateHandle 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; - } -} 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 ec89ab8931d13..0000000000000 --- 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 { - public static class Snapshot extends AbstractFsStateSnapshot, FoldingStateDescriptor> { - private static final long serialVersionUID = 1L; - - public Snapshot(TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer stateSerializer, - FoldingStateDescriptor stateDescs, - Path filePath) { - super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, filePath); - } - } - -} 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 71404abee9014..0000000000000 --- 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 { - - public static class Snapshot extends AbstractFsStateSnapshot, ListState, ListStateDescriptor> { - private static final long serialVersionUID = 1L; - - public Snapshot(TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer> stateSerializer, - ListStateDescriptor stateDescs, - Path filePath) { - super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, filePath); - } - } -} 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 153f88c53921c..0000000000000 --- 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 { - - public static class Snapshot extends AbstractFsStateSnapshot, ReducingStateDescriptor> { - private static final long serialVersionUID = 1L; - - public Snapshot(TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer stateSerializer, - ReducingStateDescriptor stateDescs, - Path filePath) { - super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, filePath); - } - } -} 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 d17751028db9d..0000000000000 --- 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; -} 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 d2ae48d23c40c..0000000000000 --- 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 { - - public static class Snapshot extends AbstractFsStateSnapshot, ValueStateDescriptor> { - private static final long serialVersionUID = 1L; - - public Snapshot(TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer stateSerializer, - ValueStateDescriptor stateDescs, - Path filePath) { - super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, filePath); - } - } -} 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 aadfe4eb0c0b9..0000000000000 --- 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> - implements KvStateSnapshot, MigrationRestoreSnapshot { - - private static final long serialVersionUID = 1L; - - /** Key Serializer */ - protected final TypeSerializer keySerializer; - - /** Namespace Serializer */ - protected final TypeSerializer namespaceSerializer; - - /** Serializer for the state value */ - protected final TypeSerializer 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 keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer 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 deserialize( - String stateName, - HeapKeyedStateBackend stateBackend) throws IOException { - - final DataInputDeserializer inView = new DataInputDeserializer(data, 0, data.length); - AbstractMigrationRestoreStrategy restoreStrategy = - new AbstractMigrationRestoreStrategy(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 getKeySerializer() { - return keySerializer; - } - - public TypeSerializer getNamespaceSerializer() { - return namespaceSerializer; - } - - public TypeSerializer getStateSerializer() { - return stateSerializer; - } - - public byte[] getData() { - return data; - } - - @Override - public String toString() { - return "AbstractMemStateSnapshot{" + - "keySerializer=" + keySerializer + - ", namespaceSerializer=" + namespaceSerializer + - ", stateSerializer=" + stateSerializer + - ", stateDesc=" + stateDesc + - '}'; - } -} 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 f58070e564b92..0000000000000 --- 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 type of key. - * @param type of namespace. - * @param type of state. - * - * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes. - */ -@Deprecated -public abstract class AbstractMigrationRestoreStrategy implements MigrationRestoreSnapshot { - - /** - * Key Serializer - */ - protected final TypeSerializer keySerializer; - - /** - * Namespace Serializer - */ - protected final TypeSerializer namespaceSerializer; - - /** - * Serializer for the state value - */ - protected final TypeSerializer stateSerializer; - - public AbstractMigrationRestoreStrategy( - TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer stateSerializer) { - - this.keySerializer = Preconditions.checkNotNull(keySerializer); - this.namespaceSerializer = Preconditions.checkNotNull(namespaceSerializer); - this.stateSerializer = Preconditions.checkNotNull(stateSerializer); - } - - @Override - public StateTable deserialize(String stateName, HeapKeyedStateBackend 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 patchedNamespaceSerializer = this.namespaceSerializer; - - if (patchedNamespaceSerializer instanceof VoidSerializer) { - patchedNamespaceSerializer = (TypeSerializer) VoidNamespaceSerializer.INSTANCE; - } - - RegisteredKeyedBackendStateMetaInfo registeredKeyedBackendStateMetaInfo = - new RegisteredKeyedBackendStateMetaInfo<>( - StateDescriptor.Type.UNKNOWN, - stateName, - patchedNamespaceSerializer, - stateSerializer); - - final StateTable 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; -} 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 c7fbab63659b0..0000000000000 --- 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 StateHandle toSerializableHandle() { - SerializedStateHandle serializableHandle = new SerializedStateHandle(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; - } -} 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 ad820e4d9234b..0000000000000 --- 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 { - - public static class Snapshot extends AbstractMemStateSnapshot, FoldingStateDescriptor> { - private static final long serialVersionUID = 1L; - - public Snapshot(TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer stateSerializer, - FoldingStateDescriptor stateDescs, byte[] data) { - super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, data); - } - } -} 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 d76cda09f9639..0000000000000 --- 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 { - - public static class Snapshot extends AbstractMemStateSnapshot, ListState, ListStateDescriptor> { - private static final long serialVersionUID = 1L; - - public Snapshot(TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer> stateSerializer, - ListStateDescriptor stateDescs, byte[] data) { - super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, data); - } - } - -} 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 c39111c5e5767..0000000000000 --- 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 The type of the key. - * @param The type of the namespace. - * @param 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 { - - public static class Snapshot extends AbstractMemStateSnapshot, ReducingStateDescriptor> { - private static final long serialVersionUID = 1L; - - public Snapshot(TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer stateSerializer, - ReducingStateDescriptor stateDescs, byte[] data) { - super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, data); - } - }} 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 940d4895569d7..0000000000000 --- 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 The type of the key. - * @param The type of the namespace. - * @param The type of the value. - * - * @deprecated Internal class for savepoint backwards compatibility. Don't use for other purposes. - */ -@Deprecated -@SuppressWarnings("deprecation") -public class MemValueState { - - public static class Snapshot extends AbstractMemStateSnapshot, ValueStateDescriptor> { - private static final long serialVersionUID = 1L; - - public Snapshot(TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - TypeSerializer stateSerializer, - ValueStateDescriptor stateDescs, byte[] data) { - super(keySerializer, namespaceSerializer, stateSerializer, stateDescs, data); - } - } -} 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 4277b56fc5be1..0000000000000 --- 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 extends Migration { - StateTable deserialize(String stateName, HeapKeyedStateBackend stateBackend) throws IOException; -} 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 49d772e3019e1..0000000000000 --- 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 The type of state represented by this state handle. - */ -@SuppressWarnings("deprecation") -public class SerializedStateHandle extends AbstractCloseableHandle implements StateHandle { - - 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.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; - } -} 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 3f1ff552d9c58..0000000000000 --- 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); - } -} 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 220191605202c..0000000000000 --- 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); - } - } -} 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 b044ffbb3d753..0000000000000 --- 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 functionState; - - private HashMap> kvStates; - - // ------------------------------------------------------------------------ - - public StateHandle getOperatorState() { - return operatorState; - } - - public void setOperatorState(StateHandle operatorState) { - this.operatorState = operatorState; - } - - public StateHandle getFunctionState() { - return functionState; - } - - public void setFunctionState(StateHandle functionState) { - this.functionState = functionState; - } - - public HashMap> getKvStates() { - return kvStates; - } - - public void setKvStates(HashMap> 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 { - - } -} 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 7643039c1f852..0000000000000 --- 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 { - - 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> 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 { - } -} 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 145ff6a978931..a5f908d7aa964 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.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 int getMaxParallelism() { 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()) { 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 296b5ab29dd32..3df9c4fc5248d 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; @@ -62,16 +59,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}. */ @@ -103,39 +90,30 @@ public class OperatorSubtaskState implements CompositeStateHandle { */ private final long stateSize; - @VisibleForTesting - public OperatorSubtaskState(StreamStateHandle legacyOperatorState) { - - this(legacyOperatorState, - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList()); - } - /** * Empty state. */ public OperatorSubtaskState() { - this(null); + this( + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList()); } public OperatorSubtaskState( - StreamStateHandle legacyOperatorState, Collection managedOperatorState, Collection rawOperatorState, Collection managedKeyedState, Collection 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 OperatorSubtaskState( * 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), @@ -182,16 +159,6 @@ private static long getSizeNullSafe(StateObject stateObject) throws Exception { // -------------------------------------------------------------------------------------------- - /** - * @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. */ @@ -228,12 +195,11 @@ public Collection getRawKeyedState() { public void discardState() { try { List 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 boolean equals(Object o) { 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 boolean equals(Object o) { @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 int hashCode() { @Override public String toString() { return "SubtaskState{" + - "legacyState=" + legacyOperatorState + - ", operatorStateFromBackend=" + managedOperatorState + + "operatorStateFromBackend=" + managedOperatorState + ", operatorStateFromStream=" + rawOperatorState + ", keyedStateFromBackend=" + managedKeyedState + ", keyedStateFromStream=" + rawKeyedState + @@ -320,8 +281,7 @@ public String toString() { } public boolean hasState() { - return legacyOperatorState != null - || hasState(managedOperatorState) + return hasState(managedOperatorState) || hasState(rawOperatorState) || hasState(managedKeyedState) || hasState(rawKeyedState); 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 b69285ed5a69e..cc9f9cd1bd10c 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.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 @@ private void assignAttemptState(ExecutionJobVertex executionJobVertex, List subNonPartitionableState = new ArrayList<>(); - Tuple2, Collection> subKeyedState = null; List> subManagedOperatorState = new ArrayList<>(); @@ -174,15 +171,6 @@ private void assignAttemptState(ExecutionJobVertex executionJobVertex, List boolean allElementsAreNull(List nonPartitionableStates) { return true; } - - private void reAssignSubNonPartitionedStates( - OperatorState operatorState, - int subTaskIndex, - int newParallelism, - int oldParallelism, - List 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 operatorStates, int newParallelism, List>> newManagedOperatorStates, @@ -524,19 +492,6 @@ private static void checkParallelismPreconditions(OperatorState operatorState, E "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); - } } /** 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 20d675b686b94..281693bc90304 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.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}. @@ -43,15 +41,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 legacyOperatorState; - /** * Snapshot from the {@link org.apache.flink.runtime.state.OperatorStateBackend}. */ @@ -80,21 +69,18 @@ public class SubtaskState implements CompositeStateHandle { private final long stateSize; public SubtaskState( - ChainedStateHandle legacyOperatorState, ChainedStateHandle managedOperatorState, ChainedStateHandle 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 @@ private static final long getSizeNullSafe(StateObject stateObject) throws Except // -------------------------------------------------------------------------------------------- - /** - * @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 getLegacyOperatorState() { - return legacyOperatorState; - } - public ChainedStateHandle getManagedOperatorState() { return managedOperatorState; } @@ -140,7 +117,6 @@ public void discardState() { try { StateUtil.bestEffortDiscardAllStateObjects( Arrays.asList( - legacyOperatorState, managedOperatorState, rawOperatorState, managedKeyedState, @@ -183,11 +159,6 @@ public boolean equals(Object o) { 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 boolean equals(Object o) { @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 int hashCode() { @Override public String toString() { return "SubtaskState{" + - "chainedStateHandle=" + legacyOperatorState + - ", operatorStateFromBackend=" + managedOperatorState + + "operatorStateFromBackend=" + managedOperatorState + ", operatorStateFromStream=" + rawOperatorState + ", keyedStateFromBackend=" + managedKeyedState + ", keyedStateFromStream=" + rawKeyedState + 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 ed847a43449d7..0f3bedbc72639 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 subtaskStates; - /** parallelism of the operator when it was checkpointed */ private final int parallelism; @@ -117,15 +116,6 @@ public int getChainLength() { 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()) { 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 c1fcf4f0d4547..12e9c5bd63782 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 @@ */ 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> 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 static SavepointSerializer getSerializer(int version) { } } + /** + * 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; + } } 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 7beb1b8dd6ace..586df57a27568 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 javax.annotation.Nonnull; import javax.annotation.Nullable; + import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; 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 f67d54ca15149..c26c983fb93a2 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.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 { 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 void serializeOld(SavepointV1 savepoint, DataOutputStream dos) throws IOE private static void serializeSubtaskState(SubtaskState subtaskState, DataOutputStream dos) throws IOException { - dos.writeLong(-1); - - ChainedStateHandle 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 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 @@ private static SubtaskState deserializeSubtaskState(DataInputStream dis) throws long ignoredDuration = dis.readLong(); int len = dis.readInt(); - List 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 operatorStateBackend = new ArrayList<>(len); @@ -196,9 +198,6 @@ private static SubtaskState deserializeSubtaskState(DataInputStream dis) throws KeyedStateHandle keyedStateStream = deserializeKeyedStateHandle(dis); - ChainedStateHandle nonPartitionableStateChain = - new ChainedStateHandle<>(nonPartitionableState); - ChainedStateHandle operatorStateBackendChain = new ChainedStateHandle<>(operatorStateBackend); @@ -206,7 +205,6 @@ private static SubtaskState deserializeSubtaskState(DataInputStream dis) throws new ChainedStateHandle<>(operatorStateStream); return new SubtaskState( - nonPartitionableStateChain, operatorStateBackendChain, operatorStateStreamChain, keyedStateBackend, 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 bd364a28958b9..9e406dfe44c64 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.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 static Savepoint convertToOperatorStateSavepointV2( continue; } - @SuppressWarnings("deprecation") - ChainedStateHandle nonPartitionedState = - subtaskState.getLegacyOperatorState(); ChainedStateHandle partitioneableState = subtaskState.getManagedOperatorState(); ChainedStateHandle rawOperatorState = @@ -240,7 +236,6 @@ public static Savepoint convertToOperatorStateSavepointV2( } OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState( - nonPartitionedState != null ? nonPartitionedState.get(operatorIndex) : null, partitioneableState != null ? partitioneableState.get(operatorIndex) : null, rawOperatorState != null ? rawOperatorState.get(operatorIndex) : null, managedKeyedState, 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 15628a0429e31..5636a52aba71e 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.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 @@ private static void serializeSubtaskState(OperatorSubtaskState subtaskState, Dat 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 @@ private static void serializeSubtaskState(OperatorSubtaskState subtaskState, Dat } 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 @@ private static OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) KeyedStateHandle keyedStateStream = deserializeKeyedStateHandle(dis); return new OperatorSubtaskState( - nonPartitionableState, operatorStateBackend, operatorStateStream, keyedStateBackend, 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 2800899280a21..8b58891e36ba6 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 @@ 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 void copy(DataInputView source, DataOutputView target) throws IOException 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()); - } } 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 d1c0466e4c665..e235b96969355 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.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.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 @@ private StateTable tryRegisterStateTable( // check compatibility results to determine if state migration is required CompatibilityResult namespaceCompatibility = CompatibilityUtil.resolveCompatibilityResult( restoredMetaInfo.getNamespaceSerializer(), - MigrationNamespaceSerializerProxy.class, + null, restoredMetaInfo.getNamespaceSerializerConfigSnapshot(), newMetaInfo.getNamespaceSerializer()); @@ -405,11 +399,7 @@ public void restore(Collection restoredState) throws Exception LOG.debug("Restoring snapshot from state handles: {}.", restoredState); } - if (MigrationUtil.isOldSavepointKeyedState(restoredState)) { - restoreOldSavepointKeyedState(restoredState); - } else { - restorePartitionedState(restoredState); - } + restorePartitionedState(restoredState); } @SuppressWarnings({"unchecked"}) @@ -559,55 +549,6 @@ public String toString() { return "HeapKeyedStateBackend"; } - /** - * @deprecated Used for backwards compatibility with previous savepoint versions. - */ - @SuppressWarnings({"unchecked", "rawtypes", "DeprecatedIsStillUsed"}) - @Deprecated - private void restoreOldSavepointKeyedState( - Collection 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> namedStates; - try (FSDataInputStream inputStream = keyGroupStateHandle.openInputStream()) { - namedStates = InstantiationUtil.deserializeObject(inputStream, userCodeClassLoader); - } - - for (Map.Entry> nameToState : namedStates.entrySet()) { - - final String stateName = nameToState.getKey(); - final KvStateSnapshot genericSnapshot = nameToState.getValue(); - - if (genericSnapshot instanceof MigrationRestoreSnapshot) { - MigrationRestoreSnapshot stateSnapshot = (MigrationRestoreSnapshot) 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. */ 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 26db772d52b29..7c95a34720e5b 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.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 void testFailingCompletedCheckpointStoreAdd() throws Exception { 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 void testFailingCompletedCheckpointStoreAdd() throws Exception { // 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(); 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 45cbbc39d12b9..4193c2c66e6a5 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.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 void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); - CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, 0L); List keyGroupPartitions1 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); List 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 void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws 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 void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws 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 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 keyGroupPartitions1 = - StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); - List 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 completedCheckpoints = coord.getSuccessfulCheckpoints(); - - assertEquals(1, completedCheckpoints.size()); - - Map 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 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s //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 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s 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 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s for (int idx = 0; idx < operatorIDs.size(); ++idx) { OperatorID operatorID = operatorIDs.get(idx); OperatorSubtaskState opState = taskStateHandles.getSubtaskStateByOperatorID(operatorID); - Assert.assertNull(opState.getLegacyOperatorState()); Collection opStateBackend = opState.getManagedOperatorState(); Collection opStateRaw = opState.getRawOperatorState(); allParallelManagedOpStates.add(opStateBackend); @@ -2593,14 +2455,11 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception 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 void testStateRecoveryWithTopologyChange(int scaleType) throws Exception expectedRawOperatorState.add(ChainedStateHandle.wrapSingleHandle(subRawOperatorState)); OperatorSubtaskState subtaskState = new OperatorSubtaskState( - null, subManagedOperatorState, subRawOperatorState, subManagedKeyedState, @@ -2735,7 +2593,6 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception OperatorSubtaskState opState = stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); - assertNull(opState.getLegacyOperatorState()); assertTrue(opState.getManagedOperatorState().isEmpty()); assertTrue(opState.getRawOperatorState().isEmpty()); } @@ -2745,16 +2602,11 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception 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 managedOperatorState = opState.getManagedOperatorState(); assertEquals(1, managedOperatorState.size()); assertTrue(CommonTestUtils.isSteamContentEqual(expectedManagedOpState.openInputStream(), @@ -2771,16 +2623,11 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception 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 managedOperatorState = opState.getManagedOperatorState(); assertEquals(1, managedOperatorState.size()); assertTrue(CommonTestUtils.isSteamContentEqual(expectedManagedOpState.openInputStream(), @@ -2816,8 +2663,6 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception actualManagedOperatorStates.add(actualSubManagedOperatorState); actualRawOperatorStates.add(actualSubRawOperatorState); - - assertNull(opState.getLegacyOperatorState()); } // operator 6 @@ -2825,7 +2670,6 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception 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 @@ static TaskStateSnapshot mockSubtaskState( 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 static void verifyStateRestore( for (int i = 0; i < executionJobVertex.getParallelism(); i++) { - final List 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 expectedOpStateBackend = generateChainedPartitionableStateHandle(jobVertexID, i, 2, 8, false); @@ -3926,7 +3762,7 @@ private void performIncrementalCheckpoint( spy(new ByteStreamStateHandle("meta", new byte[]{'m'})))); OperatorSubtaskState operatorSubtaskState = - spy(new OperatorSubtaskState(null, + spy(new OperatorSubtaskState( Collections.emptyList(), Collections.emptyList(), Collections.singletonList(managedState), 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 791bffa7635ef..1788434255bdb 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.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 serializedState = CheckpointCoordinatorTest.generateChainedStateHandle(new SerializableObject()); KeyGroupRange keyGroupRange = KeyGroupRange.of(0,0); List testStates = Collections.singletonList(new SerializableObject()); final KeyedStateHandle serializedKeyGroupStates = CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRange, testStates); @@ -125,7 +122,6 @@ public void testSetState() { subtaskStates.putSubtaskStateByOperatorID( OperatorID.fromJobVertexID(statefulId), new OperatorSubtaskState( - serializedState.get(0), Collections.emptyList(), Collections.emptyList(), Collections.singletonList(serializedKeyGroupStates), @@ -249,17 +245,13 @@ public void testNonRestoredState() throws Exception { Executors.directExecutor(), SharedStateRegistry.DEFAULT_FACTORY); - StreamStateHandle serializedState = CheckpointCoordinatorTest - .generateChainedStateHandle(new SerializableObject()) - .get(0); - // --- (2) Checkpoint misses state for a jobVertex (should work) --- Map 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 void testNonRestoredState() throws Exception { // 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); } 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 de1f599e137bc..acedb5071b252 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 static Collection createOperatorStates( 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 static Collection createOperatorStates( 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 static Collection createOperatorStates( 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 static Collection createOperatorStates( } taskState.putState(subtaskIdx, new OperatorSubtaskState( - nonPartitionableState, operatorStateHandleBackend, operatorStateHandleStream, keyedStateStream, @@ -175,15 +167,11 @@ public static Collection createTaskStates( for (int subtaskIdx = 0; subtaskIdx < numSubtasksPerTask; subtaskIdx++) { - List nonPartitionableStates = new ArrayList<>(chainLength); List operatorStatesBackend = new ArrayList<>(chainLength); List 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 static Collection createTaskStates( 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 static Collection createTaskStates( } taskState.putState(subtaskIdx, new SubtaskState( - new ChainedStateHandle<>(nonPartitionableStates), new ChainedStateHandle<>(operatorStatesBackend), new ChainedStateHandle<>(operatorStatesStream), keyedStateStream, 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 16f3769160b79..0000000000000 --- 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 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 legacyOperatorState = subtaskState.getLegacyOperatorState(); - for (int c = 0; c < legacyOperatorState.getLength(); ++c) { - StreamStateHandle stateHandle = legacyOperatorState.get(c); - try (InputStream is = stateHandle.openInputStream()) { - Tuple4 expTestState = new Tuple4<>(0, t, p, c); - Tuple4 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> 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 createTaskStatesOld( - int numTaskStates, int numSubtaskStates) throws Exception { - - List 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 testState = new Tuple4<>(0, i, j, k); - if (j % 4 != 0) { - state.setFunctionState(new SerializedStateHandle(testState)); - } - testState = new Tuple4<>(1, i, j, k); - state.setOperatorState(new SerializedStateHandle<>(testState)); - - if ((0 == k) && (i % 3 != 0)) { - HashMap> 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; - } -} 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 933c7a074ff04..173730a80e884 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.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.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.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 void setInitialState( 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 boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, Checkpoi String.valueOf(UUID.randomUUID()), InstantiationUtil.serializeObject(checkpointMetaData.getCheckpointId())); + Map 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 static void awaitCompletedCheckpoints() throws InterruptedException { public static long[] getRecoveredStates() { return recoveredStates; } + + private static OperatorStateHandle extractSingletonOperatorState(TaskStateSnapshot taskStateHandles) { + Set> subtaskStateMappings = taskStateHandles.getSubtaskStateMappings(); + Preconditions.checkNotNull(subtaskStateMappings); + Preconditions.checkState(subtaskStateMappings.size() == 1); + OperatorSubtaskState subtaskState = subtaskStateMappings.iterator().next().getValue(); + Collection managedOperatorState = + Preconditions.checkNotNull(subtaskState).getManagedOperatorState(); + Preconditions.checkNotNull(managedOperatorState); + Preconditions.checkState(managedOperatorState.size() == 1); + return managedOperatorState.iterator().next(); + } } } 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 d022cdcf59e69..b36ac86e4b85f 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 void testConfirmTaskCheckpointed() { 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())), 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 dd6148c32f0a4..0000000000000 --- 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. - * - *

DO NOT MODIFY THIS CLASS - */ -public class StreamGraphHasherV1 implements StreamGraphHasher { - - private static final Logger LOG = LoggerFactory.getLogger(StreamGraphHasherV1.class); - - @Override - public Map traverseStreamGraphAndGenerateHashes(StreamGraph streamGraph) { - // The hash function used to generate the hash - final HashFunction hashFunction = Hashing.murmur3_128(0); - final Map hashes = new HashMap<>(); - - Set visited = new HashSet<>(); - Queue 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 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 true if the node hash has been generated. - * false, 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 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 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")); - } - } -} 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 b1471b233e4ba..0000000000000 --- 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 extends TypeSerializer { - - - 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 typeSerializer; - - public MultiplexingStreamRecordSerializer(TypeSerializer 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 getContainedTypeSerializer() { - return this.typeSerializer; - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public MultiplexingStreamRecordSerializer duplicate() { - TypeSerializer copy = typeSerializer.duplicate(); - return (copy == typeSerializer) ? this : new MultiplexingStreamRecordSerializer(copy); - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - @Override - public StreamRecord createInstance() { - return new StreamRecord(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 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 fromRecord = from.asRecord(); - StreamRecord 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 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(typeSerializer.deserialize(source), timestamp); - } - else if (tag == TAG_REC_WITHOUT_TIMESTAMP) { - return new StreamRecord(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 reuseRecord = reuse.asRecord(); - reuseRecord.replace(value, timestamp); - return reuseRecord; - } - else if (tag == TAG_REC_WITHOUT_TIMESTAMP) { - T value = typeSerializer.deserialize(source); - StreamRecord 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 ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { - if (configSnapshot instanceof MultiplexingStreamRecordSerializerConfigSnapshot) { - Tuple2, TypeSerializerConfigSnapshot> previousTypeSerializerAndConfig = - ((MultiplexingStreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); - - CompatibilityResult 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 - extends CompositeTypeSerializerConfigSnapshot { - - private static final int VERSION = 1; - - /** This empty nullary constructor is required for deserializing the configuration. */ - public MultiplexingStreamRecordSerializerConfigSnapshot() {} - - public MultiplexingStreamRecordSerializerConfigSnapshot(TypeSerializer 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(); - } -} 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 e018ba0ec4595..0000000000000 --- 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, - * WITHOUStreamRecordWARRANTIES 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. - * - *

{@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 The type of value in the {@link StreamRecord} - */ -@Internal -public final class StreamRecordSerializer extends TypeSerializer> { - - private static final long serialVersionUID = 1L; - - private final TypeSerializer typeSerializer; - - public StreamRecordSerializer(TypeSerializer serializer) { - if (serializer instanceof StreamRecordSerializer) { - throw new RuntimeException("StreamRecordSerializer given to StreamRecordSerializer as value TypeSerializer: " + serializer); - } - this.typeSerializer = Preconditions.checkNotNull(serializer); - } - - public TypeSerializer getContainedTypeSerializer() { - return this.typeSerializer; - } - - // ------------------------------------------------------------------------ - // General serializer and type utils - // ------------------------------------------------------------------------ - - @Override - public StreamRecordSerializer duplicate() { - TypeSerializer serializerCopy = typeSerializer.duplicate(); - return serializerCopy == typeSerializer ? this : new StreamRecordSerializer(serializerCopy); - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public int getLength() { - return typeSerializer.getLength(); - } - - // ------------------------------------------------------------------------ - // Type serialization, copying, instantiation - // ------------------------------------------------------------------------ - - @Override - public StreamRecord createInstance() { - try { - return new StreamRecord(typeSerializer.createInstance()); - } catch (Exception e) { - throw new RuntimeException("Cannot instantiate StreamRecord.", e); - } - } - - @Override - public StreamRecord copy(StreamRecord from) { - return from.copy(typeSerializer.copy(from.getValue())); - } - - @Override - public StreamRecord copy(StreamRecord from, StreamRecord reuse) { - from.copyTo(typeSerializer.copy(from.getValue(), reuse.getValue()), reuse); - return reuse; - } - - @Override - public void serialize(StreamRecord value, DataOutputView target) throws IOException { - typeSerializer.serialize(value.getValue(), target); - } - - @Override - public StreamRecord deserialize(DataInputView source) throws IOException { - return new StreamRecord(typeSerializer.deserialize(source)); - } - - @Override - public StreamRecord deserialize(StreamRecord 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> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { - if (configSnapshot instanceof StreamRecordSerializerConfigSnapshot) { - Tuple2, TypeSerializerConfigSnapshot> previousTypeSerializerAndConfig = - ((StreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); - - CompatibilityResult 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 extends CompositeTypeSerializerConfigSnapshot { - - private static final int VERSION = 1; - - /** This empty nullary constructor is required for deserializing the configuration. */ - public StreamRecordSerializerConfigSnapshot() {} - - public StreamRecordSerializerConfigSnapshot(TypeSerializer typeSerializer) { - super(typeSerializer); - } - - @Override - public int getVersion() { - return VERSION; - } - } -} 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 cb3c7cce2943e..0000000000000 --- 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. - * - *

Deprecation and Replacement

- * 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 ( - * - *

{@code
- * public class ExampleFunction implements MapFunction, ListCheckpointed {
- *
- *     private int count;
- *
- *     public List snapshotState(long checkpointId, long timestamp) throws Exception {
- *         return Collections.singletonList(this.count);
- *     }
- *
- *     public void restoreState(List state) throws Exception {
- *         this.value = state.isEmpty() ? 0 : state.get(0);
- *     }
- *
- *     public T map(T value) {
- *         count++;
- *         return value;
- *     }
- * }
- * }
- * - * @param 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 extends CheckpointedRestoring { - - /** - * 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; -} 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 5138b49c4e36b..0000000000000 --- 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. - * - *

Deprecation and Replacement

- * 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. - * - *

{@code
- * public class ExampleFunction implements MapFunction, ListCheckpointed {
- *
- *     private int count;
- *
- *     public List snapshotState(long checkpointId, long timestamp) throws Exception {
- *         return Collections.singletonList(this.count);
- *     }
- *
- *     public void restoreState(List state) throws Exception {
- *         this.value = state.isEmpty() ? 0 : state.get(0);
- *     }
- *
- *     public T map(T value) {
- *         count++;
- *         return value;
- *     }
- * }
- * }
- * - * @deprecated Please use {@link ListCheckpointed} and {@link CheckpointedFunction} instead, - * as illustrated in the example above. - */ -@Deprecated -@PublicEvolving -public interface CheckpointedAsynchronously extends Checkpointed {} 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 cfaa505f0847b..0000000000000 --- 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 type of the restored state. - * - * @deprecated Please use {@link CheckpointedFunction} or {@link ListCheckpointed} after restoring your legacy state. - */ -@Deprecated -@PublicEvolving -public interface CheckpointedRestoring { - /** - * 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. - * - *

Note that restoreState() is called before open(). - * - * @param state The state to be restored. - */ - void restoreState(T state) throws Exception; -} 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 bb6e4bc7b68eb..0000000000000 --- 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"; - } - } -} 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 348861f221a6e..f904a10fd7ac0 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.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.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 SingleOutputStreamOperator reduce(ReduceFunction function) { //clean the closure function = input.getExecutionEnvironment().clean(function); - - String callLocation = Utils.getCallLocationName(); - String udfName = "WindowedStream." + callLocation; - - SingleOutputStreamOperator result = createFastTimeOperatorIfValid(function, input.getType(), udfName); - if (result != null) { - return result; - } - - LegacyWindowOperatorType legacyOpType = getLegacyWindowType(function); - return reduce(function, new PassThroughWindowFunction(), 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. - * - *

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 SingleOutputStreamOperator reduce(ReduceFunction reduceFunction, WindowFunction function) { - return reduce(reduceFunction, function, LegacyWindowOperatorType.NONE); + return reduce(function, new PassThroughWindowFunction()); } /** @@ -265,39 +230,15 @@ public SingleOutputStreamOperator reduce(ReduceFunction reduceFunction * * @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 SingleOutputStreamOperator reduce( - ReduceFunction reduceFunction, - WindowFunction function, - TypeInformation 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. - * - *

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 SingleOutputStreamOperator reduce( ReduceFunction reduceFunction, - WindowFunction function, - LegacyWindowOperatorType legacyWindowOpType) { + WindowFunction function) { TypeInformation inType = input.getType(); TypeInformation resultType = getWindowFunctionReturnType(function, inType); - - return reduce(reduceFunction, function, resultType, legacyWindowOpType); + return reduce(reduceFunction, function, resultType); } /** @@ -310,15 +251,12 @@ private SingleOutputStreamOperator reduce( * @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 SingleOutputStreamOperator reduce( + public SingleOutputStreamOperator reduce( ReduceFunction reduceFunction, WindowFunction function, - TypeInformation resultType, - LegacyWindowOperatorType legacyWindowOpType) { + TypeInformation resultType) { if (reduceFunction instanceof RichFunction) { throw new UnsupportedOperationException("ReduceFunction of reduce can not be a RichFunction."); @@ -374,8 +312,7 @@ private SingleOutputStreamOperator reduce( new InternalSingleValueWindowFunction<>(function), trigger, allowedLateness, - lateDataOutputTag, - legacyWindowOpType); + lateDataOutputTag); } return input.transform(opName, resultType, operator); @@ -1183,12 +1120,6 @@ private SingleOutputStreamOperator apply(InternalWindowFunction result = createFastTimeOperatorIfValid(function, resultType, udfName); - if (result != null) { - return result; - } - - LegacyWindowOperatorType legacyWindowOpType = getLegacyWindowType(function); String opName; KeySelector keySel = input.getKeySelector(); @@ -1231,8 +1162,7 @@ private SingleOutputStreamOperator apply(InternalWindowFunction aggregate(AggregationFunction aggregato 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 SingleOutputStreamOperator createFastTimeOperatorIfValid( - ReduceFunction function, - TypeInformation 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 reducer = (ReduceFunction) function; - - @SuppressWarnings("unchecked") - OneInputStreamOperator op = (OneInputStreamOperator) - 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 reducer = (ReduceFunction) function; - - @SuppressWarnings("unchecked") - OneInputStreamOperator op = (OneInputStreamOperator) - 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 SingleOutputStreamOperator createFastTimeOperatorIfValid( - InternalWindowFunction, R, K, W> function, - TypeInformation 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, R, K, TimeWindow> timeWindowFunction = - (InternalWindowFunction, R, K, TimeWindow>) function; - - OneInputStreamOperator 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, R, K, TimeWindow> timeWindowFunction = - (InternalWindowFunction, R, K, TimeWindow>) function; - - OneInputStreamOperator 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(); } 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 3c4cfbd0c9194..fedd791fc292e 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.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 @@ */ @Internal public class ContinuousFileMonitoringFunction - extends RichSourceFunction implements CheckpointedFunction, CheckpointedRestoring { + extends RichSourceFunction implements CheckpointedFunction { private static final long serialVersionUID = 1L; @@ -375,12 +374,4 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { 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); - } } 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 3a9e8e1f60c9a..e14cfda3b4608 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.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 @@ * 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. - *

- * As soon as a split descriptor is received, it is put in a queue, and have another + * + *

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 extends AbstractStreamOperator - implements OneInputStreamOperator, OutputTypeConfigurable, CheckpointedRestoringOperator { + implements OneInputStreamOperator, OutputTypeConfigurable { private static final long serialVersionUID = 1L; @@ -422,83 +415,4 @@ public void snapshotState(StateSnapshotContext context) throws Exception { 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 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; - } } 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 abaa74e7d2ecc..884b899116378 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.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 static JobGraph createJobGraph(StreamGraph streamGraph) { 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 @@ private List createChain( createChain(nonChainable.getTargetId(), nonChainable.getTargetId(), hashes, legacyHashes, 0, chainedOperatorHashes); } - List> operatorHashes = chainedOperatorHashes.get(startNodeId); - if (operatorHashes == null) { - operatorHashes = new ArrayList<>(); - chainedOperatorHashes.put(startNodeId, operatorHashes); - } + List> operatorHashes = + chainedOperatorHashes.computeIfAbsent(startNodeId, k -> new ArrayList<>()); byte[] primaryHashBytes = hashes.get(currentNodeId); - operatorHashes.add(new Tuple2<>(primaryHashBytes, legacyHashes.get(1).get(currentNodeId))); + + for (Map legacyHash : legacyHashes) { + operatorHashes.add(new Tuple2<>(primaryHashBytes, legacyHash.get(currentNodeId))); + } chainedNames.put(currentNodeId, createChainedName(currentNodeId, chainableOutputs)); chainedMinResources.put(currentNodeId, createChainedMinResources(currentNodeId, chainableOutputs)); 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 a72b9fe3491e7..a28fc304af0fb 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.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.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 final void initializeState(OperatorSubtaskState stateHandles) throws Exce 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."); - } - } } /** @@ -450,35 +412,6 @@ public void snapshotState(StateSnapshotContext context) throws Exception { } } - /** - * @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. * 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 efbebf4056ee8..329ce183ce5d0 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.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 @@ @PublicEvolving public abstract class AbstractUdfStreamOperator extends AbstractStreamOperator - implements OutputTypeConfigurable, - StreamCheckpointedOperator { + implements OutputTypeConfigurable { private static final long serialVersionUID = 1L; @@ -131,59 +122,6 @@ public void dispose() throws Exception { // checkpointing and recovery // ------------------------------------------------------------------------ - @Override - public void snapshotState(FSDataOutputStream out, long checkpointId, long timestamp) throws Exception { - if (userFunction instanceof Checkpointed) { - @SuppressWarnings("unchecked") - Checkpointed chkFunction = (Checkpointed) 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 chkFunction = (CheckpointedRestoring) 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 Configuration getUserFunctionParameters() { 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."); } } } 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 33304e413be3b..0000000000000 --- 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). - * - *

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; -} 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 986e2b76930fc..0000000000000 --- 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; - -} 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 9d5e02b1a3d37..38b4aeedb1b2c 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.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; @@ -104,21 +103,6 @@ OperatorSnapshotResult snapshotState( long timestamp, 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. * 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 252f997d0b30b..0000000000000 --- 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 - * "Fast Aligned Processing Time Window Operator" for windowing. - * - *

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: - * - *

    - *
  • No custom state backend can be selected, the operator always stores data on the Java heap.
  • - *
  • The operator does not support key groups, meaning it cannot change the parallelism.
  • - *
  • Future versions of Flink may not be able to resume from checkpoints/savepoints taken by this - * operator.
  • - *
- * - *

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. - * - *

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()); - } -} 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 83a752869e5d1..0000000000000 --- 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 - extends AbstractUdfStreamOperator - implements OneInputStreamOperator, 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 keySelector; - - private final TypeSerializer keySerializer; - private final TypeSerializer stateTypeSerializer; - - private final long windowSize; - private final long windowSlide; - private final long paneSize; - private final int numPanesPerWindow; - - // ----- fields for operator functionality ----- - - private transient AbstractKeyedTimePanes panes; - - private transient TimestampedCollector out; - - private transient RestoredState restoredState; - - private transient long nextEvaluationTime; - private transient long nextSlideTime; - - protected AbstractAlignedProcessingTimeWindowOperator( - F function, - KeySelector keySelector, - TypeSerializer keySerializer, - TypeSerializer 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 createPanes( - KeySelector 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 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 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 { - - final AbstractKeyedTimePanes panes; - final long nextEvaluationTime; - final long nextSlideTime; - - RestoredState(AbstractKeyedTimePanes panes, long nextEvaluationTime, long nextSlideTime) { - this.panes = panes; - this.nextEvaluationTime = nextEvaluationTime; - this.nextSlideTime = nextSlideTime; - } - } -} 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 d67121ab6d752..0000000000000 --- 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 - extends AbstractAlignedProcessingTimeWindowOperator, InternalWindowFunction, OUT, KEY, TimeWindow>> { - - private static final long serialVersionUID = 7305948082830843475L; - - public AccumulatingProcessingTimeWindowOperator( - InternalWindowFunction, OUT, KEY, TimeWindow> function, - KeySelector keySelector, - TypeSerializer keySerializer, - TypeSerializer valueSerializer, - long windowLength, - long windowSlide) { - super(function, keySelector, keySerializer, - new ArrayListSerializer<>(valueSerializer), windowLength, windowSlide); - } - - @Override - protected AccumulatingKeyedTimePanes createPanes(KeySelector keySelector, Function function) { - @SuppressWarnings("unchecked") - InternalWindowFunction, OUT, KEY, Window> windowFunction = (InternalWindowFunction, OUT, KEY, Window>) function; - - return new AccumulatingKeyedTimePanes<>(keySelector, windowFunction); - } -} 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 674738322dfa3..0000000000000 --- 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 - extends AbstractAlignedProcessingTimeWindowOperator> { - - private static final long serialVersionUID = 7305948082830843475L; - - public AggregatingProcessingTimeWindowOperator( - ReduceFunction function, - KeySelector keySelector, - TypeSerializer keySerializer, - TypeSerializer aggregateSerializer, - long windowLength, - long windowSlide) { - super(function, keySelector, keySerializer, aggregateSerializer, windowLength, windowSlide); - } - - @Override - protected AggregatingKeyedTimePanes createPanes(KeySelector keySelector, Function function) { - @SuppressWarnings("unchecked") - ReduceFunction windowFunction = (ReduceFunction) function; - - return new AggregatingKeyedTimePanes(keySelector, windowFunction); - } -} 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 880907dbf3251..b14739fed418d 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.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.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.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 protected transient InternalTimerService 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> restoredFromLegacyAlignedOpRecords; - - /** - * The restored processing time timers when migrating from an - * older version of the {@link WindowOperator}. - */ - private transient PriorityQueue> restoredFromLegacyProcessingTimeTimers; - - /** The restored event time timer when migrating from an - * older version of the {@link WindowOperator}. - */ - private transient PriorityQueue> restoredFromLegacyEventTimeTimers; - /** * Creates a new {@code WindowOperator} based on the given policies and user functions. */ @@ -222,25 +180,6 @@ public WindowOperator( long allowedLateness, OutputTag 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 windowAssigner, - TypeSerializer windowSerializer, - KeySelector keySelector, - TypeSerializer keySerializer, - StateDescriptor, ?> windowStateDescriptor, - InternalWindowFunction windowFunction, - Trigger trigger, - long allowedLateness, - OutputTag lateDataOutputTag, - LegacyWindowOperatorType legacyWindowOperatorType) { - super(windowFunction); checkArgument(!(windowAssigner instanceof BaseAlignedWindowAssigner), @@ -261,7 +200,6 @@ public WindowOperator( this.trigger = checkNotNull(trigger); this.allowedLateness = allowedLateness; this.lateDataOutputTag = lateDataOutputTag; - this.legacyWindowOperatorType = legacyWindowOperatorType; setChainingStrategy(ChainingStrategy.ALWAYS); } @@ -321,8 +259,6 @@ public long getCurrentProcessingTime() { getOrCreateKeyedState(VoidNamespaceSerializer.INSTANCE, mergingSetsStateDescriptor); mergingSetsState.setCurrentNamespace(VoidNamespace.INSTANCE); } - - registerRestoredLegacyStateState(); } @Override @@ -1036,256 +972,6 @@ public String toString() { } } - // ------------------------------------------------------------------------ - // 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>() { - @Override - public int compare(StreamRecord o1, StreamRecord 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 ser = new ArrayListSerializer<>((TypeSerializer) 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 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 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 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 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 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 record = restoredFromLegacyAlignedOpRecords.poll(); - setCurrentKey(keySelector.getKey(record.getValue())); - processElement(record); - } - } - - // gc friendliness - restoredFromLegacyAlignedOpRecords = null; - } - // ------------------------------------------------------------------------ // Getters for testing // ------------------------------------------------------------------------ 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 1dc0ee2500084..d0ab60a3ab347 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.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 CompatibilityResult ensureCompatibility(TypeSerializerConf if (configSnapshot instanceof StreamElementSerializerConfigSnapshot) { previousTypeSerializerAndConfig = ((StreamElementSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); - } else if (configSnapshot instanceof MultiplexingStreamRecordSerializer.MultiplexingStreamRecordSerializerConfigSnapshot) { - previousTypeSerializerAndConfig = - ((MultiplexingStreamRecordSerializer.MultiplexingStreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); } else { return CompatibilityResult.requiresMigration(); } 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 4914075d0136d..0b03b7980f52e 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.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 managedKeyedState; private final Collection rawKeyedState; private final Collection managedOperatorState; @@ -46,24 +43,18 @@ public class OperatorStateHandles { public OperatorStateHandles( int operatorChainIndex, - StreamStateHandle legacyOperatorState, Collection managedKeyedState, Collection rawKeyedState, Collection managedOperatorState, Collection 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 getManagedKeyedState() { return managedKeyedState; } 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 1ba5fb12bbdab..310df4de6c141 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.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 @@ private static final class AsyncCheckpointRunnable implements Runnable, Closeabl private final Map operatorSnapshotsInProgress; - private Map nonPartitionedStateHandles; - private final CheckpointMetaData checkpointMetaData; private final CheckpointMetrics checkpointMetrics; @@ -848,7 +844,6 @@ private static final class AsyncCheckpointRunnable implements Runnable, Closeabl AsyncCheckpointRunnable( StreamTask owner, - Map nonPartitionedStateHandles, Map operatorSnapshotsInProgress, CheckpointMetaData checkpointMetaData, CheckpointMetrics checkpointMetrics, @@ -858,7 +853,6 @@ private static final class AsyncCheckpointRunnable implements Runnable, Closeabl 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 void run() { 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 @@ private void cleanup() throws Exception { } } - // 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 @@ private static final class CheckpointingOperation { // ------------------------ - private final Map nonPartitionedStates; private final Map operatorSnapshotsInProgress; public CheckpointingOperation( @@ -1022,7 +1007,6 @@ public CheckpointingOperation( 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 void executeCheckpointing() throws Exception { } } - // 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 void executeCheckpointing() throws Exception { @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 void runAsyncCheckpointingAndAcknowledge() throws IOException { AsyncCheckpointRunnable asyncCheckpointRunnable = new AsyncCheckpointRunnable( owner, - nonPartitionedStates, operatorSnapshotsInProgress, checkpointMetaData, checkpointMetrics, 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 ff5f589302f06..4ed689d4e8626 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.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 void close() throws Exception { } private static class LifecycleTrackingStreamSource> - extends StreamSource implements Serializable, StreamCheckpointedOperator { + extends StreamSource implements Serializable { private static final long serialVersionUID = 2431488948886850562L; private transient Thread testCheckpointer; @@ -265,12 +262,6 @@ public void snapshotState(StateSnapshotContext context) throws Exception { super.snapshotState(context); } - @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"); 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 7dba4af9bfef0..0000000000000 --- 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> transformations = new ArrayList<>(); - - int initValue = 1; - - FoldApplyProcessWindowFunction foldWindowFunction = new FoldApplyProcessWindowFunction<>( - initValue, - new FoldFunction() { - @Override - public Integer fold(Integer accumulator, Integer value) throws Exception { - return accumulator + value; - } - - }, - new ProcessWindowFunction() { - @Override - public void process(Integer integer, - Context context, - Iterable input, - Collector out) throws Exception { - for (Integer in: input) { - out.collect(in); - } - } - }, - BasicTypeInfo.INT_TYPE_INFO - ); - - AccumulatingProcessingTimeWindowOperator windowOperator = new AccumulatingProcessingTimeWindowOperator<>( - new InternalIterableProcessWindowFunction<>(foldWindowFunction), - new KeySelector() { - private static final long serialVersionUID = -7951310554369722809L; - - @Override - public Integer getKey(Integer value) throws Exception { - return value; - } - }, - IntSerializer.INSTANCE, - IntSerializer.INSTANCE, - 3000, - 3000 - ); - - SourceFunction sourceFunction = new SourceFunction(){ - - private static final long serialVersionUID = 8297735565464653028L; - - @Override - public void run(SourceContext ctx) throws Exception { - - } - - @Override - public void cancel() { - - } - }; - - SourceTransformation 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 result = new ArrayList<>(); - List input = new ArrayList<>(); - List expected = new ArrayList<>(); - - input.add(1); - input.add(2); - input.add(3); - - for (int value : input) { - initValue += value; - } - - expected.add(initValue); - - FoldApplyProcessWindowFunction.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> transformations = new ArrayList<>(); - - int initValue = 1; - - FoldApplyProcessAllWindowFunction foldWindowFunction = new FoldApplyProcessAllWindowFunction<>( - initValue, - new FoldFunction() { - @Override - public Integer fold(Integer accumulator, Integer value) throws Exception { - return accumulator + value; - } - - }, - new ProcessAllWindowFunction() { - @Override - public void process(Context context, - Iterable input, - Collector out) throws Exception { - for (Integer in: input) { - out.collect(in); - } - } - }, - BasicTypeInfo.INT_TYPE_INFO - ); - - AccumulatingProcessingTimeWindowOperator windowOperator = new AccumulatingProcessingTimeWindowOperator<>( - new InternalIterableProcessAllWindowFunction<>(foldWindowFunction), - new KeySelector() { - private static final long serialVersionUID = -7951310554369722809L; - - @Override - public Byte getKey(Integer value) throws Exception { - return 0; - } - }, - ByteSerializer.INSTANCE, - IntSerializer.INSTANCE, - 3000, - 3000 - ); - - SourceFunction sourceFunction = new SourceFunction(){ - - private static final long serialVersionUID = 8297735565464653028L; - - @Override - public void run(SourceContext ctx) throws Exception { - - } - - @Override - public void cancel() { - - } - }; - - SourceTransformation 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 result = new ArrayList<>(); - List input = new ArrayList<>(); - List expected = new ArrayList<>(); - - input.add(1); - input.add(2); - input.add(3); - - for (int value : input) { - initValue += value; - } - - expected.add(initValue); - - FoldApplyProcessAllWindowFunction.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 ValueState getState(ValueStateDescriptor stateProperties) { - return null; - } - - @Override - public ListState getListState(ListStateDescriptor stateProperties) { - return null; - } - - @Override - public ReducingState getReducingState(ReducingStateDescriptor stateProperties) { - return null; - } - - @Override - public FoldingState getFoldingState(FoldingStateDescriptor stateProperties) { - return null; - } - - @Override - public MapState getMapState(MapStateDescriptor stateProperties) { - return null; - } - } - - private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment { - - @Override - public JobExecutionResult execute(String jobName) throws Exception { - return null; - } - } -} 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 7cf18ddd1a131..0000000000000 --- 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> transformations = new ArrayList<>(); - - int initValue = 1; - - FoldApplyWindowFunction foldWindowFunction = new FoldApplyWindowFunction<>( - initValue, - new FoldFunction() { - private static final long serialVersionUID = -4849549768529720587L; - - @Override - public Integer fold(Integer accumulator, Integer value) throws Exception { - return accumulator + value; - } - }, - new WindowFunction() { - @Override - public void apply(Integer integer, - TimeWindow window, - Iterable input, - Collector out) throws Exception { - for (Integer in: input) { - out.collect(in); - } - } - }, - BasicTypeInfo.INT_TYPE_INFO - ); - - AccumulatingProcessingTimeWindowOperator windowOperator = new AccumulatingProcessingTimeWindowOperator<>( - new InternalIterableWindowFunction<>( - foldWindowFunction), - new KeySelector() { - private static final long serialVersionUID = -7951310554369722809L; - - @Override - public Integer getKey(Integer value) throws Exception { - return value; - } - }, - IntSerializer.INSTANCE, - IntSerializer.INSTANCE, - 3000, - 3000 - ); - - SourceFunction sourceFunction = new SourceFunction(){ - - private static final long serialVersionUID = 8297735565464653028L; - - @Override - public void run(SourceContext ctx) throws Exception { - - } - - @Override - public void cancel() { - - } - }; - - SourceTransformation 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 result = new ArrayList<>(); - List input = new ArrayList<>(); - List 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(result)); - - Assert.assertEquals(expected, result); - } - - private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment { - - @Override - public JobExecutionResult execute(String jobName) throws Exception { - return null; - } - } -} 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 58898d8c740b1..6dd08f6d53893 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 void testUserProvidedHashing() { StreamGraph streamGraph = env.getStreamGraph(); int idx = 1; for (JobVertex jobVertex : streamGraph.getJobGraph().getVertices()) { - Assert.assertEquals(jobVertex.getIdAlternatives().get(1).toString(), userHashes.get(idx)); + List idAlternatives = jobVertex.getIdAlternatives(); + Assert.assertEquals(idAlternatives.get(idAlternatives.size() - 1).toString(), userHashes.get(idx)); --idx; } } 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 a57dcf197a38c..0000000000000 --- 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 mockFunction = mock(InternalIterableWindowFunction.class); - - @SuppressWarnings("unchecked") - private final KeySelector mockKeySelector = mock(KeySelector.class); - - private final KeySelector identitySelector = new KeySelector() { - @Override - public Integer getKey(Integer value) { - return value; - } - }; - - private final InternalIterableWindowFunction validatingIdentityFunction = - new InternalIterableWindowFunction<>(new WindowFunction() { - @Override - public void apply(Integer key, TimeWindow window, Iterable values, Collector out) throws Exception { - for (Integer val : values) { - assertEquals(key, val); - out.collect(val); - } - } - }); - - private final InternalIterableProcessWindowFunction validatingIdentityProcessFunction = - new InternalIterableProcessWindowFunction<>(new ProcessWindowFunction() { - @Override - public void process(Integer key, Context context, Iterable values, Collector 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 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 op = - new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, - StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000); - - KeyedOneInputStreamOperatorTestHarness 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, - windowSize, windowSize); - - KeyedOneInputStreamOperatorTestHarness 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 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityProcessFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, - windowSize, windowSize); - - KeyedOneInputStreamOperatorTestHarness 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 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50); - - KeyedOneInputStreamOperatorTestHarness 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 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityProcessFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50); - - KeyedOneInputStreamOperatorTestHarness 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 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, 50, 50); - - KeyedOneInputStreamOperatorTestHarness 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 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityProcessFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, 50, 50); - - KeyedOneInputStreamOperatorTestHarness 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 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50); - - KeyedOneInputStreamOperatorTestHarness 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 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityProcessFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50); - - KeyedOneInputStreamOperatorTestHarness 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 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityProcessFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, - windowSize, windowSize); - - OneInputStreamOperatorTestHarness 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 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 finalResult = new ArrayList<>(); - finalResult.addAll(resultAtSnapshot); - List 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, - windowSize, windowSize); - - OneInputStreamOperatorTestHarness 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 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 finalResult = new ArrayList<>(); - finalResult.addAll(resultAtSnapshot); - List 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityProcessFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, - windowSize, windowSlide); - - OneInputStreamOperatorTestHarness 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 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 finalResult = new ArrayList<>(resultAtSnapshot); - List 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - validatingIdentityFunction, identitySelector, - IntSerializer.INSTANCE, IntSerializer.INSTANCE, - windowSize, windowSlide); - - OneInputStreamOperatorTestHarness 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 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 finalResult = new ArrayList<>(resultAtSnapshot); - List 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 op = - new AccumulatingProcessingTimeWindowOperator<>( - new InternalIterableProcessWindowFunction<>(new StatefulFunction()), - identitySelector, - IntSerializer.INSTANCE, - IntSerializer.INSTANCE, - 50, - 50); - - OneInputStreamOperatorTestHarness 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 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( - 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 { - - // we use a concurrent map here even though there is no concurrency, to - // get "volatile" style access to entries - private static final Map globalCounts = new ConcurrentHashMap<>(); - - private ValueState 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 values, - Collector 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>()); - 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 List extractFromStreamRecords(Iterable input) { - List 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); - } - } -} 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 62f4f0baf4afe..0000000000000 --- 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 mockFunction = mock(ReduceFunction.class); - - @SuppressWarnings("unchecked") - private final KeySelector mockKeySelector = mock(KeySelector.class); - - private final KeySelector, Integer> fieldOneSelector = - new KeySelector, Integer>() { - @Override - public Integer getKey(Tuple2 value) { - return value.f0; - } - }; - - private final ReduceFunction> sumFunction = new ReduceFunction>() { - @Override - public Tuple2 reduce(Tuple2 value1, Tuple2 value2) { - return new Tuple2<>(value1.f0, value1.f1 + value2.f1); - } - }; - - private final TypeSerializer> tupleSerializer = - new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO) - .createSerializer(new ExecutionConfig()); - - private final Comparator> tupleComparator = new Comparator>() { - @Override - public int compare(Tuple2 o1, Tuple2 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 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 op = - new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, - StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000); - - KeyedOneInputStreamOperatorTestHarness 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> op = - new AggregatingProcessingTimeWindowOperator<>( - sumFunction, fieldOneSelector, - IntSerializer.INSTANCE, tupleSerializer, - windowSize, windowSize); - - KeyedOneInputStreamOperatorTestHarness, Tuple2> 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> next = new StreamRecord<>(new Tuple2<>(i, i)); - testHarness.processElement(next); - currentTime = currentTime + 10; - testHarness.setProcessingTime(currentTime); - } - - // get and verify the result - List> 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> op = - new AggregatingProcessingTimeWindowOperator<>( - sumFunction, fieldOneSelector, - IntSerializer.INSTANCE, tupleSerializer, - windowSize, windowSize); - - KeyedOneInputStreamOperatorTestHarness, Tuple2> 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> 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> 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> 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> op = - new AggregatingProcessingTimeWindowOperator<>( - sumFunction, fieldOneSelector, - IntSerializer.INSTANCE, tupleSerializer, - 150, 50); - - KeyedOneInputStreamOperatorTestHarness, Tuple2> 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> next = new StreamRecord<>(new Tuple2<>(i, i)); - testHarness.processElement(next); - currentTime = currentTime + 1; - testHarness.setProcessingTime(currentTime); - } - - // get and verify the result - List> 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 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> op = - new AggregatingProcessingTimeWindowOperator<>( - sumFunction, fieldOneSelector, - IntSerializer.INSTANCE, tupleSerializer, 150, 50); - - KeyedOneInputStreamOperatorTestHarness, Tuple2> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(op, fieldOneSelector, BasicTypeInfo.INT_TYPE_INFO); - - testHarness.open(); - - testHarness.setProcessingTime(0); - - StreamRecord> next1 = new StreamRecord<>(new Tuple2<>(1, 1)); - testHarness.processElement(next1); - - StreamRecord> next2 = new StreamRecord<>(new Tuple2<>(2, 2)); - testHarness.processElement(next2); - - testHarness.setProcessingTime(50); - testHarness.setProcessingTime(100); - testHarness.setProcessingTime(150); - - List> 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> 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> op = - new AggregatingProcessingTimeWindowOperator<>( - failingFunction, fieldOneSelector, - IntSerializer.INSTANCE, tupleSerializer, - hundredYears, hundredYears); - - KeyedOneInputStreamOperatorTestHarness, Tuple2> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(op, fieldOneSelector, BasicTypeInfo.INT_TYPE_INFO); - - testHarness.open(); - - for (int i = 0; i < 100; i++) { - StreamRecord> next = new StreamRecord<>(new Tuple2<>(1, 1)); - testHarness.processElement(next); - } - - try { - StreamRecord> 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> op = - new AggregatingProcessingTimeWindowOperator<>( - sumFunction, fieldOneSelector, - IntSerializer.INSTANCE, tupleSerializer, - windowSize, windowSize); - - OneInputStreamOperatorTestHarness, Tuple2> 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> next = new StreamRecord<>(new Tuple2<>(i, i)); - testHarness.processElement(next); - } - - // draw a snapshot - List> 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> 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> next = new StreamRecord<>(new Tuple2<>(i, i)); - testHarness.processElement(next); - } - - testHarness.setProcessingTime(200); - - // get and verify the result - List> finalResult = new ArrayList<>(resultAtSnapshot); - List> 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> op = - new AggregatingProcessingTimeWindowOperator<>( - sumFunction, fieldOneSelector, - IntSerializer.INSTANCE, tupleSerializer, - windowSize, windowSlide); - - OneInputStreamOperatorTestHarness, Tuple2> 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> next = new StreamRecord<>(new Tuple2<>(i, i)); - testHarness.processElement(next); - } - - // draw a snapshot - List> 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> 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> 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> finalResult = new ArrayList<>(resultAtSnapshot); - List> 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> op = - new AggregatingProcessingTimeWindowOperator<>( - new StatefulFunction(), fieldOneSelector, - IntSerializer.INSTANCE, tupleSerializer, twoSeconds, twoSeconds); - - KeyedOneInputStreamOperatorTestHarness, Tuple2> 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> next1 = new StreamRecord<>(new Tuple2<>(1, i)); - testHarness.processElement(next1); - - StreamRecord> 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> op = - new AggregatingProcessingTimeWindowOperator<>( - new StatefulFunction(), fieldOneSelector, - IntSerializer.INSTANCE, tupleSerializer, windowSize, windowSlide); - - KeyedOneInputStreamOperatorTestHarness, Tuple2> 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> next1 = new StreamRecord<>(new Tuple2<>(1, i)); - StreamRecord> next2 = new StreamRecord<>(new Tuple2<>(2, i)); - StreamRecord> next3 = new StreamRecord<>(new Tuple2<>(1, i)); - StreamRecord> 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> { - - private final int failAfterElements; - - private int numElements; - - FailingFunction(int failAfterElements) { - this.failAfterElements = failAfterElements; - } - - @Override - public Tuple2 reduce(Tuple2 value1, Tuple2 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> { - - private static final Map globalCounts = new ConcurrentHashMap<>(); - - private ValueState 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 reduce(Tuple2 value1, Tuple2 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 List extractFromStreamRecords(Iterable input) { - List result = new ArrayList<>(); - for (Object in : input) { - if (in instanceof StreamRecord) { - result.add((T) ((StreamRecord) in).getValue()); - } - } - return result; - } -} 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 a7c6f473a248b..f967a5b2b8d01 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.util.Collector; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import java.util.concurrent.TimeUnit; @@ -310,31 +309,6 @@ public void testReduceProcessingTime() throws Exception { 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> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); - - DataStream> window = source - .windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS))) - .reduce(new DummyReducer()); - - OneInputTransformation, Tuple2> transform = - (OneInputTransformation, Tuple2>) window.getTransformation(); - OneInputStreamOperator, Tuple2> 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 { 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 dc0e21ca43d39..d525ba6d665f1 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.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.util.Collector; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import java.util.concurrent.TimeUnit; @@ -96,60 +92,6 @@ public void apply(Tuple tuple, 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> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); - - DummyReducer reducer = new DummyReducer(); - - DataStream> window1 = source - .keyBy(0) - .window(SlidingAlignedProcessingTimeWindows.of(Time.of(1000, TimeUnit.MILLISECONDS), Time.of(100, TimeUnit.MILLISECONDS))) - .reduce(reducer); - - OneInputTransformation, Tuple2> transform1 = (OneInputTransformation, Tuple2>) window1.getTransformation(); - OneInputStreamOperator, Tuple2> 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> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); - - DataStream> window1 = source - .keyBy(0) - .window(TumblingAlignedProcessingTimeWindows.of(Time.of(1000, TimeUnit.MILLISECONDS))) - .apply(new WindowFunction, Tuple2, Tuple, TimeWindow>() { - private static final long serialVersionUID = 1L; - - @Override - public void apply(Tuple tuple, - TimeWindow window, - Iterable> values, - Collector> out) throws Exception { - - } - }); - - OneInputTransformation, Tuple2> transform1 = (OneInputTransformation, Tuple2>) window1.getTransformation(); - OneInputStreamOperator, Tuple2> operator1 = transform1.getOperator(); - Assert.assertTrue(operator1 instanceof AccumulatingProcessingTimeWindowOperator); - } - @Test @SuppressWarnings("rawtypes") public void testReduceEventTimeWindows() throws Exception { @@ -232,49 +174,6 @@ public void apply(Tuple tuple, Assert.assertTrue(winOperator1.getStateDescriptor() instanceof ListStateDescriptor); } - /** - * These tests ensure that the fast aligned time windows operator is used if the - * conditions are right. - * - *

TODO: update once the fast aligned time windows operator is in - */ - @Ignore - @Test - public void testNonParallelFastTimeWindows() throws Exception { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStream> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); - - DummyReducer reducer = new DummyReducer(); - - DataStream> window1 = source - .timeWindowAll(Time.of(1000, TimeUnit.MILLISECONDS), - Time.of(100, TimeUnit.MILLISECONDS)) - .reduce(reducer); - - OneInputTransformation, Tuple2> transform1 = (OneInputTransformation, Tuple2>) window1.getTransformation(); - OneInputStreamOperator, Tuple2> operator1 = transform1.getOperator(); - Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator); - - DataStream> window2 = source - .timeWindowAll(Time.of(1000, TimeUnit.MILLISECONDS)) - .apply(new AllWindowFunction, Tuple2, TimeWindow>() { - private static final long serialVersionUID = 1L; - - @Override - public void apply( - TimeWindow window, - Iterable> values, - Collector> out) throws Exception { - - } - }); - - OneInputTransformation, Tuple2> transform2 = (OneInputTransformation, Tuple2>) window2.getTransformation(); - OneInputStreamOperator, Tuple2> operator2 = transform2.getOperator(); - Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator); - } - // ------------------------------------------------------------------------ // UDFs // ------------------------------------------------------------------------ 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 9f1906445ec6a..d7df479094044 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.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 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 void testRestoreApplyProcessingTimeWindows() throws Exception { testHarness.close(); } - /** - * Manually run this to write binary snapshot data. - */ - @Ignore - @Test - public void writeAggregatingAlignedProcessingTimeWindowsSnapshot() throws Exception { - TypeInformation> inputType = TypeInfoParser.parse("Tuple2"); - - AggregatingProcessingTimeWindowOperator> operator = - new AggregatingProcessingTimeWindowOperator<>( - new ReduceFunction>() { - private static final long serialVersionUID = -8913160567151867987L; - - @Override - public Tuple2 reduce(Tuple2 value1, Tuple2 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> 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> inputType = TypeInfoParser.parse("Tuple2"); - - ReducingStateDescriptor> stateDesc = new ReducingStateDescriptor<>("window-contents", - new SumReducer(), - inputType.createSerializer(new ExecutionConfig())); - - WindowOperator, Tuple2, Tuple2, 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>()), - ProcessingTimeTrigger.create(), - 0, - null /* late data output tag */, - LegacyWindowOperatorType.FAST_AGGREGATING); - - OneInputStreamOperatorTestHarness, Tuple2> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); - - ConcurrentLinkedQueue 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> inputType = TypeInfoParser.parse("Tuple2"); - - AccumulatingProcessingTimeWindowOperator, Tuple2> operator = - new AccumulatingProcessingTimeWindowOperator<>( - new InternalIterableWindowFunction<>(new WindowFunction, Tuple2, String, TimeWindow>() { - - private static final long serialVersionUID = 6551516443265733803L; - - @Override - public void apply(String s, TimeWindow window, Iterable> input, Collector> out) throws Exception { - int sum = 0; - for (Tuple2 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> 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> inputType = TypeInfoParser.parse("Tuple2"); - - ReducingStateDescriptor> stateDesc = new ReducingStateDescriptor<>("window-contents", - new SumReducer(), - inputType.createSerializer(new ExecutionConfig())); - - WindowOperator, Tuple2, Tuple2, 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>()), - ProcessingTimeTrigger.create(), - 0, - null /* late data output tag */, - LegacyWindowOperatorType.FAST_ACCUMULATING); - - OneInputStreamOperatorTestHarness, Tuple2> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); - - ConcurrentLinkedQueue 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, String> { private static final long serialVersionUID = 1L; 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 8748ed4da3b56..821438e993ef5 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.util.Collector; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import java.util.concurrent.TimeUnit; @@ -336,32 +335,6 @@ public void testReduceProcessingTime() throws Exception { 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> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); - - DataStream> window = source - .keyBy(new TupleKeySelector()) - .window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS))) - .reduce(new DummyReducer()); - - OneInputTransformation, Tuple2> transform = - (OneInputTransformation, Tuple2>) window.getTransformation(); - OneInputStreamOperator, Tuple2> 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 { 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 14ae733164133..f73499c13162b 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.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 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 @@ private void testRestoreWithInterrupt(int mode) throws Exception { 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 @@ private void testRestoreWithInterrupt(int mode) throws Exception { 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 @@ private void testRestoreWithInterrupt(int mode) throws Exception { // ------------------------------------------------------------------------ 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 keyedStateFromBackend = Collections.emptyList(); Collection keyedStateFromStream = Collections.emptyList(); Collection operatorStateBackend = Collections.emptyList(); @@ -206,7 +194,7 @@ private static Task createTask( Collections.singletonList(new OperatorStateHandle(operatorStateMetadata, state)); List keyedStateHandles = - Collections.singletonList(new KeyGroupsStateHandle(keyGroupRangeOffsets, state)); + Collections.singletonList(new KeyGroupsStateHandle(keyGroupRangeOffsets, state)); switch (mode) { case OPERATOR_MANAGED: @@ -221,15 +209,11 @@ private static Task createTask( 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 @@ private static Task createTask( 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.emptyList(), Collections.emptyList()); @@ -302,11 +285,11 @@ public FSDataInputStream openInputStream() throws IOException { 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,32 +341,14 @@ public long getStateSize() { // ------------------------------------------------------------------------ - private static class TestSourceLegacy implements SourceFunction, Checkpointed { + private static class TestSource implements SourceFunction, CheckpointedFunction { private static final long serialVersionUID = 1L; + private final int testType; - @Override - public void run(SourceContext 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, CheckpointedFunction { - private static final long serialVersionUID = 1L; - @Override public void run(SourceContext ctx) throws Exception { fail("should never be called"); @@ -399,6 +364,8 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { @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(); } } 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 3190620111b48..8d80d66caed93 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.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.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.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.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 @@ public class OneInputStreamTaskTest extends TestLogger { private static final ListStateDescriptor 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 void testOpenCloseAndTimestamps() throws Exception { 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 void testWatermarkAndStreamStatusForwarding() throws Exception { 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("Hello", initialTime)); @@ -215,7 +208,7 @@ public void testWatermarkAndStreamStatusForwarding() throws Exception { 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 void testWatermarksNotForwardedWithinChainWhenIdle() throws Exception { // ------------------ 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 void testWatermarksNotForwardedWithinChainWhenIdle() throws Exception { ))); 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 void testWatermarksNotForwardedWithinChainWhenIdle() throws Exception { 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 void testSnapshottingAndRestoring() throws Exception { 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 void testSnapshottingAndRestoring() throws Exception { 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 void testSnapshottingAndRestoring() throws Exception { 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 previousOperator = new TestingStreamOperator<>(random.nextLong(), recoveryTimestamp); + TestingStreamOperator previousOperator = new TestingStreamOperator<>(); streamConfig.setStreamOperator(previousOperator); streamConfig.setOperatorID(new OperatorID(0L, 0L)); @@ -643,7 +625,7 @@ private void configureChainedTestingStreamOperator( List outputEdges = new ArrayList<>(numberChainedTasks - 1); for (int chainedIndex = 1; chainedIndex < numberChainedTasks; chainedIndex++) { - TestingStreamOperator chainedOperator = new TestingStreamOperator<>(random.nextLong(), recoveryTimestamp); + TestingStreamOperator chainedOperator = new TestingStreamOperator<>(); StreamConfig chainedConfig = new StreamConfig(new Configuration()); chainedConfig.setStreamOperator(chainedOperator); chainedConfig.setOperatorID(new OperatorID(0L, chainedIndex)); @@ -702,17 +684,17 @@ public long getCheckpointId() { } 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 TaskStateSnapshot getCheckpointStateHandles() { } private static class TestingStreamOperator - extends AbstractStreamOperator - implements OneInputStreamOperator, StreamCheckpointedOperator { + extends AbstractStreamOperator + implements OneInputStreamOperator { 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 void open() throws Exception { @Override public void snapshotState(StateSnapshotContext context) throws Exception { ListState partitionableState = - getOperatorStateBackend().getListState(TEST_DESCRIPTOR); + getOperatorStateBackend().getListState(TEST_DESCRIPTOR); partitionableState.clear(); partitionableState.add(42); @@ -778,59 +755,14 @@ public void snapshotState(StateSnapshotContext context) throws Exception { @Override public void initializeState(StateInitializationContext context) throws Exception { - - } - - TestingStreamOperator(long seed, long recoveryTimestamp) { - this.seed = seed; - this.recoveryTimestamp = recoveryTimestamp; - } - - @Override - public void processElement(StreamRecord 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 element) throws Exception { - private Integer generateOperatorState() { - return random.nextInt(); } } @@ -913,8 +845,8 @@ protected void handleWatermark(Watermark mark) { *

If it receives a watermark when it's not expecting one, it'll throw an exception and fail. */ private static class TriggerableFailOnWatermarkTestOperator - extends AbstractStreamOperator - implements OneInputStreamOperator { + extends AbstractStreamOperator + implements OneInputStreamOperator { private static final long serialVersionUID = 2048954179291813243L; 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 08c3207b546e5..a2dc6c4f2fa0d 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.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.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 void testFailingCheckpointStreamOperator() throws Exception { 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 void testFailingCheckpointStreamOperator() throws Exception { 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 void testFailingCheckpointStreamOperator() throws Exception { verify(operatorSnapshotResult1).cancel(); verify(operatorSnapshotResult2).cancel(); - - verify(streamStateHandle1).discardState(); - verify(streamStateHandle2).discardState(); - verify(streamStateHandle3).discardState(); } /** @@ -384,12 +369,12 @@ public void testFailingAsyncCheckpointRunnable() throws Exception { 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 void testFailingAsyncCheckpointRunnable() throws Exception { 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 void testFailingAsyncCheckpointRunnable() throws Exception { verify(operatorSnapshotResult1).cancel(); verify(operatorSnapshotResult2).cancel(); verify(operatorSnapshotResult3).cancel(); - - verify(streamStateHandle1).discardState(); - verify(streamStateHandle2).discardState(); - verify(streamStateHandle3).discardState(); } /** @@ -481,7 +453,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { 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 void testAsyncCheckpointingConcurrentCloseBeforeAcknowledge() throws Exce whenNew(OperatorSubtaskState.class). withArguments( - any(StreamStateHandle.class), anyCollectionOf(OperatorStateHandle.class), anyCollectionOf(OperatorStateHandle.class), anyCollectionOf(KeyedStateHandle.class), @@ -593,11 +564,10 @@ public OperatorSubtaskState answer(InvocationOnMock invocation) throws Throwable 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 OperatorSubtaskState answer(InvocationOnMock invocation) throws Throwable 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 Object answer(InvocationOnMock invocationOnMock) throws Throwable { // 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); 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 720346a561c87..9156f3413e529 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.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.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.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.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 void setup(TypeSerializer outputSerializer) { 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 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.emptyList(), - Collections.emptyList(), - Collections.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 void initializeState(OperatorStateHandles operatorStateHandles) throws Ex numSubtasks).get(subtaskIndex); OperatorSubtaskState massagedOperatorStateHandles = new OperatorSubtaskState( - operatorStateHandles.getLegacyOperatorState(), nullToEmptyCollection(localManagedOperatorState), nullToEmptyCollection(localRawOperatorState), nullToEmptyCollection(localManagedKeyGroupState), @@ -473,7 +433,6 @@ public static OperatorStateHandles repackageState(OperatorStateHandles... handle return new OperatorStateHandles( 0, - null, mergedManagedKeyedState, mergedRawKeyedState, mergedManagedOperatorState, @@ -497,8 +456,6 @@ public void open() throws Exception { */ public OperatorStateHandles snapshot(long checkpointId, long timestamp) throws Exception { - CheckpointStreamFactory streamFactory = stateBackend.createStreamFactory(new JobID(), "test_op"); - OperatorSnapshotResult operatorStateResult = operator.snapshotState( checkpointId, timestamp, @@ -510,45 +467,14 @@ public OperatorStateHandles snapshot(long checkpointId, long timestamp) throws E 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, opRaw != null ? Collections.singletonList(opRaw) : null); } - /** - * 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)} ()}. */ @@ -556,22 +482,6 @@ public void notifyOfCompletedCheckpoint(long checkpointId) throws Exception { operator.notifyOfCompletedCheckpoint(checkpointId); } - /** - * 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. */ 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 0d42d9f1e3e50..c2ec63a6474d0 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.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 KeyedStateBackend answer(InvocationOnMock invocationOnMock) throws Throwa } } - /** - * - */ - @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 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 allKeyGroupsHandles) { for (KeyedStateHandle handle : allKeyGroupsHandles) { if (handle instanceof Migration) { 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 7e32723cbe02d..33f32e9f6bf5a 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 @@ 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 static void writeStateHandle(OperatorStateHandles state, String path) thr dos.writeInt(state.getOperatorChainIndex()); - SavepointV1Serializer.serializeStreamStateHandle(state.getLegacyOperatorState(), dos); + // still required for compatibility + SavepointV1Serializer.serializeStreamStateHandle(null, dos); Collection rawOperatorState = state.getRawOperatorState(); if (rawOperatorState != null) { @@ -108,7 +108,8 @@ public static OperatorStateHandles readStateHandle(String path) throws IOExcepti 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 rawOperatorState = null; int numRawOperatorStates = dis.readInt(); @@ -154,7 +155,12 @@ public static OperatorStateHandles readStateHandle(String path) throws IOExcepti } } - return new OperatorStateHandles(index, legacyState, managedKeyedState, rawKeyedState, managedOperatorState, rawOperatorState); + return new OperatorStateHandles( + index, + managedKeyedState, + rawKeyedState, + managedOperatorState, + rawOperatorState); } } } 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 f723b345fea65..1c95a047f7865 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)); } } 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 104400f1c192a..35a56d7019a7b 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 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 cad669364fdbe..99fb6ef80f643 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.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 void restoreState(List state) throws Exception { } } - private static class PartitionedStateSource extends StateSourceBase implements CheckpointedFunction, CheckpointedRestoring { + 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 void initializeState(FunctionInitializationContext context) throws Except checkCorrectRestore[getRuntimeContext().getIndexOfThisSubtask()] = counter; } } - - @Override - public void restoreState(Integer state) throws Exception { - counterPartitions.add(state); - } } } 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 cc235458c5e87..1b7dafab6e41a 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.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 @@ protected void run() { 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 @@ protected void run() { assertTrue(errMsg, resp.getClass() == getDisposeSavepointSuccess().getClass()); // - Verification START ------------------------------------------- - // The checkpoint files List checkpointFiles = new ArrayList<>(); for (OperatorState stateForTaskGroup : savepoint.getOperatorStates()) { for (OperatorSubtaskState subtaskState : stateForTaskGroup.getStates()) { - StreamStateHandle streamTaskState = subtaskState.getLegacyOperatorState(); + Collection 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())); + } } } } 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 21be7ba84993b..eccc7e906d629 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.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.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 @@ */ public class SavepointMigrationTestBase extends TestBaseUtils { + @BeforeClass + public static void before() { + SavepointSerializers.setFailWhenLegacyStateDetected(false); + } + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); 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 da6e035741671..0000000000000 --- 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. - * - *

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>() {}.getTypeInfo(), - new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator") - .addSink(new AccumulatorCountingSink>(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>() {}.getTypeInfo(), - new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator") - .addSink(new AccumulatorCountingSink>(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>() {}.getTypeInfo(), - new RestoringCheckingUdfOperator(new RestoringCheckingFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator") - .addSink(new AccumulatorCountingSink>(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>() {}.getTypeInfo(), - new RestoringCheckingUdfOperator(new RestoringCheckingFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator") - .addSink(new AccumulatorCountingSink>(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>, Checkpointed { - - 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> 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> - implements CheckpointedRestoring { - - 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> 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> - implements Checkpointed> { - - private static final long serialVersionUID = 1L; - - public static Tuple2 checkpointedTuple = - new Tuple2<>("hello", 42L); - - @Override - public void flatMap(Tuple2 value, Collector> out) throws Exception { - out.collect(value); - } - - @Override - public void restoreState(Tuple2 state) throws Exception { - } - - @Override - public Tuple2 snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { - return checkpointedTuple; - } - } - - private static class RestoringCheckingFlatMap extends RichFlatMapFunction, Tuple2> - implements CheckpointedRestoring> { - - private static final long serialVersionUID = 1L; - - private transient Tuple2 restoredState; - - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - - getRuntimeContext().addAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR, new IntCounter()); - } - - @Override - public void flatMap(Tuple2 value, Collector> out) throws Exception { - out.collect(value); - - assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState); - getRuntimeContext().getAccumulator(SUCCESSFUL_CHECK_ACCUMULATOR).add(1); - - } - - @Override - public void restoreState(Tuple2 state) throws Exception { - restoredState = state; - } - } - - private static class LegacyCheckpointedFlatMapWithKeyedState - extends RichFlatMapFunction, Tuple2> - implements Checkpointed> { - - private static final long serialVersionUID = 1L; - - public static Tuple2 checkpointedTuple = - new Tuple2<>("hello", 42L); - - private final ValueStateDescriptor stateDescriptor = - new ValueStateDescriptor("state-name", LongSerializer.INSTANCE); - - @Override - public void flatMap(Tuple2 value, Collector> out) throws Exception { - out.collect(value); - - getRuntimeContext().getState(stateDescriptor).update(value.f1); - } - - @Override - public void restoreState(Tuple2 state) throws Exception { - } - - @Override - public Tuple2 snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { - return checkpointedTuple; - } - } - - private static class RestoringCheckingFlatMapWithKeyedState extends RichFlatMapFunction, Tuple2> - implements CheckpointedRestoring> { - - private static final long serialVersionUID = 1L; - - private transient Tuple2 restoredState; - - private final ValueStateDescriptor stateDescriptor = - new ValueStateDescriptor("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 value, Collector> out) throws Exception { - out.collect(value); - - ValueState 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 state) throws Exception { - restoredState = state; - } - } - - private static class KeyedStateSettingFlatMap extends RichFlatMapFunction, Tuple2> { - - private static final long serialVersionUID = 1L; - - private final ValueStateDescriptor stateDescriptor = - new ValueStateDescriptor("state-name", LongSerializer.INSTANCE); - - @Override - public void flatMap(Tuple2 value, Collector> out) throws Exception { - out.collect(value); - - getRuntimeContext().getState(stateDescriptor).update(value.f1); - } - } - - private static class KeyedStateCheckingFlatMap extends RichFlatMapFunction, Tuple2> { - - private static final long serialVersionUID = 1L; - - private final ValueStateDescriptor stateDescriptor = - new ValueStateDescriptor("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 value, Collector> out) throws Exception { - out.collect(value); - - ValueState 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, FlatMapFunction, Tuple2>> - implements OneInputStreamOperator, Tuple2> { - private static final long serialVersionUID = 1L; - - private static final String CHECKPOINTED_STRING = "Oh my, that's nice!"; - - public CheckpointedUdfOperator(FlatMapFunction, Tuple2> userFunction) { - super(userFunction); - } - - @Override - public void processElement(StreamRecord> 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, FlatMapFunction, Tuple2>> - implements OneInputStreamOperator, Tuple2> { - private static final long serialVersionUID = 1L; - - private String restoredState; - - public RestoringCheckingUdfOperator(FlatMapFunction, Tuple2> userFunction) { - super(userFunction); - } - - @Override - public void open() throws Exception { - super.open(); - } - - @Override - public void processElement(StreamRecord> 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 extends RichSinkFunction { - 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); - } - } -} 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 7dd1144827167..6859c2d243b6b 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.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 @@ /** * This verifies that we can restore a complete job from a Flink 1.2 savepoint. * - *

The test pipeline contains both "Checkpointed" state and keyed user state. + *

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. * *

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 @@ protected String getRocksDBSavepointPath() { } private static class LegacyCheckpointedSource - implements SourceFunction>, Checkpointed { + implements SourceFunction> { public static String checkpointedString = "Here be dragons!"; @@ -283,21 +280,10 @@ public void run(SourceContext> ctx) throws Exception { 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> - implements CheckpointedRestoring { + extends RichSourceFunction> { private static final long serialVersionUID = 1L; @@ -322,7 +308,6 @@ public void open(Configuration parameters) throws Exception { @Override public void run(SourceContext> 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 void run(SourceContext> ctx) throws Exception { public void cancel() { isRunning = false; } - - @Override - public void restoreState(String state) throws Exception { - restoredState = state; - } } - private static class LegacyCheckpointedFlatMap extends RichFlatMapFunction, Tuple2> - implements Checkpointed> { + private static class LegacyCheckpointedFlatMap extends RichFlatMapFunction, Tuple2> { private static final long serialVersionUID = 1L; @@ -362,19 +341,9 @@ private static class LegacyCheckpointedFlatMap extends RichFlatMapFunction value, Collector> out) throws Exception { out.collect(value); } - - @Override - public void restoreState(Tuple2 state) throws Exception { - } - - @Override - public Tuple2 snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { - return checkpointedTuple; - } } - private static class CheckingRestoringFlatMap extends RichFlatMapFunction, Tuple2> - implements CheckpointedRestoring> { + private static class CheckingRestoringFlatMap extends RichFlatMapFunction, Tuple2> { private static final long serialVersionUID = 1L; @@ -393,20 +362,14 @@ public void open(Configuration parameters) throws Exception { public void flatMap(Tuple2 value, Collector> out) throws Exception { out.collect(value); - assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState); getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1); } - @Override - public void restoreState(Tuple2 state) throws Exception { - restoredState = state; - } } private static class LegacyCheckpointedFlatMapWithKeyedState - extends RichFlatMapFunction, Tuple2> - implements Checkpointed> { + extends RichFlatMapFunction, Tuple2> { private static final long serialVersionUID = 1L; @@ -424,19 +387,10 @@ public void flatMap(Tuple2 value, Collector> out) assertEquals(value.f1, getRuntimeContext().getState(stateDescriptor).value()); } - - @Override - public void restoreState(Tuple2 state) throws Exception { - } - - @Override - public Tuple2 snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { - return checkpointedTuple; - } } - private static class CheckingRestoringFlatMapWithKeyedState extends RichFlatMapFunction, Tuple2> - implements CheckpointedRestoring> { + private static class CheckingRestoringFlatMapWithKeyedState + extends RichFlatMapFunction, Tuple2> { private static final long serialVersionUID = 1L; @@ -464,18 +418,12 @@ public void flatMap(Tuple2 value, Collector> out) } assertEquals(value.f1, state.value()); - assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState); getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1); } - - @Override - public void restoreState(Tuple2 state) throws Exception { - restoredState = state; - } } - private static class CheckingRestoringFlatMapWithKeyedStateInOperator extends RichFlatMapFunction, Tuple2> - implements CheckpointedRestoring> { + private static class CheckingRestoringFlatMapWithKeyedStateInOperator + extends RichFlatMapFunction, Tuple2> { private static final long serialVersionUID = 1L; @@ -503,14 +451,8 @@ public void flatMap(Tuple2 value, Collector> out) } assertEquals(value.f1, state.value()); - assertEquals(LegacyCheckpointedFlatMap.checkpointedTuple, restoredState); getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1); } - - @Override - public void restoreState(Tuple2 state) throws Exception { - restoredState = state; - } } private static class KeyedStateSettingFlatMap extends RichFlatMapFunction, Tuple2> { @@ -578,17 +520,6 @@ public void processElement(StreamRecord> element) throws Exce 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 void open() throws Exception { @Override public void processElement(StreamRecord> 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 void processElement(StreamRecord> element) throws Exce 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 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 1431d9605b840..0000000000000 --- 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 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, Checkpointed { - - private static final long serialVersionUID = 3700033137820808611L; - - public boolean running = true; - - @Override - public void run(SourceContext 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, Checkpointed, 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 { - private static final long serialVersionUID = 2381410324190818620L; - - @Override - public void invoke(String value) throws Exception { - } - } -} 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 3d78242ac8993..00d0b2c130e48 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.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; @@ -88,6 +89,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();