Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-4977

Enum serialization does not work in all cases

    Details

    • Type: Bug
    • Status: Closed
    • Priority: Minor
    • Resolution: Fixed
    • Affects Version/s: 1.1.3
    • Fix Version/s: 1.2.0, 1.1.4
    • Component/s: None
    • Labels:
      None
    • Environment:

      Java SE 1.8.0_91
      Ubuntu 14.04.4 LTS (trusty)

      Description

      Enums produce serialization failures whether they are by themselves or part of a POJO in the stream. I've tried running in IntelliJ IDEA and also via flink run. Here is a small program to reproduce:

      package org.apache.flink.testenum;
      
      import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
      
      public class TestEnumStream {
          private enum MyEnum {
              NONE, SOMETHING, EVERYTHING
          }
      
          public static void main(String[] args) throws Exception {
              final StreamExecutionEnvironment environment = StreamExecutionEnvironment.getExecutionEnvironment();
              environment.setParallelism(1);
      
              environment.fromElements(MyEnum.NONE, MyEnum.SOMETHING, MyEnum.EVERYTHING)
                      .addSink(x -> System.err.println(x));
      
              environment.execute("TestEnumStream");
          }
      }
      
      Exception in thread "main" java.lang.RuntimeException: Cannot access the constants of the enum org.apache.flink.testenum.TestEnumStream$MyEnum
      	at org.apache.flink.api.common.typeutils.base.EnumSerializer.createValues(EnumSerializer.java:132)
      	at org.apache.flink.api.common.typeutils.base.EnumSerializer.<init>(EnumSerializer.java:43)
      	at org.apache.flink.api.java.typeutils.EnumTypeInfo.createSerializer(EnumTypeInfo.java:101)
      	at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.fromCollection(StreamExecutionEnvironment.java:773)
      	at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.fromElements(StreamExecutionEnvironment.java:674)
      

      I took a look at that line in EnumSerializer.java and swapped out the reflection on the "values" method for the simpler `enumClass.getEnumConstants()`, and that seems to work after I install my custom flink-core jar. I believe this is because http://docs.oracle.com/javase/tutorial/reflect/special/enumMembers.html specifically states you cannot reflect on the "values" method since it is implicitly generated at compile time.

        Activity

        Hide
        StephanEwen Stephan Ewen added a comment -

        Fixed in

        • 1.2.0 via 852c5298ea997175ec4e968350ac32db413c50f8
        • 1.1.4 via 5133bf9ea4c9633ad3aa840f81f6dedaa0026994
        Show
        StephanEwen Stephan Ewen added a comment - Fixed in 1.2.0 via 852c5298ea997175ec4e968350ac32db413c50f8 1.1.4 via 5133bf9ea4c9633ad3aa840f81f6dedaa0026994
        Hide
        swinard Sean Winard added a comment -

        Would love to! Unfortunately, I'm not approved yet by my employer to contribute and it will probably take a few more days at least. I have no problem if anyone wants to put in the fix in the meantime.

        Show
        swinard Sean Winard added a comment - Would love to! Unfortunately, I'm not approved yet by my employer to contribute and it will probably take a few more days at least. I have no problem if anyone wants to put in the fix in the meantime.
        Hide
        StephanEwen Stephan Ewen added a comment -

        Nice, thanks for diagnosing this.

        Would be great if you could open a pull request for this!

        Show
        StephanEwen Stephan Ewen added a comment - Nice, thanks for diagnosing this. Would be great if you could open a pull request for this!
        Hide
        swinard Sean Winard added a comment -

        It looks like changing the access modifier of the enum can make this work. In the case of my small example program, you can make the enum public and it will work. In my real program, I have a separate top-level enum class which is package-private, which is then in a POJO which is streamed. If I change the enum to public, then it will work fine. Note that the access modifier of regular classes does not seem to affect their ability to be serialized properly, in my testing. And again note that the proposed change for the EnumSerializer seems to work for enums with any access modifier.

        Show
        swinard Sean Winard added a comment - It looks like changing the access modifier of the enum can make this work. In the case of my small example program, you can make the enum public and it will work. In my real program, I have a separate top-level enum class which is package-private, which is then in a POJO which is streamed. If I change the enum to public , then it will work fine. Note that the access modifier of regular classes does not seem to affect their ability to be serialized properly, in my testing. And again note that the proposed change for the EnumSerializer seems to work for enums with any access modifier.
        Hide
        swinard Sean Winard added a comment -

        Cleaned up the test program

        Show
        swinard Sean Winard added a comment - Cleaned up the test program

          People

          • Assignee:
            StephanEwen Stephan Ewen
            Reporter:
            swinard Sean Winard
          • Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development