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 2017/06/20 12:38:23 UTC

[3/3] flink git commit: [FLINK-6921] [serializer] Allow EnumValueSerializer to deal with appended enum values

[FLINK-6921] [serializer] Allow EnumValueSerializer to deal with appended enum values

The problem was that we don't check the bounds of the array with the enum names contained
in the ScalaEnumSerializerConfigSnapshot.

This PR also adds an Enumeration upgrade test which makes sure that appended fields are
supported without migration. Moreover, it checks that a field removal and an order change
leads to a required migration.

This closes #4126.


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

Branch: refs/heads/master
Commit: e520023ac768ac739537f7aa563f13aa208949ff
Parents: b9cce51
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Jun 14 16:53:49 2017 +0200
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Tue Jun 20 20:37:13 2017 +0800

----------------------------------------------------------------------
 flink-scala/pom.xml                             |  10 +
 .../scala/typeutils/EnumValueSerializer.scala   |  13 +-
 .../typeutils/EnumValueSerializerTest.scala     |  46 +++++
 .../EnumValueSerializerUpgradeTest.scala        | 199 +++++++++++++++++++
 4 files changed, 263 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e520023a/flink-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml
index 1c4d7c4..2df526e 100644
--- a/flink-scala/pom.xml
+++ b/flink-scala/pom.xml
@@ -107,6 +107,16 @@ under the License.
 				<artifactId>japicmp-maven-plugin</artifactId>
 			</plugin>
 
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<!-- We have to disable this option so that the system class loader contains all
+					dependencies as URLs -->
+					<useManifestOnlyJar>false</useManifestOnlyJar>
+				</configuration>
+			</plugin>
+
 			<!-- Scala Compiler -->
 			<plugin>
 				<groupId>net.alchim31.maven</groupId>

