You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by al...@apache.org on 2020/05/28 11:51:50 UTC
[flink] 24/42: [FLINK-13632] Port KafkaSerializer upgrade test to
TypeSerializerUpgradeTestBase
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 6762882bee9dde4cbb655950ecbf1afa0011f0a9
Author: klion26 <qc...@gmail.com>
AuthorDate: Sat Feb 15 14:05:55 2020 +0800
[FLINK-13632] Port KafkaSerializer upgrade test to TypeSerializerUpgradeTestBase
---
.../connectors/kafka/FlinkKafkaProducer.java | 14 +-
.../KafkaSerializerSnapshotsMigrationTest.java | 59 --------
.../kafka/KafkaSerializerUpgradeTest.java | 162 +++++++++++++++++++++
.../flink-1.7-context-state-serializer-data | Bin 270 -> 0 bytes
.../flink-1.7-context-state-serializer-snapshot | Bin 434 -> 0 bytes
.../flink-1.7-transaction-state-serializer-data | Bin 110 -> 0 bytes
...flink-1.7-transaction-state-serializer-snapshot | Bin 442 -> 0 bytes
7 files changed, 171 insertions(+), 64 deletions(-)
diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
index 9d033b7..b095195 100644
--- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
+++ b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
@@ -1281,7 +1281,7 @@ public class FlinkKafkaProducer<IN>
*/
@VisibleForTesting
@Internal
- protected static class KafkaTransactionState {
+ public static class KafkaTransactionState {
private final transient FlinkKafkaInternalProducer<byte[], byte[]> producer;
@@ -1292,15 +1292,18 @@ public class FlinkKafkaProducer<IN>
final short epoch;
- KafkaTransactionState(String transactionalId, FlinkKafkaInternalProducer<byte[], byte[]> producer) {
+ @VisibleForTesting
+ public KafkaTransactionState(String transactionalId, FlinkKafkaInternalProducer<byte[], byte[]> producer) {
this(transactionalId, producer.getProducerId(), producer.getEpoch(), producer);
}
- KafkaTransactionState(FlinkKafkaInternalProducer<byte[], byte[]> producer) {
+ @VisibleForTesting
+ public KafkaTransactionState(FlinkKafkaInternalProducer<byte[], byte[]> producer) {
this(null, -1, (short) -1, producer);
}
- KafkaTransactionState(
+ @VisibleForTesting
+ public KafkaTransactionState(
@Nullable String transactionalId,
long producerId,
short epoch,
@@ -1367,7 +1370,8 @@ public class FlinkKafkaProducer<IN>
public static class KafkaTransactionContext {
final Set<String> transactionalIds;
- KafkaTransactionContext(Set<String> transactionalIds) {
+ @VisibleForTesting
+ public KafkaTransactionContext(Set<String> transactionalIds) {
checkNotNull(transactionalIds);
this.transactionalIds = transactionalIds;
}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerSnapshotsMigrationTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerSnapshotsMigrationTest.java
deleted file mode 100644
index ac69205..0000000
--- a/flink-connectors/flink-connector-kafka/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 FlinkKafkaProducer.TransactionStateSerializer}
- * and {@link FlinkKafkaProducer.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_7);
-
- testSpecifications.add(
- "transaction-state-serializer",
- FlinkKafkaProducer.TransactionStateSerializer.class,
- FlinkKafkaProducer.TransactionStateSerializer.TransactionStateSerializerSnapshot.class,
- FlinkKafkaProducer.TransactionStateSerializer::new);
- testSpecifications.add(
- "context-state-serializer",
- FlinkKafkaProducer.ContextStateSerializer.class,
- FlinkKafkaProducer.ContextStateSerializer.ContextStateSerializerSnapshot.class,
- FlinkKafkaProducer.ContextStateSerializer::new);
-
- return testSpecifications.get();
- }
-}
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java
new file mode 100644
index 0000000..bb1146f
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.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.FlinkKafkaInternalProducer;
+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 {@link FlinkKafkaProducer.TransactionStateSerializer}
+ * and {@link FlinkKafkaProducer.ContextStateSerializer}.
+ */
+@RunWith(Parameterized.class)
+public class KafkaSerializerUpgradeTest extends TypeSerializerUpgradeTestBase<Object, Object> {
+
+ public KafkaSerializerUpgradeTest(TestSpecification<Object, Object> 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<>(
+ "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<FlinkKafkaProducer.KafkaTransactionState> {
+ @Override
+ public TypeSerializer<FlinkKafkaProducer.KafkaTransactionState> createPriorSerializer() {
+ return new FlinkKafkaProducer.TransactionStateSerializer();
+ }
+
+ @Override
+ public FlinkKafkaProducer.KafkaTransactionState createTestData() {
+ @SuppressWarnings("unchecked")
+ FlinkKafkaInternalProducer<byte[], byte[]> mock = Mockito.mock(FlinkKafkaInternalProducer.class);
+ return new FlinkKafkaProducer.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<FlinkKafkaProducer.KafkaTransactionState> {
+ @Override
+ public TypeSerializer<FlinkKafkaProducer.KafkaTransactionState> createUpgradedSerializer() {
+ return new FlinkKafkaProducer.TransactionStateSerializer();
+ }
+
+ @Override
+ public Matcher<FlinkKafkaProducer.KafkaTransactionState> testDataMatcher() {
+ @SuppressWarnings("unchecked")
+ FlinkKafkaInternalProducer<byte[], byte[]> mock = Mockito.mock(FlinkKafkaInternalProducer.class);
+ return is(new FlinkKafkaProducer.KafkaTransactionState("1234", 3456, (short) 789, mock));
+ }
+
+ @Override
+ public Matcher<TypeSerializerSchemaCompatibility<FlinkKafkaProducer.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<FlinkKafkaProducer.KafkaTransactionContext> {
+ @Override
+ public TypeSerializer<FlinkKafkaProducer.KafkaTransactionContext> createPriorSerializer() {
+ return new FlinkKafkaProducer.ContextStateSerializer();
+ }
+
+ @Override
+ public FlinkKafkaProducer.KafkaTransactionContext createTestData() {
+ Set<String> transactionIds = new HashSet<>();
+ transactionIds.add("123");
+ transactionIds.add("456");
+ transactionIds.add("789");
+ return new FlinkKafkaProducer.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<FlinkKafkaProducer.KafkaTransactionContext> {
+ @Override
+ public TypeSerializer<FlinkKafkaProducer.KafkaTransactionContext> createUpgradedSerializer() {
+ return new FlinkKafkaProducer.ContextStateSerializer();
+ }
+
+ @Override
+ public Matcher<FlinkKafkaProducer.KafkaTransactionContext> testDataMatcher() {
+ Set<String> transactionIds = new HashSet<>();
+ transactionIds.add("123");
+ transactionIds.add("456");
+ transactionIds.add("789");
+ return is(new FlinkKafkaProducer.KafkaTransactionContext(transactionIds));
+ }
+
+ @Override
+ public Matcher<TypeSerializerSchemaCompatibility<FlinkKafkaProducer.KafkaTransactionContext>> schemaCompatibilityMatcher(MigrationVersion version) {
+ return TypeSerializerMatchers.isCompatibleAsIs();
+ }
+ }
+
+}
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/flink-1.7-context-state-serializer-data b/flink-connectors/flink-connector-kafka/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/src/test/resources/flink-1.7-context-state-serializer-data and /dev/null differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/flink-1.7-context-state-serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/flink-1.7-context-state-serializer-snapshot
deleted file mode 100644
index 14f0dd4..0000000
Binary files a/flink-connectors/flink-connector-kafka/src/test/resources/flink-1.7-context-state-serializer-snapshot and /dev/null differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/flink-1.7-transaction-state-serializer-data b/flink-connectors/flink-connector-kafka/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/src/test/resources/flink-1.7-transaction-state-serializer-data and /dev/null differ
diff --git a/flink-connectors/flink-connector-kafka/src/test/resources/flink-1.7-transaction-state-serializer-snapshot b/flink-connectors/flink-connector-kafka/src/test/resources/flink-1.7-transaction-state-serializer-snapshot
deleted file mode 100644
index 35063ba..0000000
Binary files a/flink-connectors/flink-connector-kafka/src/test/resources/flink-1.7-transaction-state-serializer-snapshot and /dev/null differ