tvalentyn commented on code in PR #32445:
URL: https://github.com/apache/beam/pull/32445#discussion_r1803471797


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java:
##########
@@ -82,6 +83,81 @@ public static <T> Iterables<T> iterables() {
     return new Iterables<>();
   }
 
+  /**
+   * Returns a {@link PTransform} that flattens the input {@link PCollection} 
with a given a {@link

Review Comment:
   ```suggestion
      * Returns a {@link PTransform} that flattens the input {@link 
PCollection} with a given {@link
   ```



##########
sdks/python/apache_beam/examples/snippets/snippets.py:
##########
@@ -1143,6 +1143,39 @@ def model_multiple_pcollections_flatten(contents, 
output_path):
     merged | beam.io.WriteToText(output_path)
 
 
+def model_multiple_pcollections_flatten_with(contents, output_path):
+  """Merging a PCollection with FlattenWith."""
+  some_hash_fn = lambda s: ord(s[0])
+  partition_fn = lambda element, partitions: some_hash_fn(element) % partitions
+  import apache_beam as beam
+  with TestPipeline() as pipeline:  # Use TestPipeline for testing.
+
+    # Partition into deciles
+    partitioned = pipeline | beam.Create(contents) | beam.Partition(
+        partition_fn, 3)
+    pcoll1 = partitioned[0]
+    pcoll2 = partitioned[1]
+    pcoll3 = partitioned[2]
+    SomeTransform = lambda: beam.Map(lambda x: x)

Review Comment:
   Should we have other example for merging with a transform output? feel free 
to create a bug to add it. examples are just important as having the 
capability, so i think we should highlight these everywhere (beam playground, 
snippets, website docs, etc). Can be with follow up /starter bugs if you don't 
have time to do all that in one change.



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java:
##########
@@ -82,6 +83,81 @@ public static <T> Iterables<T> iterables() {
     return new Iterables<>();
   }
 
+  /**
+   * Returns a {@link PTransform} that flattens the input {@link PCollection} 
with a given a {@link
+   * PCollection} resulting in a {@link PCollection} containing all the 
elements of both {@link
+   * PCollection}s as its output.
+   *
+   * <p>This is equivalent to creating a {@link PCollectionList} containing 
both the input and
+   * {@code other} and then applying {@link #pCollections()}, but has the 
advantage that it can be
+   * more easily used inline.
+   *
+   * <p>Both {@cpde PCollections} must have equal {@link WindowFn}s. The 
output elements of {@code
+   * Flatten<T>} are in the same windows and have the same timestamps as their 
corresponding input
+   * elements. The output {@code PCollection} will have the same {@link 
WindowFn} as both inputs.
+   *
+   * @param other the other PCollection to flatten with the input
+   * @param <T> the type of the elements in the input and output {@code 
PCollection}s.
+   */
+  public static <T> PTransform<PCollection<T>, PCollection<T>> 
with(PCollection<T> other) {
+    return new FlattenWithPCollection<>(other);
+  }
+
+  /** Implementation of {@link #with(PCollection)}. */
+  private static class FlattenWithPCollection<T>
+      extends PTransform<PCollection<T>, PCollection<T>> {
+    // We only need to access this at pipeline construction time.
+    private final transient PCollection<T> other;
+
+    public FlattenWithPCollection(PCollection<T> other) {
+      this.other = other;
+    }
+
+    @Override
+    public PCollection<T> expand(PCollection<T> input) {
+      return PCollectionList.of(input).and(other).apply(pCollections());
+    }
+
+    @Override
+    public String getKindString() {
+      return "Flatten.With";
+    }
+  }
+
+  /**
+   * Returns a {@link PTransform} that flattens the input {@link PCollection} 
with the output of
+   * another {@link PTransform} resulting in a {@link PCollection} containing 
all the elements of
+   * both the input {@link PCollection}s and the output of the given {@link 
PTransform} as its
+   * output.
+   *
+   * <p>This is equivalent to creating a {@link PCollectionList} containing 
both the input and the
+   * output of {@code other} and then applying {@link #pCollections()}, but 
has the advantage that
+   * it can be more easily used inline.
+   *
+   * <p>Both {@cpde PCollections} must have equal {@link WindowFn}s. The 
output elements of {@code

Review Comment:
   ```suggestion
      * <p>Both {@code PCollections} must have equal {@link WindowFn}s. The 
output elements of {@code
   ```



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Tee.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.transforms;
+
+import java.util.function.Consumer;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+
+/**
+ * A PTransform that returns its input, but also applies its input to an 
auxiliary PTransform, akin
+ * to the shell {@code tee} command.

Review Comment:
   ```suggestion
    * to the shell {@code tee} command, which is named after the T-splitter 
used in plumbing.
   ```
   



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Flatten.java:
##########
@@ -82,6 +83,81 @@ public static <T> Iterables<T> iterables() {
     return new Iterables<>();
   }
 
+  /**
+   * Returns a {@link PTransform} that flattens the input {@link PCollection} 
with a given a {@link
+   * PCollection} resulting in a {@link PCollection} containing all the 
elements of both {@link
+   * PCollection}s as its output.
+   *
+   * <p>This is equivalent to creating a {@link PCollectionList} containing 
both the input and
+   * {@code other} and then applying {@link #pCollections()}, but has the 
advantage that it can be
+   * more easily used inline.
+   *
+   * <p>Both {@cpde PCollections} must have equal {@link WindowFn}s. The 
output elements of {@code
+   * Flatten<T>} are in the same windows and have the same timestamps as their 
corresponding input
+   * elements. The output {@code PCollection} will have the same {@link 
WindowFn} as both inputs.
+   *
+   * @param other the other PCollection to flatten with the input
+   * @param <T> the type of the elements in the input and output {@code 
PCollection}s.
+   */
+  public static <T> PTransform<PCollection<T>, PCollection<T>> 
with(PCollection<T> other) {
+    return new FlattenWithPCollection<>(other);
+  }
+
+  /** Implementation of {@link #with(PCollection)}. */
+  private static class FlattenWithPCollection<T>
+      extends PTransform<PCollection<T>, PCollection<T>> {
+    // We only need to access this at pipeline construction time.
+    private final transient PCollection<T> other;
+
+    public FlattenWithPCollection(PCollection<T> other) {
+      this.other = other;
+    }
+
+    @Override
+    public PCollection<T> expand(PCollection<T> input) {
+      return PCollectionList.of(input).and(other).apply(pCollections());
+    }
+
+    @Override
+    public String getKindString() {
+      return "Flatten.With";

Review Comment:
    Merge.With might be a possible alternative name. but maybe it adds more 
confusion since we have a pre-existing `Flatten` already for a similar concept.
   



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to