Skip to content

[FLINK-39387][Runtime / Coordination] Fix flaky scheduler benchmark tests caused by thread assertion failure in async TDD creation#27882

Open
Myracle wants to merge 1 commit intoapache:masterfrom
Myracle:FLINK-39387-scheduler-benchmark-tests-bugfix
Open

[FLINK-39387][Runtime / Coordination] Fix flaky scheduler benchmark tests caused by thread assertion failure in async TDD creation#27882
Myracle wants to merge 1 commit intoapache:masterfrom
Myracle:FLINK-39387-scheduler-benchmark-tests-bugfix

Conversation

@Myracle
Copy link
Copy Markdown
Contributor

@Myracle Myracle commented Apr 2, 2026

What is the purpose of the change

After FLINK-38114 introduced asynchronous TDD (TaskDeploymentDescriptor) creation in Execution.deploy(), all 21 scheduler benchmark tests (e.g., PartitionReleaseInBatchJobBenchmarkTest) become flaky with the following error:

IllegalStateException: Cannot leave terminal state CANCELED to transition to SCHEDULED.

The root cause is that SchedulerBenchmarkUtils.createAndInitScheduler() uses ComponentMainThreadExecutorServiceAdapter.forMainThread() as the mainThreadExecutor, which wraps a DirectScheduledExecutorService with a strict thread identity assertion in its execute() method. When the async TDD creation future completes on the scheduledExecutorService thread and dispatches the thenComposeAsync callback to jobMasterMainThreadExecutor, the assertion fails because the calling thread is not the test main thread. This triggers markFailed()notifySchedulerNgAboutInternalTaskFailure() → scheduler failover → cancellation of all executions in the affected region, causing subsequent transitionState(SCHEDULED) calls to fail.

This PR replaces ComponentMainThreadExecutorServiceAdapter.forMainThread() with a SynchronousComponentMainThreadExecutor that skips thread identity checks, following the same approach as FLINK-38970.

Brief change log

  • Introduced SynchronousComponentMainThreadExecutor as a private inner class in SchedulerBenchmarkUtils, which uses DirectScheduledExecutorService to execute tasks synchronously on the calling thread and overrides assertRunningInMainThread() as a no-op
  • Replaced ComponentMainThreadExecutorServiceAdapter.forMainThread() with the new SynchronousComponentMainThreadExecutor in createAndInitScheduler()

Verifying this change

This change is already covered by existing tests, such as:

  • PartitionReleaseInBatchJobBenchmarkTest — previously flaky, now passes consistently (verified with 5 consecutive runs)
  • All 21 scheduler benchmark tests under org.apache.flink.runtime.scheduler.benchmark.** — all pass with BUILD SUCCESS

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, 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

@Myracle Myracle force-pushed the FLINK-39387-scheduler-benchmark-tests-bugfix branch from 2b648bf to 1f80b16 Compare April 2, 2026 03:55
@flinkbot
Copy link
Copy Markdown
Collaborator

flinkbot commented Apr 2, 2026

CI report:

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

* does not perform strict thread identity checks, avoiding flaky test failures when
* CompletableFuture callbacks are dispatched from background threads.
*/
private static class SynchronousComponentMainThreadExecutor
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we use NoMainThreadCheckComponentMainThreadExecutor here.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Good catch — there's indeed an existing NoMainThreadCheckComponentMainThreadExecutor in EventReceivingTasks that does exactly the same thing. I've extracted it into a shared top-level class under o.a.f.runtime.concurrent and reused it here. This also eliminates the duplication with the one in AdaptiveBatchSchedulerTest (FLINK-38970) — though I'll leave that cleanup for a separate follow-up since it's outside this PR's scope.

…ests caused by thread assertion failure in async TDD creation
@Myracle Myracle force-pushed the FLINK-39387-scheduler-benchmark-tests-bugfix branch from 1f80b16 to f54500a Compare April 2, 2026 05:19
Copy link
Copy Markdown
Contributor

@Izeren Izeren left a comment

Choose a reason for hiding this comment

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

I would say that proper way to fix this would be to ensure that we run things in the main thread. Somewhat like:

  // In SchedulerBenchmarkUtils:

  // Add a TestingComponentMainThreadExecutor extension or create one manually
  private static final TestingComponentMainThreadExecutor MAIN_THREAD_EXECUTOR =
          new TestingComponentMainThreadExecutor();

  public static DefaultScheduler createAndInitScheduler(...) {
      ComponentMainThreadExecutor mainThreadExecutor =
              MAIN_THREAD_EXECUTOR.getExecutor();  // real thread with identity check
      // ... same builder code ...
  }

  // Then every benchmark's setup/teardown/run wraps in:
  CompletableFuture.runAsync(() -> {
      execution.transitionState(ExecutionState.SCHEDULED);
      execution.deploy();
  }, mainThreadExecutor).join();

For example: https://github.com/apache/flink/pull/27740/changes

However, I can agree to argument that testing thread correctness may not be necessary as part of benchmark tests as long as we have this branch coverage elsewhere.

@github-actions github-actions bot added the community-reviewed PR has been reviewed by the community. label Apr 2, 2026
@Myracle
Copy link
Copy Markdown
Contributor Author

Myracle commented Apr 3, 2026

I would say that proper way to fix this would be to ensure that we run things in the main thread. Somewhat like:

  // In SchedulerBenchmarkUtils:

  // Add a TestingComponentMainThreadExecutor extension or create one manually
  private static final TestingComponentMainThreadExecutor MAIN_THREAD_EXECUTOR =
          new TestingComponentMainThreadExecutor();

  public static DefaultScheduler createAndInitScheduler(...) {
      ComponentMainThreadExecutor mainThreadExecutor =
              MAIN_THREAD_EXECUTOR.getExecutor();  // real thread with identity check
      // ... same builder code ...
  }

  // Then every benchmark's setup/teardown/run wraps in:
  CompletableFuture.runAsync(() -> {
      execution.transitionState(ExecutionState.SCHEDULED);
      execution.deploy();
  }, mainThreadExecutor).join();

For example: https://github.com/apache/flink/pull/27740/changes

However, I can agree to argument that testing thread correctness may not be necessary as part of benchmark tests as long as we have this branch coverage elsewhere.

That's a fair point — using TestingComponentMainThreadExecutor with proper thread affinity would be the most rigorous approach. I did consider it, but the main concern is that wrapping every benchmark operation in CompletableFuture.runAsync(..., mainThreadExecutor).join() introduces thread-switching overhead on each call, which could skew the benchmark measurements. Since these benchmarks are meant to isolate scheduler/deployment performance rather than validate threading contracts, I'd prefer keeping the lightweight NoMainThreadCheckComponentMainThreadExecutor here.

As you noted, thread correctness is already well-covered by DefaultSchedulerTest, AdaptiveSchedulerTest, and the integration tests — so we're not losing any safety net. Happy to revisit if you feel strongly about it though!

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

Labels

community-reviewed PR has been reviewed by the community.

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants