You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/09/02 17:27:13 UTC

[GitHub] [beam] y1chi commented on a change in pull request #12710: [BEAM-10303] Refactor split logic to reuse common logic.

y1chi commented on a change in pull request #12710:
URL: https://github.com/apache/beam/pull/12710#discussion_r481585635



##########
File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
##########
@@ -1514,6 +1409,19 @@ public Object restriction() {
                   stopWindowIndex,
                   elementSplit,
                   watermarkAndState);
+        } else if (downstreamSplitResult != null) {

Review comment:
       it seems that the whole if-else branch could be simplified to just 
   ```
   windowedSplitResult =
                 computeWindowSplitResult(
                     currentElement,
                     currentRestriction,
                     currentWindow,
                     windows,
                     currentWatermarkEstimatorState,
                     (elementSplit == null && downstreamSplitResult == null) ? currentWindowIndex : newWindowStopIndex,
                     newWindowStopIndex,
                     stopWindowIndex,
                     elementSplit,
                     watermarkAndState);
   ```
   or maybe I missed something.

##########
File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
##########
@@ -1546,50 +1459,32 @@ public Object restriction() {
               currentWindowIndex,
               stopWindowIndex,
               stopWindowIndex,
-              splitResult,
+              elementSplitResult,
               watermarkAndState);
     }
-    return KV.of(windowedSplitResult, newWindowStopIndex);
+    return SplitResultsWithStopIndex.of(
+        windowedSplitResult, downstreamSplitResult, newWindowStopIndex);
   }
 
-  private HandlesSplits.SplitResult trySplitForElementAndRestriction(
-      double fractionOfRemainder, Duration resumeDelay) {
-    KV<Instant, WatermarkEstimatorStateT> watermarkAndState;
-    WindowedSplitResult windowedSplitResult = null;
-    synchronized (splitLock) {
-      // There is nothing to split if we are between element and restriction processing calls.
-      if (currentTracker == null) {
-        return null;
-      }
-      // Make sure to get the output watermark before we split to ensure that the lower bound
-      // applies to the residual.
-      watermarkAndState = currentWatermarkEstimator.getWatermarkAndState();
-      KV<WindowedSplitResult, Integer> splitResult =
-          trySplitForProcess(
-              currentElement,
-              currentRestriction,
-              currentWindow,
-              currentWindows,
-              currentWatermarkEstimatorState,
-              fractionOfRemainder,
-              currentTracker,
-              watermarkAndState,
-              windowCurrentIndex,
-              windowStopIndex);
-      if (splitResult == null) {
-        return null;
-      }
-      windowStopIndex = splitResult.getValue();
-      windowedSplitResult =
-          calculateRestrictionSize(
-              splitResult.getKey(),
-              PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN
-                  + "/GetSize");
-    }
-
+  @VisibleForTesting
+  static <WatermarkEstimatorStateT> HandlesSplits.SplitResult constructSplitResult(
+      WindowedSplitResult windowedSplitResult,
+      HandlesSplits.SplitResult downstreamElementSplit,
+      Coder fullInputCoder,
+      Instant initialWatermark,
+      KV<Instant, WatermarkEstimatorStateT> watermarkAndState,
+      String pTransformId,
+      String mainInputId,
+      Collection<String> outputIds,
+      Duration resumeDelay) {
+    // The element split cannot from both windowedSplitResult and downstreamElementSplit.
+    checkArgument(

Review comment:
       can both windowedSplitResult and downstreamElementSplit be null? if not should the check expressed with xor `^`

##########
File path: sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
##########
@@ -3858,156 +3924,225 @@ public void testScaleProgress() throws Exception {
       assertEquals(8, scaledResult.getWorkRemaining(), 0.0);
     }
 
+    @Test
+    public void testComputeSplitForProcessOrTruncateWithNullTrackerAndSplitDelegate()
+        throws Exception {
+      expected.expect(IllegalArgumentException.class);
+      FnApiDoFnRunner.computeSplitForProcessOrTruncate(
+          currentElement,
+          currentRestriction,
+          window1,
+          ImmutableList.copyOf(currentElement.getWindows()),
+          currentWatermarkEstimatorState,
+          0.0,
+          null,
+          null,
+          null,
+          0,
+          3);
+    }
+
+    @Test
+    public void testComputeSplitForProcessOrTruncateWithNotNullTrackerAndDelegate()
+        throws Exception {
+      expected.expect(IllegalArgumentException.class);
+      FnApiDoFnRunner.computeSplitForProcessOrTruncate(
+          currentElement,
+          currentRestriction,
+          window1,
+          ImmutableList.copyOf(currentElement.getWindows()),
+          currentWatermarkEstimatorState,
+          0.0,
+          new OffsetRangeTracker(currentRestriction),
+          createSplitDelegate(0.3, 0.0, null),
+          null,
+          0,
+          3);
+    }
+
+    @Test
+    public void testComputeSplitForProcessOrTruncateWithInvalidWatermarkAndState()
+        throws Exception {
+      expected.expect(NullPointerException.class);

Review comment:
       I'm curious why we are not throwing another exception with reason instead?

##########
File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
##########
@@ -1444,28 +1317,44 @@ public Object restriction() {
   }
 
   @VisibleForTesting
-  static <WatermarkEstimatorStateT> KV<WindowedSplitResult, Integer> trySplitForProcess(
+  static <WatermarkEstimatorStateT> SplitResultsWithStopIndex computeSplitForProcessOrTruncate(
       WindowedValue currentElement,
       Object currentRestriction,
       BoundedWindow currentWindow,
       List<BoundedWindow> windows,
       WatermarkEstimatorStateT currentWatermarkEstimatorState,
       double fractionOfRemainder,
       RestrictionTracker currentTracker,
+      HandlesSplits splitDelegate,
       KV<Instant, WatermarkEstimatorStateT> watermarkAndState,
       int currentWindowIndex,
       int stopWindowIndex) {
+    // We should only have currentTracker or splitDelegate.
+    checkArgument(

Review comment:
       (currentTracker == null) ^ (splitDelegate == null) ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org