Skip to content

Conversation

@pnowojski
Copy link
Contributor

With ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH enabled, final checkpoint can deadlock (or timeout after very long time) if there is a race condition between selecting tasks to trigger checkpoint on and finishing tasks. FLINK-21246 was supposed to handle it, but it doesn't work as expected, because futures from:
org.apache.flink.runtime.taskexecutor.TaskExecutor#triggerCheckpoint
and
org.apache.flink.streaming.runtime.tasks.StreamTask#triggerCheckpointAsync
are not linked together. TaskExecutor#triggerCheckpoint reports that checkpoint has been successfully triggered, while StreamTask might have actually finished.

Verifying this change

TODO: implement test

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (yes / no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (yes / no)
  • The serializers: (yes / no / don't know)
  • The runtime per-record code paths (performance sensitive): (yes / no / don't know)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / no / don't know)
  • The S3 file system connector: (yes / no / don't know)

Documentation

  • Does this pull request introduce a new feature? (yes / no)
  • If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)

@flinkbot
Copy link
Collaborator

Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
to review your pull request. We will use this comment to track the progress of the review.

Automated Checks

Last check on commit b8906e0 (Thu Aug 12 17:11:17 UTC 2021)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!

Mention the bot in a comment to re-run the automated checks.

Review Progress

  • ❓ 1. The [description] looks good.
  • ❓ 2. There is [consensus] that the contribution should go into to Flink.
  • ❓ 3. Needs [attention] from.
  • ❓ 4. The change fits into the overall [architecture].
  • ❓ 5. Overall code [quality] is good.

Please see the Pull Request Review Guide for a full explanation of the review process.

Details
The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commands
The @flinkbot bot supports the following commands:

  • @flinkbot approve description to approve one or more aspects (aspects: description, consensus, architecture and quality)
  • @flinkbot approve all to approve all aspects
  • @flinkbot approve-until architecture to approve everything until architecture
  • @flinkbot attention @username1 [@username2 ..] to require somebody's attention
  • @flinkbot disapprove architecture to remove an approval you gave earlier

@flinkbot
Copy link
Collaborator

flinkbot commented Aug 12, 2021

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run travis re-run the last Travis build
  • @flinkbot run azure re-run the last Azure build

@pnowojski
Copy link
Contributor Author

Frankly I don't have a good idea how to test this in a meaningful way. Especially the failure case. Maybe we should just relay on the WIP FLINK-21090 that actually discovered this problem?

@gaoyunhaii
Copy link
Contributor

gaoyunhaii commented Aug 13, 2021

Hi Piotr @pnowojski , very thanks for opening the PR!

It seems to me the current implementation get the result whether the trigger is successful only after it actually execute the synchronous part? It seems we might need to be that strict: currently as long as invokable.triggerCheckpointAsync is called without exception, we could ensure the checkpoint must be performed. This is due to if the method is successful, it ensures the mailbox is not prepareClose() and isRunning = true. Since isRunning is set to false after the mailbox is drained, thus we ensures when this mail is processed and the checkpoint is triggered, isRunning must be true.

Although logically we only care about the "false" result, but since Akka has a timeout, thus if the mail queued for a long time or the synchronous part takes long time, we might meet with AkkaAskTimeout and cancel the checkpoint wrongly?

@pnowojski
Copy link
Contributor Author

I would prefer to not complicate this RPC chain with having two different definitions of what Completed future mean. However I think:

the synchronous part takes long time

is a valid concern. I will just double check if indeed this would cause akka timeouts.

Comment on lines 956 to 958
CompletableFuture<Acknowledge> resultFuture = new CompletableFuture<>();
task.triggerCheckpointBarrier(checkpointId, checkpointTimestamp, checkpointOptions)
.thenApply(
triggerResult ->
triggerResult
? resultFuture.complete(Acknowledge.get())
: resultFuture.completeExceptionally(
new CheckpointException(
"Task is not running?",
CheckpointFailureReason
.TRIGGER_CHECKPOINT_FAILURE)));

return CompletableFuture.completedFuture(Acknowledge.get());
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🤦 note that in this version CompletableFuture.completedFuture(Acknowledge.get()) is returned regardless if the resultFuture succeeded or failed...

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for also missing this issue...

@gaoyunhaii
Copy link
Contributor

Very thanks Piotr @pnowojski for updating the PR! The new method looks good to me. The only concern is that there might be repeat decline some times logically, but in realistic the StreamTask only declines the checkpoints if isRunning = false, which should not happen due to the same reason as the above comment, and even if there are repeat decline there should be no problems.

And since now the change is limited to Task, perhaps we could add some UT in TaskTest to check the decline is indeed called in the three places? Might be something like:

public void testDeclineCheckpointIfTaskIsNotRunning() throws Exception {
        TestCheckpointResponder testCheckpointResponder = new TestCheckpointResponder();
        final Task task =
                createTaskBuilder().setCheckpointResponder(testCheckpointResponder).build();
        task.triggerCheckpointBarrier(
                1,
                1,
                CheckpointOptions.alignedNoTimeout(
                        CheckpointType.CHECKPOINT,
                        CheckpointStorageLocationReference.getDefault()));
        assertEquals(1, testCheckpointResponder.getDeclineReports().size());
        assertEquals(
                CheckpointFailureReason.CHECKPOINT_DECLINED_TASK_NOT_READY,
                testCheckpointResponder
                        .getDeclineReports()
                        .get(0)
                        .getCause()
                        .getCheckpointFailureReason());
    }

…f StreamTask is not running

With ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH enabled, final checkpoint can deadlock
(or timeout after very long time) if there is a race condition between selecting
tasks to trigger checkpoint on and finishing tasks. FLINK-21246 was supposed to
handle it, but it doesn't work as expected, because futures from:

org.apache.flink.runtime.taskexecutor.TaskExecutor#triggerCheckpoint
and
org.apache.flink.streaming.runtime.tasks.StreamTask#triggerCheckpointAsync

are not linked together. TaskExecutor#triggerCheckpoint reports that checkpoint
has been successfully triggered, while StreamTask might have actually finished.
@pnowojski
Copy link
Contributor Author

Thanks @gaoyunhaii for your suggestion. Your test wouldn't actually test the added cases by me, as task wouldn't be in running state, but I've easily added coverage for all 4 declining cases.

Copy link
Contributor

@gaoyunhaii gaoyunhaii left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Very thanks Piotr @pnowojski for updating the PR! LGTM and +1 to merge~

@gaoyunhaii
Copy link
Contributor

@flinkbot run azure

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants