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

ASF GitHub Bot commented on FLINK-3974:
---------------------------------------

Github user XuPingyong commented on the issue:

    https://github.com/apache/flink/pull/2110
  
    I do not agree with this pr as it always copy  StreamRecord to downstream 
operator.
    
    StreamMap change the input StreamRecord, so this pr works well. But many 
operators do not change/reuse the input StreamRecord, like StreamFlatMap.
    
    The following code no not need the extra copy.
    `DataStream<A> input = ...
    input
        .flatmap(FlatMapFunction<A,B>...)
        .addSink(...);
    
    input
        .flatmap(FlatMapFunction<A,C>...)
        ​.addSink(...);`
    
        So I think we can change StreamMap to not reuse the input StreamRecord. 
And directly send the StreamRecord if objectReuse is set true.  
        What do you think?


> enableObjectReuse fails when an operator chains to multiple downstream 
> operators
> --------------------------------------------------------------------------------
>
>                 Key: FLINK-3974
>                 URL: https://issues.apache.org/jira/browse/FLINK-3974
>             Project: Flink
>          Issue Type: Bug
>          Components: DataStream API
>    Affects Versions: 1.0.3
>            Reporter: B Wyatt
>            Assignee: Aljoscha Krettek
>            Priority: Major
>             Fix For: 1.1.0
>
>         Attachments: ReproFLINK3974.java, bwyatt-FLINK3974.1.patch
>
>
> Given a topology that looks like this:
> {code:java}
> DataStream<A> input = ...
> input
>     .map(MapFunction<A,B>...)
>     .addSink(...);
> input
>     .map(MapFunction<A,C>...)
>     ​.addSink(...);
> {code}
> enableObjectReuse() will cause an exception in the form of 
> {{"java.lang.ClassCastException: B cannot be cast to A"}} to be thrown.
> It looks like the input operator calls {{Output<StreamRecord<A>>.collect}} 
> which attempts to loop over the downstream operators and process them.
> However, the first map operation will call {{StreamRecord<>.replace}} which 
> mutates the value stored in the StreamRecord<>.  
> As a result, when the {{Output<StreamRecord<A>>.collect}} call passes the 
> {{StreamRecord<A>}} to the second map operation it is actually a 
> {{StreamRecord<B>}} and behaves as if the two map operations were serial 
> instead of parallel.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to