flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tillrohrmann <...@git.apache.org>
Subject [GitHub] flink pull request #2110: [FLINK-3974] Fix object reuse with multi-chaining
Date Wed, 22 Jun 2016 15:38:27 GMT
Github user tillrohrmann commented on a diff in the pull request:

    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
    @@ -306,8 +306,9 @@ public ChainingOutput(OneInputStreamOperator<T, ?> operator)
     		public void collect(StreamRecord<T> record) {
     			try {
    -				operator.setKeyContextElement1(record);
    -				operator.processElement(record);
    +				StreamRecord<T> shallowCopy = record.copy(record.getValue());
    +				operator.setKeyContextElement1(shallowCopy);
    +				operator.processElement(shallowCopy);
    --- End diff --
    Actually I'm wondering whether the `ChainingOutput` is the right place to do this copying.
Wouldn't it make more sense to do it in the `BroadcastingOutputCollector`, because only if
we have a branching chained data flow we have to make sure that every down stream operator
get his own copy of the record. For simple chaining it should be correct to reuse the stream
    So I would adapt the `collect` method of `BroadcastingOutputCollector` the following way:
    public void collect(StreamRecord<T> record) {
    	for (int i = 0; i < outputs.length - 1; i++) {
    		StreamRecord<T> shallowCopy = record.copy(record.getValue());
    	outputs[outputs.length - 1].collect(record);

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 infrastructure@apache.org or file a JIRA ticket
with INFRA.

View raw message