Repository: flink
Updated Branches:
  refs/heads/master 6abbba248 -> 23c82e3cc


[FLINK-6869] [scala] Specify serialVersionUID for all Scala serializers

Previously, Scala serializers did not specify the serialVersionUID, and
therefore prohibited restore from previous Flink version snapshots
because the serializers' implementations changed.

The serialVersionUIDs added in this commit are identical to what they
were (as generated by Java) in Flink 1.2, so that we can at least
restore state that were written with the Scala serializers as of 1.2.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/75ea808b
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/75ea808b
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/75ea808b

Branch: refs/heads/master
Commit: 75ea808bb163774898604a6374b8486e05520497
Parents: 6edb72d
Author: Tzu-Li (Gordon) Tai <[email protected]>
Authored: Thu Jun 8 15:29:45 2017 +0200
Committer: Tzu-Li (Gordon) Tai <[email protected]>
Committed: Tue Jun 13 06:26:48 2017 +0200

----------------------------------------------------------------------
 .../flink/api/scala/typeutils/CaseClassSerializer.scala   | 10 ++++++++++
 .../flink/api/scala/typeutils/EitherSerializer.scala      |  1 +
 .../flink/api/scala/typeutils/EnumValueSerializer.scala   |  1 +
 .../flink/api/scala/typeutils/OptionSerializer.scala      |  1 +
 .../flink/api/scala/typeutils/TraversableSerializer.scala |  1 +
 .../apache/flink/api/scala/typeutils/TrySerializer.scala  |  1 +
 .../apache/flink/api/scala/typeutils/UnitSerializer.scala |  1 +
 7 files changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/75ea808b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
index 29b4952..c8222d6 100644
--- 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
+++ 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
@@ -28,6 +28,7 @@ import org.apache.flink.types.NullFieldException
  * our Java Tuples so we have to treat them differently.
  */
 @Internal
+@SerialVersionUID(7341356073446263475L)
 abstract class CaseClassSerializer[T <: Product](
     clazz: Class[T],
     scalaFieldSerializers: Array[TypeSerializer[_]])
@@ -80,6 +81,15 @@ abstract class CaseClassSerializer[T <: Product](
     createInstance(fields)
   }
 
+  override def createSerializerInstance(
+      tupleClass: Class[T],
+      fieldSerializers: Array[TypeSerializer[_]]): TupleSerializerBase[T] = {
+    this.getClass
+      .getConstructors()(0)
+      .newInstance(tupleClass, fieldSerializers)
+      .asInstanceOf[CaseClassSerializer[T]]
+  }
+
   def copy(from: T, reuse: T): T = {
     copy(from)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/75ea808b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
index 1095aee..439e0c2 100644
--- 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
+++ 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
@@ -26,6 +26,7 @@ import org.apache.flink.core.memory.{DataInputView, 
DataOutputView}
  * Serializer for [[Either]].
  */
 @Internal
+@SerialVersionUID(9219995873023657525L)
 class EitherSerializer[A, B, T <: Either[A, B]](
     val leftSerializer: TypeSerializer[A],
     val rightSerializer: TypeSerializer[B])

http://git-wip-us.apache.org/repos/asf/flink/blob/75ea808b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
index d549623..50526f5 100644
--- 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
+++ 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
@@ -30,6 +30,7 @@ import org.apache.flink.util.{InstantiationUtil, 
Preconditions}
  * Serializer for [[Enumeration]] values.
  */
 @Internal
+@SerialVersionUID(-2403076635594572920L)
 class EnumValueSerializer[E <: Enumeration](val enum: E) extends 
TypeSerializer[E#Value] {
 
   type T = E#Value

http://git-wip-us.apache.org/repos/asf/flink/blob/75ea808b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
index 810c91c..aa4a0ea 100644
--- 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
+++ 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
@@ -25,6 +25,7 @@ import org.apache.flink.core.memory.{DataInputView, 
DataOutputView}
  * Serializer for [[Option]].
  */
 @Internal
+@SerialVersionUID(-8635243274072627338L)
 class OptionSerializer[A](val elemSerializer: TypeSerializer[A])
   extends TypeSerializer[Option[A]] {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/75ea808b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
index 5963987..b54193b 100644
--- 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
+++ 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
@@ -29,6 +29,7 @@ import scala.collection.generic.CanBuildFrom
  * Serializer for Scala Collections.
  */
 @Internal
+@SerialVersionUID(7522917416391312410L)
 abstract class TraversableSerializer[T <: TraversableOnce[E], E](
     var elementSerializer: TypeSerializer[E])
   extends TypeSerializer[T] with Cloneable {

http://git-wip-us.apache.org/repos/asf/flink/blob/75ea808b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
index a88cce7..e128157 100644
--- 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
+++ 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
@@ -29,6 +29,7 @@ import scala.util.{Failure, Success, Try}
  * Serializer for [[scala.util.Try]].
  */
 @Internal
+@SerialVersionUID(-3052182891252564491L)
 class TrySerializer[A](
     private val elemSerializer: TypeSerializer[A],
     private val executionConfig: ExecutionConfig)

http://git-wip-us.apache.org/repos/asf/flink/blob/75ea808b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitSerializer.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitSerializer.scala
 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitSerializer.scala
index d80bc9b..32c44d2 100644
--- 
a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitSerializer.scala
+++ 
b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitSerializer.scala
@@ -22,6 +22,7 @@ import 
org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton
 import org.apache.flink.core.memory.{DataInputView, DataOutputView}
 
 @Internal
+@SerialVersionUID(5413377487955047394L)
 class UnitSerializer extends TypeSerializerSingleton[Unit] {
 
   def isImmutableType(): Boolean = true

Reply via email to