Skip to content
Open
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!",
"modification": 7,
"modification": 1,
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
Expand Down Expand Up @@ -958,15 +959,24 @@ public OffsetRange getInitialRestriction() {
}

/**
* While the finalization callback hasn't been invoked, this DoFn will keep requesting
* finalization, wait one second and then checkpoint upto MAX_ATTEMPTS amount of times. Once the
* callback has been invoked, the DoFn will output the element and stop.
* While the finalization callback hasn't been invoked, this DoFn repeatedly registers a
* finalization request, sleeps ~100ms, and checkpoints (via {@link ProcessContinuation#resume()})
* up to {@link #MAX_ATTEMPTS} times. Once the callback runs and flips {@code WAS_FINALIZED}, the
* DoFn outputs the element and stops.
*
* <p>Shared state must be thread-safe: {@link BundleFinalizer} callbacks can run on a different
* thread than {@code @ProcessElement}; use {@link ConcurrentHashMap} for {@code WAS_FINALIZED} so
* {@code computeIfAbsent} / updates are not racy (a plain {@link HashMap} can hang or corrupt
* under concurrent structural access).
*/
public static class BundleFinalizingSplittableDoFn extends DoFn<String, String> {
/** Upper bound on {@link ProcessContinuation#resume()} iterations via restriction width. */
private static final long MAX_ATTEMPTS = 3000;

private static final long FINALIZATION_CALLBACK_TIMEOUT_SECS = 300;
// We use the UUID to uniquely identify this DoFn in case this test is run with
// other tests in the same JVM.
private static final Map<UUID, AtomicBoolean> WAS_FINALIZED = new HashMap();
private static final Map<UUID, AtomicBoolean> WAS_FINALIZED = new ConcurrentHashMap<>();
private final UUID uuid = UUID.randomUUID();

@NewTracker
Expand Down Expand Up @@ -996,7 +1006,7 @@ public ProcessContinuation process(
}
if (tracker.tryClaim(tracker.currentRestriction().getFrom() + 1)) {
bundleFinalizer.afterBundleCommit(
Instant.now().plus(Duration.standardSeconds(MAX_ATTEMPTS)),
Instant.now().plus(Duration.standardSeconds(FINALIZATION_CALLBACK_TIMEOUT_SECS)),
() -> WAS_FINALIZED.computeIfAbsent(uuid, (unused) -> new AtomicBoolean()).set(true));
// We sleep here instead of setting a resume time since the resume time doesn't need to
// be honored.
Expand Down
Loading