From 48e5414ef194550b08d655e3919a6f6649856483 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 9 Jun 2025 12:07:44 -0400 Subject: [PATCH 1/2] Touch trigger files for lightweight runners --- .../beam_PostCommit_Java_ValidatesRunner_Direct.json | 1 + .../beam_PostCommit_Java_ValidatesRunner_Flink.json | 1 + .../beam_PostCommit_Java_ValidatesRunner_Spark.json | 1 + ...PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json | 1 + 4 files changed, 4 insertions(+) diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json index befe325aff7b..42959ad85255 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json @@ -1,4 +1,5 @@ { + "https://github.com/apache/beam/pull/35213": "Eliminating getPane() in favor of getPaneInfo()", "https://github.com/apache/beam/pull/35177": "Introducing WindowedValueReceiver to runners", "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31761": "noting that PR #31761 should run this test", diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json index b6d8417412d5..3ce625b167aa 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json @@ -1,4 +1,5 @@ { + "https://github.com/apache/beam/pull/35213": "Eliminating getPane() in favor of getPaneInfo()", "https://github.com/apache/beam/pull/35177": "Introducing WindowedValueReceiver to runners", "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/32440": "testing datastream optimizations", diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json index 5c8755813ac7..09dc40d75c2b 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json @@ -1,4 +1,5 @@ { + "https://github.com/apache/beam/pull/35213": "Eliminating getPane() in favor of getPaneInfo()", "https://github.com/apache/beam/pull/35177": "Introducing WindowedValueReceiver to runners", "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json index 8ce9bcbb62c4..5a72b5d2a094 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json @@ -1,4 +1,5 @@ { + "https://github.com/apache/beam/pull/35213": "Eliminating getPane() in favor of getPaneInfo()", "https://github.com/apache/beam/pull/35177": "Introducing WindowedValueReceiver to runners", "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", From 0e91b58277ca72eaebce35be069b3d31a1699fc2 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 9 Jun 2025 12:04:50 -0400 Subject: [PATCH 2/2] Eliminate WindowedValue.getPane() in preparation for making it a user-facing interface --- .../complete/game/LeaderBoardTest.java | 13 +- .../core/LateDataDroppingDoFnRunner.java | 2 +- ...oundedSplittableProcessElementInvoker.java | 6 +- .../beam/runners/core/PaneInfoTracker.java | 2 +- .../runners/core/ReduceFnContextFactory.java | 12 +- .../beam/runners/core/ReduceFnRunner.java | 10 +- .../beam/runners/core/SimpleDoFnRunner.java | 4 +- .../SplittableParDoViaKeyedWorkItems.java | 6 +- .../beam/runners/core/WindowingInternals.java | 6 +- .../beam/runners/core/ReduceFnRunnerTest.java | 82 +++++++----- .../beam/runners/core/WindowMatchers.java | 6 +- .../runners/direct/SideInputContainer.java | 4 +- .../direct/WindowEvaluatorFactory.java | 2 +- .../direct/WindowEvaluatorFactoryTest.java | 18 +-- .../FlinkStreamingTransformTranslators.java | 4 +- .../functions/FlinkAssignWindows.java | 2 +- .../functions/SortingFlinkCombineRunner.java | 4 +- .../runners/dataflow/BatchViewOverrides.java | 2 +- .../RedistributeByKeyOverrideFactory.java | 6 +- .../worker/AssignWindowsParDoFnFactory.java | 2 +- .../worker/PartialGroupByKeyParDoFns.java | 6 +- ...eifyTimestampAndWindowsParDoFnFactory.java | 4 +- .../worker/UngroupedWindmillReader.java | 6 +- .../worker/WindmillKeyedWorkItem.java | 4 +- .../runners/dataflow/worker/WindmillSink.java | 6 +- .../worker/util/ValueInEmptyWindows.java | 4 +- ...TimestampAndWindowsParDoFnFactoryTest.java | 2 +- .../worker/StreamingDataflowWorkerTest.java | 13 +- .../worker/WindmillKeyedWorkItemTest.java | 4 +- .../control/BundleCheckpointHandlers.java | 2 +- .../runners/jet/processors/AssignWindowP.java | 2 +- .../beam/runners/jet/processors/ViewP.java | 12 +- .../runners/jet/processors/WindowGroupP.java | 2 +- .../beam/runners/samza/runtime/DoFnOp.java | 2 +- .../runners/samza/runtime/WindowAssignOp.java | 2 +- .../batch/GroupByKeyTranslatorBatch.java | 2 +- .../batch/WindowAssignTranslatorBatch.java | 2 +- .../translation/helpers/EncoderHelpers.java | 19 +-- .../GroupNonMergingWindowsFunctions.java | 3 +- .../ReifyTimestampsAndWindowsFunction.java | 5 +- .../translation/TransformTranslator.java | 7 +- .../spark/translation/TranslationUtils.java | 2 +- .../functions/AssignWindowsFunction.java | 2 +- .../ByteToWindowFunctionPrimitive.java | 2 +- .../functions/MapToTupleFunction.java | 4 +- .../java/org/apache/beam/sdk/io/FileIO.java | 14 +- .../beam/sdk/testing/PaneExtractors.java | 17 ++- .../apache/beam/sdk/transforms/Create.java | 6 +- .../apache/beam/sdk/transforms/Distinct.java | 8 +- .../beam/sdk/transforms/DoFnTester.java | 6 +- .../beam/sdk/transforms/Redistribute.java | 2 +- .../org/apache/beam/sdk/transforms/Reify.java | 9 +- .../apache/beam/sdk/transforms/Reshuffle.java | 2 +- .../beam/sdk/util/RowStringInterpolator.java | 2 +- .../beam/sdk/util/construction/Timer.java | 18 +-- .../values/FailsafeValueInSingleWindow.java | 4 +- .../beam/sdk/values/ValueInSingleWindow.java | 8 +- .../apache/beam/sdk/values/WindowedValue.java | 8 +- .../beam/sdk/values/WindowedValues.java | 126 +++++++++--------- .../sdk/io/DefaultFilenamePolicyTest.java | 4 +- .../org/apache/beam/sdk/io/FileIOTest.java | 4 +- .../beam/sdk/testing/TestStreamTest.java | 8 +- .../beam/sdk/transforms/CreateTest.java | 2 +- .../beam/sdk/transforms/RedistributeTest.java | 4 +- .../beam/sdk/transforms/ReshuffleTest.java | 4 +- .../beam/sdk/util/WindowedValueTest.java | 12 +- .../beam/sdk/util/construction/TimerTest.java | 2 +- .../beam/fn/harness/AssignWindowsRunner.java | 2 +- .../beam/fn/harness/FnApiDoFnRunner.java | 75 ++++++----- ...littablePairWithRestrictionDoFnRunner.java | 4 +- ...bleSplitAndSizeRestrictionsDoFnRunner.java | 4 +- .../fn/harness/AssignWindowsRunnerTest.java | 4 +- .../beam/fn/harness/FnApiDoFnRunnerTest.java | 66 ++++----- ...ablePairWithRestrictionDoFnRunnerTest.java | 12 +- ...plitAndSizeRestrictionsDoFnRunnerTest.java | 24 ++-- .../gcp/bigquery/BatchedStreamingWrite.java | 6 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 2 +- .../sdk/io/gcp/bigquery/ErrorContainer.java | 4 +- .../sdk/io/gcp/bigquery/PrepareWrite.java | 4 +- .../sdk/io/gcp/bigquery/WritePartition.java | 1 - .../io/gcp/testing/FakeDatasetService.java | 2 +- .../io/iceberg/WriteGroupedRowsToFiles.java | 4 +- .../io/iceberg/WriteUngroupedRowsToFiles.java | 4 +- .../beam/sdk/nexmark/queries/Query10.java | 8 +- 84 files changed, 422 insertions(+), 386 deletions(-) diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java index f6293f693133..13f80012cc1b 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java @@ -99,7 +99,7 @@ public void testTeamScoresOnTime() { .addElements( event(TestUser.RED_ONE, 1, Duration.standardMinutes(4)), event(TestUser.BLUE_ONE, 2, Duration.standardSeconds(270))) - // The window should close and emit an ON_TIME pane + // The window should close and emit an ON_TIME paneInfo .advanceWatermarkToInfinity(); PCollection> teamScores = @@ -137,7 +137,8 @@ public void testTeamScoresSpeculative() { // Some additional time passes and we get a speculative pane for the red team .advanceProcessingTime(Duration.standardMinutes(12)) .addElements(event(TestUser.BLUE_TWO, 3, Duration.standardSeconds(22))) - // More time passes and a speculative pane containing a refined value for the blue pane + // More time passes and a speculative pane containing a refined value for the blue + // paneInfo // is // emitted .advanceProcessingTime(Duration.standardMinutes(10)) @@ -155,7 +156,7 @@ public void testTeamScoresSpeculative() { String blueTeam = TestUser.BLUE_ONE.getTeam(); String redTeam = TestUser.RED_ONE.getTeam(); IntervalWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); - // The window contains speculative panes alongside the on-time pane + // The window contains speculative panes alongside the on-time paneInfo PAssert.that(teamScores) .inWindow(window) .containsInAnyOrder( @@ -190,7 +191,7 @@ public void testTeamScoresUnobservablyLate() { .advanceWatermarkTo( baseTime.plus(TEAM_WINDOW_DURATION).minus(Duration.standardMinutes(1))) // These events are late, but the window hasn't closed yet, so the elements are in the - // on-time pane + // on-time paneInfo .addElements( event(TestUser.RED_TWO, 2, Duration.ZERO), event(TestUser.RED_TWO, 5, Duration.standardMinutes(1)), @@ -235,7 +236,7 @@ public void testTeamScoresObservablyLate() { event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)), event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(5))) .advanceWatermarkTo(firstWindowCloses.minus(Duration.standardMinutes(1))) - // These events are late but should still appear in a late pane + // These events are late but should still appear in a late paneInfo .addElements( event(TestUser.RED_TWO, 2, Duration.ZERO), event(TestUser.RED_TWO, 5, Duration.standardMinutes(1)), @@ -244,7 +245,7 @@ public void testTeamScoresObservablyLate() { // has // not yet closed because the watermark has not advanced .advanceProcessingTime(Duration.standardMinutes(12)) - // These elements should appear in the final pane + // These elements should appear in the final paneInfo .addElements( event(TestUser.RED_TWO, 9, Duration.standardMinutes(1)), event(TestUser.RED_TWO, 1, Duration.standardMinutes(3))) 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 0fb8ff52d1e1..f89c3f36db3f 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 @@ -142,7 +142,7 @@ public Iterable> filter( } else { nonLateElements.add( WindowedValues.of( - element.getValue(), element.getTimestamp(), window, element.getPane())); + element.getValue(), element.getTimestamp(), window, element.getPaneInfo())); } } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index 7a2d47be8b0b..f8dbfd61e836 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -380,7 +380,7 @@ public Instant timestamp() { @Override public PaneInfo pane() { - return element.getPane(); + return element.getPaneInfo(); } @Override @@ -395,7 +395,7 @@ public void output(OutputT output) { @Override public void outputWithTimestamp(OutputT value, Instant timestamp) { - outputWindowedValue(value, timestamp, element.getWindows(), element.getPane()); + outputWindowedValue(value, timestamp, element.getWindows(), element.getPaneInfo()); } @Override @@ -419,7 +419,7 @@ public void output(TupleTag tag, T value) { @Override public void outputWithTimestamp(TupleTag tag, T value, Instant timestamp) { outputReceiver.output( - tag, WindowedValues.of(value, timestamp, element.getWindows(), element.getPane())); + tag, WindowedValues.of(value, timestamp, element.getWindows(), element.getPaneInfo())); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java index 12ceca976417..543b2cb5a741 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java @@ -48,7 +48,7 @@ public PaneInfoTracker(TimerInternals timerInternals) { @VisibleForTesting static final StateTag> PANE_INFO_TAG = - StateTags.makeSystemTagInternal(StateTags.value("pane", PaneInfoCoder.INSTANCE)); + StateTags.makeSystemTagInternal(StateTags.value("paneInfo", PaneInfoCoder.INSTANCE)); public void clear(StateAccessor state) { state.access(PANE_INFO_TAG).clear(); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index 2479451c2b46..1ca927f16caa 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java @@ -103,8 +103,8 @@ public ReduceFn.ProcessValueContext forValue( } public ReduceFn.OnTriggerContext forTrigger( - W window, PaneInfo pane, StateStyle style, OnTriggerCallbacks callbacks) { - return new OnTriggerContextImpl(stateAccessor(window, style), pane, callbacks); + W window, PaneInfo paneInfo, StateStyle style, OnTriggerCallbacks callbacks) { + return new OnTriggerContextImpl(stateAccessor(window, style), paneInfo, callbacks); } public ReduceFn.OnMergeContext forMerge( @@ -402,15 +402,15 @@ public Timers timers() { private class OnTriggerContextImpl extends ReduceFn.OnTriggerContext { private final StateAccessorImpl state; - private final PaneInfo pane; + private final PaneInfo paneInfo; private final OnTriggerCallbacks callbacks; private final TimersImpl timers; private OnTriggerContextImpl( - StateAccessorImpl state, PaneInfo pane, OnTriggerCallbacks callbacks) { + StateAccessorImpl state, PaneInfo paneInfo, OnTriggerCallbacks callbacks) { reduceFn.super(); this.state = state; - this.pane = pane; + this.paneInfo = paneInfo; this.callbacks = callbacks; this.timers = new TimersImpl(state.namespace()); } @@ -437,7 +437,7 @@ public StateAccessor state() { @Override public PaneInfo paneInfo() { - return pane; + return paneInfo; } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index a40f4e1205c7..4e10dd471b40 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -1037,28 +1037,28 @@ private void prefetchOnTrigger( } // Calculate the pane info. - final PaneInfo pane = paneInfoTracker.getNextPaneInfo(directContext, isFinished).read(); + final PaneInfo paneInfo = paneInfoTracker.getNextPaneInfo(directContext, isFinished).read(); // Only emit a pane if it has data or empty panes are observable. - if (needToEmit(isEmpty, isFinished, pane.getTiming())) { + if (needToEmit(isEmpty, isFinished, paneInfo.getTiming())) { // Run reduceFn.onTrigger method. final List windows = Collections.singletonList(directContext.window()); ReduceFn.OnTriggerContext renamedTriggerContext = contextFactory.forTrigger( directContext.window(), - pane, + paneInfo, StateStyle.RENAMED, toOutput -> { // We're going to output panes, so commit the (now used) PaneInfo. // This is unnecessary if the trigger isFinished since the saved // state will be immediately deleted. if (!isFinished) { - paneInfoTracker.storeCurrentPaneInfo(directContext, pane); + paneInfoTracker.storeCurrentPaneInfo(directContext, paneInfo); } // Output the actual value. outputter.output( - WindowedValues.of(KV.of(key, toOutput), outputTimestamp, windows, pane)); + WindowedValues.of(KV.of(key, toOutput), outputTimestamp, windows, paneInfo)); }); reduceFn.onTrigger(renamedTriggerContext); 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 d167bf63c8b7..840245edf7ad 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 @@ -405,7 +405,7 @@ public T sideInput(PCollectionView view) { @Override public PaneInfo pane() { - return elem.getPane(); + return elem.getPaneInfo(); } @Override @@ -437,7 +437,7 @@ public void output(TupleTag tag, T output) { public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { checkNotNull(tag, "Tag passed to outputWithTimestamp cannot be null"); checkTimestamp(elem.getTimestamp(), timestamp); - outputWindowedValue(tag, output, timestamp, elem.getWindows(), elem.getPane()); + outputWindowedValue(tag, output, timestamp, elem.getWindows(), elem.getPaneInfo()); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java index 772428f84b65..fafb02f9dd0b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java @@ -431,7 +431,7 @@ public PipelineOptions pipelineOptions() { @Override public PaneInfo paneInfo(DoFn doFn) { - return elementAndRestriction.getKey().getPane(); + return elementAndRestriction.getKey().getPaneInfo(); } @Override @@ -491,7 +491,7 @@ public PipelineOptions pipelineOptions() { @Override public PaneInfo paneInfo(DoFn doFn) { - return elementAndRestriction.getKey().getPane(); + return elementAndRestriction.getKey().getPaneInfo(); } @Override @@ -545,7 +545,7 @@ public PipelineOptions pipelineOptions() { @Override public PaneInfo paneInfo(DoFn doFn) { - return elementAndRestriction.getKey().getPane(); + return elementAndRestriction.getKey().getPaneInfo(); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowingInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowingInternals.java index 4b3438eac70a..9577be65be9d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowingInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowingInternals.java @@ -47,7 +47,7 @@ void outputWindowedValue( OutputT output, Instant timestamp, Collection windows, - PaneInfo pane); + PaneInfo paneInfo); /** Output the value to a tagged output at the specified timestamp in the listed windows. */ void outputWindowedValue( @@ -55,7 +55,7 @@ void outputWindowedValue( AdditionalOutputT output, Instant timestamp, Collection windows, - PaneInfo pane); + PaneInfo paneInfo); /** * Return the timer manager provided by the underlying system, or null if Timers need to be @@ -67,7 +67,7 @@ void outputWindowedValue( Collection windows(); /** Access the pane of the current window(s). */ - PaneInfo pane(); + PaneInfo paneInfo(); /** Return the value of the side input for a particular side input window. */ T sideInput(PCollectionView view, BoundedWindow sideInputWindow); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index 048bfb069451..85f6573be23e 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -759,7 +759,8 @@ public void testWatermarkHoldAndLateData() throws Exception { equalTo(new Instant(1)), equalTo((BoundedWindow) expectedWindow)))); assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))); + output.get(0).getPaneInfo(), + equalTo(PaneInfo.createPane(true, false, Timing.EARLY, 0, -1))); // There is no end-of-window hold, but the timer set by the trigger holds the watermark assertThat(tester.getWatermarkHold(), nullValue()); @@ -805,7 +806,8 @@ public void testWatermarkHoldAndLateData() throws Exception { 0, // window start 10))); // window end assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1))); + output.get(0).getPaneInfo(), + equalTo(PaneInfo.createPane(false, false, Timing.EARLY, 1, -1))); // Since the element hold is cleared, there is no hold remaining assertThat(tester.getWatermarkHold(), nullValue()); @@ -821,7 +823,7 @@ public void testWatermarkHoldAndLateData() throws Exception { injectElement(tester, 4); - // Fire the ON_TIME pane + // Fire the ON_TIME paneInfo when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); // To get an ON_TIME pane, we need the output watermark to be held back a little; this would @@ -831,7 +833,7 @@ public void testWatermarkHoldAndLateData() throws Exception { tester.advanceInputWatermark(expectedWindow.maxTimestamp().plus(Duration.millis(1))); tester.fireTimer(expectedWindow, expectedWindow.maxTimestamp(), TimeDomain.EVENT_TIME); - // Output time is end of the window, because all the new data was late, but the pane + // Output time is end of the window, because all the new data was late, but the paneInfo // is the ON_TIME pane. output = tester.extractOutput(); assertThat( @@ -846,7 +848,8 @@ public void testWatermarkHoldAndLateData() throws Exception { 0, // window start 10))); // window end assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0))); + output.get(0).getPaneInfo(), + equalTo(PaneInfo.createPane(false, false, Timing.ON_TIME, 2, 0))); tester.setAutoAdvanceOutputWatermark(true); @@ -879,7 +882,7 @@ public void testWatermarkHoldAndLateData() throws Exception { 0, // window start 10))); // window end assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 3, 1))); + output.get(0).getPaneInfo(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 3, 1))); assertEquals(new Instant(50), tester.getOutputWatermark()); assertEquals(null, tester.getWatermarkHold()); @@ -1254,7 +1257,7 @@ public void noEmptyPanesFinalIfNonEmpty() throws Exception { contains( // Trigger with 2 elements isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10), - // Trigger for the empty on time pane + // Trigger for the empty on time paneInfo isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10))); } @@ -1284,9 +1287,9 @@ public void noEmptyPanesFinalAlways() throws Exception { contains( // Trigger with 2 elements isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10), - // Trigger for the empty on time pane + // Trigger for the empty on time paneInfo isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10), - // Trigger for the final pane + // Trigger for the final paneInfo isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10))); } @@ -1486,7 +1489,8 @@ public void testMergeBeforeFinalizing() throws Exception { 1, // window start 20)); // window end assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0))); + output.get(0).getPaneInfo(), + equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0))); } /** @@ -1527,7 +1531,8 @@ public void testMergingWithCloseBeforeGC() throws Exception { 1, // window start 20)); // window end assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0))); + output.get(0).getPaneInfo(), + equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0))); } /** Ensure a closed trigger has its state recorded in the merge result window. */ @@ -1614,7 +1619,8 @@ public void testMergingWithReusedWindow() throws Exception { equalTo((BoundedWindow) mergedWindow))); assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0))); + output.get(0).getPaneInfo(), + equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0))); } /** @@ -1661,7 +1667,7 @@ public void testMergingWithClosedRepresentative() throws Exception { 1, // window start 18)); // window end assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0))); + output.get(0).getPaneInfo(), equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0))); } /** @@ -1702,7 +1708,7 @@ public void testMergingWithClosedDoesNotPoison() throws Exception { 2, // window start 12)); // window end assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0))); + output.get(0).getPaneInfo(), equalTo(PaneInfo.createPane(true, true, Timing.EARLY, 0, 0))); assertThat( output.get(1), isSingleWindowedValue( @@ -1711,7 +1717,8 @@ public void testMergingWithClosedDoesNotPoison() throws Exception { 1, // window start 13)); // window end assertThat( - output.get(1).getPane(), equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0))); + output.get(1).getPaneInfo(), + equalTo(PaneInfo.createPane(true, true, Timing.ON_TIME, 0, 0))); } /** @@ -1788,7 +1795,7 @@ public void testIdempotentEmptyPanesDiscarding() throws Exception { injectElement(tester, 2); tester.advanceInputWatermark(new Instant(12)); - // Fire the on-time pane + // Fire the on-time paneInfo when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); @@ -1811,7 +1818,7 @@ public void testIdempotentEmptyPanesDiscarding() throws Exception { // The late pane has the correct indices. assertThat(output.get(1).getValue(), contains(3)); assertThat( - output.get(1).getPane(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1))); + output.get(1).getPaneInfo(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); @@ -1843,14 +1850,15 @@ public void testIdempotentEmptyPanesAccumulating() throws Exception { injectElement(tester, 2); tester.advanceInputWatermark(new Instant(12)); - // Trigger the on-time pane + // Trigger the on-time paneInfo when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); List>> output = tester.extractOutput(); assertThat(output.size(), equalTo(1)); assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)); assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0))); + output.get(0).getPaneInfo(), + equalTo(PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0))); // Fire another timer with no data; the empty pane should not be output even though the // trigger is ready to fire @@ -1868,7 +1876,7 @@ public void testIdempotentEmptyPanesAccumulating() throws Exception { // The late pane has the correct indices. assertThat(output.get(0).getValue(), containsInAnyOrder(1, 2, 3)); assertThat( - output.get(0).getPane(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1))); + output.get(0).getPaneInfo(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 1, 1))); assertTrue(tester.isMarkedFinished(firstWindow)); tester.assertHasOnlyGlobalAndFinishedSetsFor(firstWindow); @@ -1916,10 +1924,10 @@ public void testEmptyOnTimeFromOrFinally() throws Exception { TimestampedValue.of(1, new Instant(7)), TimestampedValue.of(1, new Instant(5))); - // Should fire early pane + // Should fire early paneInfo tester.advanceProcessingTime(new Instant(6)); - // Should fire empty on time pane + // Should fire empty on time paneInfo tester.advanceInputWatermark(new Instant(11)); List> output = tester.extractOutput(); assertEquals(2, output.size()); @@ -1968,13 +1976,13 @@ public void testEmptyOnTimeWithOnTimeBehaviorFireIfNonEmpty() throws Exception { TimestampedValue.of(1, new Instant(7)), TimestampedValue.of(1, new Instant(5))); - // Should fire early pane + // Should fire early paneInfo tester.advanceProcessingTime(new Instant(6)); - // Should not fire empty on time pane + // Should not fire empty on time paneInfo tester.advanceInputWatermark(new Instant(11)); - // Should fire final GC pane + // Should fire final GC paneInfo tester.advanceInputWatermark(new Instant(10 + 100)); List> output = tester.extractOutput(); assertEquals(2, output.size()); @@ -2015,7 +2023,7 @@ public void testEmptyOnTimeWithOnTimeBehaviorBackwardCompatibility() throws Exce tester.injectElements(TimestampedValue.of(1, new Instant(1))); - // Should fire empty on time isFinished pane + // Should fire empty on time isFinished paneInfo tester.advanceInputWatermark(new Instant(11)); List> output = tester.extractOutput(); @@ -2064,13 +2072,13 @@ public void testEmptyOnTimeWithOnTimeBehaviorFireIfNonEmptyAndLateData() throws TimestampedValue.of(1, new Instant(7)), TimestampedValue.of(1, new Instant(5))); - // Should fire early pane + // Should fire early paneInfo tester.advanceProcessingTime(new Instant(6)); - // Should not fire empty on time pane + // Should not fire empty on time paneInfo tester.advanceInputWatermark(new Instant(11)); - // Processing late data, and should fire late pane + // Processing late data, and should fire late paneInfo tester.injectElements(TimestampedValue.of(1, new Instant(9))); tester.advanceProcessingTime(new Instant(6 + 25 + 1)); @@ -2193,8 +2201,8 @@ public void fireNonEmptyOnDrainInGlobalWindow() throws Exception { List>> output = tester.extractOutput(); assertEquals(n / 3, output.size()); for (int i = 0; i < output.size(); i++) { - assertEquals(Timing.EARLY, output.get(i).getPane().getTiming()); - assertEquals(i, output.get(i).getPane().getIndex()); + assertEquals(Timing.EARLY, output.get(i).getPaneInfo().getTiming()); + assertEquals(i, output.get(i).getPaneInfo().getIndex()); assertEquals(3, Iterables.size(output.get(i).getValue())); } @@ -2202,8 +2210,8 @@ public void fireNonEmptyOnDrainInGlobalWindow() throws Exception { output = tester.extractOutput(); assertEquals(1, output.size()); - assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming()); - assertEquals(n / 3, output.get(0).getPane().getIndex()); + assertEquals(Timing.ON_TIME, output.get(0).getPaneInfo().getTiming()); + assertEquals(n / 3, output.get(0).getPaneInfo().getIndex()); assertEquals(n - ((n / 3) * 3), Iterables.size(output.get(0).getValue())); } @@ -2231,8 +2239,8 @@ public void fireEmptyOnDrainInGlobalWindowIfRequested() throws Exception { List>> output = tester.extractOutput(); assertEquals((n + 3) / 4, output.size()); for (int i = 0; i < output.size(); i++) { - assertEquals(Timing.EARLY, output.get(i).getPane().getTiming()); - assertEquals(i, output.get(i).getPane().getIndex()); + assertEquals(Timing.EARLY, output.get(i).getPaneInfo().getTiming()); + assertEquals(i, output.get(i).getPaneInfo().getIndex()); assertEquals(4, Iterables.size(output.get(i).getValue())); } @@ -2240,8 +2248,8 @@ public void fireEmptyOnDrainInGlobalWindowIfRequested() throws Exception { output = tester.extractOutput(); assertEquals(1, output.size()); - assertEquals(Timing.ON_TIME, output.get(0).getPane().getTiming()); - assertEquals((n + 3) / 4, output.get(0).getPane().getIndex()); + assertEquals(Timing.ON_TIME, output.get(0).getPaneInfo().getTiming()); + assertEquals((n + 3) / 4, output.get(0).getPaneInfo().getIndex()); assertEquals(0, Iterables.size(output.get(0).getValue())); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java index 0cdb4042471b..33ae2f68b48f 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java @@ -155,13 +155,13 @@ public void describeTo(Description description) { @Override protected boolean matchesSafely(WindowedValue item) { - return Objects.equals(item.getPane(), paneInfo); + return Objects.equals(item.getPaneInfo(), paneInfo); } @Override protected void describeMismatchSafely( WindowedValue item, Description mismatchDescription) { - mismatchDescription.appendValue(item.getPane()); + mismatchDescription.appendValue(item.getPaneInfo()); } }; } @@ -212,7 +212,7 @@ protected boolean matchesSafely(WindowedValue windowedValue) { return valueMatcher.matches(windowedValue.getValue()) && timestampMatcher.matches(windowedValue.getTimestamp()) && windowsMatcher.matches(windowedValue.getWindows()) - && paneInfoMatcher.matches(windowedValue.getPane()); + && paneInfoMatcher.matches(windowedValue.getPaneInfo()); } } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java index 3fe6cf4a11b8..acbce34e0f95 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java @@ -159,7 +159,7 @@ private void updatePCollectionViewWindowValues( // the value had never been set, so we set it and are done. return; } - PaneInfo newPane = windowValues.iterator().next().getPane(); + PaneInfo newPane = windowValues.iterator().next().getPaneInfo(); Iterable> existingValues; long existingPane; @@ -168,7 +168,7 @@ private void updatePCollectionViewWindowValues( existingPane = Iterables.isEmpty(existingValues) ? -1L - : existingValues.iterator().next().getPane().getIndex(); + : existingValues.iterator().next().getPaneInfo().getIndex(); } while (newPane.getIndex() > existingPane && !contents.compareAndSet(existingValues, windowValues)); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java index 17139a7495c3..27de46bf102b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java @@ -92,7 +92,7 @@ public void processElement(WindowedValue compressedElement) throws Excep Collection windows = assignWindows(windowFn, element); outputBundle.add( WindowedValues.of( - element.getValue(), element.getTimestamp(), windows, element.getPane())); + element.getValue(), element.getTimestamp(), windows, element.getPaneInfo())); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java index ccf6b53fc8fb..7bb5d7dd1ebe 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java @@ -177,24 +177,24 @@ public void multipleWindowsWindowFnSucceeds() throws Exception { valueInIntervalWindow.getValue(), valueInIntervalWindow.getTimestamp(), ImmutableSet.of(wMinus1, wMinusSlide), - valueInIntervalWindow.getPane()), + valueInIntervalWindow.getPaneInfo()), // Value in three windows mapped to three windowed values in the same multiple windows isWindowedValue( valueInGlobalAndTwoIntervalWindows.getValue(), valueInGlobalAndTwoIntervalWindows.getTimestamp(), ImmutableSet.of(w1, w2), - valueInGlobalAndTwoIntervalWindows.getPane()), + valueInGlobalAndTwoIntervalWindows.getPaneInfo()), isWindowedValue( valueInGlobalAndTwoIntervalWindows.getValue(), valueInGlobalAndTwoIntervalWindows.getTimestamp(), ImmutableSet.of(w1, w2), - valueInGlobalAndTwoIntervalWindows.getPane()), + valueInGlobalAndTwoIntervalWindows.getPaneInfo()), isWindowedValue( valueInGlobalAndTwoIntervalWindows.getValue(), valueInGlobalAndTwoIntervalWindows.getTimestamp(), ImmutableSet.of(w1, w2), - valueInGlobalAndTwoIntervalWindows.getPane()))); + valueInGlobalAndTwoIntervalWindows.getPaneInfo()))); } @Test @@ -220,14 +220,14 @@ public void referencesEarlierWindowsSucceeds() throws Exception { new IntervalWindow( valueInGlobalWindow.getTimestamp(), valueInGlobalWindow.getTimestamp().plus(Duration.millis(1L))), - valueInGlobalWindow.getPane()), + valueInGlobalWindow.getPaneInfo()), // Value in interval window mapped to the same window isWindowedValue( valueInIntervalWindow.getValue(), valueInIntervalWindow.getTimestamp(), valueInIntervalWindow.getWindows(), - valueInIntervalWindow.getPane()), + valueInIntervalWindow.getPaneInfo()), // Value in global window and two interval windows exploded and mapped in both ways isSingleWindowedValue( @@ -236,17 +236,17 @@ public void referencesEarlierWindowsSucceeds() throws Exception { new IntervalWindow( valueInGlobalAndTwoIntervalWindows.getTimestamp(), valueInGlobalAndTwoIntervalWindows.getTimestamp().plus(Duration.millis(1L))), - valueInGlobalAndTwoIntervalWindows.getPane()), + valueInGlobalAndTwoIntervalWindows.getPaneInfo()), isSingleWindowedValue( valueInGlobalAndTwoIntervalWindows.getValue(), valueInGlobalAndTwoIntervalWindows.getTimestamp(), intervalWindow1, - valueInGlobalAndTwoIntervalWindows.getPane()), + valueInGlobalAndTwoIntervalWindows.getPaneInfo()), isSingleWindowedValue( valueInGlobalAndTwoIntervalWindows.getValue(), valueInGlobalAndTwoIntervalWindows.getTimestamp(), intervalWindow2, - valueInGlobalAndTwoIntervalWindows.getPane()))); + valueInGlobalAndTwoIntervalWindows.getPaneInfo()))); } private CommittedBundle createInputBundle() { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java index 8a7ec8b59f60..19ccdb76af58 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java @@ -1405,7 +1405,7 @@ public void collect(WindowedValue> element) { OutputT originalValue = element.getValue().getValue(); WindowedValue output = WindowedValues.of( - originalValue, element.getTimestamp(), element.getWindows(), element.getPane()); + originalValue, element.getTimestamp(), element.getWindows(), element.getPaneInfo()); ctx.collect(output); } @@ -1415,7 +1415,7 @@ public void collectWithTimestamp( OutputT originalValue = element.getValue().getValue(); WindowedValue output = WindowedValues.of( - originalValue, element.getTimestamp(), element.getWindows(), element.getPane()); + originalValue, element.getTimestamp(), element.getWindows(), element.getPaneInfo()); ctx.collectWithTimestamp(output, timestamp); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java index 07a47707827d..b82eab17be82 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java @@ -44,7 +44,7 @@ public void flatMap(WindowedValue input, Collector> collecto Collection windows = windowFn.assignWindows(new FlinkAssignContext<>(windowFn, input)); for (W window : windows) { collector.collect( - WindowedValues.of(input.getValue(), input.getTimestamp(), window, input.getPane())); + WindowedValues.of(input.getValue(), input.getTimestamp(), window, input.getPaneInfo())); } } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java index 976b3dfe0868..efc5ea40b441 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java @@ -163,7 +163,7 @@ private void mergeWindow(List>> elements) { elements.set( j, WindowedValues.of( - value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + value.getValue(), value.getTimestamp(), currentWindow, value.getPaneInfo())); } currentStart = i; currentWindow = nextWindow; @@ -176,7 +176,7 @@ private void mergeWindow(List>> elements) { elements.set( j, WindowedValues.of( - value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); + value.getValue(), value.getTimestamp(), currentWindow, value.getPaneInfo())); } } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index 8cb82c2ca42d..537a2d855921 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -1391,7 +1391,7 @@ public Collection getWindows() { } @Override - public PaneInfo getPane() { + public PaneInfo getPaneInfo() { return PaneInfo.NO_FIRING; } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RedistributeByKeyOverrideFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RedistributeByKeyOverrideFactory.java index 509b35b15390..cea195ed2013 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RedistributeByKeyOverrideFactory.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RedistributeByKeyOverrideFactory.java @@ -21,11 +21,7 @@ import org.apache.beam.runners.dataflow.internal.DataflowGroupByKey; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; -import org.apache.beam.sdk.runners.PTransformOverrideFactory.PTransformReplacement; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.Element; -import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Redistribute.RedistributeByKey; @@ -143,7 +139,7 @@ public void processElement( KV.of(kv.getKey(), kv.getValue().getValue()), kv.getValue().getTimestamp(), Collections.singleton(kv.getValue().getWindow()), - kv.getValue().getPane()); + kv.getValue().getPaneInfo()); } })); } 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 4a752047e256..d45e1f3a4e46 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 @@ -112,7 +112,7 @@ public BoundedWindow window() { }); WindowedValue res = - WindowedValues.of(elem.getValue(), elem.getTimestamp(), windows, elem.getPane()); + WindowedValues.of(elem.getValue(), elem.getTimestamp(), windows, elem.getPaneInfo()); receiver.process(res); } 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 a7dca89d31e0..05f537948072 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 @@ -244,7 +244,7 @@ public Object createGroupingKey(WindowedValue key) throws Exception { // Ignore timestamp for grouping purposes. // The PGBK output will inherit the timestamp of one of its inputs. return WindowedValues.of( - coder.structuralValue(key.getValue()), ignored, key.getWindows(), key.getPane()); + coder.structuralValue(key.getValue()), ignored, key.getWindows(), key.getPaneInfo()); } } @@ -305,7 +305,7 @@ public void processElement(Object elem) throws Exception { WindowedValue> input = (WindowedValue>) elem; for (BoundedWindow w : input.getWindows()) { WindowedValue> windowsExpandedInput = - WindowedValues.of(input.getValue(), input.getTimestamp(), w, input.getPane()); + WindowedValues.of(input.getValue(), input.getTimestamp(), w, input.getPaneInfo()); groupingTable.put(windowsExpandedInput, receiver); } } @@ -362,7 +362,7 @@ public void processElement(Object elem) throws Exception { WindowedValue> input = (WindowedValue>) elem; for (BoundedWindow w : input.getWindows()) { WindowedValue> windowsExpandedInput = - WindowedValues.of(input.getValue(), input.getTimestamp(), w, input.getPane()); + WindowedValues.of(input.getValue(), input.getTimestamp(), w, input.getPaneInfo()); if (!sideInputFetcher.storeIfBlocked(windowsExpandedInput)) { groupingTable.put(windowsExpandedInput, receiver); 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 39a3bf24fc24..31d846d1102d 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 @@ -78,10 +78,10 @@ public void processElement(Object untypedElem) throws Exception { typedElem.getValue().getValue(), typedElem.getTimestamp(), typedElem.getWindows(), - typedElem.getPane())), + typedElem.getPaneInfo())), typedElem.getTimestamp(), typedElem.getWindows(), - typedElem.getPane())); + typedElem.getPaneInfo())); } @Override 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 a56afd636137..e031d1bb50eb 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 @@ -116,7 +116,7 @@ protected WindowedValue decodeMessage(Windmill.Message message) throws IOExce InputStream metadata = message.getMetadata().newInput(); Collection windows = WindmillSink.decodeMetadataWindows(windowsCoder, message.getMetadata()); - PaneInfo pane = WindmillSink.decodeMetadataPane(message.getMetadata()); + PaneInfo paneInfo = WindmillSink.decodeMetadataPane(message.getMetadata()); if (valueCoder instanceof KvCoder) { KvCoder kvCoder = (KvCoder) valueCoder; InputStream key = context.getSerializedKey().newInput(); @@ -125,10 +125,10 @@ protected WindowedValue decodeMessage(Windmill.Message message) throws IOExce @SuppressWarnings("unchecked") T result = (T) KV.of(decode(kvCoder.getKeyCoder(), key), decode(kvCoder.getValueCoder(), data)); - return WindowedValues.of(result, timestampMillis, windows, pane); + return WindowedValues.of(result, timestampMillis, windows, paneInfo); } else { notifyElementRead(data.available() + metadata.available()); - return WindowedValues.of(decode(valueCoder, data), timestampMillis, windows, pane); + return WindowedValues.of(decode(valueCoder, data), timestampMillis, windows, paneInfo); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java index 29457678576b..cee4894e3d68 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java @@ -107,11 +107,11 @@ public Iterable> elementsIterable() { WindmillTimeUtils.windmillToHarnessTimestamp(message.getTimestamp()); Collection windows = WindmillSink.decodeMetadataWindows(windowsCoder, message.getMetadata()); - PaneInfo pane = WindmillSink.decodeMetadataPane(message.getMetadata()); + PaneInfo paneInfo = WindmillSink.decodeMetadataPane(message.getMetadata()); InputStream inputStream = message.getData().newInput(); ElemT value = valueCoder.decode(inputStream, Coder.Context.OUTER); - return WindowedValues.of(value, timestamp, windows, pane); + return WindowedValues.of(value, timestamp, windows, paneInfo); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java index a75374ba64af..b156ff45caf6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java @@ -72,10 +72,10 @@ class WindmillSink extends Sink> { public static ByteString encodeMetadata( Coder> windowsCoder, Collection windows, - PaneInfo pane) + PaneInfo paneInfo) throws IOException { ByteStringOutputStream stream = new ByteStringOutputStream(); - PaneInfoCoder.INSTANCE.encode(pane, stream); + PaneInfoCoder.INSTANCE.encode(paneInfo, stream); windowsCoder.encode(windows, stream, Coder.Context.OUTER); return stream.toByteString(); } @@ -155,7 +155,7 @@ private ByteString encode(Coder coder, EncodeT object) throws public long add(WindowedValue data) throws IOException { ByteString key, value; ByteString id = ByteString.EMPTY; - ByteString metadata = encodeMetadata(windowsCoder, data.getWindows(), data.getPane()); + ByteString metadata = encodeMetadata(windowsCoder, data.getWindows(), data.getPaneInfo()); if (valueCoder instanceof KvCoder) { KvCoder kvCoder = (KvCoder) valueCoder; KV kv = (KV) data.getValue(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java index ca0e2279fb03..42174629b3b8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java @@ -45,7 +45,7 @@ public ValueInEmptyWindows(T value) { } @Override - public PaneInfo getPane() { + public PaneInfo getPaneInfo() { return PaneInfo.NO_FIRING; } @@ -93,7 +93,7 @@ public int hashCode() { public String toString() { return MoreObjects.toStringHelper(getClass()) .add("value", getValue()) - .add("pane", getPane()) + .add("paneInfo", getPaneInfo()) .toString(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java index eb8f91af18b9..90071d1597ee 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java @@ -56,7 +56,7 @@ private void verifyReifiedIsInTheSameWindows(WindowedValue> elem receiver.reified.getValue().getValue().getValue(), equalTo(elem.getValue().getValue())); assertThat(receiver.reified.getValue().getValue().getTimestamp(), equalTo(elem.getTimestamp())); assertThat(receiver.reified.getValue().getValue().getWindows(), equalTo(elem.getWindows())); - assertThat(receiver.reified.getValue().getValue().getPane(), equalTo(elem.getPane())); + assertThat(receiver.reified.getValue().getValue().getPaneInfo(), equalTo(elem.getPaneInfo())); } @Test 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 11dd51f7fb3a..033d7718d1c9 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 @@ -711,11 +711,12 @@ private WorkItemCommitRequest.Builder parseCommitRequest(String output) throws E /** Sets the metadata of all the contained messages in this WorkItemCommitRequest. */ private WorkItemCommitRequest.Builder setMessagesMetadata( - PaneInfo pane, byte[] windowBytes, WorkItemCommitRequest.Builder builder) throws Exception { + PaneInfo paneInfo, byte[] windowBytes, WorkItemCommitRequest.Builder builder) + throws Exception { if (windowBytes != null) { KeyedMessageBundle.Builder bundles = builder.getOutputMessagesBuilder(0).getBundlesBuilder(0); for (int i = 0; i < bundles.getMessagesCount(); i++) { - bundles.getMessagesBuilder(i).setMetadata(addPaneTag(pane, windowBytes)); + bundles.getMessagesBuilder(i).setMetadata(addPaneTag(paneInfo, windowBytes)); } } return builder; @@ -832,9 +833,9 @@ private StreamingComputationConfig makeDefaultStreamingComputationConfig( return config; } - private ByteString addPaneTag(PaneInfo pane, byte[] windowBytes) throws IOException { + private ByteString addPaneTag(PaneInfo paneInfo, byte[] windowBytes) throws IOException { ByteStringOutputStream output = new ByteStringOutputStream(); - PaneInfo.PaneInfoCoder.INSTANCE.encode(pane, output, Context.OUTER); + PaneInfo.PaneInfoCoder.INSTANCE.encode(paneInfo, output, Context.OUTER); output.write(windowBytes); return output.toByteString(); } @@ -1747,7 +1748,7 @@ public void testMergeWindows() throws Exception { String window = "/gAAAAAAAA-joBw/"; String timerTagPrefix = "/s" + window + "+0"; ByteString bufferTag = ByteString.copyFromUtf8(window + "+ubuf"); - ByteString paneInfoTag = ByteString.copyFromUtf8(window + "+upane"); + ByteString paneInfoTag = ByteString.copyFromUtf8(window + "+upaneInfo"); String watermarkDataHoldTag = window + "+uhold"; String watermarkExtraHoldTag = window + "+uextra"; String stateFamily = "MergeWindows"; @@ -2036,7 +2037,7 @@ public void testMergeWindowsCaching() throws Exception { String window = "/gAAAAAAAA-joBw/"; String timerTagPrefix = "/s" + window + "+0"; ByteString bufferTag = ByteString.copyFromUtf8(window + "+ubuf"); - ByteString paneInfoTag = ByteString.copyFromUtf8(window + "+upane"); + ByteString paneInfoTag = ByteString.copyFromUtf8(window + "+upaneInfo"); String watermarkDataHoldTag = window + "+uhold"; String watermarkExtraHoldTag = window + "+uextra"; String stateFamily = "MergeWindows"; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java index 546b33e42f1f..ffe71176367a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java @@ -107,10 +107,10 @@ private void addElement( long timestamp, String value, IntervalWindow window, - PaneInfo pane) + PaneInfo paneInfo) throws IOException { ByteString encodedMetadata = - WindmillSink.encodeMetadata(WINDOWS_CODER, Collections.singletonList(window), pane); + WindmillSink.encodeMetadata(WINDOWS_CODER, Collections.singletonList(window), paneInfo); chunk .addMessagesBuilder() .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(new Instant(timestamp))) diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java index 14d00934a242..34d952b15608 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java @@ -132,7 +132,7 @@ public void onCheckpoint(ProcessBundleResponse response) { stateValue.getValue(), stateValue.getTimestamp(), ImmutableList.of(window), - stateValue.getPane())); + stateValue.getPaneInfo())); } } catch (Exception e) { throw new RuntimeException("Failed to set timer/state for the residual", e); diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java index 04d843414525..3afa19ccff40 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java @@ -78,7 +78,7 @@ private AssignWindowP( inputValue.getValue(), inputValue.getTimestamp(), windows, - inputValue.getPane()); + inputValue.getPaneInfo()); traverser.accept(Utils.encode(outputValue, outputCoder)); return traverser; }); diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java index 2ea7a998a896..2120f4c91006 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java @@ -75,7 +75,7 @@ protected boolean tryProcess(int ordinal, @Nonnull Object item) { values.merge( window, new TimestampAndValues( - windowedValue.getPane(), windowedValue.getTimestamp(), windowedValue.getValue()), + windowedValue.getPaneInfo(), windowedValue.getTimestamp(), windowedValue.getValue()), (o, n) -> o.merge(timestampCombiner, n)); } @@ -95,7 +95,7 @@ public boolean complete() { e.getValue().values, e.getValue().timestamp, Collections.singleton(e.getKey()), - e.getValue().pane); + e.getValue().paneInfo); return Utils.encode(outputValue, outputCoder); })); } @@ -113,10 +113,10 @@ public static SupplierEx supplier( private static class TimestampAndValues { private final List values = new ArrayList<>(); private Instant timestamp; - private PaneInfo pane; + private PaneInfo paneInfo; - TimestampAndValues(PaneInfo pane, Instant timestamp, Object value) { - this.pane = pane; + TimestampAndValues(PaneInfo paneInfo, Instant timestamp, Object value) { + this.paneInfo = paneInfo; this.timestamp = timestamp; this.values.add(value); } @@ -126,7 +126,7 @@ public Iterable getValues() { } TimestampAndValues merge(TimestampCombiner timestampCombiner, TimestampAndValues other) { - pane = other.pane; + paneInfo = other.paneInfo; timestamp = timestampCombiner.combine(timestamp, other.timestamp); values.addAll(other.values); return this; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java index 333c2c74a127..747df5a67bd3 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java @@ -133,7 +133,7 @@ private WindowGroupP( value, windowedValue.getTimestamp(), windowedValue.getWindows(), - windowedValue.getPane()); + windowedValue.getPaneInfo()); keyManagers .computeIfAbsent(keyBytes, x -> new KeyManager(key)) .processElement(updatedWindowedValue); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java index 9c508f14e00e..2f27e31d05dd 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java @@ -486,7 +486,7 @@ static CompletionStage> createOutputFuture( valueMapper.apply(res), windowedValue.getTimestamp(), windowedValue.getWindows(), - windowedValue.getPane())); + windowedValue.getPaneInfo())); } /** diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java index ed82589d2125..cd83f490bc25 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java @@ -47,7 +47,7 @@ public void processElement(WindowedValue inputElement, OpEmitter emitter) inputElement.getValue(), inputElement.getTimestamp(), window, - inputElement.getPane())) + inputElement.getPaneInfo())) .forEach(outputElement -> emitter.emitElement(outputElement)); } } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java index 7ab70f3652c8..46cda3334822 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java @@ -280,7 +280,7 @@ private static List timestampedValue(Column value, Expression ts) { } private static List windowDetails(Expression windows) { - return seqOf(lit("windows"), windows, lit("pane"), PANE_NO_FIRING).toList(); + return seqOf(lit("windows"), windows, lit("paneInfo"), PANE_NO_FIRING).toList(); } private static Expression lit(T t) { diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java index adac966f457d..25e08cd9de99 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java @@ -94,7 +94,7 @@ public T element() { return window; } }); - return WindowedValues.of(element, timestamp, windows, value.getPane()); + return WindowedValues.of(element, timestamp, windows, value.getPaneInfo()); }; } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java index 55f22f92a132..7ea277740b62 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java @@ -184,12 +184,13 @@ public static Encoder encoderFor(Coder coder) { public static Encoder> windowedValueEncoder( Encoder value, Encoder window) { Encoder timestamp = encoderOf(Instant.class); - Encoder pane = encoderOf(PaneInfo.class); + Encoder paneInfo = encoderOf(PaneInfo.class); Encoder> windows = collectionEncoder(window); Expression serializer = - serializeWindowedValue(rootRef(WINDOWED_VALUE, true), value, timestamp, windows, pane); + serializeWindowedValue(rootRef(WINDOWED_VALUE, true), value, timestamp, windows, paneInfo); Expression deserializer = - deserializeWindowedValue(rootCol(serializer.dataType()), value, timestamp, windows, pane); + deserializeWindowedValue( + rootCol(serializer.dataType()), value, timestamp, windows, paneInfo); return EncoderFactory.create(serializer, deserializer, WindowedValue.class); } @@ -333,7 +334,7 @@ private static Expression serializeWindowedValue( tuple("value", serializeField(in, valueEnc, "getValue")), tuple("timestamp", serializeField(in, timestampEnc, "getTimestamp")), tuple("windows", serializeField(in, windowsEnc, "getWindows")), - tuple("pane", serializeField(in, paneEnc, "getPane"))); + tuple("paneInfo", serializeField(in, paneEnc, "getPaneInfo"))); } private static Expression serializerObject(Expression in, Tuple2... fields) { @@ -349,13 +350,13 @@ private static Expression deserializeWindowedValue( Expression value = deserializeField(in, valueEnc, 0, "value"); Expression windows = deserializeField(in, windowsEnc, 2, "windows"); Expression timestamp = deserializeField(in, timestampEnc, 1, "timestamp"); - Expression pane = deserializeField(in, paneEnc, 3, "pane"); + Expression paneInfo = deserializeField(in, paneEnc, 3, "paneInfo"); // set timestamp to end of window (maxTimestamp) if null timestamp = ifNotNull(timestamp, invoke(Utils.class, "maxTimestamp", timestamp.dataType(), windows)); - Expression[] fields = new Expression[] {value, timestamp, windows, pane}; + Expression[] fields = new Expression[] {value, timestamp, windows, paneInfo}; - return nullSafe(pane, invoke(WindowedValues.class, "of", WINDOWED_VALUE, fields)); + return nullSafe(paneInfo, invoke(WindowedValues.class, "of", WINDOWED_VALUE, fields)); } private static Expression serializeMutablePair( @@ -544,8 +545,8 @@ public static PaneInfo paneInfoFromBytes(byte[] bytes) { return CoderHelpers.fromByteArray(bytes, PaneInfoCoder.of()); } - public static byte[] paneInfoToBytes(PaneInfo pane) { - return CoderHelpers.toByteArray(pane, PaneInfoCoder.of()); + public static byte[] paneInfoToBytes(PaneInfo paneInfo) { + return CoderHelpers.toByteArray(paneInfo, PaneInfoCoder.of()); } /** The end of the only window (max timestamp). */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java index 3944cde5bee3..3b46bee0e8cf 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java @@ -132,7 +132,8 @@ static JavaPairRDD final W window = (W) Iterables.getOnlyElement(item.getWindows()); final byte[] windowBytes = CoderHelpers.toByteArray(window, windowCoder); WindowedValue> valueOut = - WindowedValues.of(item.getValue(), item.getTimestamp(), window, item.getPane()); + WindowedValues.of( + item.getValue(), item.getTimestamp(), window, item.getPaneInfo()); final ByteArray windowedKey = new ByteArray(Bytes.concat(keyBytes, windowBytes)); return new Tuple2<>(windowedKey, mappingFn.apply(valueOut)); }); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java index 37f9d1b4e86b..5893f7f99e04 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java @@ -34,6 +34,9 @@ public KV> call(WindowedValue> elem) throws Excepti return KV.of( elem.getValue().getKey(), WindowedValues.of( - elem.getValue().getValue(), elem.getTimestamp(), elem.getWindows(), elem.getPane())); + elem.getValue().getValue(), + elem.getTimestamp(), + elem.getWindows(), + elem.getPaneInfo())); } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 12448b19e1bb..1345e99bedca 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -232,7 +232,7 @@ public void evaluate( in.getValue().getValue(), sparkCombineFn.ctxtForValue(in))), in.getTimestamp(), in.getWindows(), - in.getPane())); + in.getPaneInfo())); context.putDataset(transform, new BoundedDataset<>(outRDD)); } @@ -699,7 +699,10 @@ protected WindowedValue> computeNext() { WindowedValue wv = CoderHelpers.fromByteArray(read._2(), wvCoder); consumed(); return WindowedValues.of( - KV.of(key, wv.getValue()), wv.getTimestamp(), wv.getWindows(), wv.getPane()); + KV.of(key, wv.getValue()), + wv.getTimestamp(), + wv.getWindows(), + wv.getPaneInfo()); } } return endOfData(); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java index 8d66125efbc6..93422c6f6da7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java @@ -110,7 +110,7 @@ public WindowedValue> call(WindowedValue>> .apply(windowedKv.getValue().getValue(), fn.ctxtForValue(windowedKv))), windowedKv.getTimestamp(), windowedKv.getWindows(), - windowedKv.getPane()); + windowedKv.getPaneInfo()); } } diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java index ad235a72d2e5..9c19868a7c5b 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java @@ -71,7 +71,7 @@ public void compute(Iterator> input, RecordCollector> map(Tuple input) { KV.of(key, value.getValue()), value.getTimestamp(), value.getWindows(), - value.getPane()); + value.getPaneInfo()); return element; } diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java index da1080b697ac..dd1e0eb7d868 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java @@ -69,10 +69,10 @@ public Tuple map(WindowedValue> input) { input.getValue().getValue(), input.getTimestamp(), input.getWindows(), - input.getPane())), + input.getPaneInfo())), input.getTimestamp(), input.getWindows(), - input.getPane()); + input.getPaneInfo()); try { element = new Tuple<>( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java index c647b0e70bbf..d51a3c188afa 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java @@ -941,7 +941,7 @@ public interface FileNaming extends Serializable { */ String getFilename( BoundedWindow window, - PaneInfo pane, + PaneInfo paneInfo, int numShards, int shardIndex, Compression compression); @@ -958,9 +958,9 @@ public static FileNaming defaultNaming(final String prefix, final String suffix) */ public static FileNaming defaultNaming( final ValueProvider prefix, final ValueProvider suffix) { - return (window, pane, numShards, shardIndex, compression) -> { + return (window, paneInfo, numShards, shardIndex, compression) -> { checkArgument(window != null, "window can not be null"); - checkArgument(pane != null, "pane can not be null"); + checkArgument(paneInfo != null, "pane can not be null"); checkArgument(compression != null, "compression can not be null"); StringBuilder res = new StringBuilder(prefix.get()); if (window != GlobalWindow.INSTANCE) { @@ -976,12 +976,12 @@ public static FileNaming defaultNaming( IntervalWindow iw = (IntervalWindow) window; res.append(iw.start().toString()).append("-").append(iw.end().toString()); } - boolean isOnlyFiring = pane.isFirst() && pane.isLast(); + boolean isOnlyFiring = paneInfo.isFirst() && paneInfo.isLast(); if (!isOnlyFiring) { if (res.length() > 0) { res.append("-"); } - res.append(pane.getIndex()); + res.append(paneInfo.getIndex()); } if (res.length() > 0) { res.append("-"); @@ -999,10 +999,10 @@ public static FileNaming defaultNaming( public static FileNaming relativeFileNaming( final ValueProvider baseDirectory, final FileNaming innerNaming) { - return (window, pane, numShards, shardIndex, compression) -> + return (window, paneInfo, numShards, shardIndex, compression) -> FileSystems.matchNewResource(baseDirectory.get(), true /* isDirectory */) .resolve( - innerNaming.getFilename(window, pane, numShards, shardIndex, compression), + innerNaming.getFilename(window, paneInfo, numShards, shardIndex, compression), RESOLVE_FILE) .toString(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java index 527d17fbc85f..7210990fae1b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java @@ -83,12 +83,15 @@ private ExtractOnlyPane(PAssert.PAssertionSite site) { public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); for (ValueInSingleWindow value : input) { - if (!value.getPane().isFirst() || !value.getPane().isLast()) { + if (!value.getPaneInfo().isFirst() || !value.getPaneInfo().isLast()) { throw site.wrap( String.format( "Expected elements to be produced by a trigger that fires at most once, but got " + "a value %s in a pane that is %s.", - value, value.getPane().isFirst() ? "not the last pane" : "not the first pane")); + value, + value.getPaneInfo().isFirst() + ? "not the last paneInfo" + : "not the first paneInfo")); } outputs.add(value.getValue()); } @@ -102,7 +105,7 @@ private static class ExtractOnTimePane public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); for (ValueInSingleWindow value : input) { - if (value.getPane().getTiming().equals(Timing.ON_TIME)) { + if (value.getPaneInfo().getTiming().equals(Timing.ON_TIME)) { outputs.add(value.getValue()); } } @@ -116,7 +119,7 @@ private static class ExtractFinalPane public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); for (ValueInSingleWindow value : input) { - if (value.getPane().isLast()) { + if (value.getPaneInfo().isLast()) { outputs.add(value.getValue()); } } @@ -142,7 +145,7 @@ private static class ExtractNonLatePanes public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); for (ValueInSingleWindow value : input) { - if (value.getPane().getTiming() != PaneInfo.Timing.LATE) { + if (value.getPaneInfo().getTiming() != PaneInfo.Timing.LATE) { outputs.add(value.getValue()); } } @@ -156,7 +159,7 @@ private static class ExtractEarlyPanes public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); for (ValueInSingleWindow value : input) { - if (value.getPane().getTiming() == PaneInfo.Timing.EARLY) { + if (value.getPaneInfo().getTiming() == PaneInfo.Timing.EARLY) { outputs.add(value.getValue()); } } @@ -170,7 +173,7 @@ private static class ExtractLatePanes public Iterable apply(Iterable> input) { List outputs = new ArrayList<>(); for (ValueInSingleWindow value : input) { - if (value.getPane().getTiming() == PaneInfo.Timing.LATE) { + if (value.getPaneInfo().getTiming() == PaneInfo.Timing.LATE) { outputs.add(value.getValue()); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java index 2dce2bed7d38..88e3780384ff 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java @@ -66,7 +66,6 @@ import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; import org.apache.beam.sdk.values.TypeDescriptor; 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.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -916,7 +915,10 @@ private static class ConvertWindowedValues extends DoFn, T> @ProcessElement public void processElement(@Element WindowedValue element, OutputReceiver r) { r.outputWindowedValue( - element.getValue(), element.getTimestamp(), element.getWindows(), element.getPane()); + element.getValue(), + element.getTimestamp(), + element.getWindows(), + element.getPaneInfo()); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java index 1263d65d6760..641412772bf3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java @@ -127,8 +127,8 @@ public KV apply(T element) { new DoFn, T>() { @ProcessElement public void processElement( - @Element KV element, PaneInfo pane, OutputReceiver receiver) { - if (pane.isFirst()) { + @Element KV element, PaneInfo paneInfo, OutputReceiver receiver) { + if (paneInfo.isFirst()) { // Only output the key if it's the first time it's been seen. receiver.output(element.getKey()); } @@ -191,9 +191,9 @@ public T apply(T left, T right) { new DoFn, T>() { @ProcessElement public void processElement( - @Element KV element, PaneInfo pane, OutputReceiver receiver) { + @Element KV element, PaneInfo paneInfo, OutputReceiver receiver) { // Only output the value if it's the first time it's been seen. - if (pane.isFirst()) { + if (paneInfo.isFirst()) { receiver.output(element.getValue()); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index c0915c2dcd75..fb1947ad5ba3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -564,7 +564,7 @@ public Instant timestamp() { @Override public PaneInfo pane() { - return element.getPane(); + return element.getPaneInfo(); } @Override @@ -599,7 +599,9 @@ public void output(TupleTag tag, T output) { @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { getMutableOutput(tag) - .add(ValueInSingleWindow.of(output, timestamp, element.getWindow(), element.getPane())); + .add( + ValueInSingleWindow.of( + output, timestamp, element.getWindow(), element.getPaneInfo())); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Redistribute.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Redistribute.java index 47fb6c138a1d..ea55cbd88b36 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Redistribute.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Redistribute.java @@ -183,7 +183,7 @@ public void processElement( KV.of(kv.getKey(), kv.getValue().getValue()), kv.getValue().getTimestamp(), Collections.singleton(kv.getValue().getWindow()), - kv.getValue().getPane()); + kv.getValue().getPaneInfo()); } })); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java index 98b2b940f314..92f1b73900b2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java @@ -97,10 +97,10 @@ public void processElement( @Element T element, @DoFn.Timestamp Instant timestamp, BoundedWindow window, - PaneInfo pane, + PaneInfo paneInfo, OutputReceiver> r) { r.outputWithTimestamp( - ValueInSingleWindow.of(element, timestamp, window, pane), timestamp); + ValueInSingleWindow.of(element, timestamp, window, paneInfo), timestamp); } })) .setCoder( @@ -143,12 +143,13 @@ public void processElement( @Element KV element, @DoFn.Timestamp Instant timestamp, BoundedWindow window, - PaneInfo pane, + PaneInfo paneInfo, OutputReceiver>> r) { r.output( KV.of( element.getKey(), - ValueInSingleWindow.of(element.getValue(), timestamp, window, pane))); + ValueInSingleWindow.of( + element.getValue(), timestamp, window, paneInfo))); } })) .setCoder( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java index e06440ae6a9d..2a301d0480c0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java @@ -188,7 +188,7 @@ public void processElement( KV.of(kv.getKey(), kv.getValue().getValue()), kv.getValue().getTimestamp(), Collections.singleton(kv.getValue().getWindow()), - kv.getValue().getPane()); + kv.getValue().getPaneInfo()); } })); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowStringInterpolator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowStringInterpolator.java index 46873d77642f..513772e8ec35 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowStringInterpolator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowStringInterpolator.java @@ -129,7 +129,7 @@ public String interpolate(ValueInSingleWindow element) { val = element.getWindow().toString(); break; case PANE_INDEX: - val = element.getPane().getIndex(); + val = element.getPaneInfo().getIndex(); break; case YYYY: val = timestamp.getChronology().year().get(timestamp.getMillis()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Timer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Timer.java index 08707d6e7331..d443f008b7d2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Timer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Timer.java @@ -65,9 +65,9 @@ public static Timer of( Collection windows, Instant fireTimestamp, Instant holdTimestamp, - PaneInfo pane) { + PaneInfo paneInfo) { return new AutoValue_Timer( - userKey, dynamicTimerTag, windows, false, fireTimestamp, holdTimestamp, pane); + userKey, dynamicTimerTag, windows, false, fireTimestamp, holdTimestamp, paneInfo); } /** @@ -114,7 +114,7 @@ public static Timer cleared( * Returns the {@link PaneInfo} that is related to the timer. This field is nullable only when the * timer is being cleared. */ - public abstract @Nullable PaneInfo getPane(); + public abstract @Nullable PaneInfo getPaneInfo(); @Override public final boolean equals(@Nullable Object other) { @@ -128,7 +128,7 @@ public final boolean equals(@Nullable Object other) { && (this.getClearBit() == that.getClearBit()) && Objects.equals(this.getFireTimestamp(), that.getFireTimestamp()) && Objects.equals(this.getHoldTimestamp(), that.getHoldTimestamp()) - && Objects.equals(this.getPane(), that.getPane()); + && Objects.equals(this.getPaneInfo(), that.getPaneInfo()); } @Override @@ -144,7 +144,7 @@ public final int hashCode() { getFireTimestamp().getMillis(), getHoldTimestamp().getMillis(), getWindows(), - getPane()); + getPaneInfo()); } /** @@ -185,7 +185,7 @@ public void encode(Timer timer, OutputStream outStream) throws IOException { if (!timer.getClearBit()) { InstantCoder.of().encode(timer.getFireTimestamp(), outStream); InstantCoder.of().encode(timer.getHoldTimestamp(), outStream); - PaneInfoCoder.INSTANCE.encode(timer.getPane(), outStream); + PaneInfoCoder.INSTANCE.encode(timer.getPaneInfo(), outStream); } } @@ -200,8 +200,8 @@ public Timer decode(InputStream inStream) throws IOException { } Instant fireTimestamp = InstantCoder.of().decode(inStream); Instant holdTimestamp = InstantCoder.of().decode(inStream); - PaneInfo pane = PaneInfoCoder.INSTANCE.decode(inStream); - return Timer.of(userKey, dynamicTimerTag, windows, fireTimestamp, holdTimestamp, pane); + PaneInfo paneInfo = PaneInfoCoder.INSTANCE.decode(inStream); + return Timer.of(userKey, dynamicTimerTag, windows, fireTimestamp, holdTimestamp, paneInfo); } @Override @@ -242,7 +242,7 @@ public void registerByteSizeObserver(Timer value, ElementByteSizeObserver obs if (!value.getClearBit()) { InstantCoder.of().registerByteSizeObserver(value.getFireTimestamp(), observer); InstantCoder.of().registerByteSizeObserver(value.getHoldTimestamp(), observer); - PaneInfoCoder.INSTANCE.registerByteSizeObserver(value.getPane(), observer); + PaneInfoCoder.INSTANCE.registerByteSizeObserver(value.getPaneInfo(), observer); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java index 558552246692..f037f1ac9443 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java @@ -48,7 +48,7 @@ public abstract class FailsafeValueInSingleWindow { public abstract BoundedWindow getWindow(); /** Returns the pane of this {@code FailsafeValueInSingleWindow} in its window. */ - public abstract PaneInfo getPane(); + public abstract PaneInfo getPaneInfo(); /** Returns the failsafe value of this {@code FailsafeValueInSingleWindow}. */ public abstract ErrorT getFailsafeValue(); @@ -97,7 +97,7 @@ public void encode( throws IOException { InstantCoder.of().encode(windowedElem.getTimestamp(), outStream); windowCoder.encode(windowedElem.getWindow(), outStream); - PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream); + PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPaneInfo(), outStream); valueCoder.encode(windowedElem.getValue(), outStream); failsafeValueCoder.encode(windowedElem.getFailsafeValue(), outStream); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java index d91bb8057a05..74717fc606b2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java @@ -58,7 +58,7 @@ public T getValue() { public abstract BoundedWindow getWindow(); /** Returns the pane of this {@code ValueInSingleWindow} in its window. */ - public abstract PaneInfo getPane(); + public abstract PaneInfo getPaneInfo(); public static ValueInSingleWindow of( T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo) { @@ -95,7 +95,7 @@ public void encode(ValueInSingleWindow windowedElem, OutputStream outStream, throws IOException { InstantCoder.of().encode(windowedElem.getTimestamp(), outStream); windowCoder.encode(windowedElem.getWindow(), outStream); - PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream); + PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPaneInfo(), outStream); valueCoder.encode(windowedElem.getValue(), outStream, context); } @@ -108,9 +108,9 @@ public ValueInSingleWindow decode(InputStream inStream) throws IOException { public ValueInSingleWindow decode(InputStream inStream, Context context) throws IOException { Instant timestamp = InstantCoder.of().decode(inStream); BoundedWindow window = windowCoder.decode(inStream); - PaneInfo pane = PaneInfo.PaneInfoCoder.INSTANCE.decode(inStream); + PaneInfo paneInfo = PaneInfo.PaneInfoCoder.INSTANCE.decode(inStream); T value = valueCoder.decode(inStream, context); - return new AutoValue_ValueInSingleWindow<>(value, timestamp, window, pane); + return new AutoValue_ValueInSingleWindow<>(value, timestamp, window, paneInfo); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java index 2a5236f0147f..1dfa5feb7fd5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java @@ -37,14 +37,8 @@ public interface WindowedValue { /** Returns the windows of this {@code WindowedValue}. */ Collection getWindows(); - /** @deprecated Do not use; only here for ease of migration/container update. */ - @Deprecated - PaneInfo getPane(); - /** The {@link PaneInfo} associated with this WindowedValue. */ - default PaneInfo getPaneInfo() { - return getPane(); - } + PaneInfo getPaneInfo(); /** * A representation of each of the actual values represented by this compressed {@link diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java index 3c044990de37..9616fd845fa7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java @@ -63,40 +63,40 @@ private WindowedValues() {} // non-instantiable utility class /** Returns a {@code WindowedValue} with the given value, timestamp, and windows. */ public static WindowedValue of( - T value, Instant timestamp, Collection windows, PaneInfo pane) { - checkArgument(pane != null, "WindowedValue requires PaneInfo, but it was null"); + T value, Instant timestamp, Collection windows, PaneInfo paneInfo) { + checkArgument(paneInfo != null, "WindowedValue requires PaneInfo, but it was null"); checkArgument(windows.size() > 0, "WindowedValue requires windows, but there were none"); if (windows.size() == 1) { - return of(value, timestamp, windows.iterator().next(), pane); + return of(value, timestamp, windows.iterator().next(), paneInfo); } else { - return new TimestampedValueInMultipleWindows<>(value, timestamp, windows, pane); + return new TimestampedValueInMultipleWindows<>(value, timestamp, windows, paneInfo); } } /** @deprecated for use only in compatibility with old broken code */ @Deprecated static WindowedValue createWithoutValidation( - T value, Instant timestamp, Collection windows, PaneInfo pane) { + T value, Instant timestamp, Collection windows, PaneInfo paneInfo) { if (windows.size() == 1) { - return of(value, timestamp, windows.iterator().next(), pane); + return of(value, timestamp, windows.iterator().next(), paneInfo); } else { - return new TimestampedValueInMultipleWindows<>(value, timestamp, windows, pane); + return new TimestampedValueInMultipleWindows<>(value, timestamp, windows, paneInfo); } } /** Returns a {@code WindowedValue} with the given value, timestamp, and window. */ public static WindowedValue of( - T value, Instant timestamp, BoundedWindow window, PaneInfo pane) { - checkArgument(pane != null, "WindowedValue requires PaneInfo, but it was null"); + T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo) { + checkArgument(paneInfo != null, "WindowedValue requires PaneInfo, but it was null"); boolean isGlobal = GlobalWindow.INSTANCE.equals(window); if (isGlobal && BoundedWindow.TIMESTAMP_MIN_VALUE.equals(timestamp)) { - return valueInGlobalWindow(value, pane); + return valueInGlobalWindow(value, paneInfo); } else if (isGlobal) { - return new TimestampedValueInGlobalWindow<>(value, timestamp, pane); + return new TimestampedValueInGlobalWindow<>(value, timestamp, paneInfo); } else { - return new TimestampedValueInSingleWindow<>(value, timestamp, window, pane); + return new TimestampedValueInSingleWindow<>(value, timestamp, window, paneInfo); } } @@ -112,8 +112,8 @@ public static WindowedValue valueInGlobalWindow(T value) { * Returns a {@code WindowedValue} with the given value in the {@link GlobalWindow} using the * default timestamp and the specified pane. */ - public static WindowedValue valueInGlobalWindow(T value, PaneInfo pane) { - return new ValueInGlobalWindow<>(value, pane); + public static WindowedValue valueInGlobalWindow(T value, PaneInfo paneInfo) { + return new ValueInGlobalWindow<>(value, paneInfo); } /** @@ -179,7 +179,7 @@ public static int hashCode(WindowedValue windowedValue) { windowedValue.getValue(), windowedValue.getTimestamp().getMillis(), windowedValue.getWindows(), - windowedValue.getPane()); + windowedValue.getPaneInfo()); } private static final Collection GLOBAL_WINDOWS = @@ -199,16 +199,16 @@ public interface SingleWindowedValue { private abstract static class SimpleWindowedValue implements WindowedValue { private final T value; - private final PaneInfo pane; + private final PaneInfo paneInfo; - protected SimpleWindowedValue(T value, PaneInfo pane) { + protected SimpleWindowedValue(T value, PaneInfo paneInfo) { this.value = value; - this.pane = checkNotNull(pane); + this.paneInfo = checkNotNull(paneInfo); } @Override - public PaneInfo getPane() { - return pane; + public PaneInfo getPaneInfo() { + return paneInfo; } @Override @@ -232,8 +232,8 @@ public Iterable> explodeWindows() { /** The abstract superclass of WindowedValue representations where timestamp == MIN. */ private abstract static class MinTimestampWindowedValue extends SimpleWindowedValue { - public MinTimestampWindowedValue(T value, PaneInfo pane) { - super(value, pane); + public MinTimestampWindowedValue(T value, PaneInfo paneInfo) { + super(value, paneInfo); } @Override @@ -246,8 +246,8 @@ public Instant getTimestamp() { private static class ValueInGlobalWindow extends MinTimestampWindowedValue implements SingleWindowedValue { - public ValueInGlobalWindow(T value, PaneInfo pane) { - super(value, pane); + public ValueInGlobalWindow(T value, PaneInfo paneInfo) { + super(value, paneInfo); } @Override @@ -262,14 +262,14 @@ public BoundedWindow getWindow() { @Override public WindowedValue withValue(NewT newValue) { - return new ValueInGlobalWindow<>(newValue, getPane()); + return new ValueInGlobalWindow<>(newValue, getPaneInfo()); } @Override public boolean equals(@Nullable Object o) { if (o instanceof ValueInGlobalWindow) { ValueInGlobalWindow that = (ValueInGlobalWindow) o; - return Objects.equals(that.getPane(), this.getPane()) + return Objects.equals(that.getPaneInfo(), this.getPaneInfo()) && Objects.equals(that.getValue(), this.getValue()); } else { return super.equals(o); @@ -278,14 +278,14 @@ public boolean equals(@Nullable Object o) { @Override public int hashCode() { - return Objects.hash(getValue(), getPane()); + return Objects.hash(getValue(), getPaneInfo()); } @Override public String toString() { return MoreObjects.toStringHelper(getClass()) .add("value", getValue()) - .add("pane", getPane()) + .add("paneInfo", getPaneInfo()) .toString(); } } @@ -294,8 +294,8 @@ public String toString() { private abstract static class TimestampedWindowedValue extends SimpleWindowedValue { private final Instant timestamp; - public TimestampedWindowedValue(T value, Instant timestamp, PaneInfo pane) { - super(value, pane); + public TimestampedWindowedValue(T value, Instant timestamp, PaneInfo paneInfo) { + super(value, paneInfo); this.timestamp = checkNotNull(timestamp); } @@ -312,8 +312,8 @@ public Instant getTimestamp() { private static class TimestampedValueInGlobalWindow extends TimestampedWindowedValue implements SingleWindowedValue { - public TimestampedValueInGlobalWindow(T value, Instant timestamp, PaneInfo pane) { - super(value, timestamp, pane); + public TimestampedValueInGlobalWindow(T value, Instant timestamp, PaneInfo paneInfo) { + super(value, timestamp, paneInfo); } @Override @@ -328,7 +328,7 @@ public BoundedWindow getWindow() { @Override public WindowedValue withValue(NewT newValue) { - return new TimestampedValueInGlobalWindow<>(newValue, getTimestamp(), getPane()); + return new TimestampedValueInGlobalWindow<>(newValue, getTimestamp(), getPaneInfo()); } @Override @@ -339,7 +339,7 @@ public boolean equals(@Nullable Object o) { // Also compare timestamps according to millis-since-epoch because otherwise expensive // comparisons are made on their Chronology objects. return this.getTimestamp().isEqual(that.getTimestamp()) - && Objects.equals(that.getPane(), this.getPane()) + && Objects.equals(that.getPaneInfo(), this.getPaneInfo()) && Objects.equals(that.getValue(), this.getValue()); } else { return super.equals(o); @@ -349,7 +349,7 @@ public boolean equals(@Nullable Object o) { @Override public int hashCode() { // Hash only the millis of the timestamp to be consistent with equals - return Objects.hash(getValue(), getPane(), getTimestamp().getMillis()); + return Objects.hash(getValue(), getPaneInfo(), getTimestamp().getMillis()); } @Override @@ -357,7 +357,7 @@ public String toString() { return MoreObjects.toStringHelper(getClass()) .add("value", getValue()) .add("timestamp", getTimestamp()) - .add("pane", getPane()) + .add("paneInfo", getPaneInfo()) .toString(); } } @@ -372,14 +372,14 @@ private static class TimestampedValueInSingleWindow extends TimestampedWindow private final BoundedWindow window; public TimestampedValueInSingleWindow( - T value, Instant timestamp, BoundedWindow window, PaneInfo pane) { - super(value, timestamp, pane); + T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo) { + super(value, timestamp, paneInfo); this.window = checkNotNull(window); } @Override public WindowedValue withValue(NewT newValue) { - return new TimestampedValueInSingleWindow<>(newValue, getTimestamp(), window, getPane()); + return new TimestampedValueInSingleWindow<>(newValue, getTimestamp(), window, getPaneInfo()); } @Override @@ -401,7 +401,7 @@ public boolean equals(@Nullable Object o) { // comparisons are made on their Chronology objects. return this.getTimestamp().isEqual(that.getTimestamp()) && Objects.equals(that.getValue(), this.getValue()) - && Objects.equals(that.getPane(), this.getPane()) + && Objects.equals(that.getPaneInfo(), this.getPaneInfo()) && Objects.equals(that.window, this.window); } else { return super.equals(o); @@ -411,7 +411,7 @@ public boolean equals(@Nullable Object o) { @Override public int hashCode() { // Hash only the millis of the timestamp to be consistent with equals - return Objects.hash(getValue(), getTimestamp().getMillis(), getPane(), window); + return Objects.hash(getValue(), getTimestamp().getMillis(), getPaneInfo(), window); } @Override @@ -420,7 +420,7 @@ public String toString() { .add("value", getValue()) .add("timestamp", getTimestamp()) .add("window", window) - .add("pane", getPane()) + .add("paneInfo", getPaneInfo()) .toString(); } } @@ -430,8 +430,11 @@ private static class TimestampedValueInMultipleWindows extends TimestampedWin private Collection windows; public TimestampedValueInMultipleWindows( - T value, Instant timestamp, Collection windows, PaneInfo pane) { - super(value, timestamp, pane); + T value, + Instant timestamp, + Collection windows, + PaneInfo paneInfo) { + super(value, timestamp, paneInfo); this.windows = checkNotNull(windows); } @@ -443,7 +446,7 @@ public Collection getWindows() { @Override public WindowedValue withValue(NewT newValue) { return new TimestampedValueInMultipleWindows<>( - newValue, getTimestamp(), getWindows(), getPane()); + newValue, getTimestamp(), getWindows(), getPaneInfo()); } @Override @@ -455,7 +458,7 @@ public boolean equals(@Nullable Object o) { // comparisons are made on their Chronology objects. if (this.getTimestamp().isEqual(that.getTimestamp()) && Objects.equals(that.getValue(), this.getValue()) - && Objects.equals(that.getPane(), this.getPane())) { + && Objects.equals(that.getPaneInfo(), this.getPaneInfo())) { ensureWindowsAreASet(); that.ensureWindowsAreASet(); return that.windows.equals(this.windows); @@ -471,7 +474,7 @@ public boolean equals(@Nullable Object o) { public int hashCode() { // Hash only the millis of the timestamp to be consistent with equals ensureWindowsAreASet(); - return Objects.hash(getValue(), getTimestamp().getMillis(), getPane(), windows); + return Objects.hash(getValue(), getTimestamp().getMillis(), getPaneInfo(), windows); } @Override @@ -480,7 +483,7 @@ public String toString() { .add("value", getValue()) .add("timestamp", getTimestamp()) .add("windows", windows) - .add("pane", getPane()) + .add("paneInfo", getPaneInfo()) .toString(); } @@ -581,7 +584,7 @@ public void encode(WindowedValue windowedElem, OutputStream outStream, Contex throws CoderException, IOException { InstantCoder.of().encode(windowedElem.getTimestamp(), outStream); windowsCoder.encode(windowedElem.getWindows(), outStream); - PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream); + PaneInfoCoder.INSTANCE.encode(windowedElem.getPaneInfo(), outStream); valueCoder.encode(windowedElem.getValue(), outStream, context); } @@ -595,12 +598,12 @@ public WindowedValue decode(InputStream inStream, Context context) throws CoderException, IOException { Instant timestamp = InstantCoder.of().decode(inStream); Collection windows = windowsCoder.decode(inStream); - PaneInfo pane = PaneInfoCoder.INSTANCE.decode(inStream); + PaneInfo paneInfo = PaneInfoCoder.INSTANCE.decode(inStream); T value = valueCoder.decode(inStream, context); // Because there are some remaining (incorrect) uses of WindowedValue with no windows, // we call this deprecated no-validation path when decoding - return WindowedValues.createWithoutValidation(value, timestamp, windows, pane); + return WindowedValues.createWithoutValidation(value, timestamp, windows, paneInfo); } @Override @@ -616,7 +619,7 @@ public void registerByteSizeObserver(WindowedValue value, ElementByteSizeObse throws Exception { InstantCoder.of().registerByteSizeObserver(value.getTimestamp(), observer); windowsCoder.registerByteSizeObserver(value.getWindows(), observer); - PaneInfoCoder.INSTANCE.registerByteSizeObserver(value.getPane(), observer); + PaneInfoCoder.INSTANCE.registerByteSizeObserver(value.getPaneInfo(), observer); valueCoder.registerByteSizeObserver(value.getValue(), observer); } @@ -729,8 +732,8 @@ public static ParamWindowedValueCoder of( Coder windowCoder, Instant timestamp, Collection windows, - PaneInfo pane) { - return new ParamWindowedValueCoder<>(valueCoder, windowCoder, timestamp, windows, pane); + PaneInfo paneInfo) { + return new ParamWindowedValueCoder<>(valueCoder, windowCoder, timestamp, windows, paneInfo); } /** @@ -763,15 +766,15 @@ public static ParamWindowedValueCoder of(Coder valueCoder) { Coder windowCoder, Instant timestamp, Collection windows, - PaneInfo pane) { + PaneInfo paneInfo) { super(valueCoder, windowCoder); - this.windowedValuePrototype = WindowedValues.of(EMPTY_BYTES, timestamp, windows, pane); + this.windowedValuePrototype = WindowedValues.of(EMPTY_BYTES, timestamp, windows, paneInfo); } @Override public WindowedValueCoder withValueCoder(Coder valueCoder) { return new ParamWindowedValueCoder<>( - valueCoder, getWindowCoder(), getTimestamp(), getWindows(), getPane()); + valueCoder, getWindowCoder(), getTimestamp(), getWindows(), getPaneInfo()); } @Override @@ -817,8 +820,8 @@ public Collection getWindows() { return windowedValuePrototype.getWindows(); } - public PaneInfo getPane() { - return windowedValuePrototype.getPane(); + public PaneInfo getPaneInfo() { + return windowedValuePrototype.getPaneInfo(); } /** Returns the serialized payload that will be provided when deserializing this coder. */ @@ -827,7 +830,8 @@ public static byte[] getPayload(ParamWindowedValueCoder from) { // ParamWindowedValueCoder ByteArrayOutputStream baos = new ByteArrayOutputStream(); WindowedValue windowedValue = - WindowedValues.of(EMPTY_BYTES, from.getTimestamp(), from.getWindows(), from.getPane()); + WindowedValues.of( + EMPTY_BYTES, from.getTimestamp(), from.getWindows(), from.getPaneInfo()); WindowedValues.FullWindowedValueCoder windowedValueCoder = WindowedValues.FullWindowedValueCoder.of(ByteArrayCoder.of(), from.getWindowCoder()); try { @@ -855,7 +859,7 @@ public static WindowedValues.ParamWindowedValueCoder fromComponents( windowCoder, windowedValue.getTimestamp(), windowedValue.getWindows(), - windowedValue.getPane()); + windowedValue.getPaneInfo()); } catch (IOException e) { throw new RuntimeException( "Unable to decode constant members from payload for ParamWindowedValueCoder: ", e); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java index fa6aa1cdfde8..0b41f75f7c51 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java @@ -122,7 +122,7 @@ public void testConstructWindowedName() { "myPaneStr3", "slidingWindow1")); - // test first/last pane + // test first/last paneInfo assertEquals( "/out.txt/part-00042-myWindowStr-pane-11-true-false", constructName( @@ -130,7 +130,7 @@ public void testConstructWindowedName() { assertEquals( "/path/to/out.txt", - constructName("/path/to/ou", "t.t", "xt", 1, 1, "pane", "anotherWindowStr")); + constructName("/path/to/ou", "t.t", "xt", 1, 1, "paneInfo", "anotherWindowStr")); assertEquals( "/out0102shard-oneMoreWindowStr-pane--1-false-false-pane--1-false-false.txt", diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java index 90b0822d9dca..d3c1f6680bee 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java @@ -454,7 +454,7 @@ private static String getDefaultFileName(FileIO.Write write) throws Except @Test public void testFilenameFnResolution() throws Exception { - FileIO.Write.FileNaming foo = (window, pane, numShards, shardIndex, compression) -> "foo"; + FileIO.Write.FileNaming foo = (window, paneInfo, numShards, shardIndex, compression) -> "foo"; String expected = FileSystems.matchNewResource("test", true).resolve("foo", RESOLVE_FILE).toString(); @@ -526,7 +526,7 @@ public void testFileIoDynamicNaming() throws IOException { Contextful.Fn fileNaming = (element, c) -> - (window, pane, numShards, shardIndex, compression) -> + (window, paneInfo, numShards, shardIndex, compression) -> c.sideInput(outputFileNameView) + "-" + shardIndex; p.apply(Create.of("")) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java index 4bf865475e34..00b6027f945d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java @@ -363,12 +363,12 @@ public void testEarlyPanesOfWindow() { TestStream source = TestStream.create(VarLongCoder.of()) .addElements(TimestampedValue.of(1L, new Instant(1000L))) - .advanceProcessingTime(Duration.standardMinutes(6)) // Fire early pane + .advanceProcessingTime(Duration.standardMinutes(6)) // Fire early paneInfo .addElements(TimestampedValue.of(2L, new Instant(2000L))) - .advanceProcessingTime(Duration.standardMinutes(6)) // Fire early pane + .advanceProcessingTime(Duration.standardMinutes(6)) // Fire early paneInfo .addElements(TimestampedValue.of(3L, new Instant(3000L))) - .advanceProcessingTime(Duration.standardMinutes(6)) // Fire early pane - .advanceWatermarkToInfinity(); // Fire on-time pane + .advanceProcessingTime(Duration.standardMinutes(6)) // Fire early paneInfo + .advanceWatermarkToInfinity(); // Fire on-time paneInfo PCollection> sum = p.apply(source) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java index e9cb2664341f..03f4f1c67c67 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java @@ -367,7 +367,7 @@ public void testCreateWindowedValues() { windowedValue.getValue(), windowedValue.getTimestamp(), w, - windowedValue.getPane()))) + windowedValue.getPaneInfo()))) .collect(Collectors.toList()); PCollection output = diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java index d21eb77819a2..ea46ffec4496 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java @@ -224,8 +224,8 @@ public void testRedistributePreservesMetadata() { afterRedistributeTimestamp, equalTo(originalTimestamp)); - PaneInfo originalPaneInfo = elem.getValue().getPane(); - PaneInfo afterRedistributePaneInfo = elem.getPane(); + PaneInfo originalPaneInfo = elem.getValue().getPaneInfo(); + PaneInfo afterRedistributePaneInfo = elem.getPaneInfo(); assertThat( "Redistribute did not preserve pane info for " + elem, afterRedistributePaneInfo, diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java index 099dcb78a6e0..10f65dfce68d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java @@ -215,8 +215,8 @@ public void testReshufflePreservesMetadata() { afterReshuffleTimestamp, equalTo(originalTimestamp)); - PaneInfo originalPaneInfo = elem.getValue().getPane(); - PaneInfo afterReshufflePaneInfo = elem.getPane(); + PaneInfo originalPaneInfo = elem.getValue().getPaneInfo(); + PaneInfo afterReshufflePaneInfo = elem.getPaneInfo(); assertThat( "Reshuffle did not preserve pane info for " + elem, afterReshufflePaneInfo, diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java index db1579333e57..18660c5e6c36 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java @@ -122,21 +122,21 @@ public void testExplodeWindowsManyWindowsMultipleWindowedValues() { BoundedWindow futureWindow = new IntervalWindow(now.minus(Duration.millis(500L)), now.plus(Duration.millis(1500L))); BoundedWindow futureFutureWindow = new IntervalWindow(now, now.plus(Duration.millis(2000L))); - PaneInfo pane = PaneInfo.createPane(false, false, Timing.ON_TIME, 3L, 0L); + PaneInfo paneInfo = PaneInfo.createPane(false, false, Timing.ON_TIME, 3L, 0L); WindowedValue value = WindowedValues.of( "foo", now, ImmutableList.of(pastWindow, centerWindow, futureWindow, futureFutureWindow), - pane); + paneInfo); assertThat( value.explodeWindows(), containsInAnyOrder( - WindowedValues.of("foo", now, futureFutureWindow, pane), - WindowedValues.of("foo", now, futureWindow, pane), - WindowedValues.of("foo", now, centerWindow, pane), - WindowedValues.of("foo", now, pastWindow, pane))); + WindowedValues.of("foo", now, futureFutureWindow, paneInfo), + WindowedValues.of("foo", now, futureWindow, paneInfo), + WindowedValues.of("foo", now, centerWindow, paneInfo), + WindowedValues.of("foo", now, pastWindow, paneInfo))); assertThat(value, not(instanceOf(WindowedValues.SingleWindowedValue.class))); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java index f3e7c392664d..e96bf27c6ed9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java @@ -63,7 +63,7 @@ public void testTimer() { assertEquals(FIRE_TIME, timer.getFireTimestamp()); assertEquals(HOLD_TIME, timer.getHoldTimestamp()); assertEquals(Collections.singleton(GlobalWindow.INSTANCE), timer.getWindows()); - assertEquals(PaneInfo.NO_FIRING, timer.getPane()); + assertEquals(PaneInfo.NO_FIRING, timer.getPaneInfo()); assertFalse(timer.getClearBit()); } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java index 957fa057054f..0b3c677bb54d 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java @@ -110,6 +110,6 @@ public BoundedWindow window() { } }; Collection windows = windowFn.assignWindows(ctxt); - return WindowedValues.of(input.getValue(), input.getTimestamp(), windows, input.getPane()); + return WindowedValues.of(input.getValue(), input.getTimestamp(), windows, input.getPaneInfo()); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index 460c4cc6cab7..4258681a494f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -1016,28 +1016,28 @@ public Object restriction() { KV.of(splitResult.getPrimaryInFullyProcessedWindowsRoot().getValue(), fullSize), splitResult.getPrimaryInFullyProcessedWindowsRoot().getTimestamp(), splitResult.getPrimaryInFullyProcessedWindowsRoot().getWindows(), - splitResult.getPrimaryInFullyProcessedWindowsRoot().getPane()), + splitResult.getPrimaryInFullyProcessedWindowsRoot().getPaneInfo()), splitResult.getPrimarySplitRoot() == null ? null : WindowedValues.of( KV.of(splitResult.getPrimarySplitRoot().getValue(), primarySize), splitResult.getPrimarySplitRoot().getTimestamp(), splitResult.getPrimarySplitRoot().getWindows(), - splitResult.getPrimarySplitRoot().getPane()), + splitResult.getPrimarySplitRoot().getPaneInfo()), splitResult.getResidualSplitRoot() == null ? null : WindowedValues.of( KV.of(splitResult.getResidualSplitRoot().getValue(), residualSize), splitResult.getResidualSplitRoot().getTimestamp(), splitResult.getResidualSplitRoot().getWindows(), - splitResult.getResidualSplitRoot().getPane()), + splitResult.getResidualSplitRoot().getPaneInfo()), splitResult.getResidualInUnprocessedWindowsRoot() == null ? null : WindowedValues.of( KV.of(splitResult.getResidualInUnprocessedWindowsRoot().getValue(), fullSize), splitResult.getResidualInUnprocessedWindowsRoot().getTimestamp(), splitResult.getResidualInUnprocessedWindowsRoot().getWindows(), - splitResult.getResidualInUnprocessedWindowsRoot().getPane())); + splitResult.getResidualInUnprocessedWindowsRoot().getPaneInfo())); } private HandlesSplits.SplitResult trySplitForWindowObservingTruncateRestriction( @@ -1120,7 +1120,7 @@ private static WindowedSplitResult computeWindowSplit KV.of(currentRestriction, currentWatermarkEstimatorState)), currentElement.getTimestamp(), primaryFullyProcessedWindows, - currentElement.getPane()), + currentElement.getPaneInfo()), splitResult == null ? null : WindowedValues.of( @@ -1129,7 +1129,7 @@ private static WindowedSplitResult computeWindowSplit KV.of(splitResult.getPrimary(), currentWatermarkEstimatorState)), currentElement.getTimestamp(), currentWindow, - currentElement.getPane()), + currentElement.getPaneInfo()), splitResult == null ? null : WindowedValues.of( @@ -1138,7 +1138,7 @@ private static WindowedSplitResult computeWindowSplit KV.of(splitResult.getResidual(), watermarkAndState.getValue())), currentElement.getTimestamp(), currentWindow, - currentElement.getPane()), + currentElement.getPaneInfo()), residualUnprocessedWindows.isEmpty() ? null : WindowedValues.of( @@ -1147,7 +1147,7 @@ private static WindowedSplitResult computeWindowSplit KV.of(currentRestriction, currentWatermarkEstimatorState)), currentElement.getTimestamp(), residualUnprocessedWindows, - currentElement.getPane())); + currentElement.getPaneInfo())); return windowedSplitResult; } @@ -1980,7 +1980,7 @@ public void output(OutputT output) { outputTo( mainOutputConsumer, WindowedValues.of( - output, currentElement.getTimestamp(), currentWindow, currentElement.getPane())); + output, currentElement.getTimestamp(), currentWindow, currentElement.getPaneInfo())); } @Override @@ -1994,7 +1994,7 @@ public void output(TupleTag tag, T output) { outputTo( consumer, WindowedValues.of( - output, currentElement.getTimestamp(), currentWindow, currentElement.getPane())); + output, currentElement.getTimestamp(), currentWindow, currentElement.getPaneInfo())); } @Override @@ -2003,7 +2003,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { // runners can provide proper timestamps. outputTo( mainOutputConsumer, - WindowedValues.of(output, timestamp, currentWindow, currentElement.getPane())); + WindowedValues.of(output, timestamp, currentWindow, currentElement.getPaneInfo())); } @Override @@ -2027,7 +2027,8 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } outputTo( - consumer, WindowedValues.of(output, timestamp, currentWindow, currentElement.getPane())); + consumer, + WindowedValues.of(output, timestamp, currentWindow, currentElement.getPaneInfo())); } @Override @@ -2084,7 +2085,7 @@ public org.apache.beam.sdk.state.Timer timer(String timerId) { currentWindow, currentElement.getTimestamp(), currentElement.getTimestamp(), - currentElement.getPane(), + currentElement.getPaneInfo(), timeDomain); } @@ -2096,7 +2097,7 @@ public TimerMap timerFamily(String timerFamilyId) { currentWindow, currentElement.getTimestamp(), currentElement.getTimestamp(), - currentElement.getPane()); + currentElement.getPaneInfo()); } } @@ -2143,7 +2144,7 @@ public Instant timestamp(DoFn doFn) { size), currentElement.getTimestamp(), currentWindow, - currentElement.getPane())); + currentElement.getPaneInfo())); } @Override @@ -2188,7 +2189,7 @@ public Instant timestamp(DoFn doFn) { size), timestamp, currentWindow, - currentElement.getPane())); + currentElement.getPaneInfo())); } @Override @@ -2356,7 +2357,7 @@ public Instant timestamp(DoFn doFn) { size), timestamp, currentElement.getWindows(), - currentElement.getPane())); + currentElement.getPaneInfo())); } @Override @@ -2452,7 +2453,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { outputTo( mainOutputConsumer, WindowedValues.of( - output, timestamp, currentElement.getWindows(), currentElement.getPane())); + output, timestamp, currentElement.getWindows(), currentElement.getPaneInfo())); } @Override @@ -2476,7 +2477,7 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp outputTo( consumer, WindowedValues.of( - output, timestamp, currentElement.getWindows(), currentElement.getPane())); + output, timestamp, currentElement.getWindows(), currentElement.getPaneInfo())); } @Override @@ -2786,7 +2787,7 @@ public Instant timestamp() { @Override public PaneInfo pane() { - return currentElement.getPane(); + return currentElement.getPaneInfo(); } @Override @@ -2826,7 +2827,10 @@ public void output(OutputT output) { outputTo( mainOutputConsumer, WindowedValues.of( - output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); + output, + currentTimer.getHoldTimestamp(), + currentWindow, + currentTimer.getPaneInfo())); } @Override @@ -2834,7 +2838,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { checkOnWindowExpirationTimestamp(timestamp); outputTo( mainOutputConsumer, - WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); + WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPaneInfo())); } @Override @@ -2857,7 +2861,10 @@ public void output(TupleTag tag, T output) { outputTo( consumer, WindowedValues.of( - output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); + output, + currentTimer.getHoldTimestamp(), + currentWindow, + currentTimer.getPaneInfo())); } @Override @@ -2869,7 +2876,8 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } outputTo( - consumer, WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); + consumer, + WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPaneInfo())); } @Override @@ -3122,7 +3130,10 @@ public void output(OutputT output) { outputTo( mainOutputConsumer, WindowedValues.of( - output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); + output, + currentTimer.getHoldTimestamp(), + currentWindow, + currentTimer.getPaneInfo())); } @Override @@ -3130,7 +3141,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { checkTimerTimestamp(timestamp); outputTo( mainOutputConsumer, - WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); + WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPaneInfo())); } @Override @@ -3154,7 +3165,10 @@ public void output(TupleTag tag, T output) { outputTo( consumer, WindowedValues.of( - output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); + output, + currentTimer.getHoldTimestamp(), + currentWindow, + currentTimer.getPaneInfo())); } @Override @@ -3166,7 +3180,8 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } outputTo( - consumer, WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); + consumer, + WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPaneInfo())); } @Override @@ -3409,7 +3424,7 @@ public org.apache.beam.sdk.state.Timer timer(String timerId) { currentWindow, currentTimer.getHoldTimestamp(), currentTimer.getFireTimestamp(), - currentTimer.getPane(), + currentTimer.getPaneInfo(), timeDomain); } @@ -3421,7 +3436,7 @@ public TimerMap timerFamily(String timerFamilyId) { currentWindow, currentTimer.getHoldTimestamp(), currentTimer.getFireTimestamp(), - currentTimer.getPane()); + currentTimer.getPaneInfo()); } @Override diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java index 72565f608ce5..3bfb914a4a93 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java @@ -208,7 +208,7 @@ private void processElementForWindowObservingPairWithRestriction(WindowedValue doFn) { @Override public PaneInfo paneInfo(DoFn doFn) { - return getCurrentElementOrFail().getPane(); + return getCurrentElementOrFail().getPaneInfo(); } @Override diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunner.java index 3df6462d04ad..b701c468abca 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunner.java @@ -353,7 +353,7 @@ public void output(RestrictionT subrestriction) { size), getCurrentElement().getTimestamp(), getCurrentWindow(), - getCurrentElement().getPane())); + getCurrentElement().getPaneInfo())); } } @@ -412,7 +412,7 @@ public Instant timestamp(DoFn doFn) { @Override public PaneInfo paneInfo(DoFn doFn) { - return getCurrentElement().getPane(); + return getCurrentElement().getPaneInfo(); } @Override diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java index 9fbf21683b63..78563dc24690 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java @@ -121,7 +121,7 @@ public void singleInputMultipleOutputSucceeds() throws Exception { -3, new Instant(-12), ImmutableSet.of(firstWindow, secondWindow), - firstValue.getPane()))); + firstValue.getPaneInfo()))); WindowedValue secondValue = WindowedValues.of( 3, @@ -136,7 +136,7 @@ public void singleInputMultipleOutputSucceeds() throws Exception { 3, new Instant(12), ImmutableSet.of(secondWindow, thirdWindow), - secondValue.getPane()))); + secondValue.getPaneInfo()))); } @Test diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index b9c879ad827a..f50508cd1b79 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -2267,7 +2267,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc 3.0), firstValue.getTimestamp(), window1, - firstValue.getPane())); + firstValue.getPaneInfo())); assertEquals( decode(inputCoder, residualRoot.getApplication().getElement()), WindowedValues.of( @@ -2280,7 +2280,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc 2.0), firstValue.getTimestamp(), window1, - firstValue.getPane())); + firstValue.getPaneInfo())); assertEquals( decode(inputCoder, residualRootForUnprocessedWindows.getApplication().getElement()), WindowedValues.of( @@ -2293,7 +2293,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc 5.0), firstValue.getTimestamp(), window2, - firstValue.getPane())); + firstValue.getPaneInfo())); splitListener.clear(); // Check that before processing an element we don't report progress @@ -2320,37 +2320,37 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc "5:5", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(5)), window1, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( "5:6", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(6)), window1, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( "5:7", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(7)), window1, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( "2:0", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(0)), window1, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( "2:1", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)), window1, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( "2:0", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(0)), window2, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( "2:1", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)), window2, - firstValue.getPane()))); + firstValue.getPaneInfo()))); assertTrue(splitListener.getPrimaryRoots().isEmpty()); assertTrue(splitListener.getResidualRoots().isEmpty()); mainOutputValues.clear(); @@ -2412,22 +2412,22 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc "7:0", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(0)), window1, - splitValue.getPane()), + splitValue.getPaneInfo()), WindowedValues.of( "7:1", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)), window1, - splitValue.getPane()), + splitValue.getPaneInfo()), WindowedValues.of( "7:2", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(2)), window1, - splitValue.getPane()), + splitValue.getPaneInfo()), WindowedValues.of( "7:3", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(3)), window1, - splitValue.getPane()))); + splitValue.getPaneInfo()))); BundleApplication primaryRoot = Iterables.getOnlyElement(trySplitResult.getPrimaryRoots()); assertEquals(2, trySplitResult.getResidualRoots().size()); @@ -2501,7 +2501,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc 5.0), splitValue.getTimestamp(), window2, - splitValue.getPane()), + splitValue.getPaneInfo()), inputCoder.decode( residualRootInUnprocessedWindows.getApplication().getElement().newInput())); @@ -2717,14 +2717,14 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser 3.0), splitValue.getTimestamp(), window1, - splitValue.getPane()), + splitValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("7", KV.of(new OffsetRange(0, 3), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), splitValue.getTimestamp(), window2, - splitValue.getPane()))); + splitValue.getPaneInfo()))); SplitResult expectedElementSplit = createSplitResult(0); BundleApplication expectedElementSplitPrimary = @@ -2736,7 +2736,7 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser KV.of("7", KV.of(new OffsetRange(0, 6), GlobalWindow.TIMESTAMP_MIN_VALUE)), 6.0), splitValue.getTimestamp(), window1, - splitValue.getPane()), + splitValue.getPaneInfo()), primaryBytes); BundleApplication expectedWindowedPrimary = BundleApplication.newBuilder() @@ -2753,7 +2753,7 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser KV.of("7", KV.of(new OffsetRange(0, 6), GlobalWindow.TIMESTAMP_MIN_VALUE)), 6.0), splitValue.getTimestamp(), window3, - splitValue.getPane()), + splitValue.getPaneInfo()), residualBytes); DelayedBundleApplication expectedWindowedResidual = DelayedBundleApplication.newBuilder() @@ -3027,28 +3027,28 @@ public void testProcessElementForWindowedTruncateAndSizeRestriction() throws Exc 2.0), firstValue.getTimestamp(), window1, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), window2, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), firstValue.getTimestamp(), window1, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), firstValue.getTimestamp(), window2, - firstValue.getPane()))); + firstValue.getPaneInfo()))); mainOutputValues.clear(); assertTrue(context.getFinishBundleFunctions().isEmpty()); @@ -3150,14 +3150,14 @@ public void testProcessElementForWindowedTruncateAndSizeRestriction() throws Exc 2.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), - firstValue.getPane()))); + firstValue.getPaneInfo()))); mainOutputValues.clear(); assertTrue(context.getFinishBundleFunctions().isEmpty()); @@ -3340,14 +3340,14 @@ private KV createSplitInWindow( KV.of(primaryRestriction, currentWatermarkEstimatorState)), currentElement.getTimestamp(), window, - currentElement.getPane()), + currentElement.getPaneInfo()), WindowedValues.of( KV.of( currentElement.getValue(), KV.of(residualRestriction, watermarkAndState.getValue())), currentElement.getTimestamp(), window, - currentElement.getPane())); + currentElement.getPaneInfo())); } private KV createSplitAcrossWindows( @@ -3361,7 +3361,7 @@ private KV createSplitAcrossWindows( KV.of(currentRestriction, currentWatermarkEstimatorState)), currentElement.getTimestamp(), primaryWindows, - currentElement.getPane()), + currentElement.getPaneInfo()), residualWindows.isEmpty() ? null : WindowedValues.of( @@ -3370,7 +3370,7 @@ private KV createSplitAcrossWindows( KV.of(currentRestriction, currentWatermarkEstimatorState)), currentElement.getTimestamp(), residualWindows, - currentElement.getPane())); + currentElement.getPaneInfo())); } private KV createSplitWithSizeInWindow( @@ -3384,7 +3384,7 @@ private KV createSplitWithSizeInWindow( (double) (primaryRestriction.getTo() - primaryRestriction.getFrom())), currentElement.getTimestamp(), window, - currentElement.getPane()), + currentElement.getPaneInfo()), WindowedValues.of( KV.of( KV.of( @@ -3393,7 +3393,7 @@ private KV createSplitWithSizeInWindow( (double) (residualRestriction.getTo() - residualRestriction.getFrom())), currentElement.getTimestamp(), window, - currentElement.getPane())); + currentElement.getPaneInfo())); } private KV createSplitWithSizeAcrossWindows( @@ -3409,7 +3409,7 @@ private KV createSplitWithSizeAcrossWindows( (double) (currentRestriction.getTo() - currentRestriction.getFrom())), currentElement.getTimestamp(), primaryWindows, - currentElement.getPane()), + currentElement.getPaneInfo()), residualWindows.isEmpty() ? null : WindowedValues.of( @@ -3420,7 +3420,7 @@ private KV createSplitWithSizeAcrossWindows( (double) (currentRestriction.getTo() - currentRestriction.getFrom())), currentElement.getTimestamp(), residualWindows, - currentElement.getPane())); + currentElement.getPaneInfo())); } @Before diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java index 370fd6e6de2c..13a0b105ec39 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java @@ -362,7 +362,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))), firstValue.getTimestamp(), window1, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( "5", @@ -371,7 +371,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))), firstValue.getTimestamp(), window2, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( "2", @@ -380,7 +380,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))), secondValue.getTimestamp(), window1, - secondValue.getPane()), + secondValue.getPaneInfo()), WindowedValues.of( KV.of( "2", @@ -389,7 +389,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))), secondValue.getTimestamp(), window2, - secondValue.getPane()))); + secondValue.getPaneInfo()))); mainOutputValues.clear(); assertTrue(context.getFinishBundleFunctions().isEmpty()); @@ -477,7 +477,7 @@ public void testProcessElementForWindowedPairWithRestrictionWithNonWindowObservi GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))), firstValue.getTimestamp(), ImmutableList.of(window1, window2), - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( "2", @@ -486,7 +486,7 @@ public void testProcessElementForWindowedPairWithRestrictionWithNonWindowObservi GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))), secondValue.getTimestamp(), ImmutableList.of(window1, window2), - secondValue.getPane()))); + secondValue.getPaneInfo()))); mainOutputValues.clear(); assertTrue(context.getFinishBundleFunctions().isEmpty()); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java index a19f65bd5716..1336d2f4ba9f 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java @@ -334,56 +334,56 @@ public void testProcessElementForWindowedSplitAndSizeRestriction() throws Except 2.0), firstValue.getTimestamp(), window1, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(2, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), firstValue.getTimestamp(), window1, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), window2, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(2, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), firstValue.getTimestamp(), window2, - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), secondValue.getTimestamp(), window1, - secondValue.getPane()), + secondValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(1, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), secondValue.getTimestamp(), window1, - secondValue.getPane()), + secondValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), secondValue.getTimestamp(), window2, - secondValue.getPane()), + secondValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(1, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), secondValue.getTimestamp(), window2, - secondValue.getPane()))); + secondValue.getPaneInfo()))); mainOutputValues.clear(); assertTrue(context.getFinishBundleFunctions().isEmpty()); @@ -477,28 +477,28 @@ public void testProcessElementForWindowedSplitAndSizeRestriction() throws Except 2.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(2, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), - firstValue.getPane()), + firstValue.getPaneInfo()), WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(1, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), - firstValue.getPane()))); + firstValue.getPaneInfo()))); mainOutputValues.clear(); assertTrue(context.getFinishBundleFunctions().isEmpty()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java index 2569f71bf036..8ea7f31b3b2e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchedStreamingWrite.java @@ -257,7 +257,7 @@ public void processElement( @Element KV> element, @Timestamp Instant timestamp, BoundedWindow window, - PaneInfo pane) { + PaneInfo paneInfo) { Map>> tableRows = this.tableRows; Map> uniqueIdsForTableRows = this.uniqueIdsForTableRows; String tableSpec = element.getKey(); @@ -265,7 +265,9 @@ public void processElement( TableRow failsafeTableRow = toFailsafeTableRow.apply(element.getValue().tableRow); tableRows .computeIfAbsent(tableSpec, k -> new ArrayList<>()) - .add(FailsafeValueInSingleWindow.of(tableRow, timestamp, window, pane, failsafeTableRow)); + .add( + FailsafeValueInSingleWindow.of( + tableRow, timestamp, window, paneInfo, failsafeTableRow)); uniqueIdsForTableRows .computeIfAbsent(tableSpec, k -> new ArrayList<>()) .add(element.getValue().uniqueId); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 2765a0e0b523..390ffa1aa991 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -1304,7 +1304,7 @@ long insertAll( rowsToPublish.get(i).getValue(), rowsToPublish.get(i).getTimestamp(), rowsToPublish.get(i).getWindow(), - rowsToPublish.get(i).getPane())); + rowsToPublish.get(i).getPaneInfo())); } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java index 652c91a4a78c..2bd7dd69e590 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java @@ -45,13 +45,13 @@ void add( Preconditions.checkArgumentNotNull(tableRow.getFailsafeValue()), tableRow.getTimestamp(), tableRow.getWindow(), - tableRow.getPane())); + tableRow.getPaneInfo())); ErrorContainer BIG_QUERY_INSERT_ERROR_ERROR_CONTAINER = (failedInserts, error, ref, tableRow) -> { BigQueryInsertError err = new BigQueryInsertError(tableRow.getFailsafeValue(), error, ref); failedInserts.add( ValueInSingleWindow.of( - err, tableRow.getTimestamp(), tableRow.getWindow(), tableRow.getPane())); + err, tableRow.getTimestamp(), tableRow.getWindow(), tableRow.getPaneInfo())); }; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java index 5c44497fdbd9..a5d4c0e43215 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java @@ -62,11 +62,11 @@ public void processElement( @Element InputT element, @Timestamp Instant timestamp, BoundedWindow window, - PaneInfo pane) + PaneInfo paneInfo) throws IOException { dynamicDestinations.setSideInputAccessorFromProcessContext(context); ValueInSingleWindow windowedElement = - ValueInSingleWindow.of(element, timestamp, window, pane); + ValueInSingleWindow.of(element, timestamp, window, paneInfo); DestinationT tableDestination = dynamicDestinations.getDestination(windowedElement); checkArgument( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java index 31036c58f9dc..6fde170e3792 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java @@ -28,7 +28,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.gcp.bigquery.WriteBundlesToFiles.Result; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java index 24229643eca3..3d100413cb2d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java @@ -538,7 +538,7 @@ public long insertAll( row, rowList.get(i).getTimestamp(), rowList.get(i).getWindow(), - rowList.get(i).getPane())); + rowList.get(i).getPaneInfo())); } } else { errorContainer.add( diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java index 42de901587b6..7db1ac426595 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java @@ -93,13 +93,13 @@ public void processElement( ProcessContext c, @Element KV, Iterable> element, BoundedWindow window, - PaneInfo pane) + PaneInfo paneInfo) throws Exception { String tableIdentifier = element.getKey().getKey(); IcebergDestination destination = dynamicDestinations.instantiateDestination(tableIdentifier); WindowedValue windowedDestination = - WindowedValues.of(destination, window.maxTimestamp(), window, pane); + WindowedValues.of(destination, window.maxTimestamp(), window, paneInfo); RecordWriterManager writer; try (RecordWriterManager openWriter = new RecordWriterManager(getCatalog(), filePrefix, maxFileSize, Integer.MAX_VALUE)) { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java index e76d18ae7106..bf2a5a3535fb 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java @@ -227,14 +227,14 @@ public void startBundle() { public void processElement( @Element KV element, BoundedWindow window, - PaneInfo pane, + PaneInfo paneInfo, MultiOutputReceiver out) throws Exception { String dest = element.getKey(); Row data = element.getValue(); IcebergDestination destination = dynamicDestinations.instantiateDestination(dest); WindowedValue windowedDestination = - WindowedValues.of(destination, window.maxTimestamp(), window, pane); + WindowedValues.of(destination, window.maxTimestamp(), window, paneInfo); // Attempt to write record. If the writer is saturated and cannot accept // the record, spill it over to WriteGroupedRowsToFiles diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java index ae9ac559a798..c894e55a6c09 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java @@ -146,7 +146,7 @@ private String timingToString(PaneInfo.Timing timing) { } /** Construct an {@link OutputFile} for {@code pane} in {@code window} for {@code shard}. */ - private OutputFile outputFileFor(BoundedWindow window, String shard, PaneInfo pane) { + private OutputFile outputFileFor(BoundedWindow window, String shard, PaneInfo paneInfo) { @Nullable String filename = outputPath == null @@ -156,11 +156,11 @@ private OutputFile outputFileFor(BoundedWindow window, String shard, PaneInfo pa outputPath, window.maxTimestamp(), shard, - pane.getIndex(), - timingToString(pane.getTiming()), + paneInfo.getIndex(), + timingToString(paneInfo.getTiming()), ThreadLocalRandom.current().nextLong()); return new OutputFile( - window.maxTimestamp(), shard, pane.getIndex(), pane.getTiming(), filename); + window.maxTimestamp(), shard, paneInfo.getIndex(), paneInfo.getTiming(), filename); } /**