Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
17 commits
Select commit Hold shift + click to select a range
fda75e0
Adds bundleFinalizer support to non-portable worker.
acrites Feb 26, 2026
1ec825d
Removes check preventing stateful DoFn's with bundle finalizers from …
acrites Feb 26, 2026
4694de8
Changes finalize method to only acquire the lock once. Runs finalize …
acrites Feb 27, 2026
8570168
Addresses PR review comments, mostly around improving tests.
acrites Mar 2, 2026
5e1642c
Addresses minor code comments. Adds back in tests for batch, non-port…
acrites Mar 4, 2026
a65b537
Merge branch 'master' into bundle-finalizer
acrites Mar 4, 2026
6dcafc4
Ran spotlessApply
acrites Mar 4, 2026
081a900
Merge branch 'bundle-finalizer' of https://github.com/acrites/beam in…
acrites Mar 4, 2026
7413001
Addresses more PR review comments. Triggering validates runner tests.
acrites Mar 6, 2026
5b24e50
Adds finalization callbacks directly when flushing instead of copying…
acrites Mar 9, 2026
4f01761
Ran spotlessApply and retriggering streaming ValidatesRunner tests.
acrites Mar 9, 2026
591aeaa
Retriggering validates runner tests.
acrites Mar 13, 2026
d7df552
Changes Long to long for build error.
acrites Mar 13, 2026
a50b859
Starts finalization callback cleanup thread lazily. This is for tests…
acrites Mar 16, 2026
047592e
Merge remote-tracking branch 'upstream/master' into bundle-finalizer
acrites Mar 17, 2026
7c62f6f
Excludes bundle finalizer tests for appliance only and retriggers val…
acrites Mar 17, 2026
24ce8f6
Switches which Pair library being imported since it was causing undec…
acrites Mar 18, 2026
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,5 +1,5 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run!",
"modification": 3,
"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": 1,
"modification": 6,
}
Original file line number Diff line number Diff line change
Expand Up @@ -311,9 +311,14 @@ public DoFn<InputT, OutputT>.StartBundleContext startBundleContext(DoFn<InputT,
public String getErrorContext() {
return "SimpleDoFnRunner/StartBundle";
}

@Override
public BundleFinalizer bundleFinalizer() {
return stepContext.bundleFinalizer();
}
}

