This is an automated email from the ASF dual-hosted git repository. aljoscha pushed a commit to branch release-1.11 in repository https://gitbox.apache.org/repos/asf/flink.git
commit c967de544bdcbd3ea091ea562c99df77d50fff3f Author: klion26 <[email protected]> AuthorDate: Tue Jan 21 18:19:57 2020 +0800 [FLINK-13632] Port TimerSerializer test to TypeSerializerUpgradeTestBase --- .../api/operators/TimerHeapInternalTimer.java | 2 +- .../TimerSerializerSnapshotMigrationTest.java | 61 ------------ .../api/operators/TimerSerializerUpgradeTest.java | 107 +++++++++++++++++++++ .../test/resources/flink-1.6-timer-serializer-data | Bin 240 -> 0 bytes .../resources/flink-1.6-timer-serializer-snapshot | Bin 1406 -> 0 bytes .../test/resources/flink-1.7-timer-serializer-data | Bin 240 -> 0 bytes .../resources/flink-1.7-timer-serializer-snapshot | Bin 1414 -> 0 bytes 7 files changed, 108 insertions(+), 62 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java index a6194ed..e02901b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java @@ -50,7 +50,7 @@ public final class TimerHeapInternalTimer<K, N> implements InternalTimer<K, N>, */ private transient int timerHeapIndex; - TimerHeapInternalTimer(long timestamp, @Nonnull K key, @Nonnull N namespace) { + public TimerHeapInternalTimer(long timestamp, @Nonnull K key, @Nonnull N namespace) { this.timestamp = timestamp; this.key = key; this.namespace = namespace; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimerSerializerSnapshotMigrationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimerSerializerSnapshotMigrationTest.java deleted file mode 100644 index 6b07a09..0000000 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimerSerializerSnapshotMigrationTest.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.operators; - -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotMigrationTestBase; -import org.apache.flink.api.common.typeutils.base.IntSerializer; -import org.apache.flink.api.common.typeutils.base.StringSerializer; -import org.apache.flink.testutils.migration.MigrationVersion; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Collection; - -/** - * Migration test for {@link TimerSerializer}. - */ -@RunWith(Parameterized.class) -public class TimerSerializerSnapshotMigrationTest - extends TypeSerializerSnapshotMigrationTestBase<TimerHeapInternalTimer<String, Integer>> { - - public TimerSerializerSnapshotMigrationTest( - TestSpecification<TimerHeapInternalTimer<String, Integer>> testSpecification) { - super(testSpecification); - } - - @SuppressWarnings("unchecked") - @Parameterized.Parameters(name = "Test Specification = {0}") - public static Collection<TestSpecification<?>> testSpecifications() { - - final TestSpecifications testSpecifications = new TestSpecifications(MigrationVersion.v1_6, MigrationVersion.v1_7); - - testSpecifications.add( - "timer-serializer", - TimerSerializer.class, - TimerSerializerSnapshot.class, - TimerSerializerSnapshotMigrationTest::stringIntTimerSerializerSupplier); - - return testSpecifications.get(); - } - - private static TypeSerializer<TimerHeapInternalTimer<String, Integer>> stringIntTimerSerializerSupplier() { - return new TimerSerializer<>(StringSerializer.INSTANCE, IntSerializer.INSTANCE); - } -} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimerSerializerUpgradeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimerSerializerUpgradeTest.java new file mode 100644 index 0000000..21c12f4 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/TimerSerializerUpgradeTest.java @@ -0,0 +1,107 @@ +/* + * 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.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerMatchers; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.testutils.migration.MigrationVersion; + +import org.hamcrest.Matcher; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Collection; + +import static org.hamcrest.Matchers.is; + +/** + * Migration test for {@link TimerSerializer}. + */ +@RunWith(Parameterized.class) +public class TimerSerializerUpgradeTest + extends TypeSerializerUpgradeTestBase<TimerHeapInternalTimer<String, Integer>, TimerHeapInternalTimer<String, Integer>> { + + public TimerSerializerUpgradeTest( + TestSpecification<TimerHeapInternalTimer<String, Integer>, TimerHeapInternalTimer<String, Integer>> testSpecification) { + super(testSpecification); + } + + @SuppressWarnings("unchecked") + @Parameterized.Parameters(name = "Test Specification = {0}") + public static Collection<TestSpecification<?, ?>> testSpecifications() throws Exception { + + ArrayList<TestSpecification<?, ?>> testSpecifications = new ArrayList<>(); + for (MigrationVersion migrationVersion : MIGRATION_VERSIONS) { + testSpecifications.add( + new TestSpecification<>( + "timer-serializer", + migrationVersion, + TimerSerializerSetup.class, + TimerSerializerVerifier.class)); + } + return testSpecifications; + } + + private static TypeSerializer<TimerHeapInternalTimer<String, Integer>> stringIntTimerSerializerSupplier() { + return new TimerSerializer<>(StringSerializer.INSTANCE, IntSerializer.INSTANCE); + } + + // ---------------------------------------------------------------------------------------------- + // Specification for "TimerSerializer" + // ---------------------------------------------------------------------------------------------- + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class TimerSerializerSetup implements TypeSerializerUpgradeTestBase.PreUpgradeSetup<TimerHeapInternalTimer<String, Integer>> { + @Override + public TypeSerializer<TimerHeapInternalTimer<String, Integer>> createPriorSerializer() { + return new TimerSerializer<>(StringSerializer.INSTANCE, IntSerializer.INSTANCE); + } + + @Override + public TimerHeapInternalTimer<String, Integer> createTestData() { + return new TimerHeapInternalTimer<>(12345, "key", 678); + } + } + + /** + * This class is only public to work with {@link org.apache.flink.api.common.typeutils.ClassRelocator}. + */ + public static final class TimerSerializerVerifier implements TypeSerializerUpgradeTestBase.UpgradeVerifier<TimerHeapInternalTimer<String, Integer>> { + @Override + public TypeSerializer<TimerHeapInternalTimer<String, Integer>> createUpgradedSerializer() { + return new TimerSerializer<>(StringSerializer.INSTANCE, IntSerializer.INSTANCE); + } + + @Override + public Matcher<TimerHeapInternalTimer<String, Integer>> testDataMatcher() { + return is(new TimerHeapInternalTimer<>(12345, "key", 678)); + } + + @Override + public Matcher<TypeSerializerSchemaCompatibility<TimerHeapInternalTimer<String, Integer>>> schemaCompatibilityMatcher(MigrationVersion version) { + return TypeSerializerMatchers.isCompatibleAsIs(); + } + } +} diff --git a/flink-streaming-java/src/test/resources/flink-1.6-timer-serializer-data b/flink-streaming-java/src/test/resources/flink-1.6-timer-serializer-data deleted file mode 100644 index 02f84d9..0000000 Binary files a/flink-streaming-java/src/test/resources/flink-1.6-timer-serializer-data and /dev/null differ diff --git a/flink-streaming-java/src/test/resources/flink-1.6-timer-serializer-snapshot b/flink-streaming-java/src/test/resources/flink-1.6-timer-serializer-snapshot deleted file mode 100644 index fee77f2..0000000 Binary files a/flink-streaming-java/src/test/resources/flink-1.6-timer-serializer-snapshot and /dev/null differ diff --git a/flink-streaming-java/src/test/resources/flink-1.7-timer-serializer-data b/flink-streaming-java/src/test/resources/flink-1.7-timer-serializer-data deleted file mode 100644 index 9069def..0000000 Binary files a/flink-streaming-java/src/test/resources/flink-1.7-timer-serializer-data and /dev/null differ diff --git a/flink-streaming-java/src/test/resources/flink-1.7-timer-serializer-snapshot b/flink-streaming-java/src/test/resources/flink-1.7-timer-serializer-snapshot deleted file mode 100644 index d5f3fda..0000000 Binary files a/flink-streaming-java/src/test/resources/flink-1.7-timer-serializer-snapshot and /dev/null differ
