flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-6948) EnumValueSerializer cannot handle removed enum values
Date Mon, 19 Jun 2017 12:13:00 GMT

    [ https://issues.apache.org/jira/browse/FLINK-6948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16053885#comment-16053885
] 

ASF GitHub Bot commented on FLINK-6948:
---------------------------------------

Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4142#discussion_r122686133
  
    --- Diff: flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala
---
    @@ -0,0 +1,119 @@
    +/*
    + * 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 org.apache.flink.api.scala.typeutils.EnumValueSerializer.ScalaEnumSerializerConfigSnapshot
    +import org.apache.flink.core.testutils.CommonTestUtils
    +import org.apache.flink.runtime.util.{DataInputDeserializer, DataOutputSerializer}
    +import org.apache.flink.util.TestLogger
    +import org.junit.{Ignore, 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)
    +  }
    +
    +  @Ignore("This test case is only used to create a " +
    +    "TypeSerializerConfigurationSnapshot for the EnumValueSerializer")
    +  @Test
    +  def createEnumValueSerializerConfigSnapshot(): Unit = {
    +    val enumValueSerializer = new EnumValueSerializer(Letters)
    +
    +    val snapshot = enumValueSerializer.snapshotConfiguration()
    +
    +    val out = new DataOutputSerializer(128)
    +
    +    snapshot.write(out)
    +    val buffer = out.getCopyOfBuffer
    +
    +    val tmpDir = CommonTestUtils.getTempDir
    +    val configFile = new File(tmpDir, "EnumValueSerializerConfigSnapshot-1.3.1")
    +
    +    log.info(s"Writing EnumValueSerializerConfigSnapshot-1.3.1 to $configFile")
    +
    +    val outputStream = new DataOutputStream(new FileOutputStream(configFile))
    +
    +    try {
    +      outputStream.writeInt(buffer.length)
    +      outputStream.write(buffer)
    +    } finally {
    +      outputStream.close()
    +    }
    +  }
    +
    +  /**
    +    * Check backwards compatibility between 1.3.1 and 1.3.2 since the
    +    * ScalaEnumSerializerConfigSnapshot format changed.
    +    *
    +    * FLINK-694X
    --- End diff --
    
    nit: I would use `@see` for this.
    Also, directly refer to FLINK-6948, because that's the exact issue that required the format
change.


> EnumValueSerializer cannot handle removed enum values
> -----------------------------------------------------
>
>                 Key: FLINK-6948
>                 URL: https://issues.apache.org/jira/browse/FLINK-6948
>             Project: Flink
>          Issue Type: Bug
>          Components: Type Serialization System
>    Affects Versions: 1.3.1, 1.4.0
>            Reporter: Till Rohrmann
>            Assignee: Till Rohrmann
>            Priority: Critical
>
> The {{EnumValueSerializer}} cannot handle removed enum values or enum values whose ordinal
value have been changed. We should try to detect these cases and require migration. The problem
is that we take the values of the new enum and iterate over them without checking that all
old enum values contained in the config snapshot are actually contained.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message