/** An {@link DoFnInvoker.ArgumentProvider} for {@link DoFn.StartBundle @StartBundle}. */
/** An {@link DoFnInvoker.ArgumentProvider} for {@link DoFn.FinishBundle @FinishBundle}. */
private class DoFnFinishBundleArgumentProvider
extends DoFnInvoker.BaseArgumentProvider<InputT, OutputT> {
/** A concrete implementation of {@link DoFn.FinishBundleContext}. */
Expand Down Expand Up @@ -356,6 +361,11 @@ public DoFn<InputT, OutputT>.FinishBundleContext finishBundleContext(
public String getErrorContext() {
return "SimpleDoFnRunner/FinishBundle";
}

@Override
public BundleFinalizer bundleFinalizer() {
return stepContext.bundleFinalizer();
}
}

/**
Expand Down Expand Up @@ -1030,7 +1040,7 @@ public <T> void outputWindowedValue(
@Override
public BundleFinalizer bundleFinalizer() {
throw new UnsupportedOperationException(
"Bundle finalization is not supported in non-portable pipelines.");
"Bundle finalization is not supported in OnTimer calls.");
}
}

Expand Down Expand Up @@ -1289,7 +1299,7 @@ public <T> void outputWindowedValue(
@Override
public BundleFinalizer bundleFinalizer() {
throw new UnsupportedOperationException(
"Bundle finalization is not supported in non-portable pipelines.");
"Bundle finalization is not supported in OnWindowExpiration calls.");
}
}

Expand Down
7 changes: 5 additions & 2 deletions runners/google-cloud-dataflow-java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -205,7 +205,6 @@ def commonLegacyExcludeCategories = [
'org.apache.beam.sdk.testing.UsesGaugeMetrics',
'org.apache.beam.sdk.testing.UsesTestStream',
'org.apache.beam.sdk.testing.UsesMetricsPusher',
'org.apache.beam.sdk.testing.UsesBundleFinalizer',
'org.apache.beam.sdk.testing.UsesBoundedTrieMetrics', // Dataflow QM as of now does not support returning back BoundedTrie in metric result.
]

Expand Down Expand Up @@ -456,7 +455,9 @@ task validatesRunner {
'org.apache.beam.sdk.transforms.ParDoLifecycleTest.testTeardownCalledAfterExceptionInSetupStateful',
'org.apache.beam.sdk.transforms.ParDoLifecycleTest.testTeardownCalledAfterExceptionInStartBundle',
'org.apache.beam.sdk.transforms.ParDoLifecycleTest.testTeardownCalledAfterExceptionInStartBundleStateful',
]
],
// Batch legacy worker does not support bundle finalization.
excludedCategories: [ 'org.apache.beam.sdk.testing.UsesBundleFinalizer', ],
))
}

Expand Down Expand Up @@ -490,6 +491,8 @@ task validatesRunnerStreaming {
description "Validates Dataflow runner forcing streaming mode"
dependsOn(createLegacyWorkerValidatesRunnerTest(validatesRunnerStreamingConfig + [
name: 'validatesRunnerLegacyWorkerTestStreaming',
// Streaming appliance currently fails bundle finalizer tests.
excludedCategories: validatesRunnerStreamingConfig.excludedCategories + [ 'org.apache.beam.sdk.testing.UsesBundleFinalizer', ],
]))
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2737,11 +2737,11 @@ static void verifyDoFnSupported(
DataflowRunner.class.getSimpleName()));
}
boolean isUnifiedWorker = useUnifiedWorker(options);
if (DoFnSignatures.usesBundleFinalizer(fn) && !isUnifiedWorker) {
if (DoFnSignatures.usesBundleFinalizer(fn) && !isUnifiedWorker && !streaming) {
throw new UnsupportedOperationException(
String.format(
"%s does not currently support %s when not using unified worker because it uses "
+ "BundleFinalizers in its implementation. Set the `--experiments=use_runner_v2` "
"%s does not currently support %s in batch mode when not using unified worker because it "
+ "uses BundleFinalizers in its implementation. Set the `--experiments=use_runner_v2` "
+ "option to use this DoFn.",
DataflowRunner.class.getSimpleName(), fn.getClass().getSimpleName()));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -156,10 +156,7 @@ public DoFnRunner<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> crea
// in the event of a crash.
10000,
Duration.standardSeconds(10),
() -> {
throw new UnsupportedOperationException(
"BundleFinalizer unsupported by non-portable Dataflow.");
}));
stepContext::bundleFinalizer));
DoFnRunner<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> simpleRunner =
new SimpleDoFnRunner<>(
options,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicLong;
import javax.annotation.concurrent.NotThreadSafe;
import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair;
import org.apache.beam.runners.core.SideInputReader;
import org.apache.beam.runners.core.StateInternals;
import org.apache.beam.runners.core.StateNamespace;
Expand Down Expand Up @@ -73,6 +74,7 @@
import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
import org.apache.beam.sdk.metrics.MetricsContainer;
import org.apache.beam.sdk.state.TimeDomain;
import org.apache.beam.sdk.transforms.DoFn.BundleFinalizer;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.ByteStringOutputStream;
import org.apache.beam.sdk.values.PCollectionView;
Expand All @@ -82,8 +84,10 @@
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.PeekingIterator;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets;
Expand Down Expand Up @@ -446,11 +450,27 @@ public void invalidateCache() {
}
}

public Map<Long, Runnable> flushState() {
Map<Long, Runnable> callbacks = new HashMap<>();
public Map<Long, Pair<Instant, Runnable>> flushState() {
Map<Long, Pair<Instant, Runnable>> callbacks = new HashMap<>();

for (StepContext stepContext : getAllStepContexts()) {
stepContext.flushState();
for (Pair<Instant, BundleFinalizer.Callback> bundleFinalizer :
stepContext.flushBundleFinalizerCallbacks()) {
long id = ThreadLocalRandom.current().nextLong();
callbacks.put(
id,
Pair.of(
bundleFinalizer.getLeft(),
() -> {
try {
bundleFinalizer.getRight().onBundleSuccess();
} catch (Exception e) {
throw new RuntimeException("Exception while running bundle finalizer", e);
}
}));
outputBuilder.addFinalizeIds(id);
}
}

if (activeReader != null) {
Expand All @@ -462,13 +482,15 @@ public Map<Long, Runnable> flushState() {
sourceStateBuilder.addFinalizeIds(id);
callbacks.put(
id,
() -> {
try {
checkpointMark.finalizeCheckpoint();
} catch (IOException e) {
throw new RuntimeException("Exception while finalizing checkpoint", e);
}
});
Pair.of(
Instant.now().plus(Duration.standardMinutes(5)),
() -> {
try {
checkpointMark.finalizeCheckpoint();
} catch (IOException e) {
throw new RuntimeException("Exception while finalizing checkpoint", e);
}
}));

@SuppressWarnings("unchecked")
Coder<UnboundedSource.CheckpointMark> checkpointCoder =
Expand Down Expand Up @@ -699,6 +721,11 @@ public <W extends BoundedWindow> void setStateCleanupTimer(
public DataflowStepContext namespacedToUser() {
return this;
}

@Override
public BundleFinalizer bundleFinalizer() {
return wrapped.bundleFinalizer();
}
}

/** A {@link SideInputReader} that fetches side inputs from the streaming worker's cache. */
Expand Down Expand Up @@ -771,6 +798,7 @@ class StepContext extends DataflowExecutionContext.DataflowStepContext
// A list of timer keys that were modified by user processing earlier in this bundle. This
// serves a tombstone, so that we know not to fire any bundle timers that were modified.
private Table<String, StateNamespace, TimerData> modifiedUserTimerKeys = null;
private final WindmillBundleFinalizer bundleFinalizer = new WindmillBundleFinalizer();

public StepContext(DataflowOperationContext operationContext) {
super(operationContext.nameContext());
Expand Down Expand Up @@ -1043,9 +1071,37 @@ public TimerInternals timerInternals() {
return checkNotNull(systemTimerInternals);
}

@Override
public BundleFinalizer bundleFinalizer() {
return bundleFinalizer;
}

public TimerInternals userTimerInternals() {
ensureStateful("Tried to access user timers");
return checkNotNull(userTimerInternals);
}

public ImmutableList<Pair<Instant, BundleFinalizer.Callback>> flushBundleFinalizerCallbacks() {
return bundleFinalizer.flushCallbacks();
}
}

private static class WindmillBundleFinalizer implements BundleFinalizer {
private ImmutableList.Builder<Pair<Instant, Callback>> callbacks = ImmutableList.builder();

private WindmillBundleFinalizer() {}

private ImmutableList<Pair<Instant, Callback>> flushCallbacks() {
ImmutableList<Pair<Instant, Callback>> flushedCallbacks = callbacks.build();
if (!Iterables.isEmpty(flushedCallbacks)) {
callbacks = ImmutableList.builder();
}
return flushedCallbacks;
}

@Override
public void afterBundleCommit(Instant callbackExpiry, Callback callback) {
callbacks.add(Pair.of(callbackExpiry, callback));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -54,24 +54,27 @@ final class GetWorkResponseChunkAssembler {
private final WorkItem.Builder workItemBuilder; // Reused to reduce GC overhead.
private ByteString data;
private long bufferedSize;
private final List<Long> appliedFinalizeIds;

GetWorkResponseChunkAssembler() {
workTimingInfosTracker = new GetWorkTimingInfosTracker(System::currentTimeMillis);
data = ByteString.EMPTY;
bufferedSize = 0;
metadata = null;
workItemBuilder = WorkItem.newBuilder();
appliedFinalizeIds = new ArrayList<>();
}

/**
* Appends the response chunk bytes to the {@link #data }byte buffer. Return the assembled
* Appends the response chunk bytes to the {@link #data} byte buffer. Return the assembled
* WorkItem if all response chunks for a WorkItem have been received.
*/
List<AssembledWorkItem> append(Windmill.StreamingGetWorkResponseChunk chunk) {
if (chunk.hasComputationMetadata()) {
metadata = ComputationMetadata.fromProto(chunk.getComputationMetadata());
}
workTimingInfosTracker.addTimingInfo(chunk.getPerWorkItemTimingInfosList());
appliedFinalizeIds.addAll(chunk.getAppliedFinalizeIdsList());

List<AssembledWorkItem> response = new ArrayList<>();
for (int i = 0; i < chunk.getSerializedWorkItemList().size(); i++) {
Expand All @@ -90,13 +93,14 @@ List<AssembledWorkItem> append(Windmill.StreamingGetWorkResponseChunk chunk) {
}

/**
* Attempt to flush the {@link #data} bytes into a {@link WorkItem} w/ it's metadata. Resets the
* Attempt to flush the {@link #data} bytes into a {@link WorkItem} w/ its metadata. Resets the
* data byte string and tracking metadata afterwards, whether the {@link WorkItem} deserialization
* was successful or not.
*/
private Optional<AssembledWorkItem> flushToWorkItem() {
try {
workItemBuilder.mergeFrom(data);
workItemBuilder.addAllAppliedFinalizeIds(appliedFinalizeIds);
return Optional.of(
AssembledWorkItem.create(
workItemBuilder.build(),
Copy link
Contributor

Choose a reason for hiding this comment

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

add the appliedFinalizeIds to the work item builder before building instead of passing to create? or could remove the list and just add to the builder where we are currently adding to the list.

Otherwise I had a concern we were passing out mutable list outside this class. It is safe since we can that create below is just iterating and not keeping a reference to it but seems like we might as well do it here so it's clearer.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Nice. That seems a bit clearer. Now I don't even have to pass them to the AssembledWorkItem.

Expand All @@ -110,6 +114,7 @@ private Optional<AssembledWorkItem> flushToWorkItem() {
workTimingInfosTracker.reset();
data = ByteString.EMPTY;
bufferedSize = 0;
appliedFinalizeIds.clear();
}

return Optional.empty();
Expand Down
Loading
Loading