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 bd37c3c27f47d7d163ca4ca9203a47b738497ddf
Author: klion26 <[email protected]>
AuthorDate: Fri Feb 21 19:51:18 2020 +0800

    [FLINK-13632] Port Kafka011Serializer upgrade test to 
TypeSerializerUpgradeTestBase
---
 .../connectors/kafka/FlinkKafkaProducer011.java    |  18 ++-
 .../kafka/Kafka011SerializerUpgradeTest.java       | 162 +++++++++++++++++++++
 .../KafkaSerializerSnapshotsMigrationTest.java     |  59 --------
 .../flink-1.6-context-state-serializer-data        | Bin 270 -> 0 bytes
 .../flink-1.6-context-state-serializer-snapshot    | Bin 452 -> 0 bytes
 .../flink-1.6-transaction-state-serializer-data    | Bin 110 -> 0 bytes
 ...flink-1.6-transaction-state-serializer-snapshot | Bin 460 -> 0 bytes
 .../flink-1.7-context-state-serializer-data        | Bin 270 -> 0 bytes
 .../flink-1.7-context-state-serializer-snapshot    | Bin 440 -> 0 bytes
 .../flink-1.7-transaction-state-serializer-data    | Bin 110 -> 0 bytes
 ...flink-1.7-transaction-state-serializer-snapshot | Bin 448 -> 0 bytes
 11 files changed, 175 insertions(+), 64 deletions(-)

diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 
b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
index 32bd558..f8642f9 100644
--- 
a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
+++ 
b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
@@ -1083,7 +1083,7 @@ public class FlinkKafkaProducer011<IN>
         */
        @VisibleForTesting
        @Internal
