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 04cd45a8cb0a03410ac170b65a11b32cd206f7b8
Author: klion26 <qcx978132...@gmail.com>
AuthorDate: Wed Jan 22 11:14:29 2020 +0800

    [FLINK-13632] Port StreamElementSerializer test to 
TypeSerializerUpgradeTestBase
---
 .../StreamElementSerializerMigrationTest.java      |  55 -----------
 .../StreamElementSerializerUpgradeTest.java        | 103 +++++++++++++++++++++
 .../flink-1.6-stream-element-serializer-data       | Bin 158 -> 0 bytes
 .../flink-1.6-stream-element-serializer-snapshot   | Bin 931 -> 0 bytes
 .../flink-1.7-stream-element-serializer-data       | Bin 158 -> 0 bytes
 .../flink-1.7-stream-element-serializer-snapshot   | Bin 932 -> 0 bytes
 6 files changed, 103 insertions(+), 55 deletions(-)

diff --git 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializerMigrationTest.java
 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializerMigrationTest.java
deleted file mode 100644
index b6169d4..0000000
--- 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializerMigrationTest.java
+++ /dev/null
@@ -1,55 +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.streamrecord;
-
-import 
org.apache.flink.api.common.typeutils.TypeSerializerSnapshotMigrationTestBase;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import 
org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer.StreamElementSerializerSnapshot;
-import org.apache.flink.testutils.migration.MigrationVersion;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Collection;
-
-/**
- * Migration tests for {@link StreamElementSerializer}.
- */
-@RunWith(Parameterized.class)
-public class StreamElementSerializerMigrationTest extends 
TypeSerializerSnapshotMigrationTestBase<StreamElement> {
-
-       public 
StreamElementSerializerMigrationTest(TestSpecification<StreamElement> 
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(
-                       "stream-element-serializer",
-                       StreamElementSerializer.class,
-                       StreamElementSerializerSnapshot.class,
-                       () -> new 
StreamElementSerializer<>(StringSerializer.INSTANCE));
-
-               return testSpecifications.get();
-       }
-}
diff --git 
a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializerUpgradeTest.java
 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializerUpgradeTest.java
new file mode 100644
index 0000000..9d02c60
--- /dev/null
+++ 
b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializerUpgradeTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.streamrecord;
+
+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.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.apache.flink.streaming.util.StreamRecordMatchers.streamRecord;
+import static org.hamcrest.Matchers.is;
+
+/**
+ * Migration tests for {@link StreamElementSerializer}.
+ */
+@RunWith(Parameterized.class)
+public class StreamElementSerializerUpgradeTest extends 
TypeSerializerUpgradeTestBase<StreamElement, StreamElement> {
+
+       public 
StreamElementSerializerUpgradeTest(TestSpecification<StreamElement, 
StreamElement> 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<>(
+                                       "stream-element-serializer",
+                                       migrationVersion,
+                                       StreamElementSetup.class,
+                                       StreamElementVerifier.class));
+               }
+
+               return testSpecifications;
+       }
+
+       // 
----------------------------------------------------------------------------------------------
+       //  Specification for "StreamElement-serializer"
+       // 
----------------------------------------------------------------------------------------------
+
+       /**
+        * This class is only public to work with {@link 
org.apache.flink.api.common.typeutils.ClassRelocator}.
+        */
+       public static final class StreamElementSetup implements 
TypeSerializerUpgradeTestBase.PreUpgradeSetup<StreamElement> {
+               @Override
+               public TypeSerializer<StreamElement> createPriorSerializer() {
+                       return new 
StreamElementSerializer<>(StringSerializer.INSTANCE);
+               }
+
+               @Override
+               public StreamElement createTestData() {
+                       return new StreamRecord<>("key", 123456);
+               }
+       }
+
+       /**
+        * This class is only public to work with {@link 
org.apache.flink.api.common.typeutils.ClassRelocator}.
+        */
+       public static final class StreamElementVerifier implements 
TypeSerializerUpgradeTestBase.UpgradeVerifier<StreamElement> {
+               @Override
+               public TypeSerializer<StreamElement> createUpgradedSerializer() 
{
+                       return new 
StreamElementSerializer<>(StringSerializer.INSTANCE);
+               }
+
+               @SuppressWarnings({"unchecked", "rawtypes"})
+               @Override
+               public Matcher<StreamElement> testDataMatcher() {
+                       return (Matcher) streamRecord(is("key"), is(123456L));
+               }
+
+               @Override
+               public 
Matcher<TypeSerializerSchemaCompatibility<StreamElement>> 
schemaCompatibilityMatcher(MigrationVersion version) {
+                       return TypeSerializerMatchers.isCompatibleAsIs();
+               }
+       }
+}
diff --git 
a/flink-streaming-java/src/test/resources/flink-1.6-stream-element-serializer-data
 
b/flink-streaming-java/src/test/resources/flink-1.6-stream-element-serializer-data
deleted file mode 100644
index 81b80c3..0000000
Binary files 
a/flink-streaming-java/src/test/resources/flink-1.6-stream-element-serializer-data
 and /dev/null differ
diff --git 
a/flink-streaming-java/src/test/resources/flink-1.6-stream-element-serializer-snapshot
 
b/flink-streaming-java/src/test/resources/flink-1.6-stream-element-serializer-snapshot
deleted file mode 100644
index 8ffdb43..0000000
Binary files 
a/flink-streaming-java/src/test/resources/flink-1.6-stream-element-serializer-snapshot
 and /dev/null differ
diff --git 
a/flink-streaming-java/src/test/resources/flink-1.7-stream-element-serializer-data
 
b/flink-streaming-java/src/test/resources/flink-1.7-stream-element-serializer-data
deleted file mode 100644
index 01f05e7..0000000
Binary files 
a/flink-streaming-java/src/test/resources/flink-1.7-stream-element-serializer-data
 and /dev/null differ
diff --git 
a/flink-streaming-java/src/test/resources/flink-1.7-stream-element-serializer-snapshot
 
b/flink-streaming-java/src/test/resources/flink-1.7-stream-element-serializer-snapshot
deleted file mode 100644
index dc7f76b..0000000
Binary files 
a/flink-streaming-java/src/test/resources/flink-1.7-stream-element-serializer-snapshot
 and /dev/null differ

Reply via email to