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/06/27 02:51:16 UTC

[GitHub] [beam] boyuanzz commented on a change in pull request #12093: [BEAM-10303] Add support for the non-window observing optimization to DoFn execution in portable Beam Java

boyuanzz commented on a change in pull request #12093:
URL: https://github.com/apache/beam/pull/12093#discussion_r446475260



##########
File path: sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
##########
@@ -488,60 +438,201 @@ public void accept(WindowedValue input) throws Exception {
     this.doFnInvoker.invokeSetup();
 
     this.startBundleArgumentProvider = new StartBundleArgumentProvider();
+    // Register the appropriate handlers.
+    switch (pTransform.getSpec().getUrn()) {
+      case PTransformTranslation.PAR_DO_TRANSFORM_URN:
+      case PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN:
+        startFunctionRegistry.register(pTransformId, this::startBundle);
+        break;
+      case PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN:
+        // startBundle should not be invoked
+      case PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN:
+        // startBundle should not be invoked
+      default:
+        // no-op
+    }
+
+    String mainInput;
+    try {
+      mainInput = ParDoTranslation.getMainInputName(pTransform);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    final FnDataReceiver<WindowedValue> mainInputConsumer;
+    switch (pTransform.getSpec().getUrn()) {
+      case PTransformTranslation.PAR_DO_TRANSFORM_URN:
+        if (doFnSignature.processElement().observesWindow() || !sideInputMapping.isEmpty()) {
+          mainInputConsumer = this::processElementForWindowObservingParDo;
+          this.processContext = new WindowObservingProcessBundleContext();
+        } else {
+          mainInputConsumer = this::processElementForParDo;
+          this.processContext = new NonWindowObservingProcessBundleContext();
+        }
+        break;
+      case PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN:
+        if (doFnSignature.getInitialRestriction().observesWindow()
+            || (doFnSignature.getInitialWatermarkEstimatorState() != null
+                && doFnSignature.getInitialWatermarkEstimatorState().observesWindow())
+            || !sideInputMapping.isEmpty()) {
+          mainInputConsumer = this::processElementForWindowObservingPairWithRestriction;
+          this.processContext = new WindowObservingProcessBundleContext();
+        } else {
+          mainInputConsumer = this::processElementForPairWithRestriction;
+          this.processContext = new NonWindowObservingProcessBundleContext();
+        }
+        break;
+      case PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN:
+        if ((doFnSignature.splitRestriction() != null
+                && doFnSignature.splitRestriction().observesWindow())
+            || (doFnSignature.newTracker() != null && doFnSignature.newTracker().observesWindow())
+            || (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()));
+                }
+              };
+        } 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()));
+                }
+              };
+        }
+        break;
+      case PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN:
+        if (doFnSignature.processElement().observesWindow()
+            || (doFnSignature.newTracker() != null && doFnSignature.newTracker().observesWindow())
+            || (doFnSignature.getSize() != null && doFnSignature.getSize().observesWindow())
+            || (doFnSignature.newWatermarkEstimator() != null
+                && doFnSignature.newWatermarkEstimator().observesWindow())
+            || !sideInputMapping.isEmpty()) {
+          mainInputConsumer =
+              new SplittableFnDataReceiver() {
+                @Override
+                public void accept(WindowedValue input) throws Exception {
+                  processElementForWindowObservingSizedElementAndRestriction(input);
+                }
+              };
+          this.processContext = new WindowObservingProcessBundleContext();
+        } else {
+          mainInputConsumer =
+              new SplittableFnDataReceiver() {
+                @Override
+                public void accept(WindowedValue input) throws Exception {
+                  // TODO(BEAM-10303): Create a variant which is optimized to not observe the
+                  // windows.
+                  processElementForWindowObservingSizedElementAndRestriction(input);
+                }
+              };
+          this.processContext = new WindowObservingProcessBundleContext();
+        }
+        break;
+      default:
+        throw new IllegalStateException("Unknown urn: " + pTransform.getSpec().getUrn());
+    }
+    pCollectionConsumerRegistry.register(
+        pTransform.getInputsOrThrow(mainInput), pTransformId, (FnDataReceiver) mainInputConsumer);
+
     switch (pTransform.getSpec().getUrn()) {

Review comment:
       It seems like we should remove this duplicate part.




----------------------------------------------------------------
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