Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix silent failures in dispatch loop from stalling the pipeline #32922

Open
wants to merge 3 commits into
base: master
Choose a base branch
from

Conversation

m-trieu
Copy link
Contributor

@m-trieu m-trieu commented Oct 24, 2024

fix silent failures in dispatch loop from stalling the pipeline

changed all executors used by the Streaming harness to use TerminatingExecutors which attaches a WorkerUncaughtExceptionHandler to all of the thread factories used to create an Executor/ExecutorService. This means that any uncaught exception will terminate the JVM. This seems to be the original intention in DataflowWorkerHarnessHelper.initializeLogging() since it attaches an uncaught exception handler that will terminate the JVM on uncaught exceptions, however that was only applied to raw threads.

added test in SingleSourceWorkerHarness to ensure exception is propogated to main thread.

WorkerUncaughtExceptionHandler already has a unit test to ensure that it works.


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Mention the appropriate issue in your description (for example: addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.

Copy link
Contributor

Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment assign set of reviewers

@m-trieu
Copy link
Contributor Author

m-trieu commented Oct 25, 2024

R: @arunpandianp @scwhittle

Copy link
Contributor

Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control. If you'd like to restart, comment assign set of reviewers

});

try {
dispatchLoopFuture.get();
Copy link
Contributor

Choose a reason for hiding this comment

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

this is changing start() to be blocking.

If we want that we shoudl have clearer method name and also just get rid of the workProviderExecutor and use this thread directly.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

removed

SingleSourceWorkerHarness.GetWorkSender getWorkSender =
SingleSourceWorkerHarness.GetWorkSender.forAppliance(
() -> {
throw expected;
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm confused why the test of this isn't causing the runtime to halt. It makes me think the unhandled exception handler isn't being called and this is just working because of the changes to make start() blocking and call get on the future().

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added test to ensure that the runtime is being terminated

private static ThreadFactory terminatingThreadFactory(
ThreadFactoryBuilder threadFactoryBuilder, Logger logger) {
return threadFactoryBuilder
.setUncaughtExceptionHandler(new WorkerUncaughtExceptionHandler(logger))
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm not sure this is working if the executor thread is catchign the exception and putting it in the future.

I think we might need instead to wrap the executor so that scheduled tasks are wrapped and then catch exceptions and call termination method like com/google/common/util/concurrent/WrappingExecutorService.java

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added to test to ensure that we are halting the JVM

@m-trieu
Copy link
Contributor Author

m-trieu commented Oct 25, 2024

back to you thank you!
@scwhittle

@@ -157,6 +157,19 @@ private static Optional<StreamingConfigTask> fetchConfigWithRetry(
}
}

private static Optional<ComputationConfig> createComputationConfig(StreamingConfigTask config) {
Copy link
Contributor

Choose a reason for hiding this comment

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

can we revert the unrelated change, to keep the diff small and on topic?

@@ -228,6 +228,22 @@ private static void shutdownExecutor(ScheduledExecutorService executor) {
}
}

// Calculates the PerWorkerMetrics reporting frequency, ensuring alignment with the
Copy link
Contributor

Choose a reason for hiding this comment

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

can we revert the unrelated change, to keep the diff small and on topic?

@@ -346,10 +348,7 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o
.setSizeMb(options.getWorkerCacheMb())
.setSupportMapViaMultimap(options.isEnableStreamingEngine())
.build();
Function<String, ScheduledExecutorService> executorSupplier =
threadName ->
Executors.newSingleThreadScheduledExecutor(
Copy link
Contributor

Choose a reason for hiding this comment

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

I think changing Executors.newSingleThreadScheduledExecutor to Executors.newSingleThreadExecutor should fix the problem? We don't need to introduce a new Executor.

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.

3 participants