Skip to content

Conversation

@Jiayi-Liao
Copy link
Contributor

#12956 What is the purpose of the change

Users may execute heavy I/O operations in finalizeOnMaster function, which may block the operations in main executor such as client's request and heartbeats. This change is to move the job vertex's finalizeOnMaster call into dispatcher's IO executor, and transition the job status in main executor after all the finalizeOnMaster operations finish.

Brief change log

Use CompletableFuture.runAsync to execute the finalizeOnMaster in ioExecutor.

Verifying this change

The current unit testing in FinalizeOnMasterTest and MiniClusterITCase should cover this change.

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

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

Documentation

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

@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 85d8066 (Thu Sep 17 13:41:29 UTC 2020)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!
  • This pull request references an unassigned Jira ticket. According to the code contribution guide, tickets need to be assigned before starting with the implementation work.

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 Sep 17, 2020

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

@Jiayi-Liao
Copy link
Contributor Author

@flinkbot run azure

@Jiayi-Liao
Copy link
Contributor Author

@rmetzger Could you help me confirm the failed test? I found the ExecutionGraphRestartTest in the error stack but I'm not sure whether this is it because I rerun the tests on my laptop and it works.

@rmetzger
Copy link
Contributor

It is a common phenomenon that tests are failing more frequently, since it is a slower environment (only 2 cores, 8gb memory). On the first run, the test is passing, but if I configure IntelliJ to run until failure, the test gets stuck on my MacBook Pro locally as well. I'm pretty sure this failure is caused by the proposed changes.

if (transitionState(JobStatus.RUNNING, JobStatus.FINISHED)) {
onTerminalState(JobStatus.FINISHED);
}
FutureUtils.combineAll(futures).whenCompleteAsync((ignored, t) -> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
FutureUtils.combineAll(futures).whenCompleteAsync((ignored, t) -> {
FutureUtils.assertNoException(FutureUtils.combineAll(futures).whenCompleteAsync((ignored, t) -> {

Otherwise, the ExceptionUtils.rethrowIfFatalError won't do much.

ClusterEntryPointExceptionUtils.tryEnrichClusterEntryPointError(t);
failGlobal(new Exception("Failed to finalize execution on master", t));
} else {
if (transitionState(JobStatus.RUNNING, JobStatus.FINISHED)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

What do we do if the job has transitioned into a terminal state in the meantime (say CANCELLED?)
If I'm reading the code correctly, will throw a IllegalStateException if the current state is terminal.

Copy link
Contributor

Choose a reason for hiding this comment

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

Also, I believe we should not confirm any terminal state until the finalizeOnMaster method has finished.

@rmetzger
Copy link
Contributor

rmetzger commented Sep 21, 2020

I looked a bit into your change. I can not guarantee that my comments are pointing you in the right direction. I have no prior experience with the ExecutionGraph.
I believe it would be helpful to add test cases for edge cases such as "cancellation while finalizeOnMaster is blocked" and maybe an integration test with a slow finalizeOnMaster call to make sure no other components are affected (I see calls to release partitions "nearby").

@StephanEwen
Copy link
Contributor

Just leaving a quick comment here that the new sinks that @aljoscha and @guoweiM are working on will be different and not affected by this. So this here is more of a temporary fix.

@Jiayi-Liao
Copy link
Contributor Author

@rmetzger The test fails because we don't use a real executor as the jobMasterMainThreadExecutor, see #ComponentMainThreadExecutorServiceAdapter#forMainThread(), which means we cannot execute any async runnable on jobMasterMainThreadExecutor in the whole ExecutionGraphRestartTest. (don't know why it's designed at first)

@rmetzger
Copy link
Contributor

That is a very good remark @StephanEwen! I would actually propose to not address this issue anymore.
The finalizeOnMaster method has been executed in the main thread for many releases in Flink, and it seems that we are fixing this with the new sinks to be released with 1.12.
I don't think it's worth the effort implementing this for 1.11: We might potentially introduce new stability bugs into an old release.

@rmetzger
Copy link
Contributor

rmetzger commented Oct 5, 2020

I'm closing this PR for now. Please re-open or comment if you have a different opinion.

@rmetzger rmetzger closed this Oct 5, 2020
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