pnowojski commented on a change in pull request #16905:
URL: https://github.com/apache/flink/pull/16905#discussion_r696468015
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java
##########
@@ -244,6 +234,28 @@ protected void createInputProcessor(
return resultFuture;
}
+ private CompletableFuture<Boolean> triggerStopWithSavepointWithDrainAsync(
+ CheckpointMetaData checkpointMetaData, CheckpointOptions
checkpointOptions) {
+
+ List<CompletableFuture<Void>> sourceFinishedFutures = new
ArrayList<>();
+ mainMailboxExecutor.execute(
+ () -> {
+
setSynchronousSavepoint(checkpointMetaData.getCheckpointId(), true);
+ operatorChain
+ .getSourceTaskInputs()
+ .forEach(s ->
sourceFinishedFutures.add(s.getOperator().stop()));
+ },
+ "stop chained Flip-27 source for stop-with-savepoint --drain");
+
+ CompletableFuture<Void> sourcesStopped =
FutureUtils.waitForAll(sourceFinishedFutures);
Review comment:
I think you have a race condition. You are accessing not thread safe
`sourceFinishedFutures` from two different threads.
In my previous suggestion I meant to use something like:
`Future<T> MailboxExecutor#submit(java.util.concurrent.Callable<T>,
java.lang.String)`. You can submit a callable that will collect the sources
stopping futures and return them as a `Future<CompletableFuture<Void>>`, that
you could chain in the line below.
One caveat might be that you might need to either modify `submit()` to
return `CompletableFuture`, or to introduce another version that will do just
that? `MailboxExecutor` is `@PublicEvolving`, and probably not used very
widely, and only by the power users, so we should be ok with changing it's
interface.
--
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]