http://git-wip-us.apache.org/repos/asf/flink/blob/e520023a/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 119db93..8be5e0b 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
@@ -92,11 +92,14 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[
 
           if (previousEnumConstants != null) {
             for (i <- enum.values.iterator) {
-              if (!previousEnumConstants(i.id).equals(i.toString)) {
-                // compatible only if new enum constants are only appended,
-                // and original constants must be in the exact same order
-
-                return CompatibilityResult.requiresMigration()
+              // skip the check for all newly added fields
+              if (i.id < previousEnumConstants.length) {
+                if (!previousEnumConstants(i.id).equals(i.toString)) {
+                  // compatible only if new enum constants are only appended,
+                  // and original constants must be in the exact same order
+
+                  return CompatibilityResult.requiresMigration()
+                }
               }
             }
           }

http://git-wip-us.apache.org/repos/asf/flink/blob/e520023a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala
new file mode 100644
index 0000000..5009271
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala
@@ -0,0 +1,46 @@
+/*
+ * 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.scala.typeutils
+
+import org.apache.flink.util.TestLogger
+import org.junit.Test
+import org.junit.Assert._
+import org.scalatest.junit.JUnitSuiteLike
+
+class EnumValueSerializerTest extends TestLogger with JUnitSuiteLike {
+
+  /**
+    * Tests that the snapshot configuration can be created and that the serializer
+    * is compatible when being called with the created serializer snapshot
+    *
+    * FLINK-6914
+    */
+  @Test
+  def testEnumValueSerializerEnsureCompatibilityIdempotency() {
+    val enumSerializer = new EnumValueSerializer(Letters)
+
+    val snapshot = enumSerializer.snapshotConfiguration()
+
+    assertFalse(enumSerializer.ensureCompatibility(snapshot).isRequiresMigration)
+  }
+}
+
+object Letters extends Enumeration {
+  val A, B, C = Value
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e520023a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala
new file mode 100644
index 0000000..c470cd0
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala
@@ -0,0 +1,199 @@
+/*
+ * 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.scala.typeutils
+
+import java.io._
+import java.net.{URL, URLClassLoader}
+
+import org.apache.flink.api.common.typeutils.{CompatibilityResult, TypeSerializerSerializationUtil}
+import org.apache.flink.core.memory.{DataInputViewStreamWrapper, DataOutputViewStreamWrapper}
+import org.apache.flink.util.TestLogger
+import org.junit.rules.TemporaryFolder
+import org.junit.{Rule, Test}
+import org.junit.Assert._
+import org.scalatest.junit.JUnitSuiteLike
+
+import scala.reflect.NameTransformer
+import scala.tools.nsc.reporters.ConsoleReporter
+import scala.tools.nsc.{GenericRunnerSettings, Global}
+
+class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike {
+
+  private val _tempFolder = new TemporaryFolder()
+
+  @Rule
+  def tempFolder = _tempFolder
+
+  val enumName = "EnumValueSerializerUpgradeTestEnum"
+
+  val enumA =
+    s"""
+      |@SerialVersionUID(1L)
+      |object $enumName extends Enumeration {
+      |  val A, B, C = Value
+      |}
+    """.stripMargin
+
+  val enumB =
+    s"""
+       |@SerialVersionUID(1L)
+       |object $enumName extends Enumeration {
+       |  val A, B, C, D = Value
+       |}
+    """.stripMargin
+
+  val enumC =
+    s"""
+       |@SerialVersionUID(1L)
+       |object $enumName extends Enumeration {
+       |  val A, C = Value
+       |}
+    """.stripMargin
+
+  val enumD =
+    s"""
+       |@SerialVersionUID(1L)
+       |object $enumName extends Enumeration {
+       |  val A, C, B = Value
+       |}
+    """.stripMargin
+
+  /**
+    * Check that identical enums don't require migration
+    */
+  @Test
+  def checkIdenticalEnums(): Unit = {
+    assertFalse(checkCompatibility(enumA, enumA).isRequiresMigration)
+  }
+
+  /**
+    * Check that appending fields to the enum does not require migration
+    */
+  @Test
+  def checkAppendedField(): Unit = {
+    assertFalse(checkCompatibility(enumA, enumB).isRequiresMigration)
+  }
+
+  /**
+    * Check that removing enum fields requires migration
+    */
+  @Test
+  def checkRemovedField(): Unit = {
+    assertTrue(checkCompatibility(enumA, enumC).isRequiresMigration)
+  }
+
+  /**
+    * Check that changing the enum field order requires migration
+    */
+  @Test
+  def checkDifferentFieldOrder(): Unit = {
+    assertTrue(checkCompatibility(enumA, enumD).isRequiresMigration)
+  }
+
+  def checkCompatibility(enumSourceA: String, enumSourceB: String)
+    : CompatibilityResult[Enumeration#Value] = {
+    import EnumValueSerializerUpgradeTest._
+
+    val classLoader = compileAndLoadEnum(tempFolder.newFolder(), s"$enumName.scala", enumSourceA)
+
+    val enum = instantiateEnum[Enumeration](classLoader, enumName)
+
+    val enumValueSerializer = new EnumValueSerializer(enum)
+    val snapshot = enumValueSerializer.snapshotConfiguration()
+
+    val baos = new ByteArrayOutputStream()
+    val output = new DataOutputViewStreamWrapper(baos)
+    TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(output, snapshot)
+
+    output.close()
+    baos.close()
+
+    val bais = new ByteArrayInputStream(baos.toByteArray)
+    val input=  new DataInputViewStreamWrapper(bais)
+
+    val classLoader2 = compileAndLoadEnum(tempFolder.newFolder(), s"$enumName.scala", enumSourceB)
+
+    val snapshot2 = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+      input,
+      classLoader2)
+    val enum2 = instantiateEnum[Enumeration](classLoader2, enumName)
+
+    val enumValueSerializer2 = new EnumValueSerializer(enum2)
+    enumValueSerializer2.ensureCompatibility(snapshot2)
+  }
+}
+
+object EnumValueSerializerUpgradeTest {
+  def compileAndLoadEnum(root: File, filename: String, source: String): ClassLoader = {
+    val file = writeSourceFile(root, filename, source)
+
+    compileScalaFile(file)
+
+    new URLClassLoader(
+      Array[URL](root.toURI.toURL),
+      Thread.currentThread().getContextClassLoader)
+  }
+
+  def instantiateEnum[T <: Enumeration](classLoader: ClassLoader, enumName: String): T = {
+    val clazz = classLoader.loadClass(enumName + "$").asInstanceOf[Class[_ <: Enumeration]]
+    val field = clazz.getField(NameTransformer.MODULE_INSTANCE_NAME)
+
+    field.get(null).asInstanceOf[T]
+  }
+
+  def writeSourceFile(root: File, filename: String, source: String): File = {
+    val file = new File(root, filename)
+    val fileWriter = new FileWriter(file)
+
+    fileWriter.write(source)
+
+    fileWriter.close()
+
+    file
+  }
+
+  def compileScalaFile(file: File): Unit = {
+    val in = new BufferedReader(new StringReader(""))
+    val out = new PrintWriter(new BufferedWriter(
+      new OutputStreamWriter(System.out)))
+
+    val settings = new GenericRunnerSettings(out.println _)
+
+    val classLoader = Thread.currentThread().getContextClassLoader
+
+    val urls = classLoader match {
+      case urlClassLoader: URLClassLoader =>
+        urlClassLoader.getURLs
+      case x => throw new IllegalStateException(s"Not possible to extract URLs " +
+        s"from class loader $x.")
+    }
+
+    settings.classpath.value = urls.map(_.toString).mkString(java.io.File.pathSeparator)
+    settings.outdir.value = file.getParent
+
+    val reporter = new ConsoleReporter(settings)
+    val global = new Global(settings, reporter)
+    val run = new global.Run
+
+    run.compile(List(file.getAbsolutePath))
+
+    reporter.printSummary()
+  }
+}
+