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

Sean Winard commented on FLINK-4977:
------------------------------------

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. 

> Enum serialization does not work properly
> -----------------------------------------
>
>                 Key: FLINK-4977
>                 URL: https://issues.apache.org/jira/browse/FLINK-4977
>             Project: Flink
>          Issue Type: Bug
>    Affects Versions: 1.1.3
>         Environment: Java SE 1.8.0_91
> Ubuntu 14.04.4 LTS (trusty)
>            Reporter: Sean Winard
>            Priority: Minor
>
> 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:
> {code:java}
> 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");
>     }
> }
> {code}
> {noformat}
> 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)
> {noformat}
> 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.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to