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 b139cb7df0c17bb275a6bac64c1317c93d5fe5a4
Author: klion26 <[email protected]>
AuthorDate: Sat Feb 15 11:46:43 2020 +0800

    [FLINK-13632] Port UnionSerializer upgrade test to 
TypeSerializerUpgradeTestBase
---
 .../streaming/api/datastream/CoGroupedStreams.java |  20 ++-
 .../datastream/UnionSerializerMigrationTest.java   |  67 ---------
 .../api/datastream/UnionSerializerUpgradeTest.java | 154 +++++++++++++++++++++
 .../test/resources/flink-1.6-union-serializer-data |   1 -
 .../resources/flink-1.6-union-serializer-snapshot  | Bin 1393 -> 0 bytes
 .../test/resources/flink-1.7-union-serializer-data |   1 -
 .../resources/flink-1.7-union-serializer-snapshot  | Bin 1403 -> 0 bytes
 7 files changed, 173 insertions(+), 70 deletions(-)

diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
index 3c30bb9..d60a465 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
@@ -50,6 +50,7 @@ import org.apache.flink.util.Preconditions;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
 import static java.util.Objects.requireNonNull;
 
@@ -425,6 +426,20 @@ public class CoGroupedStreams<T1, T2> {
                public static <T1, T2> TaggedUnion<T1, T2> two(T2 two) {
                        return new TaggedUnion<>(null, two);
                }
+
+               @Override
+               public boolean equals(Object obj) {
+                       if (obj == this) {
+                               return true;
+                       }
+
+                       if (!(obj instanceof TaggedUnion)) {
+                               return false;
+                       }
+
+                       TaggedUnion other = (TaggedUnion) obj;
+                       return Objects.equals(one, other.one) && 
Objects.equals(two, other.two);
+               }
        }
 
        private static class UnionTypeInfo<T1, T2> extends 
TypeInformation<TaggedUnion<T1, T2>> {
@@ -503,9 +518,12 @@ public class CoGroupedStreams<T1, T2> {
                }
        }
 
+       /**
+        * {@link TypeSerializer} for {@link TaggedUnion}.
+        */
        @VisibleForTesting
        @Internal
-       static class UnionSerializer<T1, T2> extends 
TypeSerializer<TaggedUnion<T1, T2>> {
+       public static class UnionSerializer<T1, T2> extends 
TypeSerializer<TaggedUnion<T1, T2>> {
                private static final long serialVersionUID = 1L;
 
                private final TypeSerializer<T1> oneSerializer;
diff --git 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/UnionSerializerMigrationTest.java
 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/UnionSerializerMigrationTest.java
deleted file mode 100644
index 2b85d7e..0000000
--- 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/UnionSerializerMigrationTest.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.streaming.api.datastream;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import 
org.apache.flink.api.common.typeutils.TypeSerializerSnapshotMigrationTestBase;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.streaming.api.datastream.CoGroupedStreams.TaggedUnion;
-import 
org.apache.flink.streaming.api.datastream.CoGroupedStreams.UnionSerializer;
-import 
org.apache.flink.streaming.api.datastream.CoGroupedStreams.UnionSerializerSnapshot;
-import org.apache.flink.testutils.migration.MigrationVersion;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Collection;
-
-/**
- * State migration tests for {@link UnionSerializer}.
- */
-@RunWith(Parameterized.class)
-public class UnionSerializerMigrationTest extends 
TypeSerializerSnapshotMigrationTestBase<TaggedUnion<String, Long>> {
-
-       public 
UnionSerializerMigrationTest(TestSpecification<TaggedUnion<String, Long>> 
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(
-                       "union-serializer",
-                       UnionSerializer.class,
-                       UnionSerializerSnapshot.class,
-                       UnionSerializerMigrationTest::stringLongRowSupplier);
-
-               return testSpecifications.get();
-       }
-
-       private static TypeSerializer<TaggedUnion<String, Long>> 
stringLongRowSupplier() {
-               return new UnionSerializer<>(StringSerializer.INSTANCE, 
LongSerializer.INSTANCE);
-       }
-
-}
-
-
-
diff --git 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/UnionSerializerUpgradeTest.java
 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/UnionSerializerUpgradeTest.java
new file mode 100644
index 0000000..a767797
--- /dev/null
+++ 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/UnionSerializerUpgradeTest.java
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+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.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.streaming.api.datastream.CoGroupedStreams.TaggedUnion;
+import 
org.apache.flink.streaming.api.datastream.CoGroupedStreams.UnionSerializer;
+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;
+
+/**
+ * A {@link TypeSerializerUpgradeTestBase} for {@link UnionSerializer}.
+ */
+@RunWith(Parameterized.class)
+public class UnionSerializerUpgradeTest extends 
TypeSerializerUpgradeTestBase<TaggedUnion<String, Long>, TaggedUnion<String, 
Long>> {
+
+       public UnionSerializerUpgradeTest(TestSpecification<TaggedUnion<String, 
Long>, TaggedUnion<String, Long>> testSpecification) {
+               super(testSpecification);
+       }
+
+       @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<>(
+                                       "union-serializer-one",
+                                       migrationVersion,
+                                       UnionSerializerOneSetup.class,
+                                       UnionSerializerOneVerifier.class));
+                       testSpecifications.add(
+                               new TestSpecification<>(
+                                       "union-serializer-two",
+                                       migrationVersion,
+                                       UnionSerializerTwoSetup.class,
+                                       UnionSerializerTwoVerifier.class));
+               }
+               return testSpecifications;
+       }
+
+       private static TypeSerializer<TaggedUnion<String, Long>> 
stringLongRowSupplier() {
+               return new UnionSerializer<>(StringSerializer.INSTANCE, 
LongSerializer.INSTANCE);
+       }
+
+       // 
----------------------------------------------------------------------------------------------
+       //  Specification for "union-serializer-for-TaggedUnion.one"
+       // 
----------------------------------------------------------------------------------------------
+
+       /**
+        * This class is only public to work with {@link 
org.apache.flink.api.common.typeutils.ClassRelocator}.
+        */
+       public static final class UnionSerializerOneSetup implements 
TypeSerializerUpgradeTestBase.PreUpgradeSetup<TaggedUnion<String, Long>> {
+               @Override
+               public TypeSerializer<TaggedUnion<String, Long>> 
createPriorSerializer() {
+                       return new UnionSerializer<>(StringSerializer.INSTANCE, 
LongSerializer.INSTANCE);
+               }
+
+               @Override
+               public TaggedUnion<String, Long> createTestData() {
+                       return TaggedUnion.one("flink");
+               }
+       }
+
+       /**
+        * This class is only public to work with {@link 
org.apache.flink.api.common.typeutils.ClassRelocator}.
+        */
+       public static final class UnionSerializerOneVerifier implements 
TypeSerializerUpgradeTestBase.UpgradeVerifier<TaggedUnion<String, Long>> {
+               @Override
+               public TypeSerializer<TaggedUnion<String, Long>> 
createUpgradedSerializer() {
+                       return new UnionSerializer<>(StringSerializer.INSTANCE, 
LongSerializer.INSTANCE);
+               }
+
+               @Override
+               public Matcher<TaggedUnion<String, Long>> testDataMatcher() {
+                       return is(TaggedUnion.one("flink"));
+               }
+
+               @Override
+               public 
Matcher<TypeSerializerSchemaCompatibility<TaggedUnion<String, Long>>> 
schemaCompatibilityMatcher(MigrationVersion version) {
+                       return TypeSerializerMatchers.isCompatibleAsIs();
+               }
+       }
+
+       // 
----------------------------------------------------------------------------------------------
+       //  Specification for "union-serializer-for-TaggedUnion.two"
+       // 
----------------------------------------------------------------------------------------------
+
+       /**
+        * This class is only public to work with {@link 
org.apache.flink.api.common.typeutils.ClassRelocator}.
+        */
+       public static final class UnionSerializerTwoSetup implements 
TypeSerializerUpgradeTestBase.PreUpgradeSetup<TaggedUnion<String, Long>> {
+               @Override
+               public TypeSerializer<TaggedUnion<String, Long>> 
createPriorSerializer() {
+                       return new UnionSerializer<>(StringSerializer.INSTANCE, 
LongSerializer.INSTANCE);
+               }
+
+               @Override
+               public TaggedUnion<String, Long> createTestData() {
+                       return TaggedUnion.two(23456L);
+               }
+       }
+
+       /**
+        * This class is only public to work with {@link 
org.apache.flink.api.common.typeutils.ClassRelocator}.
+        */
+       public static final class UnionSerializerTwoVerifier implements 
TypeSerializerUpgradeTestBase.UpgradeVerifier<TaggedUnion<String, Long>> {
+               @Override
+               public TypeSerializer<TaggedUnion<String, Long>> 
createUpgradedSerializer() {
+                       return new UnionSerializer<>(StringSerializer.INSTANCE, 
LongSerializer.INSTANCE);
+               }
+
+               @Override
+               public Matcher<TaggedUnion<String, Long>> testDataMatcher() {
+                       return is(TaggedUnion.two(23456L));
+               }
+
+               @Override
+               public 
Matcher<TypeSerializerSchemaCompatibility<TaggedUnion<String, Long>>> 
schemaCompatibilityMatcher(MigrationVersion version) {
+                       return TypeSerializerMatchers.isCompatibleAsIs();
+               }
+       }
+}
+
+
+
diff --git 
a/flink-streaming-java/src/test/resources/flink-1.6-union-serializer-data 
b/flink-streaming-java/src/test/resources/flink-1.6-union-serializer-data
deleted file mode 100644
index cb29a99..0000000
--- a/flink-streaming-java/src/test/resources/flink-1.6-union-serializer-data
+++ /dev/null
@@ -1 +0,0 @@
-53778725243338537787315927955377873178348653778731961974537787321387515377873232740853778732510096537787327007985377873288617553778733069270
\ No newline at end of file
diff --git 
a/flink-streaming-java/src/test/resources/flink-1.6-union-serializer-snapshot 
b/flink-streaming-java/src/test/resources/flink-1.6-union-serializer-snapshot
deleted file mode 100644
index 178d007..0000000
Binary files 
a/flink-streaming-java/src/test/resources/flink-1.6-union-serializer-snapshot 
and /dev/null differ
diff --git 
a/flink-streaming-java/src/test/resources/flink-1.7-union-serializer-data 
b/flink-streaming-java/src/test/resources/flink-1.7-union-serializer-data
deleted file mode 100644
index d2f04fd..0000000
--- a/flink-streaming-java/src/test/resources/flink-1.7-union-serializer-data
+++ /dev/null
@@ -1 +0,0 @@
-53711258937562537112614539985371126164162753711261837231537112620123325371126218543153711262413321537112625282785371126263382753711262735393
\ No newline at end of file
diff --git 
a/flink-streaming-java/src/test/resources/flink-1.7-union-serializer-snapshot 
b/flink-streaming-java/src/test/resources/flink-1.7-union-serializer-snapshot
deleted file mode 100644
index 747eb82..0000000
Binary files 
a/flink-streaming-java/src/test/resources/flink-1.7-union-serializer-snapshot 
and /dev/null differ

Reply via email to