rkhachatryan commented on a change in pull request #10151: [FLINK-14231] Handle
the processing-time timers before closing operator to properly support endInput
URL: https://github.com/apache/flink/pull/10151#discussion_r346763174
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -1395,6 +1381,114 @@ private void
checkpointStreamOperator(StreamOperator<?> op) throws Exception {
}
}
+ /**
+ * This class executes {@link StreamOperator#close()} of all operators
in the chain
+ * of this {@link StreamTask} one by one through the mailbox thread.
Closing happens
+ * from <b>head to tail</b> operator in the chain, contrary to {@link
StreamOperator#open()}
+ * which happens <b>tail to head</b> (see {@link #openAllOperators()}.
Review comment:
Here we are traversing chain "from the outside". Alternative approach would
be to allow operators to pass "close" message to the next one (when they're
ready). So when the 1st op receives it, it waits for all its timers to fire and
then emits "close" message to all its outputs.
Pros:
1. will work for any topology, not only for linear chains
2. less coupling, e.g. no need to maintain a map of processingTimeServices
3. (subjective) more simple, better reflects domain; no need to play with
indexes and less with Futures
Cons:
1. (without wrapper) changes to PublicEvolving API (though we probably have
to do it anyway to ask operator for maximum wait time)
2. (without wrapper) the responsibility of passing the signal is moved to
the user code
Wrapper implementation could look like this:
1. add class StreamOperatorWrapper
```
class StreamOperatorWrapper {
private StreamOperator wrapped;
private StreamOperatorWrapper next;
private ProcessingTimeService timerService;
public CompletableFuture<> endOfInput() {
timerService.shutdown()
.thenApply(wrapped.close()); // todo: sync (mailbox?), handle errors,
supply thread?
.thenApply(next.endOfInput);
}
}
```
2. And then wrap operators while building operator chain
3. Expose OperatorChain.headWrapper.close and call it from StreamTask
Non-wrapper implementation could like this:
1. add constant StreamStatus.FINISHED
2. in StreamProcessor: call output.emitStreamStatus(StreamStatus.FINISHED)
if END_OF_INPUT
3. add method endOfInput() to org.apache.flink.streaming.api.operators.Output
4. add method endOfInput() to StreamOperator // contract: call
output.endOfInput when ready (e.g. using ProcessingTimeService)
5. in operatorChain.toggleStreamStatus call operator.endOfInput if status is
EOI
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services