You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tz...@apache.org on 2019/01/23 09:22:49 UTC

[flink] 04/05: [FLINK-11328] [e2e] Do not use deprecated ParameterlessTypeSerializerConfig in e2e tests

This is an automated email from the ASF dual-hosted git repository.

tzulitai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit eb1241eec517d3f906625bc67e722d097a9514cd
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
AuthorDate: Fri Jan 18 14:12:51 2019 +0100

    [FLINK-11328] [e2e] Do not use deprecated ParameterlessTypeSerializerConfig in e2e tests
---
 .../StatefulComplexPayloadSerializer.java          | 32 ++++++++++------------
 1 file changed, 14 insertions(+), 18 deletions(-)

diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/StatefulComplexPayloadSerializer.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/StatefulComplexPayloadSerializer.java
index 89b16b6..294241b 100644
--- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/StatefulComplexPayloadSerializer.java
+++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/StatefulComplexPayloadSerializer.java
@@ -18,10 +18,8 @@
 
 package org.apache.flink.streaming.tests.artificialstate;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.ParameterlessTypeSerializerConfig;
+import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -143,21 +141,8 @@ public class StatefulComplexPayloadSerializer extends TypeSerializer<ComplexPayl
 	}
 
 	@Override
-	public TypeSerializerConfigSnapshot<ComplexPayload> snapshotConfiguration() {
-		// type serializer singletons should always be parameter-less
-		return new ParameterlessTypeSerializerConfig<>(getSerializationFormatIdentifier());
-	}
-
-	@Override
-	public CompatibilityResult<ComplexPayload> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
-		if (configSnapshot instanceof ParameterlessTypeSerializerConfig
-			&& isCompatibleSerializationFormatIdentifier(
-			((ParameterlessTypeSerializerConfig<?>) configSnapshot).getSerializationFormatIdentifier())) {
-
-			return CompatibilityResult.compatible();
-		} else {
-			return CompatibilityResult.requiresMigration();
-		}
+	public Snapshot snapshotConfiguration() {
+		return new Snapshot();
 	}
 
 	private boolean isCompatibleSerializationFormatIdentifier(String identifier) {
@@ -167,4 +152,15 @@ public class StatefulComplexPayloadSerializer extends TypeSerializer<ComplexPayl
 	private String getSerializationFormatIdentifier() {
 		return getClass().getCanonicalName();
 	}
+
+	// ----------------------------------------------------------------------------------------
+
+	/**
+	 * Snapshot for the {@link StatefulComplexPayloadSerializer}.
+	 */
+	public static class Snapshot extends SimpleTypeSerializerSnapshot<ComplexPayload> {
+		public Snapshot() {
+			super(StatefulComplexPayloadSerializer::new);
+		}
+	}
 }