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

    https://github.com/apache/flink/pull/3484#discussion_r106139593
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
 ---
    @@ -441,26 +486,53 @@ public void close() {
        private static final class CopyingChainingOutput<T> extends 
ChainingOutput<T> {
                
                private final TypeSerializer<T> serializer;
    -           
    +
                public CopyingChainingOutput(
                                OneInputStreamOperator<T, ?> operator,
                                TypeSerializer<T> serializer,
    +                           OutputTag<T> outputTag,
                                StreamStatusProvider streamStatusProvider) {
    -                   super(operator, streamStatusProvider);
    +                   super(operator, streamStatusProvider, outputTag);
                        this.serializer = serializer;
                }
     
                @Override
                public void collect(StreamRecord<T> record) {
    +                   if (this.outputTag != null) {
    +                           // we are only responsible for emitting to the 
main input
    +                           return;
    +                   }
    +
    +                   pushToOperator(record);
    +           }
    +
    +           @Override
    +           public <X> void collect(OutputTag<?> outputTag, StreamRecord<X> 
record) {
    +                   if (this.outputTag == null || 
!this.outputTag.equals(outputTag)) {
    +                           // we are only responsible for emitting to the 
side-output specified by our
    +                           // OutputTag.
    +                           return;
    +                   }
    +
    +                   pushToOperator(record);
    +           }
    +
    +           @Override
    +           protected <X> void pushToOperator(StreamRecord<X> record) {
                        try {
    +                           // we know that the given outputTag matches our 
OutputTag so the record
    +                           // must be of the type that our operator (and 
Serializer) expects.
    +                           @SuppressWarnings("unchecked")
    +                           StreamRecord<T> castRecord = (StreamRecord<T>) 
record;
    +
                                numRecordsIn.inc();
    -                           StreamRecord<T> copy = 
record.copy(serializer.copy(record.getValue()));
    +                           StreamRecord<T> copy = 
castRecord.copy(serializer.copy(castRecord.getValue()));
                                operator.setKeyContextElement1(copy);
    -                           operator.processElement(copy);
    -                   }
    -                   catch (Exception e) {
    +                           operator.processElement(castRecord);
    --- End diff --
    
    This should be `copy`, not `castRecord`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to