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

Invoke teardown when DoFn throws in portable runners #32522

Merged
merged 3 commits into from
Oct 9, 2024

Conversation

Abacn
Copy link
Contributor

@Abacn Abacn commented Sep 20, 2024

Fix (partly) #31381

Please add a meaningful description for your change here


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.

@Abacn
Copy link
Contributor Author

Abacn commented Sep 20, 2024

java.lang.RuntimeException: Dataflow job 2024-09-20_12_32_18-8696550461908579531 terminated in state RUNNING but did not return a failure reason.
	at org.apache.beam.runners.dataflow.TestDataflowRunner.run(TestDataflowRunner.java:149)

however the job does succeeded. The worker log shows

2024-09-20 15:34:47.080 EDT
Entering instruction processing loop
2024-09-20 15:35:02.066 EDT
Memory is used/total/max = 392/890/4951 MB, GC last/max = 3.47/3.47 % (configured threshold: 50.00%), #pushbacks=0, gc thrashing=false
2024-09-20 15:40:02.073 EDT
Memory is used/total/max = 392/890/4951 MB, GC last/max = 0.00/3.47 % (configured threshold: 50.00%), #pushbacks=0, gc thrashing=false
2024-09-20 15:45:02.078 EDT
Memory is used/total/max = 88/930/4951 MB, GC last/max = 1.06/3.47 % (configured threshold: 50.00%), #pushbacks=0, gc thrashing=false
2024-09-20 15:47:46.638 EDT
Filepattern gs://apache-beam-samples/shakespeare/sonnets.txt matched 1 files with total size 95662

took extended period of time to match the file source (actually start to process data). Intermittent UW issue?

// going to be re-used.
for (ThrowingRunnable teardownFunction : Lists.reverse(this.getTearDownFunctions())) {
try {
teardownFunction.run();
Copy link
Contributor

Choose a reason for hiding this comment

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

if we throw the exception above during one of the start functions we may not have called all of them. That could be confusing since then finish could be called without corresponding start. If we knew which start/teardown functions corresponeded (ie change to a single list with both matched up) we coudl remember the index we made it to on start functions to call teardown only on the right ones.

Copy link
Contributor Author

@Abacn Abacn Sep 23, 2024

Choose a reason for hiding this comment

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

The naming here was confusing. startFunction are for startBundle. tearDown corresponds to setup in terms of DoFns, where setup was called as early as:

org.apache.beam.sdk.transforms.reflect.DoFnInvokers.tryInvokeSetupFor(DoFnInvokers.java:53)
org.apache.beam.fn.harness.FnApiDoFnRunner.<init>(FnApiDoFnRunner.java:505)
org.apache.beam.fn.harness.FnApiDoFnRunner$Factory.createRunnerForPTransform(FnApiDoFnRunner.java:195)
org.apache.beam.fn.harness.FnApiDoFnRunner$Factory.createRunnerForPTransform(FnApiDoFnRunner.java:163)
org.apache.beam.fn.harness.control.ProcessBundleHandler.createRunnerAndConsumersForPTransformRecursively(ProcessBundleHandler.java:307)
org.apache.beam.fn.harness.control.ProcessBundleHandler.createRunnerAndConsumersForPTransformRecursively(ProcessBundleHandler.java:261)
org.apache.beam.fn.harness.control.ProcessBundleHandler.createRunnerAndConsumersForPTransformRecursively(ProcessBundleHandler.java:261)
org.apache.beam.fn.harness.control.ProcessBundleHandler.createRunnerAndConsumersForPTransformRecursively(ProcessBundleHandler.java:261)
org.apache.beam.fn.harness.control.ProcessBundleHandler.createBundleProcessor(ProcessBundleHandler.java:861)
org.apache.beam.fn.harness.control.ProcessBundleHandler.lambda$processBundle$0(ProcessBundleHandler.java:511)
org.apache.beam.fn.harness.control.ProcessBundleHandler$BundleProcessorCache.get(ProcessBundleHandler.java:972)
org.apache.beam.fn.harness.control.ProcessBundleHandler.processBundle(ProcessBundleHandler.java:507)

that was when BundleProcessor gets created. Therefore in bundleProcessor.discard call DoFn's @Setup have been called before.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If the question is about throw the exception in setupFunction, then tearDown is invoked earlier, at

and createBundleProcessor won't complete and discard() method won't invoked

Copy link
Contributor

Choose a reason for hiding this comment

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

the DoFnInvoker link where teardown is called if exception during startup is for a single DoFn.

If we have fused DoFnA->DoFnB we start in reverse order. If A throws exception in setup, tearDown is invoked on DoFnA. Do we call teardown on B which had startup invoked? If not we probably need some try catch in createBundleProcessor to invoke the added teardown functions.

Copy link
Contributor Author

@Abacn Abacn Sep 26, 2024

Choose a reason for hiding this comment

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

Tested that teardown on B isn't called, indeed. However this is same for legacy runner, except for legacy runner setup is called in sequence order (not reverse), which means if B throws in setup, tearDown in A won't be called.

Prefer to note this as a separate issue for now.

@Abacn
Copy link
Contributor Author

Abacn commented Sep 23, 2024

Java PVR Spark Batch failed twice, each time different test. This is due to #30512. None of them are related to ParDo lifecycle.

@Abacn
Copy link
Contributor Author

Abacn commented Sep 23, 2024

Rerun Dataflow Example and test passed. Looks like intermittent issue (cause the SDK harness start to work 12 minutes late)

@Abacn Abacn marked this pull request as ready for review September 23, 2024 15:28
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

@Abacn
Copy link
Contributor Author

Abacn commented Sep 24, 2024

PTAL @scwhittle @robertwb thanks!

Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @kennknowles for label java.
R: @damccorm for label build.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

} catch (Throwable e) {
LOG.warn(
"Exceptions are thrown from DoFn.teardown method when trying to discard "
+ "ProcessBundleHandler",
Copy link
Contributor Author

Choose a reason for hiding this comment

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

adjusted to warning and wording of it. Originally it was an Error, which may cause confuse on the cause of the error in case of a failed pipeline (the cause is upstream error in processElement, etc. not tearDown throws)

Copy link
Contributor

github-actions bot commented Oct 4, 2024

Reminder, please take a look at this pr: @kennknowles @damccorm

@Abacn
Copy link
Contributor Author

Abacn commented Oct 4, 2024

R: @scwhittle

Copy link
Contributor

github-actions bot commented Oct 4, 2024

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

@Abacn
Copy link
Contributor Author

Abacn commented Oct 8, 2024

rebased onto latest HEAD to resolve merge conflict (on CHANGES.md)

@scwhittle
Copy link
Contributor

Looks like the spark runner failure might be real from a test that is now not being filtered.

@Abacn
Copy link
Contributor Author

Abacn commented Oct 9, 2024

PostCommit Java PVR Spark Batch two attempt different random test failing due to #30512, not related to this PR, merging for now

@Abacn Abacn merged commit c31d81c into apache:master Oct 9, 2024
30 of 32 checks passed
@Abacn Abacn deleted the invoketeardown branch October 9, 2024 14:58
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.

2 participants