pnowojski commented on a change in pull request #16589:
URL: https://github.com/apache/flink/pull/16589#discussion_r681172483
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java
##########
@@ -116,20 +116,21 @@ protected void finishTask() throws Exception {
public CompletableFuture<Boolean> triggerCheckpointAsync(
CheckpointMetaData checkpointMetaData, CheckpointOptions
checkpointOptions) {
if (!isExternallyInducedSource) {
- return super.triggerCheckpointAsync(checkpointMetaData,
checkpointOptions);
+ if (checkpointOptions.getCheckpointType().shouldDrain()) {
+ return triggerStopWithSavepointWithDrain(checkpointMetaData,
checkpointOptions);
+ } else {
+ return super.triggerCheckpointAsync(checkpointMetaData,
checkpointOptions);
+ }
} else {
return CompletableFuture.completedFuture(isRunning());
}
}
- @Override
- protected CompletableFuture<Void> stopIfSourceMailboxAction() throws
Exception {
+ private CompletableFuture<Boolean> triggerStopWithSavepointWithDrain(
Review comment:
`triggerStopWithSavepointWithDrainAsync`
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java
##########
@@ -195,6 +195,22 @@ protected void createInputProcessor(
// EndOfPartitionEvent, we would not complement barriers for the
// unfinished network inputs, and the checkpoint would be triggered
// after received all the EndOfPartitionEvent.
+ if (options.getCheckpointType().shouldDrain()) {
+ CompletableFuture<Void> sourcesStopped =
+ FutureUtils.waitForAll(
+ operatorChain.getSourceTaskInputs().stream()
+ .map(s -> s.getOperator().stop())
+ .collect(Collectors.toList()));
+
+ return sourcesStopped.thenCompose(
+ ignore -> triggerSourcesCheckpointInMailbox(metadata,
options));
Review comment:
Don't we need something like `FutureUtils.assertNoException` wrapped
around this? In case of an exception happening after completing
`sourcesStopped`, but before enqueuing mailbox action?
ditto about other places when we are composing futures?
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
##########
@@ -261,6 +277,29 @@ private void interruptSourceThread(boolean interrupt) {
}
}
+ private CompletableFuture<Boolean> triggerStopWithSavepointWithDrain(
Review comment:
bump
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]