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

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

commit 13cc1ca34ee42ea9270453039c99b7c6bc3d0d5f
Author: Tzu-Li (Gordon) Tai <tzuli...@apache.org>
AuthorDate: Mon Feb 25 16:22:16 2019 +0800

    [FLINK-11741] Remove ensureCompatibility implementation from dummy 
serializers
---
 .../flink/api/common/typeutils/TypeDeserializerAdapter.java      | 8 +-------
 .../api/common/typeutils/UnloadableDummyTypeSerializer.java      | 7 +------
 .../org/apache/flink/api/scala/typeutils/NothingSerializer.scala | 6 +-----
 .../api/operators/InternalTimersSnapshotReaderWriters.java       | 9 +--------
 4 files changed, 4 insertions(+), 26 deletions(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeDeserializerAdapter.java
 
b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeDeserializerAdapter.java
index 7b8e9b5..86883e8 100644
--- 
a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeDeserializerAdapter.java
+++ 
b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeDeserializerAdapter.java
@@ -126,14 +126,8 @@ public final class TypeDeserializerAdapter<T> extends 
TypeSerializer<T> implemen
                        "This is a TypeDeserializerAdapter used only for 
deserialization; this method should not be used.");
        }
 
-       public TypeSerializerConfigSnapshot<T> snapshotConfiguration() {
+       public TypeSerializerSnapshot<T> snapshotConfiguration() {
                throw new UnsupportedOperationException(
                        "This is a TypeDeserializerAdapter used only for 
deserialization; this method should not be used.");
        }
-
-       public CompatibilityResult<T> 
ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
-               throw new UnsupportedOperationException(
-                       "This is a TypeDeserializerAdapter used only for 
deserialization; this method should not be used.");
-       }
-
 }
diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/UnloadableDummyTypeSerializer.java
 
b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/UnloadableDummyTypeSerializer.java
index 6660ad8..c34f7e1 100644
--- 
a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/UnloadableDummyTypeSerializer.java
+++ 
b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/UnloadableDummyTypeSerializer.java
@@ -109,12 +109,7 @@ public class UnloadableDummyTypeSerializer<T> extends 
TypeSerializer<T> {
        }
 
        @Override
-       public TypeSerializerConfigSnapshot<T> snapshotConfiguration() {
-               throw new UnsupportedOperationException("This object is a dummy 
TypeSerializer.");
-       }
-
-       @Override
-       public CompatibilityResult<T> 
ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
+       public TypeSerializerSnapshot<T> snapshotConfiguration() {
                throw new UnsupportedOperationException("This object is a dummy 
TypeSerializer.");
        }
 
diff --git 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
index aefca07..10f26a9 100644
--- 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
+++ 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
@@ -18,7 +18,7 @@
 package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.annotation.Internal
-import org.apache.flink.api.common.typeutils.{CompatibilityResult, 
TypeSerializer, TypeSerializerConfigSnapshot}
+import org.apache.flink.api.common.typeutils.{TypeSerializer, 
TypeSerializerConfigSnapshot}
 import org.apache.flink.core.memory.{DataInputView, DataOutputView}
 
 /**
@@ -59,10 +59,6 @@ class NothingSerializer extends TypeSerializer[Any] {
   override def snapshotConfiguration(): TypeSerializerConfigSnapshot[Any] =
     throw new RuntimeException("This must not be used. You encountered a bug.")
 
-  override def ensureCompatibility(
-      configSnapshot: TypeSerializerConfigSnapshot[_]): 
CompatibilityResult[Any] =
-    throw new RuntimeException("This must not be used. You encountered a bug.")
-
   override def equals(obj: Any): Boolean = {
     obj match {
       case nothingSerializer: NothingSerializer => true
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshotReaderWriters.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshotReaderWriters.java
index 9dcd959..9a0b0ae 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshotReaderWriters.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshotReaderWriters.java
@@ -20,9 +20,7 @@ package org.apache.flink.streaming.api.operators;
 
 import org.apache.flink.annotation.Internal;
 import 
org.apache.flink.api.common.typeutils.BackwardsCompatibleSerializerSnapshot;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
@@ -456,12 +454,7 @@ public class InternalTimersSnapshotReaderWriters {
                }
 
                @Override
-               public TypeSerializerConfigSnapshot snapshotConfiguration() {
-                       throw new UnsupportedOperationException("This 
serializer is not registered for managed state.");
-               }
-
-               @Override
-               public CompatibilityResult<TimerHeapInternalTimer<K, N>> 
ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+               public TypeSerializerSnapshot<TimerHeapInternalTimer<K, N>> 
snapshotConfiguration() {
                        throw new UnsupportedOperationException("This 
serializer is not registered for managed state.");
                }
        }

Reply via email to