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

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

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2110#discussion_r68240257
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
 ---
    @@ -386,4 +402,23 @@ public void close() {
                        }
                }
        }
    +
    +   /**
    +    * Special version of {@link BroadcastingOutputCollector} that performs 
a shallow copy of the
    +    * {@link StreamRecord} to ensure that multi-chaining works correctly.
    +    */
    +   private static final class CopyingBroadcastingOutputCollector<T> 
extends BroadcastingOutputCollector<T> {
    +
    +           public 
CopyingBroadcastingOutputCollector(Output<StreamRecord<T>>[] outputs) {
    +                   super(outputs);
    +           }
    +
    +           @Override
    +           public void collect(StreamRecord<T> record) {
    +                   for (Output<StreamRecord<T>> output : outputs) {
    +                           StreamRecord<T> shallowCopy = 
record.copy(record.getValue());
    +                           output.collect(shallowCopy);
    +                   }
    --- End diff --
    
    Here, the same. I think we could save one copying operation by giving the 
original `record` to the last output.


> 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