Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id EA5D6200BC0 for ; Mon, 31 Oct 2016 19:00:01 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id E9053160B06; Mon, 31 Oct 2016 18:00:01 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 207CC160B05 for ; Mon, 31 Oct 2016 19:00:00 +0100 (CET) Received: (qmail 77241 invoked by uid 500); 31 Oct 2016 17:59:59 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 77200 invoked by uid 99); 31 Oct 2016 17:59:59 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 31 Oct 2016 17:59:59 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id B12392C2A6E for ; Mon, 31 Oct 2016 17:59:59 +0000 (UTC) Date: Mon, 31 Oct 2016 17:59:59 +0000 (UTC) From: "Sean Winard (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (FLINK-4977) Enum serialization does not work properly MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Mon, 31 Oct 2016 18:00:02 -0000 [ https://issues.apache.org/jira/browse/FLINK-4977?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sean Winard updated FLINK-4977: ------------------------------- 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: {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.(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 type. was: 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) { final StreamExecutionEnvironment environment = StreamExecutionEnvironment.getExecutionEnvironment(); environment.setParallelism(1); environment.fromElements(MyEnum.NONE, MyEnum.SOMETHING, MyEnum.EVERYTHING) .addSink(x -> System.err.println(x.name())); } } {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.(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 type. > 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.(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 type. -- This message was sent by Atlassian JIRA (v6.3.4#6332)