diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json index 090751435f20..e623d3373a93 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run!", - "modification": 7, + "modification": 1, } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java index 8ce9330b9ab4..94b6926ba695 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java @@ -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; @@ -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. + * + *
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