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 2022/11/29 20:30:14 UTC

[GitHub] [beam] lukecwik commented on a diff in pull request #24025: [#24024] Stop wrapping light weight functions with Contextful as they add a lot of overhead for functions that are meant to do almost no work.

lukecwik commented on code in PR #24025:
URL: https://github.com/apache/beam/pull/24025#discussion_r1035251386


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java:
##########
@@ -45,16 +45,13 @@
     extends PTransform<PCollection<? extends InputT>, PCollection<OutputT>> {
   private final transient @Nullable TypeDescriptor<InputT> inputType;
   private final transient @Nullable TypeDescriptor<OutputT> outputType;
-  private final transient @Nullable Object originalFnForDisplayData;
-  private final @Nullable Contextful<Fn<InputT, Iterable<OutputT>>> fn;
+  private final @Nullable Object fn;
 
   private FlatMapElements(
-      @Nullable Contextful<Fn<InputT, Iterable<OutputT>>> fn,
-      @Nullable Object originalFnForDisplayData,
+      @Nullable Object fn,

Review Comment:
   You're correct, the type system was too painful to specify `OR` between `ProcessFunction` and `Contextful`.
   
   Note that the constructor is private so this detail isn't exposed to users.



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java:
##########
@@ -137,57 +132,90 @@ public <NewInputT> FlatMapElements<NewInputT, OutputT> via(
   @Experimental(Kind.CONTEXTFUL)
   public <NewInputT> FlatMapElements<NewInputT, OutputT> via(
       Contextful<Fn<NewInputT, Iterable<OutputT>>> fn) {
-    return new FlatMapElements<>(
-        fn, fn.getClosure(), TypeDescriptors.inputOf(fn.getClosure()), outputType);
+    return new FlatMapElements<>(fn, TypeDescriptors.inputOf(fn.getClosure()), outputType);
   }
 
   @Override
   public PCollection<OutputT> expand(PCollection<? extends InputT> input) {
     checkArgument(fn != null, ".via() is required");
-    return input.apply(
-        "FlatMap",
-        ParDo.of(
-                new DoFn<InputT, OutputT>() {
-                  @ProcessElement
-                  public void processElement(ProcessContext c) throws Exception {
-                    Iterable<OutputT> res =
-                        fn.getClosure().apply(c.element(), Fn.Context.wrapProcessContext(c));
-                    for (OutputT output : res) {
-                      c.output(output);
+    if (fn instanceof Contextful) {

Review Comment:
   Yes, half the cost of these trivial functions was going to creating and destroying a Contextful that wraps a ProcessBundleDescriptor that is passed in.
   
   Also note that the branch only happens during pipeline construction so it won't show up on the execution side.



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

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

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