From f91b5af808b38dad5c8b8f2c74fdaa664f40fc9e Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Sun, 10 May 2026 10:04:54 +0000 Subject: [PATCH 01/20] [Dataflow] Add Operation::finishKey() and move timers into it Moving the processTimers call from finish() to finishKey(). In upcoming changes there'll be multiple streaming work items in a single beam bundle. With multiple work items, we've to process elements and timers of each work item before moving to the next work items. finishKey() will be called by the NativeIterator classes after iterating through all elements from a work item. Batch processes timers in BatchModeUngroupingParDoFn and does not rely on the processTimers() in ParDoOperation::finish(). So removing the processTimers() call from ParDoOperation::finish() is safe. Batch also does not use the new finishKey() method. --- ...tCommit_Java_ValidatesRunner_Dataflow.json | 2 +- ...va_ValidatesRunner_Dataflow_Streaming.json | 2 +- .../runners/dataflow/worker/PubsubReader.java | 14 +---- .../worker/StreamingModeExecutionContext.java | 12 +++- .../worker/UngroupedWindmillReader.java | 14 +---- .../worker/WindmillReaderIteratorBase.java | 15 ++++- .../worker/WindowingWindmillReader.java | 5 ++ .../WorkerCustomSourceOperationExecutor.java | 3 + .../streaming/ComputationWorkExecutor.java | 2 +- .../util/common/worker/FlattenOperation.java | 3 + .../util/common/worker/MapTaskExecutor.java | 7 +++ .../worker/util/common/worker/Operation.java | 3 + .../util/common/worker/ParDoOperation.java | 7 ++- .../util/common/worker/ReadOperation.java | 3 + .../util/common/worker/WorkExecutor.java | 3 + .../util/common/worker/WriteOperation.java | 3 + .../worker/IntrinsicMapTaskExecutorTest.java | 12 ++++ .../worker/StreamingDataflowWorkerTest.java | 4 +- .../StreamingModeExecutionContextTest.java | 11 +++- .../WindmillReaderIteratorBaseTest.java | 60 ++++++++++++++++++- .../worker/WorkerCustomSourcesTest.java | 7 ++- .../util/common/worker/ExecutorTestUtils.java | 3 + .../common/worker/MapTaskExecutorTest.java | 12 ++++ .../common/worker/ParDoOperationTest.java | 2 + 24 files changed, 170 insertions(+), 39 deletions(-) diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json index e9d869cc508c..8144784f5f02 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run!", - "modification": 1, + "modification": 2, } 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 e623d3373a93..50d17c108f2e 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": 1, + "modification": 2, } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java index b60cb84415ff..0c80909a0f3b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java @@ -117,20 +117,12 @@ public NativeReader create( @Override public NativeReaderIterator> iterator() throws IOException { - return new PubsubReaderIterator(context.getWorkItem()); + return new PubsubReaderIterator(); } class PubsubReaderIterator extends WindmillReaderIteratorBase { - protected PubsubReaderIterator(Windmill.WorkItem work) { - super(work, skipUndecodableElements); - } - - @Override - public boolean advance() throws IOException { - if (context.workIsFailed()) { - return false; - } - return super.advance(); + protected PubsubReaderIterator() { + super(context, skipUndecodableElements); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index f75d452b211b..0255d5c577ec 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -56,6 +56,7 @@ import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInput; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputState; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; +import org.apache.beam.runners.dataflow.worker.util.common.worker.WorkExecutor; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataId; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest; @@ -157,6 +158,8 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext activeReader; + private @Nullable WorkExecutor workExecutor; + public StreamingModeExecutionContext( CounterFactory counterFactory, String computationId, @@ -240,9 +243,11 @@ public void start( Work work, WindmillStateReader stateReader, SideInputStateFetcher sideInputStateFetcher, - Windmill.WorkItemCommitRequest.Builder outputBuilder) { + Windmill.WorkItemCommitRequest.Builder outputBuilder, + WorkExecutor workExecutor) { this.key = key; this.work = work; + this.workExecutor = workExecutor; this.computationKey = WindmillComputationKey.create(computationId, work.getShardedKey()); this.sideInputStateFetcher = sideInputStateFetcher; StreamingGlobalConfig config = globalConfigHandle.getConfig(); @@ -270,6 +275,11 @@ public void start( } } + public void finishKey() throws Exception { + checkNotNull(workExecutor, "workExecutor must be set before calling finishKey()"); + workExecutor.finishKey(); + } + /** * Ensure that the processing time is greater than any fired processing time timers. Otherwise, a * trigger could ignore the timer and orphan the window. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java index 8ef0bf80323a..f6924493f190 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java @@ -109,20 +109,12 @@ public NativeReader create( @Override public NativeReaderIterator> iterator() throws IOException { - return new UngroupedWindmillReaderIterator(context.getWorkItem()); + return new UngroupedWindmillReaderIterator(); } class UngroupedWindmillReaderIterator extends WindmillReaderIteratorBase { - UngroupedWindmillReaderIterator(Windmill.WorkItem work) { - super(work, skipUndecodableElements); - } - - @Override - public boolean advance() throws IOException { - if (context.workIsFailed()) { - return false; - } - return super.advance(); + UngroupedWindmillReaderIterator() { + super(context, skipUndecodableElements); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java index 7e6508a4788c..b8232ca9f61a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java @@ -34,6 +34,7 @@ */ public abstract class WindmillReaderIteratorBase extends NativeReader.NativeReaderIterator> { + private final StreamingModeExecutionContext context; private final Windmill.WorkItem work; private int bundleIndex = 0; private int messageIndex = -1; @@ -42,9 +43,10 @@ public abstract class WindmillReaderIteratorBase private static final Logger LOG = LoggerFactory.getLogger(WindmillReaderIteratorBase.class); protected WindmillReaderIteratorBase( - Windmill.WorkItem work, ValueProvider skipUndecodableElements) { + StreamingModeExecutionContext context, ValueProvider skipUndecodableElements) { + this.context = context; this.skipUndecodableElements = skipUndecodableElements; - this.work = work; + this.work = context.getWorkItem(); } @Override @@ -54,9 +56,18 @@ public boolean start() throws IOException { @Override public boolean advance() throws IOException { + if (context.workIsFailed()) { + throw new WorkItemCancelledException(context.getWorkItem().getShardingKey()); + } + while (true) { if (bundleIndex >= work.getMessageBundlesCount()) { current = null; + try { + context.finishKey(); + } catch (Exception e) { + throw new RuntimeException(e); + } return false; } Windmill.InputMessageBundle bundle = work.getMessageBundles(bundleIndex); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java index 173b254f6395..7483f2d138a2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java @@ -182,6 +182,11 @@ public boolean start() throws IOException { @Override public boolean advance() throws IOException { current = null; + try { + context.finishKey(); + } catch (Exception e) { + throw new RuntimeException(e); + } return false; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java index 31528e96e07f..a1321d57ebb6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java @@ -89,6 +89,9 @@ public void execute() throws Exception { LOG.debug("Source operation execution complete"); } + @Override + public void finishKey() throws Exception {} + @Override public SourceOperationResponse getResponse() { return response; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationWorkExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationWorkExecutor.java index 8dc681fc640c..b4f3a22a7f52 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationWorkExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationWorkExecutor.java @@ -74,7 +74,7 @@ public final void executeWork( SideInputStateFetcher sideInputStateFetcher, Windmill.WorkItemCommitRequest.Builder outputBuilder) throws Exception { - context().start(key, work, stateReader, sideInputStateFetcher, outputBuilder); + context().start(key, work, stateReader, sideInputStateFetcher, outputBuilder, workExecutor()); workExecutor().execute(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java index 4847e9f2ea9c..af1b2b9c48bd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java @@ -43,6 +43,9 @@ public void process(Object elem) throws Exception { } } + @Override + public void finishKey() throws Exception {} + @Override public boolean supportsRestart() { return true; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java index 58b95f286d55..3c33e1904069 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java @@ -112,6 +112,13 @@ public void execute() throws Exception { // TODO: support for success / failure ports? } + @Override + public void finishKey() throws Exception { + for (Operation op : operations) { + op.finishKey(); + } + } + @Override public NativeReader.Progress getWorkerProgress() throws Exception { return getReadOperation().getProgress(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java index b7b4e255cfa5..b630da33cfad 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java @@ -137,6 +137,9 @@ public void finish() throws Exception { } } + /** Called when all elements for a specific key have been processed. */ + public abstract void finishKey() throws Exception; + /** Aborts this Operation's execution. */ public void abort() throws Exception { synchronized (initializationStateLock) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java index 27b6e9d1fb35..5aec82073366 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java @@ -46,12 +46,17 @@ public void process(Object elem) throws Exception { } @Override - public void finish() throws Exception { + // Batch mode does not use this method and instead relies on BatchModeUngroupingParDoFn + // to process timers per key. + public void finishKey() throws Exception { try (Closeable scope = context.enterProcessTimers()) { checkStarted(); fn.processTimers(); } + } + @Override + public void finish() throws Exception { try (Closeable scope = context.enterFinish()) { fn.finishBundle(); super.finish(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java index d6b020483d4c..fabc8d6af25b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java @@ -271,6 +271,9 @@ public void finish() throws Exception { } } + @Override + public void finishKey() throws Exception {} + @Override public void abort() throws Exception { if (readerIterator != null) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java index b7170c80ced9..1083fdbb9c42 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java @@ -34,6 +34,9 @@ public interface WorkExecutor extends AutoCloseable { /** Executes the task. */ public abstract void execute() throws Exception; + /** Called when all elements for a specific key have been processed. */ + void finishKey() throws Exception; + /** * Returns the worker's current progress. * diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java index 673140d58d89..d28e7f3e5d3d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java @@ -105,6 +105,9 @@ public void finish() throws Exception { } } + @Override + public void finishKey() throws Exception {} + @Override public void abort() throws Exception { if (writer == null) { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java index c519efd4172c..396c8db87e6b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java @@ -104,6 +104,9 @@ public void abort() throws Exception { aborted = true; super.abort(); } + + @Override + public void finishKey() throws Exception {} } // A mock ReadOperation fed to a MapTaskExecutor in test. @@ -312,6 +315,9 @@ public void start() throws Exception { Metrics.counter("TestMetric", "MetricCounter").inc(1L); } } + + @Override + public void finishKey() throws Exception {} }, new Operation(new OutputReceiver[] {}, context2) { @Override @@ -321,6 +327,9 @@ public void start() throws Exception { Metrics.counter("TestMetric", "MetricCounter").inc(2L); } } + + @Override + public void finishKey() throws Exception {} }, new Operation(new OutputReceiver[] {}, context3) { @Override @@ -330,6 +339,9 @@ public void start() throws Exception { Metrics.counter("TestMetric", "MetricCounter").inc(3L); } } + + @Override + public void finishKey() throws Exception {} }); try (IntrinsicMapTaskExecutor executor = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index d8a1d1b90d47..ff82a1ab5c4c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -3657,8 +3657,8 @@ public void testActiveWorkFailure() throws Exception { server.waitForAndGetCommitsWithTimeout(1, Duration.standardSeconds(5)); assertEquals(1, commits.size()); - assertEquals(0, BlockingFn.teardownCounter.get()); - assertEquals(1, BlockingFn.setupCounter.get()); + assertEquals(1, BlockingFn.teardownCounter.get()); + assertEquals(2, BlockingFn.setupCounter.get()); worker.stop(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 4bfa6efc8880..970a9a537c0d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -62,6 +62,7 @@ import org.apache.beam.runners.dataflow.worker.streaming.config.FakeGlobalConfigHandle; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingGlobalConfig; import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher; +import org.apache.beam.runners.dataflow.worker.util.common.worker.WorkExecutor; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.FakeGetDataClient; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; @@ -99,6 +100,7 @@ public class StreamingModeExecutionContextTest { @Rule public transient Timeout globalTimeout = Timeout.seconds(600); @Mock private SideInputStateFetcher sideInputStateFetcher; @Mock private WindmillStateReader stateReader; + @Mock private WorkExecutor workExecutor; private static final String COMPUTATION_ID = "computationId"; @@ -168,7 +170,8 @@ public void testTimerInternalsSetTimer() { Watermarks.builder().setInputDataWatermark(new Instant(1000)).build()), stateReader, sideInputStateFetcher, - outputBuilder); + outputBuilder, + workExecutor); TimerInternals timerInternals = stepContext.timerInternals(); @@ -218,7 +221,8 @@ public void testTimerInternalsProcessingTimeSkew() { Watermarks.builder().setInputDataWatermark(new Instant(1000)).build()), stateReader, sideInputStateFetcher, - outputBuilder); + outputBuilder, + workExecutor); TimerInternals timerInternals = stepContext.timerInternals(); assertTrue(timerTimestamp.isBefore(timerInternals.currentProcessingTime())); } @@ -427,7 +431,8 @@ public void testStateTagEncodingBasedOnConfig() { Watermarks.builder().setInputDataWatermark(new Instant(1000)).build()), stateReader, sideInputStateFetcher, - outputBuilder); + outputBuilder, + workExecutor); assertEquals(expectedEncoding, executionContext.getWindmillTagEncoding().getClass()); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java index 61e2f4250d06..539c38eeb1da 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java @@ -19,6 +19,11 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import java.io.IOException; import java.util.ArrayList; @@ -40,8 +45,8 @@ public class WindmillReaderIteratorBaseTest { private static class TestWindmillReaderIterator extends WindmillReaderIteratorBase { protected TestWindmillReaderIterator( - Windmill.WorkItem work, ValueProvider skipUndecodableElements) { - super(work, skipUndecodableElements); + StreamingModeExecutionContext context, ValueProvider skipUndecodableElements) { + super(context, skipUndecodableElements); } @Override @@ -81,6 +86,51 @@ public void testSkipErrors() throws IOException { testForMessageBundleCounts(true, 0, 0, 1, 3, 0, 1, 0, 0, 0, 0); } + @Test + public void testWorkItemCancelledException() throws IOException { + StreamingModeExecutionContext mockContext = mock(StreamingModeExecutionContext.class); + when(mockContext.workIsFailed()).thenReturn(true); + Windmill.WorkItem workItem = + Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(0L).build(); + when(mockContext.getWorkItem()).thenReturn(workItem); + + try (TestWindmillReaderIterator iter = + new TestWindmillReaderIterator(mockContext, ValueProvider.StaticValueProvider.of(false))) { + iter.start(); + fail("Expected WorkItemCancelledException"); + } catch (WorkItemCancelledException e) { + // Expected + } + } + + @Test + public void testFinishKeyCalled() throws Exception { + StreamingModeExecutionContext mockContext = mock(StreamingModeExecutionContext.class); + when(mockContext.workIsFailed()).thenReturn(false); + Windmill.WorkItem workItem = + Windmill.WorkItem.newBuilder() + .setKey(ByteString.EMPTY) + .setWorkToken(0L) + .addMessageBundles( + Windmill.InputMessageBundle.newBuilder() + .setSourceComputationId("foo") + .addMessages( + Windmill.Message.newBuilder() + .setTimestamp(0) + .setData(ByteString.EMPTY) + .build()) + .build()) + .build(); + when(mockContext.getWorkItem()).thenReturn(workItem); + + try (TestWindmillReaderIterator iter = + new TestWindmillReaderIterator(mockContext, ValueProvider.StaticValueProvider.of(false))) { + assertTrue(iter.start()); + assertFalse(iter.advance()); // This should trigger finishKey + verify(mockContext).finishKey(); + } + } + private void testForMessageBundleCounts(int... messageBundleCounts) throws IOException { testForMessageBundleCounts(false, messageBundleCounts); } @@ -111,9 +161,13 @@ private void testForMessageBundleCounts(boolean skipErrors, int... messageBundle .setWorkToken(0L) .addAllMessageBundles(bundles) .build(); + + StreamingModeExecutionContext mockContext = mock(StreamingModeExecutionContext.class); + when(mockContext.getWorkItem()).thenReturn(workItem); + try (TestWindmillReaderIterator iter = new TestWindmillReaderIterator( - workItem, ValueProvider.StaticValueProvider.of(skipErrors))) { + mockContext, ValueProvider.StaticValueProvider.of(skipErrors))) { List actual = ReaderTestUtils.windowedValuesToValues( ReaderUtils.readRemainingFromIterator(iter, false)); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index ce8ad32f71aa..5dfd65b46123 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -97,6 +97,7 @@ import org.apache.beam.runners.dataflow.worker.testing.TestCountingSource; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader.NativeReaderIterator; +import org.apache.beam.runners.dataflow.worker.util.common.worker.WorkExecutor; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.FakeGetDataClient; import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; @@ -643,7 +644,8 @@ public void testReadUnboundedReader() throws Exception { Watermarks.builder().setInputDataWatermark(new Instant(0)).build()), mock(WindmillStateReader.class), mock(SideInputStateFetcher.class), - Windmill.WorkItemCommitRequest.newBuilder()); + Windmill.WorkItemCommitRequest.newBuilder(), + mock(WorkExecutor.class)); @SuppressWarnings({"unchecked", "rawtypes"}) NativeReader>>> reader = @@ -1023,7 +1025,8 @@ public void testFailedWorkItemsAbort() throws Exception { dummyWork, mock(WindmillStateReader.class), mock(SideInputStateFetcher.class), - Windmill.WorkItemCommitRequest.newBuilder()); + Windmill.WorkItemCommitRequest.newBuilder(), + mock(WorkExecutor.class)); @SuppressWarnings({"unchecked", "rawtypes"}) NativeReader>>> reader = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java index 2c35f4bf99db..d5e3b9c87139 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java @@ -59,6 +59,9 @@ private static OutputReceiver[] createOutputReceivers(int numOutputs, CounterSet } return receivers; } + + @Override + public void finishKey() throws Exception {} } /** A {@code Reader} that yields a specified set of values. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java index 188466a50572..5d8f8eebb6f6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java @@ -100,6 +100,9 @@ public void abort() throws Exception { aborted = true; super.abort(); } + + @Override + public void finishKey() throws Exception {} } // A mock ReadOperation fed to a MapTaskExecutor in test. @@ -309,6 +312,9 @@ public void start() throws Exception { Metrics.counter("TestMetric", "MetricCounter").inc(1L); } } + + @Override + public void finishKey() throws Exception {} }, new Operation(new OutputReceiver[] {}, context2) { @Override @@ -318,6 +324,9 @@ public void start() throws Exception { Metrics.counter("TestMetric", "MetricCounter").inc(2L); } } + + @Override + public void finishKey() throws Exception {} }, new Operation(new OutputReceiver[] {}, context3) { @Override @@ -327,6 +336,9 @@ public void start() throws Exception { Metrics.counter("TestMetric", "MetricCounter").inc(3L); } } + + @Override + public void finishKey() throws Exception {} }); assertEquals(TimeUnit.MINUTES.toMillis(10), stateTracker.getNextBundleLullDurationReportMs()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java index ba327f92cc44..5d058b1968cb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java @@ -104,6 +104,7 @@ public void testRunParDoOperation() throws Exception { parDoOperation.process(""); parDoOperation.process("bob"); + parDoOperation.finishKey(); parDoOperation.finish(); parDoOperation.abort(); @@ -147,6 +148,7 @@ public void testParDoOperationContext() throws Exception { operation.start(); operation.process("hello"); + operation.finishKey(); operation.finish(); InOrder inOrder = From a8e4b0eef083d3a7125947f9bd89be4a9f35b11d Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Mon, 11 May 2026 05:48:08 +0000 Subject: [PATCH 02/20] Update empty work item iterator --- .../runners/dataflow/worker/WindowingWindmillReader.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java index 7483f2d138a2..7108b4b7992d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java @@ -156,6 +156,11 @@ public NativeReaderIterator>> iterator() throw return new NativeReaderIterator>>() { @Override public boolean start() throws IOException { + try { + context.finishKey(); + } catch (Exception e) { + throw new RuntimeException(e); + } return false; } From 986b90331df73c0b29f41b75bb3b3381d3c956f7 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Mon, 11 May 2026 06:28:43 +0000 Subject: [PATCH 03/20] Check if finishKey is called before flushState --- .../dataflow/worker/StreamingModeExecutionContext.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index 0255d5c577ec..b5ced4400848 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -159,6 +159,7 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext activeReader; private @Nullable WorkExecutor workExecutor; + private boolean finishKeyCalled = false; public StreamingModeExecutionContext( CounterFactory counterFactory, @@ -248,6 +249,7 @@ public void start( this.key = key; this.work = work; this.workExecutor = workExecutor; + this.finishKeyCalled = false; this.computationKey = WindmillComputationKey.create(computationId, work.getShardedKey()); this.sideInputStateFetcher = sideInputStateFetcher; StreamingGlobalConfig config = globalConfigHandle.getConfig(); @@ -276,8 +278,10 @@ public void start( } public void finishKey() throws Exception { + checkState(!finishKeyCalled, "finishKey was already called"); checkNotNull(workExecutor, "workExecutor must be set before calling finishKey()"); workExecutor.finishKey(); + this.finishKeyCalled = true; } /** @@ -461,6 +465,7 @@ public void invalidateCache() { } public Map> flushState() { + checkState(finishKeyCalled, "finishKey must be called before flushState"); Map> callbacks = new HashMap<>(); for (StepContext stepContext : getAllStepContexts()) { From 107517b169c5c4aeb36a6376a03d33a96b8b2c41 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Mon, 11 May 2026 07:25:35 +0000 Subject: [PATCH 04/20] Fix tests --- .../dataflow/worker/StreamingModeExecutionContextTest.java | 3 ++- .../beam/runners/dataflow/worker/WorkerCustomSourcesTest.java | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 970a9a537c0d..850c0988ac8a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -154,7 +154,7 @@ COMPUTATION_ID, new FakeGetDataClient(), ignored -> {}, mock(HeartbeatSender.cla } @Test - public void testTimerInternalsSetTimer() { + public void testTimerInternalsSetTimer() throws Exception { Windmill.WorkItemCommitRequest.Builder outputBuilder = Windmill.WorkItemCommitRequest.newBuilder(); NameContext nameContext = NameContextsForTests.nameContextForTest(); @@ -182,6 +182,7 @@ public void testTimerInternalsSetTimer() { new Instant(5000), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL)); + executionContext.finishKey(); executionContext.flushState(); Windmill.Timer timer = outputBuilder.buildPartial().getOutputTimers(0); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index 5dfd65b46123..bbc8bed7187e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -680,6 +680,7 @@ public void testReadUnboundedReader() throws Exception { numReadOnThisIteration, lessThanOrEqualTo(debugOptions.getUnboundedReaderMaxElements())); // Extract and verify state modifications. + context.finishKey(); context.flushState(); state = context.getOutputBuilder().getSourceStateUpdates().getState(); // CountingSource's watermark is the last record + 1. i is now one past the last record, From 9e83f33a4a620d4c947e5a1459d8e7c819654d3c Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Mon, 11 May 2026 08:01:57 +0000 Subject: [PATCH 05/20] Call finishKey from WorkerCustomSources --- .../worker/StreamingModeExecutionContext.java | 8 ++++++-- .../dataflow/worker/WindmillReaderIteratorBase.java | 6 +----- .../dataflow/worker/WindowingWindmillReader.java | 12 ++---------- .../runners/dataflow/worker/WorkerCustomSources.java | 9 +++++++-- 4 files changed, 16 insertions(+), 19 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index b5ced4400848..1ff2c1bc4a1c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -277,10 +277,14 @@ public void start( } } - public void finishKey() throws Exception { + public void finishKey() { checkState(!finishKeyCalled, "finishKey was already called"); checkNotNull(workExecutor, "workExecutor must be set before calling finishKey()"); - workExecutor.finishKey(); + try { + workExecutor.finishKey(); + } catch (Exception e) { + throw new RuntimeException(e); + } this.finishKeyCalled = true; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java index b8232ca9f61a..075a1a8a4250 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java @@ -63,11 +63,7 @@ public boolean advance() throws IOException { while (true) { if (bundleIndex >= work.getMessageBundlesCount()) { current = null; - try { - context.finishKey(); - } catch (Exception e) { - throw new RuntimeException(e); - } + context.finishKey(); return false; } Windmill.InputMessageBundle bundle = work.getMessageBundles(bundleIndex); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java index 7108b4b7992d..488684769bd9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java @@ -156,11 +156,7 @@ public NativeReaderIterator>> iterator() throw return new NativeReaderIterator>>() { @Override public boolean start() throws IOException { - try { - context.finishKey(); - } catch (Exception e) { - throw new RuntimeException(e); - } + context.finishKey(); return false; } @@ -187,11 +183,7 @@ public boolean start() throws IOException { @Override public boolean advance() throws IOException { current = null; - try { - context.finishKey(); - } catch (Exception e) { - throw new RuntimeException(e); - } + context.finishKey(); return false; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java index c00edffeaf95..29d5fb3561a1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java @@ -824,6 +824,7 @@ public boolean start() throws IOException { } try { if (!reader.start()) { + context.finishKey(); return false; } } catch (Exception e) { @@ -841,10 +842,13 @@ public boolean advance() throws IOException { // that there are regular checkpoints and that state does not become too large. BackOff backoff = backoffFactory.backoff(); while (true) { + if (context.workIsFailed()) { + throw new WorkItemCancelledException(context.getWorkItem().getShardingKey()); + } if (elemsRead >= maxElems || Instant.now().isAfter(endTime) - || context.isSinkFullHintSet() - || context.workIsFailed()) { + || context.isSinkFullHintSet()) { + context.finishKey(); return false; } try { @@ -857,6 +861,7 @@ public boolean advance() throws IOException { } long nextBackoff = backoff.nextBackOffMillis(); if (nextBackoff == BackOff.STOP) { + context.finishKey(); return false; } Uninterruptibles.sleepUninterruptibly(nextBackoff, TimeUnit.MILLISECONDS); From d5ef9e5909d2991d878d7b778deb0c01b1aba560 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Mon, 11 May 2026 19:39:02 +0000 Subject: [PATCH 06/20] Fix tests --- .../worker/WorkerCustomSourcesTest.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index bbc8bed7187e..d5cf2948d928 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -680,7 +680,6 @@ public void testReadUnboundedReader() throws Exception { numReadOnThisIteration, lessThanOrEqualTo(debugOptions.getUnboundedReaderMaxElements())); // Extract and verify state modifications. - context.finishKey(); context.flushState(); state = context.getOutputBuilder().getSourceStateUpdates().getState(); // CountingSource's watermark is the last record + 1. i is now one past the last record, @@ -1042,14 +1041,19 @@ public void testFailedWorkItemsAbort() throws Exception { NativeReaderIterator>>> readerIterator = reader.iterator(); int numReads = 0; - while ((numReads == 0) ? readerIterator.start() : readerIterator.advance()) { - WindowedValue>> value = readerIterator.getCurrent(); - assertEquals(KV.of(0, numReads), value.getValue().getValue()); - numReads++; - // Fail the work item after reading two elements. - if (numReads == 2) { - dummyWork.setFailed(); + try { + while ((numReads == 0) ? readerIterator.start() : readerIterator.advance()) { + WindowedValue>> value = readerIterator.getCurrent(); + assertEquals(KV.of(0, numReads), value.getValue().getValue()); + numReads++; + // Fail the work item after reading two elements. + if (numReads == 2) { + dummyWork.setFailed(); + } } + fail("Expected WorkItemCancelledException"); + } catch (WorkItemCancelledException e) { + // Expected } assertThat(numReads, equalTo(2)); } From 08a7f7df50e6bdd62108bfd5b5828d39df20ddeb Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 12 May 2026 01:29:32 +0000 Subject: [PATCH 07/20] Add DoFnRunner::finishKey method. In upcoming changes there'll be multiple dataflow streaming work items in a single beam bundle. With multiple work items, we've to process elements and timers of each work item before moving to the next work items. The new finishKey method allows the DoFnRunners to cleanup/persist state (that should not be carried over) before switching work items on multi key bundles. Streaming SideInputDoFnRunners are the only classes using the finishKey method right now. The finishKey() is not exposed to DoFns and is not visible in user apis. --- .../java/org/apache/beam/runners/core/DoFnRunner.java | 6 ++++++ .../beam/runners/core/LateDataDroppingDoFnRunner.java | 5 +++++ .../apache/beam/runners/core/SimpleDoFnRunner.java | 5 +++++ .../apache/beam/runners/core/StatefulDoFnRunner.java | 5 +++++ .../core/SimplePushbackSideInputDoFnRunnerTest.java | 3 +++ .../dataflow/worker/AssignWindowsParDoFnFactory.java | 5 +++++ .../dataflow/worker/BatchModeUngroupingParDoFn.java | 5 +++++ .../CreateIsmShardKeyAndSortKeyDoFnFactory.java | 5 +++++ .../dataflow/worker/DataflowProcessFnRunner.java | 5 +++++ .../runners/dataflow/worker/ForwardingParDoFn.java | 5 +++++ .../dataflow/worker/GroupAlsoByWindowFnRunner.java | 3 +++ .../dataflow/worker/GroupAlsoByWindowsParDoFn.java | 7 +++++++ .../worker/PairWithConstantKeyDoFnFactory.java | 5 +++++ .../dataflow/worker/PartialGroupByKeyParDoFns.java | 11 ++++++++++- .../ReifyTimestampAndWindowsParDoFnFactory.java | 5 +++++ .../beam/runners/dataflow/worker/SimpleParDoFn.java | 7 +++++++ .../StreamingKeyedWorkItemSideInputDoFnRunner.java | 5 +++++ .../worker/StreamingPCollectionViewWriterParDoFn.java | 5 +++++ .../dataflow/worker/StreamingSideInputDoFnRunner.java | 5 +++++ .../worker/ToIsmRecordForMultimapDoFnFactory.java | 5 +++++ .../runners/dataflow/worker/ValuesDoFnFactory.java | 5 +++++ .../dataflow/worker/util/common/worker/ParDoFn.java | 2 ++ .../worker/util/common/worker/ParDoOperation.java | 1 + .../common/worker/SimplePartialGroupByKeyParDoFn.java | 5 +++++ .../dataflow/worker/IntrinsicMapTaskExecutorTest.java | 3 +++ .../util/common/worker/MapTaskExecutorTest.java | 3 +++ .../worker/util/common/worker/ParDoOperationTest.java | 3 +++ 27 files changed, 128 insertions(+), 1 deletion(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index 500aedecb5d6..0f747a106d2b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -63,6 +63,12 @@ public interface DoFnRunner void onWindowExpiration( BoundedWindow window, Instant timestamp, KeyT key); + /** + * Performs per-key cleanup or processing after all elements and timers for a key have been + * processed. + */ + void finishKey(); + /** * Returns the underlying fn instance. * diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index 2ce94533d9e2..c3469a464e70 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -101,6 +101,11 @@ public void finishBundle() { doFnRunner.finishBundle(); } + @Override + public void finishKey() { + doFnRunner.finishKey(); + } + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { doFnRunner.onWindowExpiration(window, timestamp, key); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index d553a7be2d44..4ce5d128d09d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -229,6 +229,11 @@ public void finishBundle() { } } + @Override + public void finishKey() { + // Do nothing by default. + } + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { invoker.invokeOnWindowExpiration( diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java index 779138834669..b42abcd9c27c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java @@ -131,6 +131,11 @@ public void finishBundle() { doFnRunner.finishBundle(); } + @Override + public void finishKey() { + doFnRunner.finishKey(); + } + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { doFnRunner.onWindowExpiration(window, timestamp, key); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java index 1ae937b7a836..7d6bec47f93f 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java @@ -379,6 +379,9 @@ public void finishBundle() { finished = true; } + @Override + public void finishKey() {} + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java index 83cbc3aa62c7..730cc2cf65f4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java @@ -119,6 +119,11 @@ public void processTimers() throws Exception { // Nothing. } + @Override + public void finishKey() throws Exception { + // Nothing. + } + @Override public void finishBundle() throws Exception { receiver = null; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java index 8e2b325b580a..a03538a49c1c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java @@ -73,6 +73,11 @@ public void processTimers() throws Exception { // The timers for the underlying ParDoFn are processed at the end of each element } + @Override + public void finishKey() throws Exception { + // Nothing. + } + @Override public void finishBundle() throws Exception { underlyingParDoFn.finishBundle(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java index bd991560c186..51cec7429a14 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java @@ -114,6 +114,11 @@ public void processElement(Object untypedElem) throws Exception { @Override public void processTimers() {} + @Override + public void finishKey() throws Exception { + // Nothing. + } + @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java index ec1fcd6c8432..a95349b3d076 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java @@ -128,6 +128,11 @@ public void finishBundle() { simpleRunner.finishBundle(); } + @Override + public void finishKey() { + simpleRunner.finishKey(); + } + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { simpleRunner.onWindowExpiration(window, timestamp, key); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java index d55181559322..9a15a3a02004 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java @@ -47,6 +47,11 @@ public void processTimers() throws Exception { delegate.processTimers(); } + @Override + public void finishKey() throws Exception { + delegate.finishKey(); + } + @Override public void finishBundle() throws Exception { delegate.finishBundle(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java index 4845bb0c98e4..61913ecf325b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java @@ -102,6 +102,9 @@ private void invokeProcessElement(WindowedValue elem) { @Override public void finishBundle() {} + @Override + public void finishKey() {} + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java index 882dd497e3f5..663fe00a7628 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java @@ -142,6 +142,13 @@ public void processTimers() throws Exception { // it here to build a KeyedWorkItem } + @Override + public void finishKey() throws Exception { + if (fnRunner != null) { + fnRunner.finishKey(); + } + } + @Override public void finishBundle() throws Exception { checkState(fnRunner != null); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java index 6951e3a95b20..514cc3c8afd2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java @@ -98,6 +98,11 @@ public void processElement(Object untypedElem) throws Exception { @Override public void processTimers() {} + @Override + public void finishKey() throws Exception { + // Nothing. + } + @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java index 399258d7dbb9..82e7360f0c10 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java @@ -317,6 +317,11 @@ public void processElement(Object elem) throws Exception { @Override public void processTimers() {} + @Override + public void finishKey() throws Exception { + // Nothing. + } + @Override public void finishBundle() throws Exception { groupingTable.flush(receiver); @@ -377,10 +382,14 @@ public void processElement(Object elem) throws Exception { @Override public void processTimers() {} + @Override + public void finishKey() throws Exception { + sideInputFetcher.persist(); + } + @Override public void finishBundle() throws Exception { groupingTable.flush(receiver); - sideInputFetcher.persist(); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java index 746c09404f6e..0232f83dacf3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java @@ -86,6 +86,11 @@ public void processElement(Object untypedElem) throws Exception { @Override public void processTimers() {} + @Override + public void finishKey() throws Exception { + // Nothing. + } + @Override public void finishBundle() throws Exception { this.receiver = null; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java index 434d46c20a5b..1fca7c4b9ec3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java @@ -363,6 +363,13 @@ public void processTimers() throws Exception { processTimers(TimerType.SYSTEM, stepContext, windowCoder); } + @Override + public void finishKey() throws Exception { + if (fnRunner != null) { + fnRunner.finishKey(); + } + } + private void processUserTimer(TimerData timer) throws Exception { if (fnSignature.timerDeclarations().containsKey(timer.getTimerId()) || fnSignature.timerFamilyDeclarations().containsKey(timer.getTimerFamilyId())) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java index 0b9ccd1f37c6..72c3f84ced83 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java @@ -152,6 +152,11 @@ public void onTimer( @Override public void finishBundle() { simpleDoFnRunner.finishBundle(); + } + + @Override + public void finishKey() { + simpleDoFnRunner.finishKey(); sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java index 61730b0c8d88..18843a35348e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java @@ -73,6 +73,11 @@ public void processElement(Object element) throws Exception { @Override public void processTimers() {} + @Override + public void finishKey() throws Exception { + // Nothing. + } + @Override public void finishBundle() throws Exception {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java index 3b7891c5378d..9f88b6a0988a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java @@ -94,6 +94,11 @@ public void onTimer( @Override public void finishBundle() { simpleDoFnRunner.finishBundle(); + } + + @Override + public void finishKey() { + simpleDoFnRunner.finishKey(); sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java index f9e2d6de2461..48c2007f12f4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java @@ -149,6 +149,11 @@ public void processElement(Object untypedElem) throws Exception { @Override public void processTimers() {} + @Override + public void finishKey() throws Exception { + // Nothing. + } + @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java index 3ddb3c2003db..012ebee1c661 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java @@ -80,6 +80,11 @@ public void processElement(Object untypedElem) throws Exception { @Override public void processTimers() {} + @Override + public void finishKey() throws Exception { + // Nothing. + } + @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java index 84dbbd627b08..eb3393cd4602 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java @@ -30,6 +30,8 @@ public interface ParDoFn { void processTimers() throws Exception; + void finishKey() throws Exception; + void finishBundle() throws Exception; void abort() throws Exception; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java index 5aec82073366..d64eba5dc98f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java @@ -52,6 +52,7 @@ public void finishKey() throws Exception { try (Closeable scope = context.enterProcessTimers()) { checkStarted(); fn.processTimers(); + fn.finishKey(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java index 7a9fcfdf0694..145693a74885 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java @@ -39,6 +39,11 @@ public void processElement(Object elem) throws Exception { @Override public void processTimers() {} + @Override + public void finishKey() throws Exception { + // Nothing. + } + @Override public void finishBundle() throws Exception { groupingTable.flush(receiver); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java index 396c8db87e6b..259b699dc383 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java @@ -220,6 +220,9 @@ public void finishBundle() {} @Override public void abort() {} + + @Override + public void finishKey() throws Exception {} } /** Verify counts for the per-element-output-time counter are correct. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java index 5d8f8eebb6f6..649b41483706 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java @@ -216,6 +216,9 @@ public void finishBundle() {} @Override public void abort() {} + + @Override + public void finishKey() throws Exception {} } /** Verify counts for the per-element-output-time counter are correct. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java index 5d058b1968cb..9819f6e957a1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java @@ -85,6 +85,9 @@ public void finishBundle() throws Exception { public void abort() throws Exception { outputReceiver.process("a-aborted"); } + + @Override + public void finishKey() throws Exception {} } @Test From e3f45c7f29329f827b4bb7fd5cc56b4e2ec29152 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 12 May 2026 03:03:21 +0000 Subject: [PATCH 08/20] Fix build --- .../java/org/apache/beam/runners/core/SimpleDoFnRunner.java | 4 +--- .../runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java | 3 +++ .../wrappers/streaming/ExecutableStageDoFnOperator.java | 3 +++ .../wrappers/streaming/stableinput/BufferingDoFnRunner.java | 3 +++ .../translation/wrappers/streaming/DoFnOperatorTest.java | 3 +++ .../runners/dataflow/worker/AssignWindowsParDoFnFactory.java | 4 +--- .../runners/dataflow/worker/BatchModeUngroupingParDoFn.java | 4 +--- .../worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java | 4 +--- .../dataflow/worker/PairWithConstantKeyDoFnFactory.java | 4 +--- .../runners/dataflow/worker/PartialGroupByKeyParDoFns.java | 4 +--- .../worker/ReifyTimestampAndWindowsParDoFnFactory.java | 4 +--- .../worker/StreamingPCollectionViewWriterParDoFn.java | 4 +--- .../dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java | 4 +--- .../beam/runners/dataflow/worker/ValuesDoFnFactory.java | 4 +--- .../util/common/worker/SimplePartialGroupByKeyParDoFn.java | 4 +--- .../dataflow/worker/StreamingSideInputDoFnRunnerTest.java | 4 ++++ .../translation/batch/DoFnRunnerFactory.java | 3 +++ .../translation/batch/DoFnRunnerWithMetrics.java | 3 +++ .../beam/runners/spark/translation/DoFnRunnerWithMetrics.java | 3 +++ .../spark/translation/SparkInputDataProcessorTest.java | 3 +++ 20 files changed, 39 insertions(+), 33 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 4ce5d128d09d..b5ba4c158592 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -230,9 +230,7 @@ public void finishBundle() { } @Override - public void finishKey() { - // Do nothing by default. - } + public void finishKey() {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java index 56e077253ae6..c70d5fb622c1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java @@ -105,6 +105,9 @@ public void finishBundle() { container.updateMetrics(stepName); } + @Override + public void finishKey() {} + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { delegate.onWindowExpiration(window, timestamp, key); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java index 4ebb359fceae..32dbcdcdb7da 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java @@ -1064,6 +1064,9 @@ public void finishBundle() { } } + @Override + public void finishKey() {} + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java index 73b20238ef05..5388cb5febd7 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java @@ -255,6 +255,9 @@ public void finishBundle() { Optional.ofNullable(finishBundleCallback).ifPresent(Runnable::run); } + @Override + public void finishKey() {} + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java index 5c4975ffab01..5a23468f404c 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java @@ -523,6 +523,9 @@ public void finishBundle() { wrappedRunner.finishBundle(); } + @Override + public void finishKey() {} + @Override public void onWindowExpiration( BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java index 730cc2cf65f4..6319997f5546 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java @@ -120,9 +120,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey() throws Exception { - // Nothing. - } + public void finishKey() throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java index a03538a49c1c..3bf1ac30837b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java @@ -74,9 +74,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey() throws Exception { - // Nothing. - } + public void finishKey() throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java index 51cec7429a14..c3aec27ab14f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java @@ -115,9 +115,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception { - // Nothing. - } + public void finishKey() throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java index 514cc3c8afd2..e744c1de0cea 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java @@ -99,9 +99,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception { - // Nothing. - } + public void finishKey() throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java index 82e7360f0c10..75e830e713ab 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java @@ -318,9 +318,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception { - // Nothing. - } + public void finishKey() throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java index 0232f83dacf3..a681cb78fe8f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java @@ -87,9 +87,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception { - // Nothing. - } + public void finishKey() throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java index 18843a35348e..6afc0dd9b1b6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java @@ -74,9 +74,7 @@ public void processElement(Object element) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception { - // Nothing. - } + public void finishKey() throws Exception {} @Override public void finishBundle() throws Exception {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java index 48c2007f12f4..18901fbc4073 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java @@ -150,9 +150,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception { - // Nothing. - } + public void finishKey() throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java index 012ebee1c661..9b469adc3a28 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java @@ -81,9 +81,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception { - // Nothing. - } + public void finishKey() throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java index 145693a74885..455dd8278c97 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java @@ -40,9 +40,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception { - // Nothing. - } + public void finishKey() throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java index d18bc512723e..45bbc831a2ae 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java @@ -150,6 +150,7 @@ public void testSideInputNotReady() throws Exception { runner.startBundle(); runner.processElement(createDatum("e", 0)); + runner.finishKey(); runner.finishBundle(); assertTrue(outputManager.getOutput(mainOutputTag).isEmpty()); @@ -214,6 +215,7 @@ public void testMultipleWindowsNotReady() throws Exception { runner.startBundle(); runner.processElement(elem); + runner.finishKey(); runner.finishBundle(); assertTrue(outputManager.getOutput(mainOutputTag).isEmpty()); @@ -317,6 +319,7 @@ public void testSideInputNotification() throws Exception { when(mockSideInputReader.get(eq(view), any(BoundedWindow.class))).thenReturn("data"); runner.startBundle(); + runner.finishKey(); runner.finishBundle(); assertThat(outputManager.getOutput(mainOutputTag), contains(createDatum("e:data", 0))); @@ -373,6 +376,7 @@ public void testMultipleSideInputs() throws Exception { runner.startBundle(); runner.processElement(createDatum("e2", 2)); + runner.finishKey(); runner.finishBundle(); assertThat( diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java index 99ce3dc69889..cf514b4ca499 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java @@ -284,6 +284,9 @@ public void finishBundle() { } } + @Override + public void finishKey() {} + @Override public DoFn getFn() { throw new UnsupportedOperationException(); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java index 28dbf44cb8fe..543042098b63 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java @@ -104,6 +104,9 @@ public void finishBundle() { } } + @Override + public void finishKey() {} + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { delegate.onWindowExpiration(window, timestamp, key); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java index c8cd7eb5f262..5dbfe5b096a7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java @@ -103,6 +103,9 @@ public void finishBundle() { } } + @Override + public void finishKey() {} + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { delegate.onWindowExpiration(window, timestamp, key); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java index 2dc428d5a6b2..6a5e03e30ab8 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java @@ -259,6 +259,9 @@ public void onTimer( @Override public void finishBundle() {} + @Override + public void finishKey() {} + @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} From 5085977a4e6ef5e78f0630fd619be660ab541bcd Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 12 May 2026 05:20:10 +0000 Subject: [PATCH 09/20] run test --- ...beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 50d17c108f2e..e1b083e439cc 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": 2, + "modification": 3, } From 70803b4f4c7d2f6623062ee6e5342e96c8a1bf29 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Thu, 28 May 2026 09:08:31 +0000 Subject: [PATCH 10/20] address comments --- .../main/java/org/apache/beam/runners/core/DoFnRunner.java | 4 ++-- .../runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java | 5 ++--- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index 0f747a106d2b..e757e9909967 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -64,8 +64,8 @@ public interface DoFnRunner Date: Thu, 28 May 2026 09:21:07 +0000 Subject: [PATCH 11/20] address comments --- .../main/java/org/apache/beam/runners/core/DoFnRunner.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index e757e9909967..3442c2f0d896 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -66,6 +66,10 @@ public interface DoFnRunner Date: Mon, 1 Jun 2026 09:05:30 +0000 Subject: [PATCH 12/20] spotless fix --- .../main/java/org/apache/beam/runners/core/DoFnRunner.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index 3442c2f0d896..031b7a9d989d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -64,10 +64,10 @@ public interface DoFnRunnerThis is an optional method that can be used by runners as a hook to reset any per key state * before moving to a different key in the same bundle. Currently used only by the Dataflow * Streaming runner. */ From d2cb29801e0bb47256178233164a992297e59a64 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Thu, 4 Jun 2026 06:03:05 +0000 Subject: [PATCH 13/20] Add key parameter to finishKey --- .../java/org/apache/beam/runners/core/DoFnRunner.java | 4 +++- .../beam/runners/core/LateDataDroppingDoFnRunner.java | 5 +++-- .../org/apache/beam/runners/core/SimpleDoFnRunner.java | 2 +- .../apache/beam/runners/core/StatefulDoFnRunner.java | 5 +++-- .../core/SimplePushbackSideInputDoFnRunnerTest.java | 3 ++- .../flink/metrics/DoFnRunnerWithMetricsUpdate.java | 3 ++- .../streaming/ExecutableStageDoFnOperator.java | 2 +- .../streaming/stableinput/BufferingDoFnRunner.java | 2 +- .../wrappers/streaming/DoFnOperatorTest.java | 2 +- .../dataflow/worker/AssignWindowsParDoFnFactory.java | 2 +- .../dataflow/worker/BatchModeUngroupingParDoFn.java | 2 +- .../worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java | 2 +- .../dataflow/worker/DataflowProcessFnRunner.java | 5 +++-- .../runners/dataflow/worker/ForwardingParDoFn.java | 4 ++-- .../dataflow/worker/GroupAlsoByWindowFnRunner.java | 3 ++- .../dataflow/worker/GroupAlsoByWindowsParDoFn.java | 4 ++-- .../worker/PairWithConstantKeyDoFnFactory.java | 2 +- .../dataflow/worker/PartialGroupByKeyParDoFns.java | 4 ++-- .../worker/ReifyTimestampAndWindowsParDoFnFactory.java | 2 +- .../beam/runners/dataflow/worker/SimpleParDoFn.java | 4 ++-- .../StreamingKeyedWorkItemSideInputDoFnRunner.java | 5 +++-- .../dataflow/worker/StreamingModeExecutionContext.java | 2 +- .../worker/StreamingPCollectionViewWriterParDoFn.java | 2 +- .../dataflow/worker/StreamingSideInputDoFnRunner.java | 5 +++-- .../worker/ToIsmRecordForMultimapDoFnFactory.java | 2 +- .../runners/dataflow/worker/ValuesDoFnFactory.java | 2 +- .../worker/WorkerCustomSourceOperationExecutor.java | 2 +- .../worker/util/common/worker/FlattenOperation.java | 2 +- .../worker/util/common/worker/MapTaskExecutor.java | 4 ++-- .../dataflow/worker/util/common/worker/Operation.java | 2 +- .../dataflow/worker/util/common/worker/ParDoFn.java | 2 +- .../worker/util/common/worker/ParDoOperation.java | 4 ++-- .../worker/util/common/worker/ReadOperation.java | 2 +- .../common/worker/SimplePartialGroupByKeyParDoFn.java | 2 +- .../worker/util/common/worker/WorkExecutor.java | 2 +- .../worker/util/common/worker/WriteOperation.java | 2 +- .../dataflow/worker/IntrinsicMapTaskExecutorTest.java | 10 +++++----- .../worker/StreamingSideInputDoFnRunnerTest.java | 8 ++++---- .../worker/util/common/worker/ExecutorTestUtils.java | 2 +- .../worker/util/common/worker/MapTaskExecutorTest.java | 10 +++++----- .../worker/util/common/worker/ParDoOperationTest.java | 7 ++++--- .../translation/batch/DoFnRunnerFactory.java | 3 ++- .../translation/batch/DoFnRunnerWithMetrics.java | 3 ++- .../spark/translation/DoFnRunnerWithMetrics.java | 3 ++- .../spark/translation/SparkInputDataProcessorTest.java | 3 ++- 45 files changed, 84 insertions(+), 69 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index 031b7a9d989d..d1278eddaef0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -70,8 +70,10 @@ public interface DoFnRunnerThis is an optional method that can be used by runners as a hook to reset any per key state * before moving to a different key in the same bundle. Currently used only by the Dataflow * Streaming runner. + * + * @param key current key to clean up or finish processing */ - void finishKey(); + void finishKey(KeyT key); /** * Returns the underlying fn instance. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index f95de8c5ed15..dfab198f8932 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** @@ -102,8 +103,8 @@ public void finishBundle() { } @Override - public void finishKey() { - doFnRunner.finishKey(); + public void finishKey(KeyT key) { + doFnRunner.finishKey(key); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index fb76ef20812f..61f8476df415 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -231,7 +231,7 @@ public void finishBundle() { } @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java index b42abcd9c27c..f5de79652f23 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java @@ -43,6 +43,7 @@ import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; @@ -132,8 +133,8 @@ public void finishBundle() { } @Override - public void finishKey() { - doFnRunner.finishKey(); + public void finishKey(KeyT key) { + doFnRunner.finishKey(key); } @Override diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java index 7d6bec47f93f..aa61122a7547 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java @@ -65,6 +65,7 @@ import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; @@ -380,7 +381,7 @@ public void finishBundle() { } @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java index c70d5fb622c1..c327c8d91bea 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.CausedByDrain; import org.apache.beam.sdk.values.WindowedValue; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** @@ -106,7 +107,7 @@ public void finishBundle() { } @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java index 32dbcdcdb7da..f7bc62a8ec5a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java @@ -1065,7 +1065,7 @@ public void finishBundle() { } @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java index 5388cb5febd7..6d1d4085f0bc 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java @@ -256,7 +256,7 @@ public void finishBundle() { } @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java index 5a23468f404c..cce8f808c242 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java @@ -524,7 +524,7 @@ public void finishBundle() { } @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration( diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java index 6319997f5546..673130a6048a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java @@ -120,7 +120,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java index 3bf1ac30837b..c3cf6d9e67e2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java @@ -74,7 +74,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java index c3aec27ab14f..afe2e28b2a67 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java @@ -115,7 +115,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java index a95349b3d076..400dbc047443 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** @@ -129,8 +130,8 @@ public void finishBundle() { } @Override - public void finishKey() { - simpleRunner.finishKey(); + public void finishKey(KeyT key) { + simpleRunner.finishKey(key); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java index 9a15a3a02004..31356632d065 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java @@ -48,8 +48,8 @@ public void processTimers() throws Exception { } @Override - public void finishKey() throws Exception { - delegate.finishKey(); + public void finishKey(Object key) throws Exception { + delegate.finishKey(key); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java index 61913ecf325b..aa60a61af8af 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.util.WindowedValueReceiver; import org.apache.beam.sdk.values.CausedByDrain; import org.apache.beam.sdk.values.WindowedValue; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** @@ -103,7 +104,7 @@ private void invokeProcessElement(WindowedValue elem) { public void finishBundle() {} @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java index b5a341151f6a..e204a78a7d2e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java @@ -143,9 +143,9 @@ public void processTimers() throws Exception { } @Override - public void finishKey() throws Exception { + public void finishKey(Object key) throws Exception { checkState(fnRunner != null); - fnRunner.finishKey(); + fnRunner.finishKey(key); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java index e744c1de0cea..425184a4a126 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java @@ -99,7 +99,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java index 75e830e713ab..932a495d0836 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java @@ -318,7 +318,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { @@ -381,7 +381,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception { + public void finishKey(Object key) throws Exception { sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java index a681cb78fe8f..0438b525b6b9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java @@ -87,7 +87,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java index 1fca7c4b9ec3..9acb836bda9f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java @@ -364,9 +364,9 @@ public void processTimers() throws Exception { } @Override - public void finishKey() throws Exception { + public void finishKey(Object key) throws Exception { if (fnRunner != null) { - fnRunner.finishKey(); + fnRunner.finishKey(key); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java index 72c3f84ced83..d8ce0a96807d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java @@ -39,6 +39,7 @@ 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.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** @@ -155,8 +156,8 @@ public void finishBundle() { } @Override - public void finishKey() { - simpleDoFnRunner.finishKey(); + public void finishKey(KeyT key) { + simpleDoFnRunner.finishKey(key); sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index 25ce299adf7a..3fdb8ffbef9a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -283,7 +283,7 @@ public void finishKey() { checkState(!finishKeyCalled, "finishKey was already called"); checkStateNotNull(workExecutor, "workExecutor must be set before calling finishKey()"); try { - workExecutor.finishKey(); + workExecutor.finishKey(key); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java index 6afc0dd9b1b6..6b51427bb930 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java @@ -74,7 +74,7 @@ public void processElement(Object element) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java index 9f88b6a0988a..722452136ab7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.CausedByDrain; import org.apache.beam.sdk.values.WindowedValue; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** @@ -97,8 +98,8 @@ public void finishBundle() { } @Override - public void finishKey() { - simpleDoFnRunner.finishKey(); + public void finishKey(KeyT key) { + simpleDoFnRunner.finishKey(key); sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java index 18901fbc4073..261ed69fb5d7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java @@ -150,7 +150,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java index 9b469adc3a28..acf143a5467b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java @@ -81,7 +81,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java index a1321d57ebb6..3220dbade61c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java @@ -90,7 +90,7 @@ public void execute() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public SourceOperationResponse getResponse() { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java index af1b2b9c48bd..3a131f94aa71 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java @@ -44,7 +44,7 @@ public void process(Object elem) throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public boolean supportsRestart() { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java index 3c33e1904069..bbd28c6f55ba 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java @@ -113,9 +113,9 @@ public void execute() throws Exception { } @Override - public void finishKey() throws Exception { + public void finishKey(Object key) throws Exception { for (Operation op : operations) { - op.finishKey(); + op.finishKey(key); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java index b630da33cfad..7594c9fc6192 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java @@ -138,7 +138,7 @@ public void finish() throws Exception { } /** Called when all elements for a specific key have been processed. */ - public abstract void finishKey() throws Exception; + public abstract void finishKey(Object key) throws Exception; /** Aborts this Operation's execution. */ public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java index eb3393cd4602..25008e310d03 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java @@ -30,7 +30,7 @@ public interface ParDoFn { void processTimers() throws Exception; - void finishKey() throws Exception; + void finishKey(Object key) throws Exception; void finishBundle() throws Exception; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java index f676162b3477..b91d7ef94eda 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java @@ -48,11 +48,11 @@ public void process(Object elem) throws Exception { // Batch mode does not use this method and instead relies on BatchModeUngroupingParDoFn // to process timers per key. @Override - public void finishKey() throws Exception { + public void finishKey(Object key) throws Exception { try (Closeable scope = context.enterProcessTimers()) { checkStarted(); fn.processTimers(); - fn.finishKey(); + fn.finishKey(key); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java index fabc8d6af25b..8ce0d7dbfa2c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java @@ -272,7 +272,7 @@ public void finish() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java index 455dd8278c97..bef27d6cea4d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java @@ -40,7 +40,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java index 1083fdbb9c42..022f12b74ddf 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java @@ -35,7 +35,7 @@ public interface WorkExecutor extends AutoCloseable { public abstract void execute() throws Exception; /** Called when all elements for a specific key have been processed. */ - void finishKey() throws Exception; + void finishKey(Object key) throws Exception; /** * Returns the worker's current progress. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java index d28e7f3e5d3d..d0206a183946 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java @@ -106,7 +106,7 @@ public void finish() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java index 259b699dc383..73d69bd0f617 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java @@ -106,7 +106,7 @@ public void abort() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} } // A mock ReadOperation fed to a MapTaskExecutor in test. @@ -222,7 +222,7 @@ public void finishBundle() {} public void abort() {} @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} } /** Verify counts for the per-element-output-time counter are correct. */ @@ -320,7 +320,7 @@ public void start() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context2) { @Override @@ -332,7 +332,7 @@ public void start() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context3) { @Override @@ -344,7 +344,7 @@ public void start() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} }); try (IntrinsicMapTaskExecutor executor = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java index 45bbc831a2ae..c110cc0d2bf7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java @@ -150,7 +150,7 @@ public void testSideInputNotReady() throws Exception { runner.startBundle(); runner.processElement(createDatum("e", 0)); - runner.finishKey(); + runner.finishKey("key"); runner.finishBundle(); assertTrue(outputManager.getOutput(mainOutputTag).isEmpty()); @@ -215,7 +215,7 @@ public void testMultipleWindowsNotReady() throws Exception { runner.startBundle(); runner.processElement(elem); - runner.finishKey(); + runner.finishKey("key"); runner.finishBundle(); assertTrue(outputManager.getOutput(mainOutputTag).isEmpty()); @@ -319,7 +319,7 @@ public void testSideInputNotification() throws Exception { when(mockSideInputReader.get(eq(view), any(BoundedWindow.class))).thenReturn("data"); runner.startBundle(); - runner.finishKey(); + runner.finishKey("key"); runner.finishBundle(); assertThat(outputManager.getOutput(mainOutputTag), contains(createDatum("e:data", 0))); @@ -376,7 +376,7 @@ public void testMultipleSideInputs() throws Exception { runner.startBundle(); runner.processElement(createDatum("e2", 2)); - runner.finishKey(); + runner.finishKey("key"); runner.finishBundle(); assertThat( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java index d5e3b9c87139..ac7c787b1d26 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java @@ -61,7 +61,7 @@ private static OutputReceiver[] createOutputReceivers(int numOutputs, CounterSet } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} } /** A {@code Reader} that yields a specified set of values. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java index 649b41483706..40f4c7b0e715 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java @@ -102,7 +102,7 @@ public void abort() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} } // A mock ReadOperation fed to a MapTaskExecutor in test. @@ -218,7 +218,7 @@ public void finishBundle() {} public void abort() {} @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} } /** Verify counts for the per-element-output-time counter are correct. */ @@ -317,7 +317,7 @@ public void start() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context2) { @Override @@ -329,7 +329,7 @@ public void start() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context3) { @Override @@ -341,7 +341,7 @@ public void start() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} }); assertEquals(TimeUnit.MINUTES.toMillis(10), stateTracker.getNextBundleLullDurationReportMs()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java index 9819f6e957a1..0c9667acf38b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java @@ -87,7 +87,7 @@ public void abort() throws Exception { } @Override - public void finishKey() throws Exception {} + public void finishKey(Object key) throws Exception {} } @Test @@ -107,7 +107,7 @@ public void testRunParDoOperation() throws Exception { parDoOperation.process(""); parDoOperation.process("bob"); - parDoOperation.finishKey(); + parDoOperation.finishKey("key"); parDoOperation.finish(); parDoOperation.abort(); @@ -151,7 +151,7 @@ public void testParDoOperationContext() throws Exception { operation.start(); operation.process("hello"); - operation.finishKey(); + operation.finishKey("key"); operation.finish(); InOrder inOrder = @@ -166,6 +166,7 @@ public void testParDoOperationContext() throws Exception { inOrder.verify(processCloseable).close(); inOrder.verify(context).enterProcessTimers(); inOrder.verify(fn).processTimers(); + inOrder.verify(fn).finishKey("key"); inOrder.verify(processTimersCloseable).close(); inOrder.verify(context).enterFinish(); inOrder.verify(fn).finishBundle(); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java index cf514b4ca499..5e8703a05b06 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java @@ -49,6 +49,7 @@ 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.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** @@ -285,7 +286,7 @@ public void finishBundle() { } @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public DoFn getFn() { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java index 543042098b63..7202de0f0aa8 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java @@ -30,6 +30,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.CausedByDrain; import org.apache.beam.sdk.values.WindowedValue; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** DoFnRunner decorator which registers {@link MetricsContainer}. */ @@ -105,7 +106,7 @@ public void finishBundle() { } @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java index 5dbfe5b096a7..bc434b2117de 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java @@ -29,6 +29,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.CausedByDrain; import org.apache.beam.sdk.values.WindowedValue; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** DoFnRunner decorator which registers {@link MetricsContainerImpl}. */ @@ -104,7 +105,7 @@ public void finishBundle() { } @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java index 6a5e03e30ab8..35fe7b745ea3 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.values.WindowedValues; 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.Lists; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; @@ -260,7 +261,7 @@ public void onTimer( public void finishBundle() {} @Override - public void finishKey() {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} From 01d83b87d4c4428a97eb4d9adb43dedfa63eb7d1 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Fri, 5 Jun 2026 04:54:19 +0000 Subject: [PATCH 14/20] change finishKey signature to finishKey(@Nullable Object key) --- .../java/org/apache/beam/runners/core/DoFnRunner.java | 4 ++-- .../beam/runners/core/LateDataDroppingDoFnRunner.java | 2 +- .../apache/beam/runners/core/SimpleDoFnRunner.java | 2 +- .../apache/beam/runners/core/StatefulDoFnRunner.java | 2 +- .../core/SimplePushbackSideInputDoFnRunnerTest.java | 2 +- .../flink/metrics/DoFnRunnerWithMetricsUpdate.java | 2 +- .../streaming/ExecutableStageDoFnOperator.java | 2 +- .../streaming/stableinput/BufferingDoFnRunner.java | 2 +- .../wrappers/streaming/DoFnOperatorTest.java | 2 +- .../dataflow/worker/AssignWindowsParDoFnFactory.java | 2 +- .../dataflow/worker/BatchModeUngroupingParDoFn.java | 3 ++- .../CreateIsmShardKeyAndSortKeyDoFnFactory.java | 3 ++- .../dataflow/worker/DataflowProcessFnRunner.java | 2 +- .../runners/dataflow/worker/ForwardingParDoFn.java | 3 ++- .../dataflow/worker/GroupAlsoByWindowFnRunner.java | 2 +- .../dataflow/worker/GroupAlsoByWindowsParDoFn.java | 2 +- .../worker/PairWithConstantKeyDoFnFactory.java | 3 ++- .../dataflow/worker/PartialGroupByKeyParDoFns.java | 4 ++-- .../ReifyTimestampAndWindowsParDoFnFactory.java | 2 +- .../beam/runners/dataflow/worker/SimpleParDoFn.java | 2 +- .../StreamingKeyedWorkItemSideInputDoFnRunner.java | 2 +- .../worker/StreamingPCollectionViewWriterParDoFn.java | 3 ++- .../dataflow/worker/StreamingSideInputDoFnRunner.java | 2 +- .../worker/ToIsmRecordForMultimapDoFnFactory.java | 3 ++- .../runners/dataflow/worker/ValuesDoFnFactory.java | 3 ++- .../worker/WorkerCustomSourceOperationExecutor.java | 3 ++- .../worker/util/common/worker/FlattenOperation.java | 3 ++- .../worker/util/common/worker/MapTaskExecutor.java | 2 +- .../dataflow/worker/util/common/worker/Operation.java | 4 +++- .../dataflow/worker/util/common/worker/ParDoFn.java | 4 +++- .../worker/util/common/worker/ParDoOperation.java | 3 ++- .../worker/util/common/worker/ReadOperation.java | 2 +- .../common/worker/SimplePartialGroupByKeyParDoFn.java | 4 +++- .../worker/util/common/worker/WorkExecutor.java | 2 +- .../worker/util/common/worker/WriteOperation.java | 3 ++- .../dataflow/worker/IntrinsicMapTaskExecutorTest.java | 11 ++++++----- .../worker/util/common/worker/ExecutorTestUtils.java | 3 ++- .../util/common/worker/MapTaskExecutorTest.java | 11 ++++++----- .../worker/util/common/worker/ParDoOperationTest.java | 3 ++- .../translation/batch/DoFnRunnerFactory.java | 2 +- .../translation/batch/DoFnRunnerWithMetrics.java | 2 +- .../spark/translation/DoFnRunnerWithMetrics.java | 2 +- .../translation/SparkInputDataProcessorTest.java | 2 +- 43 files changed, 74 insertions(+), 53 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index d1278eddaef0..addb723c0d74 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -71,9 +71,9 @@ public interface DoFnRunner void finishKey(KeyT key); + void finishKey(@Nullable Object key); /** * Returns the underlying fn instance. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index dfab198f8932..32446e8a6b40 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -103,7 +103,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) { + public void finishKey(@Nullable Object key) { doFnRunner.finishKey(key); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 61f8476df415..4c89b263d548 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -231,7 +231,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java index f5de79652f23..b8f50e376573 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java @@ -133,7 +133,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) { + public void finishKey(@Nullable Object key) { doFnRunner.finishKey(key); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java index aa61122a7547..ac7b4da8113c 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java @@ -381,7 +381,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java index c327c8d91bea..2be613cc6a5e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java @@ -107,7 +107,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java index f7bc62a8ec5a..82360b2b1d27 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java @@ -1065,7 +1065,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java index 6d1d4085f0bc..742b2a9a741c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java @@ -256,7 +256,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java index cce8f808c242..c77da4b23c4d 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java @@ -524,7 +524,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public void onWindowExpiration( diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java index 673130a6048a..14777835d749 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java @@ -120,7 +120,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java index c3cf6d9e67e2..5ae93416d1a6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java @@ -21,6 +21,7 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowedValue; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** @@ -74,7 +75,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java index afe2e28b2a67..7da27a0811ba 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link ParDoFnFactory} that creates a system {@link ParDoFn} responsible for limiting the users @@ -115,7 +116,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java index 400dbc047443..702b9a7293cb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java @@ -130,7 +130,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) { + public void finishKey(@Nullable Object key) { simpleRunner.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java index 31356632d065..3a864d6caf2f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java @@ -19,6 +19,7 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A base class for {@link ParDoFn} implementations for overriding particular methods while @@ -48,7 +49,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(Object key) throws Exception { + public void finishKey(@Nullable Object key) throws Exception { delegate.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java index aa60a61af8af..1b66eb1eea62 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java @@ -104,7 +104,7 @@ private void invokeProcessElement(WindowedValue elem) { public void finishBundle() {} @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java index e204a78a7d2e..ef227ea65b33 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java @@ -143,7 +143,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(Object key) throws Exception { + public void finishKey(@Nullable Object key) throws Exception { checkState(fnRunner != null); fnRunner.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java index 425184a4a126..0ad69803087f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowedValue; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link ParDoFnFactory} which returns a {@link ParDoFn} that transforms all {@code @@ -99,7 +100,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java index 932a495d0836..0b32def31f9d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java @@ -318,7 +318,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() throws Exception { @@ -381,7 +381,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception { + public void finishKey(@Nullable Object key) throws Exception { sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java index 0438b525b6b9..cc7c05e4faf3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java @@ -87,7 +87,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java index 9acb836bda9f..d422c1ca0e7a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java @@ -364,7 +364,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(Object key) throws Exception { + public void finishKey(@Nullable Object key) throws Exception { if (fnRunner != null) { fnRunner.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java index d8ce0a96807d..bf5ff3753c43 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java @@ -156,7 +156,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) { + public void finishKey(@Nullable Object key) { simpleDoFnRunner.finishKey(key); sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java index 6b51427bb930..bc01c5cd77db 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link ParDoFn} that writes side input data using {@link @@ -74,7 +75,7 @@ public void processElement(Object element) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() throws Exception {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java index 722452136ab7..734c53ac6113 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java @@ -98,7 +98,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) { + public void finishKey(@Nullable Object key) { simpleDoFnRunner.finishKey(key); sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java index 261ed69fb5d7..4d7516f72cbc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java @@ -39,6 +39,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link ParDoFnFactory} that creates a system {@link ParDoFn} responsible for transforming @@ -150,7 +151,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java index acf143a5467b..9ea2ce42c006 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java @@ -30,6 +30,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowedValue; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link ParDoFnFactory} which returns a {@link ParDoFn} with similar behavior to {@link @@ -81,7 +82,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java index 3220dbade61c..f28f2ffa230a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java @@ -27,6 +27,7 @@ import org.apache.beam.runners.core.metrics.ExecutionStateTracker.ExecutionState; import org.apache.beam.runners.dataflow.worker.counters.CounterSet; import org.apache.beam.sdk.options.PipelineOptions; +import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,7 +91,7 @@ public void execute() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public SourceOperationResponse getResponse() { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java index 3a131f94aa71..7ef011266507 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java @@ -19,6 +19,7 @@ import java.io.Closeable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.checkerframework.checker.nullness.qual.Nullable; /** A flatten operation. */ public class FlattenOperation extends ReceivingOperation { @@ -44,7 +45,7 @@ public void process(Object elem) throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public boolean supportsRestart() { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java index bbd28c6f55ba..e4b4d986a0a2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java @@ -113,7 +113,7 @@ public void execute() throws Exception { } @Override - public void finishKey(Object key) throws Exception { + public void finishKey(@Nullable Object key) throws Exception { for (Operation op : operations) { op.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java index 7594c9fc6192..138ce525e26b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow.worker.util.common.worker; +import org.checkerframework.checker.nullness.qual.Nullable; + /** * The abstract base class for Operations, which correspond to Instructions in the original MapTask * InstructionGraph. @@ -138,7 +140,7 @@ public void finish() throws Exception { } /** Called when all elements for a specific key have been processed. */ - public abstract void finishKey(Object key) throws Exception; + public abstract void finishKey(@Nullable Object key) throws Exception; /** Aborts this Operation's execution. */ public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java index 25008e310d03..75ecdb67dd42 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow.worker.util.common.worker; +import org.checkerframework.checker.nullness.qual.Nullable; + /** * Interface for functions invokable by {@link ParDoOperation} instances. * @@ -30,7 +32,7 @@ public interface ParDoFn { void processTimers() throws Exception; - void finishKey(Object key) throws Exception; + void finishKey(@Nullable Object key) throws Exception; void finishBundle() throws Exception; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java index b91d7ef94eda..a814b90cae21 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java @@ -19,6 +19,7 @@ import java.io.Closeable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.checkerframework.checker.nullness.qual.Nullable; /** A ParDo mapping function. */ public class ParDoOperation extends ReceivingOperation { @@ -48,7 +49,7 @@ public void process(Object elem) throws Exception { // Batch mode does not use this method and instead relies on BatchModeUngroupingParDoFn // to process timers per key. @Override - public void finishKey(Object key) throws Exception { + public void finishKey(@Nullable Object key) throws Exception { try (Closeable scope = context.enterProcessTimers()) { checkStarted(); fn.processTimers(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java index 8ce0d7dbfa2c..5d118626f18d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java @@ -272,7 +272,7 @@ public void finish() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java index bef27d6cea4d..cce77f3aceb6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow.worker.util.common.worker; +import org.checkerframework.checker.nullness.qual.Nullable; + /** A partial group-by-key {@link ParDoFn} implementation. */ public class SimplePartialGroupByKeyParDoFn implements ParDoFn { private final GroupingTable groupingTable; @@ -40,7 +42,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java index 022f12b74ddf..8aab050fb22c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java @@ -35,7 +35,7 @@ public interface WorkExecutor extends AutoCloseable { public abstract void execute() throws Exception; /** Called when all elements for a specific key have been processed. */ - void finishKey(Object key) throws Exception; + void finishKey(@Nullable Object key) throws Exception; /** * Returns the worker's current progress. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java index d0206a183946..a97c9920b9a3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java @@ -21,6 +21,7 @@ import org.apache.beam.runners.dataflow.worker.counters.Counter; import org.apache.beam.runners.dataflow.worker.counters.CounterName; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.checkerframework.checker.nullness.qual.Nullable; /** A write operation. */ @SuppressWarnings({ @@ -106,7 +107,7 @@ public void finish() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java index 73d69bd0f617..70705e502633 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java @@ -70,6 +70,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -106,7 +107,7 @@ public void abort() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} } // A mock ReadOperation fed to a MapTaskExecutor in test. @@ -222,7 +223,7 @@ public void finishBundle() {} public void abort() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} } /** Verify counts for the per-element-output-time counter are correct. */ @@ -320,7 +321,7 @@ public void start() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context2) { @Override @@ -332,7 +333,7 @@ public void start() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context3) { @Override @@ -344,7 +345,7 @@ public void start() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} }); try (IntrinsicMapTaskExecutor executor = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java index ac7c787b1d26..c822f2416ec1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java @@ -30,6 +30,7 @@ import org.apache.beam.runners.dataflow.worker.counters.CounterSet; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.checkerframework.checker.nullness.qual.Nullable; /** Utilities for tests. */ @SuppressWarnings({ @@ -61,7 +62,7 @@ private static OutputReceiver[] createOutputReceivers(int numOutputs, CounterSet } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} } /** A {@code Reader} that yields a specified set of values. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java index 40f4c7b0e715..bd2944849981 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java @@ -66,6 +66,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -102,7 +103,7 @@ public void abort() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} } // A mock ReadOperation fed to a MapTaskExecutor in test. @@ -218,7 +219,7 @@ public void finishBundle() {} public void abort() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} } /** Verify counts for the per-element-output-time counter are correct. */ @@ -317,7 +318,7 @@ public void start() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context2) { @Override @@ -329,7 +330,7 @@ public void start() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context3) { @Override @@ -341,7 +342,7 @@ public void start() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} }); assertEquals(TimeUnit.MINUTES.toMillis(10), stateTracker.getNextBundleLullDurationReportMs()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java index 0c9667acf38b..421cf9865fcf 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java @@ -32,6 +32,7 @@ import org.apache.beam.runners.dataflow.worker.counters.CounterFactory.LongCounterMean; import org.apache.beam.runners.dataflow.worker.counters.CounterSet; import org.apache.beam.runners.dataflow.worker.counters.NameContext; +import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.CoreMatchers; import org.junit.Test; import org.junit.runner.RunWith; @@ -87,7 +88,7 @@ public void abort() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} } @Test diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java index 5e8703a05b06..e267d14fb312 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java @@ -286,7 +286,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public DoFn getFn() { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java index 7202de0f0aa8..d3a074ddb3b1 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java @@ -106,7 +106,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java index bc434b2117de..d4de751b38b6 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java @@ -105,7 +105,7 @@ public void finishBundle() { } @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java index 35fe7b745ea3..b70787d5a521 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java @@ -261,7 +261,7 @@ public void onTimer( public void finishBundle() {} @Override - public void finishKey(KeyT key) {} + public void finishKey(@Nullable Object key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} From c7aa197af075cfea77889c88e8897167536057c0 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Mon, 8 Jun 2026 01:45:53 +0000 Subject: [PATCH 15/20] restore postsubmit files --- .../beam_PostCommit_Java_ValidatesRunner_Dataflow.json | 2 +- ...beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json index 8144784f5f02..e9d869cc508c 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run!", - "modification": 2, + "modification": 1, } 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 e1b083e439cc..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": 3, + "modification": 1, } From 84b4520ca0672bcae19d676ea0d0c847a8950102 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Thu, 11 Jun 2026 15:02:08 +0000 Subject: [PATCH 16/20] Revert "change finishKey signature to finishKey(@Nullable Object key)" This reverts commit 01d83b87d4c4428a97eb4d9adb43dedfa63eb7d1. --- .../java/org/apache/beam/runners/core/DoFnRunner.java | 4 ++-- .../beam/runners/core/LateDataDroppingDoFnRunner.java | 2 +- .../apache/beam/runners/core/SimpleDoFnRunner.java | 2 +- .../apache/beam/runners/core/StatefulDoFnRunner.java | 2 +- .../core/SimplePushbackSideInputDoFnRunnerTest.java | 2 +- .../flink/metrics/DoFnRunnerWithMetricsUpdate.java | 2 +- .../streaming/ExecutableStageDoFnOperator.java | 2 +- .../streaming/stableinput/BufferingDoFnRunner.java | 2 +- .../wrappers/streaming/DoFnOperatorTest.java | 2 +- .../dataflow/worker/AssignWindowsParDoFnFactory.java | 2 +- .../dataflow/worker/BatchModeUngroupingParDoFn.java | 3 +-- .../CreateIsmShardKeyAndSortKeyDoFnFactory.java | 3 +-- .../dataflow/worker/DataflowProcessFnRunner.java | 2 +- .../runners/dataflow/worker/ForwardingParDoFn.java | 3 +-- .../dataflow/worker/GroupAlsoByWindowFnRunner.java | 2 +- .../dataflow/worker/GroupAlsoByWindowsParDoFn.java | 2 +- .../worker/PairWithConstantKeyDoFnFactory.java | 3 +-- .../dataflow/worker/PartialGroupByKeyParDoFns.java | 4 ++-- .../ReifyTimestampAndWindowsParDoFnFactory.java | 2 +- .../beam/runners/dataflow/worker/SimpleParDoFn.java | 2 +- .../StreamingKeyedWorkItemSideInputDoFnRunner.java | 2 +- .../worker/StreamingPCollectionViewWriterParDoFn.java | 3 +-- .../dataflow/worker/StreamingSideInputDoFnRunner.java | 2 +- .../worker/ToIsmRecordForMultimapDoFnFactory.java | 3 +-- .../runners/dataflow/worker/ValuesDoFnFactory.java | 3 +-- .../worker/WorkerCustomSourceOperationExecutor.java | 3 +-- .../worker/util/common/worker/FlattenOperation.java | 3 +-- .../worker/util/common/worker/MapTaskExecutor.java | 2 +- .../dataflow/worker/util/common/worker/Operation.java | 4 +--- .../dataflow/worker/util/common/worker/ParDoFn.java | 4 +--- .../worker/util/common/worker/ParDoOperation.java | 3 +-- .../worker/util/common/worker/ReadOperation.java | 2 +- .../common/worker/SimplePartialGroupByKeyParDoFn.java | 4 +--- .../worker/util/common/worker/WorkExecutor.java | 2 +- .../worker/util/common/worker/WriteOperation.java | 3 +-- .../dataflow/worker/IntrinsicMapTaskExecutorTest.java | 11 +++++------ .../worker/util/common/worker/ExecutorTestUtils.java | 3 +-- .../util/common/worker/MapTaskExecutorTest.java | 11 +++++------ .../worker/util/common/worker/ParDoOperationTest.java | 3 +-- .../translation/batch/DoFnRunnerFactory.java | 2 +- .../translation/batch/DoFnRunnerWithMetrics.java | 2 +- .../spark/translation/DoFnRunnerWithMetrics.java | 2 +- .../translation/SparkInputDataProcessorTest.java | 2 +- 43 files changed, 53 insertions(+), 74 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index addb723c0d74..d1278eddaef0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -71,9 +71,9 @@ public interface DoFnRunner void finishKey(KeyT key); /** * Returns the underlying fn instance. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index 32446e8a6b40..dfab198f8932 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -103,7 +103,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) { + public void finishKey(KeyT key) { doFnRunner.finishKey(key); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 4c89b263d548..61f8476df415 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -231,7 +231,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java index b8f50e376573..f5de79652f23 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java @@ -133,7 +133,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) { + public void finishKey(KeyT key) { doFnRunner.finishKey(key); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java index ac7b4da8113c..aa61122a7547 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java @@ -381,7 +381,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java index 2be613cc6a5e..c327c8d91bea 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java @@ -107,7 +107,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java index 82360b2b1d27..f7bc62a8ec5a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java @@ -1065,7 +1065,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java index 742b2a9a741c..6d1d4085f0bc 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java @@ -256,7 +256,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java index c77da4b23c4d..cce8f808c242 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java @@ -524,7 +524,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration( diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java index 14777835d749..673130a6048a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java @@ -120,7 +120,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java index 5ae93416d1a6..c3cf6d9e67e2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java @@ -21,7 +21,6 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowedValue; -import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; /** @@ -75,7 +74,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java index 7da27a0811ba..afe2e28b2a67 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java @@ -36,7 +36,6 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link ParDoFnFactory} that creates a system {@link ParDoFn} responsible for limiting the users @@ -116,7 +115,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java index 702b9a7293cb..400dbc047443 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java @@ -130,7 +130,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) { + public void finishKey(KeyT key) { simpleRunner.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java index 3a864d6caf2f..31356632d065 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java @@ -19,7 +19,6 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; -import org.checkerframework.checker.nullness.qual.Nullable; /** * A base class for {@link ParDoFn} implementations for overriding particular methods while @@ -49,7 +48,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception { + public void finishKey(Object key) throws Exception { delegate.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java index 1b66eb1eea62..aa60a61af8af 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java @@ -104,7 +104,7 @@ private void invokeProcessElement(WindowedValue elem) { public void finishBundle() {} @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java index ef227ea65b33..e204a78a7d2e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java @@ -143,7 +143,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception { + public void finishKey(Object key) throws Exception { checkState(fnRunner != null); fnRunner.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java index 0ad69803087f..425184a4a126 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java @@ -35,7 +35,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowedValue; -import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link ParDoFnFactory} which returns a {@link ParDoFn} that transforms all {@code @@ -100,7 +99,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java index 0b32def31f9d..932a495d0836 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java @@ -318,7 +318,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { @@ -381,7 +381,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey(@Nullable Object key) throws Exception { + public void finishKey(Object key) throws Exception { sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java index cc7c05e4faf3..0438b525b6b9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java @@ -87,7 +87,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java index d422c1ca0e7a..9acb836bda9f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java @@ -364,7 +364,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception { + public void finishKey(Object key) throws Exception { if (fnRunner != null) { fnRunner.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java index bf5ff3753c43..d8ce0a96807d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java @@ -156,7 +156,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) { + public void finishKey(KeyT key) { simpleDoFnRunner.finishKey(key); sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java index bc01c5cd77db..6b51427bb930 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; -import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link ParDoFn} that writes side input data using {@link @@ -75,7 +74,7 @@ public void processElement(Object element) throws Exception { public void processTimers() {} @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java index 734c53ac6113..722452136ab7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java @@ -98,7 +98,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) { + public void finishKey(KeyT key) { simpleDoFnRunner.finishKey(key); sideInputFetcher.persist(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java index 4d7516f72cbc..261ed69fb5d7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java @@ -39,7 +39,6 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link ParDoFnFactory} that creates a system {@link ParDoFn} responsible for transforming @@ -151,7 +150,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java index 9ea2ce42c006..acf143a5467b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowedValue; -import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link ParDoFnFactory} which returns a {@link ParDoFn} with similar behavior to {@link @@ -82,7 +81,7 @@ public void processElement(Object untypedElem) throws Exception { public void processTimers() {} @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() {} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java index f28f2ffa230a..3220dbade61c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java @@ -27,7 +27,6 @@ import org.apache.beam.runners.core.metrics.ExecutionStateTracker.ExecutionState; import org.apache.beam.runners.dataflow.worker.counters.CounterSet; import org.apache.beam.sdk.options.PipelineOptions; -import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -91,7 +90,7 @@ public void execute() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public SourceOperationResponse getResponse() { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java index 7ef011266507..3a131f94aa71 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java @@ -19,7 +19,6 @@ import java.io.Closeable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.checkerframework.checker.nullness.qual.Nullable; /** A flatten operation. */ public class FlattenOperation extends ReceivingOperation { @@ -45,7 +44,7 @@ public void process(Object elem) throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public boolean supportsRestart() { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java index e4b4d986a0a2..bbd28c6f55ba 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java @@ -113,7 +113,7 @@ public void execute() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception { + public void finishKey(Object key) throws Exception { for (Operation op : operations) { op.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java index 138ce525e26b..7594c9fc6192 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java @@ -17,8 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.util.common.worker; -import org.checkerframework.checker.nullness.qual.Nullable; - /** * The abstract base class for Operations, which correspond to Instructions in the original MapTask * InstructionGraph. @@ -140,7 +138,7 @@ public void finish() throws Exception { } /** Called when all elements for a specific key have been processed. */ - public abstract void finishKey(@Nullable Object key) throws Exception; + public abstract void finishKey(Object key) throws Exception; /** Aborts this Operation's execution. */ public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java index 75ecdb67dd42..25008e310d03 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java @@ -17,8 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.util.common.worker; -import org.checkerframework.checker.nullness.qual.Nullable; - /** * Interface for functions invokable by {@link ParDoOperation} instances. * @@ -32,7 +30,7 @@ public interface ParDoFn { void processTimers() throws Exception; - void finishKey(@Nullable Object key) throws Exception; + void finishKey(Object key) throws Exception; void finishBundle() throws Exception; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java index a814b90cae21..b91d7ef94eda 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java @@ -19,7 +19,6 @@ import java.io.Closeable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.checkerframework.checker.nullness.qual.Nullable; /** A ParDo mapping function. */ public class ParDoOperation extends ReceivingOperation { @@ -49,7 +48,7 @@ public void process(Object elem) throws Exception { // Batch mode does not use this method and instead relies on BatchModeUngroupingParDoFn // to process timers per key. @Override - public void finishKey(@Nullable Object key) throws Exception { + public void finishKey(Object key) throws Exception { try (Closeable scope = context.enterProcessTimers()) { checkStarted(); fn.processTimers(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java index 5d118626f18d..8ce0d7dbfa2c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java @@ -272,7 +272,7 @@ public void finish() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java index cce77f3aceb6..bef27d6cea4d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java @@ -17,8 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.util.common.worker; -import org.checkerframework.checker.nullness.qual.Nullable; - /** A partial group-by-key {@link ParDoFn} implementation. */ public class SimplePartialGroupByKeyParDoFn implements ParDoFn { private final GroupingTable groupingTable; @@ -42,7 +40,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java index 8aab050fb22c..022f12b74ddf 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java @@ -35,7 +35,7 @@ public interface WorkExecutor extends AutoCloseable { public abstract void execute() throws Exception; /** Called when all elements for a specific key have been processed. */ - void finishKey(@Nullable Object key) throws Exception; + void finishKey(Object key) throws Exception; /** * Returns the worker's current progress. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java index a97c9920b9a3..d0206a183946 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java @@ -21,7 +21,6 @@ import org.apache.beam.runners.dataflow.worker.counters.Counter; import org.apache.beam.runners.dataflow.worker.counters.CounterName; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; -import org.checkerframework.checker.nullness.qual.Nullable; /** A write operation. */ @SuppressWarnings({ @@ -107,7 +106,7 @@ public void finish() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} @Override public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java index 70705e502633..73d69bd0f617 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java @@ -70,7 +70,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -107,7 +106,7 @@ public void abort() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} } // A mock ReadOperation fed to a MapTaskExecutor in test. @@ -223,7 +222,7 @@ public void finishBundle() {} public void abort() {} @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} } /** Verify counts for the per-element-output-time counter are correct. */ @@ -321,7 +320,7 @@ public void start() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context2) { @Override @@ -333,7 +332,7 @@ public void start() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context3) { @Override @@ -345,7 +344,7 @@ public void start() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} }); try (IntrinsicMapTaskExecutor executor = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java index c822f2416ec1..ac7c787b1d26 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java @@ -30,7 +30,6 @@ import org.apache.beam.runners.dataflow.worker.counters.CounterSet; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; /** Utilities for tests. */ @SuppressWarnings({ @@ -62,7 +61,7 @@ private static OutputReceiver[] createOutputReceivers(int numOutputs, CounterSet } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} } /** A {@code Reader} that yields a specified set of values. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java index bd2944849981..40f4c7b0e715 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java @@ -66,7 +66,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -103,7 +102,7 @@ public void abort() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} } // A mock ReadOperation fed to a MapTaskExecutor in test. @@ -219,7 +218,7 @@ public void finishBundle() {} public void abort() {} @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} } /** Verify counts for the per-element-output-time counter are correct. */ @@ -318,7 +317,7 @@ public void start() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context2) { @Override @@ -330,7 +329,7 @@ public void start() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} }, new Operation(new OutputReceiver[] {}, context3) { @Override @@ -342,7 +341,7 @@ public void start() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} }); assertEquals(TimeUnit.MINUTES.toMillis(10), stateTracker.getNextBundleLullDurationReportMs()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java index 421cf9865fcf..0c9667acf38b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperationTest.java @@ -32,7 +32,6 @@ import org.apache.beam.runners.dataflow.worker.counters.CounterFactory.LongCounterMean; import org.apache.beam.runners.dataflow.worker.counters.CounterSet; import org.apache.beam.runners.dataflow.worker.counters.NameContext; -import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.CoreMatchers; import org.junit.Test; import org.junit.runner.RunWith; @@ -88,7 +87,7 @@ public void abort() throws Exception { } @Override - public void finishKey(@Nullable Object key) throws Exception {} + public void finishKey(Object key) throws Exception {} } @Test diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java index e267d14fb312..5e8703a05b06 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java @@ -286,7 +286,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public DoFn getFn() { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java index d3a074ddb3b1..7202de0f0aa8 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java @@ -106,7 +106,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java index d4de751b38b6..bc434b2117de 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java @@ -105,7 +105,7 @@ public void finishBundle() { } @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) { diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java index b70787d5a521..35fe7b745ea3 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java @@ -261,7 +261,7 @@ public void onTimer( public void finishBundle() {} @Override - public void finishKey(@Nullable Object key) {} + public void finishKey(KeyT key) {} @Override public void onWindowExpiration(BoundedWindow window, Instant timestamp, KeyT key) {} From 61e2e8046a914512f53269536e718c652219464f Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Thu, 11 Jun 2026 15:34:56 +0000 Subject: [PATCH 17/20] Remove sideinput logic from finishKey, Make finishKey methods a noop --- .../runners/dataflow/worker/PartialGroupByKeyParDoFns.java | 2 +- .../apache/beam/runners/dataflow/worker/SimpleParDoFn.java | 2 -- .../beam/runners/dataflow/worker/SimpleParDoFnHelpers.java | 6 ------ .../worker/StreamingKeyedWorkItemSideInputDoFnRunner.java | 2 +- 4 files changed, 2 insertions(+), 10 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java index 932a495d0836..5201ef710deb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java @@ -382,12 +382,12 @@ public void processTimers() {} @Override public void finishKey(Object key) throws Exception { - sideInputFetcher.persist(); } @Override public void finishBundle() throws Exception { groupingTable.flush(receiver); + sideInputFetcher.persist(); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java index 6fd2d025e081..82a5fa194469 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java @@ -181,7 +181,6 @@ public void processTimers() throws Exception { .getWindowingStrategy() .getWindowFn() .windowCoder(); - helpers.processTimers( SimpleParDoFnHelpers.TimerType.USER, helpers.userStepContext, @@ -198,7 +197,6 @@ public void processTimers() throws Exception { @Override public void finishKey(Object key) throws Exception { - helpers.finishKey(key); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnHelpers.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnHelpers.java index 940e5fbd13e6..964cf2323d51 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnHelpers.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnHelpers.java @@ -233,12 +233,6 @@ public void output(TupleTag tag, WindowedValue output) { fnRunner.startBundle(); } - public void finishKey(Object key) { - if (fnRunner != null) { - fnRunner.finishKey(key); - } - } - void finishBundle(StreamingSideInputProcessor sideInputProcessor) throws Exception { if (fnRunner != null) { fnRunner.finishBundle(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java index d8ce0a96807d..3de7c0f3a9b9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java @@ -153,12 +153,12 @@ public void onTimer( @Override public void finishBundle() { simpleDoFnRunner.finishBundle(); + sideInputFetcher.persist(); } @Override public void finishKey(KeyT key) { simpleDoFnRunner.finishKey(key); - sideInputFetcher.persist(); } @Override From 762622dae9d27b9d4b5f3d6db7173551ac0cef64 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Thu, 11 Jun 2026 16:17:53 +0000 Subject: [PATCH 18/20] spotless --- .../runners/dataflow/worker/PartialGroupByKeyParDoFns.java | 3 +-- .../org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java index 5201ef710deb..a6d7810412a1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java @@ -381,8 +381,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception { - } + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java index 82a5fa194469..e0f1e0f410cd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java @@ -196,8 +196,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(Object key) throws Exception { - } + public void finishKey(Object key) throws Exception {} @Override public void finishBundle() throws Exception { From a2b3c00fd918f71070fc39c35b819648d5d585ed Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Thu, 11 Jun 2026 18:58:16 +0000 Subject: [PATCH 19/20] fix compile --- .../worker/StreamingKeyedWorkItemSideInputParDoFn.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputParDoFn.java index 225bc6af0ea9..63de0b8d55db 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputParDoFn.java @@ -193,6 +193,9 @@ public void processTimers() throws Exception { () -> sideInputProcessor); } + @Override + public void finishKey(Object key) throws Exception {} + @Override public void finishBundle() throws Exception { helpers.finishBundle(sideInputProcessor); From 79887799de6b4e22fd33595510b41d3a5814436d Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Thu, 11 Jun 2026 19:56:30 +0000 Subject: [PATCH 20/20] fix nullness --- .../beam/runners/dataflow/worker/ForwardingParDoFn.java | 3 ++- .../dataflow/worker/util/common/worker/FlattenOperation.java | 3 ++- .../dataflow/worker/util/common/worker/MapTaskExecutor.java | 2 +- .../runners/dataflow/worker/util/common/worker/Operation.java | 4 +++- .../runners/dataflow/worker/util/common/worker/ParDoFn.java | 4 +++- .../dataflow/worker/util/common/worker/ParDoOperation.java | 3 ++- .../dataflow/worker/util/common/worker/ReadOperation.java | 2 +- .../util/common/worker/SimplePartialGroupByKeyParDoFn.java | 4 +++- .../dataflow/worker/util/common/worker/WorkExecutor.java | 2 +- .../dataflow/worker/util/common/worker/WriteOperation.java | 3 ++- 10 files changed, 20 insertions(+), 10 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java index 31356632d065..3a864d6caf2f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ForwardingParDoFn.java @@ -19,6 +19,7 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A base class for {@link ParDoFn} implementations for overriding particular methods while @@ -48,7 +49,7 @@ public void processTimers() throws Exception { } @Override - public void finishKey(Object key) throws Exception { + public void finishKey(@Nullable Object key) throws Exception { delegate.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java index 3a131f94aa71..7ef011266507 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/FlattenOperation.java @@ -19,6 +19,7 @@ import java.io.Closeable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.checkerframework.checker.nullness.qual.Nullable; /** A flatten operation. */ public class FlattenOperation extends ReceivingOperation { @@ -44,7 +45,7 @@ public void process(Object elem) throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public boolean supportsRestart() { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java index fc09986d57c9..90fc76276940 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java @@ -110,7 +110,7 @@ public void execute() throws Exception { } @Override - public void finishKey(Object key) throws Exception { + public void finishKey(@Nullable Object key) throws Exception { for (Operation op : operations) { op.finishKey(key); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java index 7594c9fc6192..138ce525e26b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/Operation.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow.worker.util.common.worker; +import org.checkerframework.checker.nullness.qual.Nullable; + /** * The abstract base class for Operations, which correspond to Instructions in the original MapTask * InstructionGraph. @@ -138,7 +140,7 @@ public void finish() throws Exception { } /** Called when all elements for a specific key have been processed. */ - public abstract void finishKey(Object key) throws Exception; + public abstract void finishKey(@Nullable Object key) throws Exception; /** Aborts this Operation's execution. */ public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java index 25008e310d03..75ecdb67dd42 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoFn.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow.worker.util.common.worker; +import org.checkerframework.checker.nullness.qual.Nullable; + /** * Interface for functions invokable by {@link ParDoOperation} instances. * @@ -30,7 +32,7 @@ public interface ParDoFn { void processTimers() throws Exception; - void finishKey(Object key) throws Exception; + void finishKey(@Nullable Object key) throws Exception; void finishBundle() throws Exception; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java index b91d7ef94eda..a814b90cae21 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ParDoOperation.java @@ -19,6 +19,7 @@ import java.io.Closeable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.checkerframework.checker.nullness.qual.Nullable; /** A ParDo mapping function. */ public class ParDoOperation extends ReceivingOperation { @@ -48,7 +49,7 @@ public void process(Object elem) throws Exception { // Batch mode does not use this method and instead relies on BatchModeUngroupingParDoFn // to process timers per key. @Override - public void finishKey(Object key) throws Exception { + public void finishKey(@Nullable Object key) throws Exception { try (Closeable scope = context.enterProcessTimers()) { checkStarted(); fn.processTimers(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java index 8ce0d7dbfa2c..5d118626f18d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java @@ -272,7 +272,7 @@ public void finish() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void abort() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java index bef27d6cea4d..cce77f3aceb6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/SimplePartialGroupByKeyParDoFn.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow.worker.util.common.worker; +import org.checkerframework.checker.nullness.qual.Nullable; + /** A partial group-by-key {@link ParDoFn} implementation. */ public class SimplePartialGroupByKeyParDoFn implements ParDoFn { private final GroupingTable groupingTable; @@ -40,7 +42,7 @@ public void processElement(Object elem) throws Exception { public void processTimers() {} @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void finishBundle() throws Exception { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java index 022f12b74ddf..8aab050fb22c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java @@ -35,7 +35,7 @@ public interface WorkExecutor extends AutoCloseable { public abstract void execute() throws Exception; /** Called when all elements for a specific key have been processed. */ - void finishKey(Object key) throws Exception; + void finishKey(@Nullable Object key) throws Exception; /** * Returns the worker's current progress. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java index d0206a183946..a97c9920b9a3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java @@ -21,6 +21,7 @@ import org.apache.beam.runners.dataflow.worker.counters.Counter; import org.apache.beam.runners.dataflow.worker.counters.CounterName; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.checkerframework.checker.nullness.qual.Nullable; /** A write operation. */ @SuppressWarnings({ @@ -106,7 +107,7 @@ public void finish() throws Exception { } @Override - public void finishKey(Object key) throws Exception {} + public void finishKey(@Nullable Object key) throws Exception {} @Override public void abort() throws Exception {