[ https://issues.apache.org/jira/browse/FLINK-12113?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
yankai zhang updated FLINK-12113: --------------------------------- Description: {code:java} interface IS<E> extends Iterator<E>, Serializable { } StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.fromCollection(new IS<Object>() { @Override public boolean hasNext() { return false; } @Override public Object next() { return null; } }, Object.class); {code} Code piece above throws exception: {code:java} org.apache.flink.api.common.InvalidProgramException: The implementation of the SourceFunction is not serializable. The object probably contains or references non serializable fields. at org.apache.flink.api.java.ClosureCleaner.clean(ClosureCleaner.java:99) ....{code} And my workaround is wrapping clean around iterator instance, like this: {code:java} StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.fromCollection(env.clean(new IS<Object>() { @Override public boolean hasNext() { return false; } @Override public Object next() { return null; } }), Object.class); {code} was: {code:java} interface IS<E> extends Iterator<E>, Serializable { } StreamExecutionEnvironment .getExecutionEnvironment() .fromCollection(new IS<Object>() { @Override public boolean hasNext() { return false; } @Override public Object next() { return null; } }, Object.class); {code} Code piece above throws exception: {code:java} org.apache.flink.api.common.InvalidProgramException: The implementation of the SourceFunction is not serializable. The object probably contains or references non serializable fields. at org.apache.flink.api.java.ClosureCleaner.clean(ClosureCleaner.java:99) ....{code} > User code passing to fromCollection(Iterator, Class) not cleaned > ---------------------------------------------------------------- > > Key: FLINK-12113 > URL: https://issues.apache.org/jira/browse/FLINK-12113 > Project: Flink > Issue Type: Bug > Components: API / DataStream > Affects Versions: 1.7.2 > Reporter: yankai zhang > Priority: Major > > > {code:java} > interface IS<E> extends Iterator<E>, Serializable { } > StreamExecutionEnvironment env = > StreamExecutionEnvironment.getExecutionEnvironment(); > env.fromCollection(new IS<Object>() { > @Override > public boolean hasNext() { > return false; > } > @Override > public Object next() { > return null; > } > }, Object.class); > {code} > Code piece above throws exception: > {code:java} > org.apache.flink.api.common.InvalidProgramException: The implementation of > the SourceFunction is not serializable. The object probably contains or > references non serializable fields. > at org.apache.flink.api.java.ClosureCleaner.clean(ClosureCleaner.java:99) > ....{code} > And my workaround is wrapping clean around iterator instance, like this: > > {code:java} > StreamExecutionEnvironment env = > StreamExecutionEnvironment.getExecutionEnvironment(); > env.fromCollection(env.clean(new IS<Object>() { > @Override > public boolean hasNext() { > return false; > } > @Override > public Object next() { > return null; > } > }), Object.class); > {code} > > > -- This message was sent by Atlassian JIRA (v7.6.3#76005)