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/08 16:49:23 UTC

[flink] 10/12: [FLINK-11073] [core] Replace EitherSerializerSnapshot with new JavaEitherSerializerSnapshot

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 301be5552290788864676b131e09741f769ef471
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
AuthorDate: Mon Jan 7 12:50:27 2019 +0100

    [FLINK-11073] [core] Replace EitherSerializerSnapshot with new JavaEitherSerializerSnapshot
---
 .../java/typeutils/runtime/EitherSerializer.java   |  4 +-
 .../runtime/EitherSerializerSnapshot.java          | 12 +++--
 .../runtime/JavaEitherSerializerSnapshot.java      | 61 ++++++++++++++++++++++
 ...mpositeTypeSerializerSnapshotMigrationTest.java |  4 +-
 4 files changed, 72 insertions(+), 9 deletions(-)

diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java
index 3d4e8e9..0128640 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java
@@ -205,7 +205,7 @@ public class EitherSerializer<L, R> extends TypeSerializer<Either<L, R>> {
 	// ------------------------------------------------------------------------
 
 	@Override
-	public EitherSerializerSnapshot<L, R> snapshotConfiguration() {
-		return new EitherSerializerSnapshot<>(leftSerializer, rightSerializer);
+	public JavaEitherSerializerSnapshot<L, R> snapshotConfiguration() {
+		return new JavaEitherSerializerSnapshot<>(this);
 	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerSnapshot.java
index 016fd04..3b7a8e7 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerSnapshot.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerSnapshot.java
@@ -35,8 +35,12 @@ import static org.apache.flink.util.Preconditions.checkState;
 
 /**
  * Configuration snapshot for the {@link EitherSerializer}.
+ *
+ * @deprecated this snapshot class is no longer used by any serializers.
+ *             Instead, {@link JavaEitherSerializerSnapshot} is used.
  */
 @Internal
+@Deprecated
 public final class EitherSerializerSnapshot<L, R> implements TypeSerializerSnapshot<Either<L, R>> {
 
 	private static final int CURRENT_VERSION = 2;
@@ -110,12 +114,10 @@ public final class EitherSerializerSnapshot<L, R> implements TypeSerializerSnaps
 		checkState(nestedSnapshot != null);
 
 		if (newSerializer instanceof EitherSerializer) {
+			// delegate compatibility check to the new snapshot class
 			EitherSerializer<L, R> serializer = (EitherSerializer<L, R>) newSerializer;
-
-			return nestedSnapshot.resolveCompatibilityWithNested(
-					TypeSerializerSchemaCompatibility.compatibleAsIs(),
-					serializer.getLeftSerializer(),
-					serializer.getRightSerializer());
+			JavaEitherSerializerSnapshot<L, R> newSnapshot = new JavaEitherSerializerSnapshot<>(serializer);
+			return newSnapshot.resolveSchemaCompatibility(serializer);
 		}
 		else {
 			return TypeSerializerSchemaCompatibility.incompatible();
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/JavaEitherSerializerSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/JavaEitherSerializerSnapshot.java
new file mode 100644
index 0000000..5036345
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/JavaEitherSerializerSnapshot.java
@@ -0,0 +1,61 @@
+/*
+ * 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.api.java.typeutils.runtime;
+
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.types.Either;
+
+/**
+ * Snapshot class for the {@link EitherSerializer}.
+ */
+public class JavaEitherSerializerSnapshot<L, R> extends CompositeTypeSerializerSnapshot<Either<L, R>, EitherSerializer> {
+
+	private static final int CURRENT_VERSION = 1;
+
+	/**
+	 * Constructor for read instantiation.
+	 */
+	@SuppressWarnings("unused")
+	public JavaEitherSerializerSnapshot() {
+		super(EitherSerializer.class);
+	}
+
+	/**
+	 * Constructor to create the snapshot for writing.
+	 */
+	public JavaEitherSerializerSnapshot(EitherSerializer<L, R> eitherSerializer) {
+		super(eitherSerializer);
+	}
+
+	@Override
+	protected int getCurrentOuterSnapshotVersion() {
+		return CURRENT_VERSION;
+	}
+
+	@Override
+	protected EitherSerializer createOuterSerializerWithNestedSerializers(TypeSerializer<?>[] nestedSerializers) {
+		return new EitherSerializer<>(nestedSerializers[0], nestedSerializers[1]);
+	}
+
+	@Override
+	protected TypeSerializer<?>[] getNestedSerializers(EitherSerializer outerSerializer) {
+		return new TypeSerializer<?>[]{ outerSerializer.getLeftSerializer(), outerSerializer.getRightSerializer() };
+	}
+}
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerSnapshotMigrationTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerSnapshotMigrationTest.java
index c6b49a4..62135d7 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerSnapshotMigrationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerSnapshotMigrationTest.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeutils.base.GenericArraySerializerSnapshot
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.java.typeutils.runtime.EitherSerializer;
-import org.apache.flink.api.java.typeutils.runtime.EitherSerializerSnapshot;
+import org.apache.flink.api.java.typeutils.runtime.JavaEitherSerializerSnapshot;
 import org.apache.flink.types.Either;
 
 import org.junit.runner.RunWith;
@@ -48,7 +48,7 @@ public class CompositeTypeSerializerSnapshotMigrationTest extends TypeSerializer
 
 		// Either<String, Integer>
 
-		final TestSpecification<Either<String, Integer>> either = TestSpecification.<Either<String, Integer>>builder("1.6-either", EitherSerializer.class, EitherSerializerSnapshot.class)
+		final TestSpecification<Either<String, Integer>> either = TestSpecification.<Either<String, Integer>>builder("1.6-either", EitherSerializer.class, JavaEitherSerializerSnapshot.class)
 			.withSerializerProvider(() -> new EitherSerializer<>(StringSerializer.INSTANCE, IntSerializer.INSTANCE))
 			.withSnapshotDataLocation("flink-1.6-either-type-serializer-snapshot")
 			.withTestData("flink-1.6-either-type-serializer-data", 10);