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

Enum serialization does not work in all cases

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Minor
    • Resolution: Fixed
    • 1.1.3
    • 1.1.4, 1.2.0
    • None
    • None
    • 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.

      Attachments

        Activity

          People

            sewen Stephan Ewen
            swinard Sean Winard
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: