Skip to content

Commit

Permalink
[FLINK-11741] Remove ensureCompatibility implementation from dummy se…
Browse files Browse the repository at this point in the history
…rializers
  • Loading branch information
tzulitai committed Feb 28, 2019
1 parent 852f876 commit fab31b2
Show file tree
Hide file tree
Showing 4 changed files with 4 additions and 26 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -126,14 +126,8 @@ public void copy(DataInputView source, DataOutputView target) throws IOException
"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.");
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -109,12 +109,7 @@ public void copy(DataInputView source, DataOutputView target) throws IOException
}

@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.");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}

/**
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,7 @@

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;
Expand Down Expand Up @@ -456,12 +454,7 @@ public int hashCode() {
}

@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.");
}
}
Expand Down

0 comments on commit fab31b2

Please sign in to comment.