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
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test"
"modification": 1
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 0
"modification": 1
}
2 changes: 1 addition & 1 deletion .github/trigger_files/beam_PostCommit_Java_PVR_Samza.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 0
"modification": 1
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 0
"modification": 1
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 0
"modification": 1
}
1 change: 1 addition & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@
## Bugfixes

* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)).
* (Java) Fixed tearDown not invoked when DoFn throws on Portable Runners ([#18592](https://github.com/apache/beam/issues/18592), [#31381](https://github.com/apache/beam/issues/31381)).

## Security Fixes
* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)).
Expand Down
1 change: 0 additions & 1 deletion runners/flink/job-server/flink_job_server.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,6 @@ def portableValidatesRunnerTask(String name, boolean streaming, boolean checkpoi
excludeCategories 'org.apache.beam.sdk.testing.UsesCustomWindowMerging'
excludeCategories 'org.apache.beam.sdk.testing.UsesFailureMessage'
excludeCategories 'org.apache.beam.sdk.testing.UsesGaugeMetrics'
excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
excludeCategories 'org.apache.beam.sdk.testing.UsesMapState'
excludeCategories 'org.apache.beam.sdk.testing.UsesMultimapState'
excludeCategories 'org.apache.beam.sdk.testing.UsesSetState'
Expand Down
2 changes: 1 addition & 1 deletion runners/google-cloud-dataflow-java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ def commonLegacyExcludeCategories = [
'org.apache.beam.sdk.testing.UsesGaugeMetrics',
'org.apache.beam.sdk.testing.UsesMultimapState',
'org.apache.beam.sdk.testing.UsesTestStream',
'org.apache.beam.sdk.testing.UsesParDoLifecycle',
'org.apache.beam.sdk.testing.UsesParDoLifecycle', // doesn't support remote runner
'org.apache.beam.sdk.testing.UsesMetricsPusher',
'org.apache.beam.sdk.testing.UsesBundleFinalizer',
]
Expand Down
3 changes: 2 additions & 1 deletion runners/samza/job-server/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,6 @@ def portableValidatesRunnerTask(String name, boolean docker) {
excludeCategories 'org.apache.beam.sdk.testing.UsesCustomWindowMerging'
excludeCategories 'org.apache.beam.sdk.testing.UsesFailureMessage'
excludeCategories 'org.apache.beam.sdk.testing.UsesGaugeMetrics'
excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
excludeCategories 'org.apache.beam.sdk.testing.UsesMapState'
excludeCategories 'org.apache.beam.sdk.testing.UsesMultimapState'
excludeCategories 'org.apache.beam.sdk.testing.UsesSetState'
Expand Down Expand Up @@ -127,6 +126,8 @@ def portableValidatesRunnerTask(String name, boolean docker) {
excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testParDoShiftTimestampInvalid'
// TODO(https://github.com/apache/beam/issues/21144)
excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testParDoShiftTimestampInvalidZeroAllowed'
// TODO(https://github.com/apache/beam/issues/32520)
excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoLifecycleTest.testTeardownCalledAfterExceptionIn*Stateful'
// TODO(https://github.com/apache/beam/issues/21145)
excludeTestsMatching 'org.apache.beam.sdk.transforms.DeduplicateTest.testEventTime'
// TODO(https://github.com/apache/beam/issues/21146)
Expand Down
2 changes: 0 additions & 2 deletions runners/spark/job-server/spark_job_server.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,6 @@ def portableValidatesRunnerTask(String name, boolean streaming, boolean docker,
excludeCategories 'org.apache.beam.sdk.testing.UsesFailureMessage'
excludeCategories 'org.apache.beam.sdk.testing.UsesGaugeMetrics'
excludeCategories 'org.apache.beam.sdk.testing.UsesPerKeyOrderedDelivery'
excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
excludeCategories 'org.apache.beam.sdk.testing.UsesMapState'
excludeCategories 'org.apache.beam.sdk.testing.UsesSetState'
excludeCategories 'org.apache.beam.sdk.testing.UsesOrderedListState'
Expand Down Expand Up @@ -187,7 +186,6 @@ def portableValidatesRunnerTask(String name, boolean streaming, boolean docker,
excludeCategories 'org.apache.beam.sdk.testing.UsesGaugeMetrics'
excludeCategories 'org.apache.beam.sdk.testing.UsesPerKeyOrderedDelivery'
excludeCategories 'org.apache.beam.sdk.testing.UsesPerKeyOrderInBundle'
excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
excludeCategories 'org.apache.beam.sdk.testing.UsesMapState'
excludeCategories 'org.apache.beam.sdk.testing.UsesMultimapState'
excludeCategories 'org.apache.beam.sdk.testing.UsesSetState'
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -596,7 +596,11 @@ public BeamFnApi.InstructionResponse.Builder processBundle(BeamFnApi.Instruction
request.getProcessBundle().getProcessBundleDescriptorId(), bundleProcessor);
return BeamFnApi.InstructionResponse.newBuilder().setProcessBundle(response);
} catch (Exception e) {
// Make sure we clean-up from the active set of bundle processors.
// Make sure we clean up from the active set of bundle processors.
LOG.debug(
"Discard bundleProcessor for {} after exception: {}",
request.getProcessBundle().getProcessBundleDescriptorId(),
e.getMessage());
bundleProcessorCache.discard(bundleProcessor);
throw e;
}
Expand Down Expand Up @@ -1168,13 +1172,26 @@ void discard() {
if (this.bundleCache != null) {
this.bundleCache.clear();
}
// setupFunctions are invoked in createBundleProcessor. Invoke teardownFunction here as the
// BundleProcessor is already removed from cache and won't 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.

} 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)

e);
}
}
getMetricsEnvironmentStateForBundle().discard();
for (BeamFnDataOutboundAggregator aggregator : getOutboundAggregators().values()) {
aggregator.discard();
}
}
}

// this is called in cachedBundleProcessors removal listener
void shutdown() {
for (ThrowingRunnable tearDownFunction : getTearDownFunctions()) {
LOG.debug("Tearing down function {}", tearDownFunction);
Expand Down
Loading