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

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

GitHub user aljoscha opened a pull request:

    https://github.com/apache/flink/pull/2110

    [FLINK-3974] Fix object reuse with multi-chaining

    Before, a job would fail if object reuse was enabled and multiple
    operators were chained to one upstream operator. Now, we always create a
    shallow copy of the StreamRecord in OperatorChain.ChainingOutput because
    downstream operations change/reuse the StreamRecord.
    
    This fix was contributed by @wanderingbort (if this is the right github 
handle) as a patch on the Flink Jira. I can change the commit to attribute it 
to him but so far he didn't respond to my question about this on Jira.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/aljoscha/flink chaining/fix

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2110.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2110
    
----
commit 092f350cccbda32331f527c4eaf7ad3304fa1811
Author: Aljoscha Krettek <aljoscha.kret...@gmail.com>
Date:   2016-06-14T10:18:35Z

    [FLINK-3974] Fix object reuse with multi-chaining
    
    Before, a job would fail if object reuse was enabled and multiple
    operators were chained to one upstream operator. Now, we always create a
    shallow copy of the StreamRecord in OperatorChain.ChainingOutput because
    downstream operations change/reuse the StreamRecord.

----


> 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
>         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.3.4#6332)

Reply via email to