-       static class KafkaTransactionState {
+       public static class KafkaTransactionState {
 
                private final transient FlinkKafkaProducer<byte[], byte[]> 
producer;
 
@@ -1094,15 +1094,21 @@ public class FlinkKafkaProducer011<IN>
 
                final short epoch;
 
-               KafkaTransactionState(String transactionalId, 
FlinkKafkaProducer<byte[], byte[]> producer) {
+               @VisibleForTesting
+               @Internal
+               public KafkaTransactionState(String transactionalId, 
FlinkKafkaProducer<byte[], byte[]> producer) {
                        this(transactionalId, producer.getProducerId(), 
producer.getEpoch(), producer);
                }
 
-               KafkaTransactionState(FlinkKafkaProducer<byte[], byte[]> 
producer) {
+               @VisibleForTesting
+               @Internal
+               public KafkaTransactionState(FlinkKafkaProducer<byte[], byte[]> 
producer) {
                        this(null, -1, (short) -1, producer);
                }
 
-               KafkaTransactionState(
+               @VisibleForTesting
+               @Internal
+               public KafkaTransactionState(
                                @Nullable String transactionalId,
                                long producerId,
                                short epoch,
@@ -1165,7 +1171,9 @@ public class FlinkKafkaProducer011<IN>
        public static class KafkaTransactionContext {
                final Set<String> transactionalIds;
 
-               KafkaTransactionContext(Set<String> transactionalIds) {
+               @VisibleForTesting
+               @Internal
+               public KafkaTransactionContext(Set<String> transactionalIds) {
                        checkNotNull(transactionalIds);
                        this.transactionalIds = transactionalIds;
                }
diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011SerializerUpgradeTest.java
 
b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011SerializerUpgradeTest.java
new file mode 100644
index 0000000..2d1144b
--- /dev/null
+++ 
b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011SerializerUpgradeTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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.kafka;
+
+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.streaming.connectors.kafka.internal.FlinkKafkaProducer;
+import org.apache.flink.testutils.migration.MigrationVersion;
+
+import org.hamcrest.Matcher;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.hamcrest.Matchers.is;
+
+/**
+ * A {@link TypeSerializerUpgradeTestBase} for the {@link 
FlinkKafkaProducer011.TransactionStateSerializer}
+ * and {@link FlinkKafkaProducer011.ContextStateSerializer}.
+ */
+@RunWith(Parameterized.class)
+public class Kafka011SerializerUpgradeTest extends 
TypeSerializerUpgradeTestBase<Object, Object> {
+
+       public Kafka011SerializerUpgradeTest(TestSpecification<Object, Object> 
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<>(
+                                       "transaction-state-serializer",
+                                       migrationVersion,
+                                       TransactionStateSerializerSetup.class,
+                                       
TransactionStateSerializerVerifier.class));
+                       testSpecifications.add(
+                               new TestSpecification<>(
+                                       "context-state-serializer",
+                                       migrationVersion,
+                                       ContextStateSerializerSetup.class,
+                                       ContextStateSerializerVerifier.class));
+               }
+               return testSpecifications;
+       }
+
+       // 
----------------------------------------------------------------------------------------------
+       //  Specification for "transaction-state-serializer"
+       // 
----------------------------------------------------------------------------------------------
+
+       /**
+        * This class is only public to work with {@link 
org.apache.flink.api.common.typeutils.ClassRelocator}.
+        */
+       public static final class TransactionStateSerializerSetup implements 
TypeSerializerUpgradeTestBase.PreUpgradeSetup<FlinkKafkaProducer011.KafkaTransactionState>
 {
+               @Override
+               public 
TypeSerializer<FlinkKafkaProducer011.KafkaTransactionState> 
createPriorSerializer() {
+                       return new 
FlinkKafkaProducer011.TransactionStateSerializer();
+               }
+
+               @Override
+               public FlinkKafkaProducer011.KafkaTransactionState 
createTestData() {
+                       @SuppressWarnings("unchecked")
+                       FlinkKafkaProducer<byte[], byte[]> mock = 
Mockito.mock(FlinkKafkaProducer.class);
+                       return new 
FlinkKafkaProducer011.KafkaTransactionState("1234", 3456, (short) 789, mock);
+               }
+       }
+
+       /**
+        * This class is only public to work with {@link 
org.apache.flink.api.common.typeutils.ClassRelocator}.
+        */
+       public static final class TransactionStateSerializerVerifier implements 
TypeSerializerUpgradeTestBase.UpgradeVerifier<FlinkKafkaProducer011.KafkaTransactionState>
 {
+               @Override
+               public 
TypeSerializer<FlinkKafkaProducer011.KafkaTransactionState> 
createUpgradedSerializer() {
+                       return new 
FlinkKafkaProducer011.TransactionStateSerializer();
+               }
+
+               @Override
+               public Matcher<FlinkKafkaProducer011.KafkaTransactionState> 
testDataMatcher() {
+                       @SuppressWarnings("unchecked")
+                       FlinkKafkaProducer<byte[], byte[]> mock = 
Mockito.mock(FlinkKafkaProducer.class);
+                       return is(new 
FlinkKafkaProducer011.KafkaTransactionState("1234", 3456, (short) 789, mock));
+               }
+
+               @Override
+               public 
Matcher<TypeSerializerSchemaCompatibility<FlinkKafkaProducer011.KafkaTransactionState>>
 schemaCompatibilityMatcher(MigrationVersion version) {
+                       return TypeSerializerMatchers.isCompatibleAsIs();
+               }
+       }
+
+       // 
----------------------------------------------------------------------------------------------
+       //  Specification for "context-state-serializer"
+       // 
----------------------------------------------------------------------------------------------
+
+       /**
+        * This class is only public to work with {@link 
org.apache.flink.api.common.typeutils.ClassRelocator}.
+        */
+       public static final class ContextStateSerializerSetup implements 
TypeSerializerUpgradeTestBase.PreUpgradeSetup<FlinkKafkaProducer011.KafkaTransactionContext>
 {
+               @Override
+               public 
TypeSerializer<FlinkKafkaProducer011.KafkaTransactionContext> 
createPriorSerializer() {
+                       return new 
FlinkKafkaProducer011.ContextStateSerializer();
+               }
+
+               @Override
+               public FlinkKafkaProducer011.KafkaTransactionContext 
createTestData() {
+                       Set<String> transactionIds = new HashSet<>();
+                       transactionIds.add("123");
+                       transactionIds.add("456");
+                       transactionIds.add("789");
+                       return new 
FlinkKafkaProducer011.KafkaTransactionContext(transactionIds);
+               }
+       }
+
+       /**
+        * This class is only public to work with {@link 
org.apache.flink.api.common.typeutils.ClassRelocator}.
+        */
+       public static final class ContextStateSerializerVerifier implements 
TypeSerializerUpgradeTestBase.UpgradeVerifier<FlinkKafkaProducer011.KafkaTransactionContext>
 {
+               @Override
+               public 
TypeSerializer<FlinkKafkaProducer011.KafkaTransactionContext> 
createUpgradedSerializer() {
+                       return new 
FlinkKafkaProducer011.ContextStateSerializer();
+               }
+
+               @Override
+               public Matcher<FlinkKafkaProducer011.KafkaTransactionContext> 
testDataMatcher() {
+                       Set<String> transactionIds = new HashSet<>();
+                       transactionIds.add("123");
+                       transactionIds.add("456");
+                       transactionIds.add("789");
+                       return is(new 
FlinkKafkaProducer011.KafkaTransactionContext(transactionIds));
+               }
+
+               @Override
+               public 
Matcher<TypeSerializerSchemaCompatibility<FlinkKafkaProducer011.KafkaTransactionContext>>
 schemaCompatibilityMatcher(MigrationVersion version) {
+                       return TypeSerializerMatchers.isCompatibleAsIs();
+               }
+       }
+}
diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerSnapshotsMigrationTest.java
 
b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerSnapshotsMigrationTest.java
deleted file mode 100644
index 85272c8..0000000
--- 
a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerSnapshotsMigrationTest.java
+++ /dev/null
@@ -1,59 +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.kafka;
-
-import 
org.apache.flink.api.common.typeutils.TypeSerializerSnapshotMigrationTestBase;
-import org.apache.flink.testutils.migration.MigrationVersion;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.Collection;
-
-/**
- * Migration tests for the {@link 
FlinkKafkaProducer011.TransactionStateSerializer}
- * and {@link FlinkKafkaProducer011.ContextStateSerializer}.
- */
-@RunWith(Parameterized.class)
-public class KafkaSerializerSnapshotsMigrationTest extends 
TypeSerializerSnapshotMigrationTestBase<Object> {
-
-       public KafkaSerializerSnapshotsMigrationTest(TestSpecification<Object> 
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(
-                       "transaction-state-serializer",
-                       FlinkKafkaProducer011.TransactionStateSerializer.class,
-                       
FlinkKafkaProducer011.TransactionStateSerializer.TransactionStateSerializerSnapshot.class,
-                       FlinkKafkaProducer011.TransactionStateSerializer::new);
-               testSpecifications.add(
-                       "context-state-serializer",
-                       FlinkKafkaProducer011.ContextStateSerializer.class,
-                       
FlinkKafkaProducer011.ContextStateSerializer.ContextStateSerializerSnapshot.class,
-                       FlinkKafkaProducer011.ContextStateSerializer::new);
-
-               return testSpecifications.get();
-       }
-}
diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-context-state-serializer-data
 
b/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-context-state-serializer-data
deleted file mode 100644
index e7d439a..0000000
Binary files 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-context-state-serializer-data
 and /dev/null differ
diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-context-state-serializer-snapshot
 
b/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-context-state-serializer-snapshot
deleted file mode 100644
index f6f36e7..0000000
Binary files 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-context-state-serializer-snapshot
 and /dev/null differ
diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-transaction-state-serializer-data
 
b/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-transaction-state-serializer-data
deleted file mode 100644
index 2476af5..0000000
Binary files 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-transaction-state-serializer-data
 and /dev/null differ
diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-transaction-state-serializer-snapshot
 
b/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-transaction-state-serializer-snapshot
deleted file mode 100644
index 87182ea..0000000
Binary files 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.6-transaction-state-serializer-snapshot
 and /dev/null differ
diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-context-state-serializer-data
 
b/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-context-state-serializer-data
deleted file mode 100644
index e7d439a..0000000
Binary files 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-context-state-serializer-data
 and /dev/null differ
diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-context-state-serializer-snapshot
 
b/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-context-state-serializer-snapshot
deleted file mode 100644
index 870f4b4..0000000
Binary files 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-context-state-serializer-snapshot
 and /dev/null differ
diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-transaction-state-serializer-data
 
b/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-transaction-state-serializer-data
deleted file mode 100644
index 2476af5..0000000
Binary files 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-transaction-state-serializer-data
 and /dev/null differ
diff --git 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-transaction-state-serializer-snapshot
 
b/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-transaction-state-serializer-snapshot
deleted file mode 100644
index 52370eb..0000000
Binary files 
a/flink-connectors/flink-connector-kafka-0.11/src/test/resources/flink-1.7-transaction-state-serializer-snapshot
 and /dev/null differ

Reply via email to