lukecwik commented on a change in pull request #12287:
URL: https://github.com/apache/beam/pull/12287#discussion_r456154654



##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
##########
@@ -488,90 +493,77 @@
             || (doFnSignature.getSize() != null && 
doFnSignature.getSize().observesWindow())
             || !sideInputMapping.isEmpty()) {
           mainInputConsumer = 
this::processElementForWindowObservingSplitRestriction;
-          // OutputT == RestrictionT
           this.processContext =
-              new WindowObservingProcessBundleContext() {
-                @Override
-                public void outputWithTimestamp(OutputT output, Instant 
timestamp) {
-                  double size =
-                      doFnInvoker.invokeGetSize(
-                          new DelegatingArgumentProvider<InputT, OutputT>(
-                              this,
-                              
PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN
-                                  + "/GetSize") {
-                            @Override
-                            public Object restriction() {
-                              return output;
-                            }
-
-                            @Override
-                            public Instant timestamp(DoFn<InputT, OutputT> 
doFn) {
-                              return timestamp;
-                            }
-
-                            @Override
-                            public RestrictionTracker<?, ?> 
restrictionTracker() {
-                              return doFnInvoker.invokeNewTracker(this);
-                            }
-                          });
-
-                  outputTo(
-                      mainOutputConsumers,
-                      (WindowedValue<OutputT>)
-                          WindowedValue.of(
-                              KV.of(
-                                  KV.of(
-                                      currentElement.getValue(),
-                                      KV.of(output, 
currentWatermarkEstimatorState)),
-                                  size),
-                              timestamp,
-                              currentWindow,
-                              currentElement.getPane()));
-                }
-              };
+              new SizedRestrictionWindowObservingProcessBundleContext(
+                  
PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN);
+
         } else {
           mainInputConsumer = this::processElementForSplitRestriction;
-          // OutputT == RestrictionT
           this.processContext =
-              new NonWindowObservingProcessBundleContext() {
-                @Override
-                public void outputWithTimestamp(OutputT output, Instant 
timestamp) {
-                  double size =
-                      doFnInvoker.invokeGetSize(
-                          new DelegatingArgumentProvider<InputT, OutputT>(
-                              this,
-                              
PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN
-                                  + "/GetSize") {
-                            @Override
-                            public Object restriction() {
-                              return output;
-                            }
-
-                            @Override
-                            public Instant timestamp(DoFn<InputT, OutputT> 
doFn) {
-                              return timestamp;
-                            }
-
-                            @Override
-                            public RestrictionTracker<?, ?> 
restrictionTracker() {
-                              return doFnInvoker.invokeNewTracker(this);
-                            }
-                          });
-
-                  outputTo(
-                      mainOutputConsumers,
-                      (WindowedValue<OutputT>)
-                          WindowedValue.of(
-                              KV.of(
-                                  KV.of(
-                                      currentElement.getValue(),
-                                      KV.of(output, 
currentWatermarkEstimatorState)),
-                                  size),
-                              timestamp,
-                              currentElement.getWindows(),
-                              currentElement.getPane()));
-                }
-              };
+              new SizedRestrictionNonWindowObservingProcessBundleContext(
+                  
PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN);
+        }
+        break;
+      case PTransformTranslation.SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN:
+        if ((doFnSignature.truncateRestriction() != null
+                && doFnSignature.truncateRestriction().observesWindow())
+            || (doFnSignature.newTracker() != null && 
doFnSignature.newTracker().observesWindow())
+            || (doFnSignature.getSize() != null && 
doFnSignature.getSize().observesWindow())
+            || !sideInputMapping.isEmpty()) {
+          if (Iterables.get(mainOutputConsumers, 0) instanceof HandlesSplits) {

Review comment:
       This should only handle splits if the mainOutputConsumers has only one 
value.

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
##########
@@ -488,90 +493,77 @@
             || (doFnSignature.getSize() != null && 
doFnSignature.getSize().observesWindow())
             || !sideInputMapping.isEmpty()) {
           mainInputConsumer = 
this::processElementForWindowObservingSplitRestriction;
-          // OutputT == RestrictionT
           this.processContext =
-              new WindowObservingProcessBundleContext() {
-                @Override
-                public void outputWithTimestamp(OutputT output, Instant 
timestamp) {
-                  double size =
-                      doFnInvoker.invokeGetSize(
-                          new DelegatingArgumentProvider<InputT, OutputT>(
-                              this,
-                              
PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN
-                                  + "/GetSize") {
-                            @Override
-                            public Object restriction() {
-                              return output;
-                            }
-
-                            @Override
-                            public Instant timestamp(DoFn<InputT, OutputT> 
doFn) {
-                              return timestamp;
-                            }
-
-                            @Override
-                            public RestrictionTracker<?, ?> 
restrictionTracker() {
-                              return doFnInvoker.invokeNewTracker(this);
-                            }
-                          });
-
-                  outputTo(
-                      mainOutputConsumers,
-                      (WindowedValue<OutputT>)
-                          WindowedValue.of(
-                              KV.of(
-                                  KV.of(
-                                      currentElement.getValue(),
-                                      KV.of(output, 
currentWatermarkEstimatorState)),
-                                  size),
-                              timestamp,
-                              currentWindow,
-                              currentElement.getPane()));
-                }
-              };
+              new SizedRestrictionWindowObservingProcessBundleContext(
+                  
PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN);
+
         } else {
           mainInputConsumer = this::processElementForSplitRestriction;
-          // OutputT == RestrictionT
           this.processContext =
-              new NonWindowObservingProcessBundleContext() {
-                @Override
-                public void outputWithTimestamp(OutputT output, Instant 
timestamp) {
-                  double size =
-                      doFnInvoker.invokeGetSize(
-                          new DelegatingArgumentProvider<InputT, OutputT>(
-                              this,
-                              
PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN
-                                  + "/GetSize") {
-                            @Override
-                            public Object restriction() {
-                              return output;
-                            }
-
-                            @Override
-                            public Instant timestamp(DoFn<InputT, OutputT> 
doFn) {
-                              return timestamp;
-                            }
-
-                            @Override
-                            public RestrictionTracker<?, ?> 
restrictionTracker() {
-                              return doFnInvoker.invokeNewTracker(this);
-                            }
-                          });
-
-                  outputTo(
-                      mainOutputConsumers,
-                      (WindowedValue<OutputT>)
-                          WindowedValue.of(
-                              KV.of(
-                                  KV.of(
-                                      currentElement.getValue(),
-                                      KV.of(output, 
currentWatermarkEstimatorState)),
-                                  size),
-                              timestamp,
-                              currentElement.getWindows(),
-                              currentElement.getPane()));
-                }
-              };
+              new SizedRestrictionNonWindowObservingProcessBundleContext(
+                  
PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN);
+        }
+        break;
+      case PTransformTranslation.SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN:
+        if ((doFnSignature.truncateRestriction() != null
+                && doFnSignature.truncateRestriction().observesWindow())
+            || (doFnSignature.newTracker() != null && 
doFnSignature.newTracker().observesWindow())
+            || (doFnSignature.getSize() != null && 
doFnSignature.getSize().observesWindow())
+            || !sideInputMapping.isEmpty()) {
+          if (Iterables.get(mainOutputConsumers, 0) instanceof HandlesSplits) {
+            mainInputConsumer =
+                new SplittableFnDataReceiver() {
+                  @Override
+                  public void accept(WindowedValue input) throws Exception {
+                    processElementForWindowObservingTruncateRestriction(input);
+                  }
+
+                  // TODO(BEAM-10303): Split should work with window observing 
optimization.
+                  @Override
+                  public SplitResult trySplit(double fractionOfRemainder) {
+                    return null;
+                  }
+
+                  // TODO(BEAM-10303): Progress should work with window 
observing optimization.
+                  @Override
+                  public double getProgress() {
+                    return 0;
+                  }
+                };
+          } else {
+            mainInputConsumer = 
this::processElementForWindowObservingTruncateRestriction;
+          }
+          this.processContext =
+              new SizedRestrictionWindowObservingProcessBundleContext(
+                  
PTransformTranslation.SPLITTABLE_TRUNCATE_SIZED_RESTRICTION_URN);
+        } else {
+          if (Iterables.get(mainOutputConsumers, 0) instanceof HandlesSplits) {

Review comment:
       ditto for handling splits if there is only one output consumer




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to