[ 
https://issues.apache.org/jira/browse/BEAM-5413?focusedWorklogId=148250&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-148250
 ]

ASF GitHub Bot logged work on BEAM-5413:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 26/Sep/18 18:26
            Start Date: 26/Sep/18 18:26
    Worklog Time Spent: 10m 
      Work Description: lukecwik closed pull request #6414: [BEAM-5413] Add 
PTransform::compose for lambda-based composite transforms
URL: https://github.com/apache/beam/pull/6414
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
index 4021c66757c..49c40f76938 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
@@ -24,6 +24,7 @@
 import java.util.Map;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -292,4 +293,30 @@ private void readObject(ObjectInputStream oos) {
    */
   @Override
   public void populateDisplayData(Builder builder) {}
+
+  /**
+   * For a {@code SerializableFunction<InputT, OutputT>} {@code fn}, returns a 
{@code PTransform}
+   * given by applying {@code fn.apply(v)} to the input {@code 
PCollection<InputT>}.
+   *
+   * <p>Allows users to define a concise composite transform using a Java 8 
lambda expression. For
+   * example:
+   *
+   * <pre>{@code
+   * PCollection<String> words = wordsAndErrors.apply(
+   *   (PCollectionTuple input) -> {
+   *     input.get(errorsTag).apply(new WriteErrorOutput());
+   *     return input.get(wordsTag);
+   *   });
+   * }</pre>
+   */
+  @Experimental
+  public static <InputT extends PInput, OutputT extends POutput>
+      PTransform<InputT, OutputT> compose(SerializableFunction<InputT, 
OutputT> fn) {
+    return new PTransform<InputT, OutputT>() {
+      @Override
+      public OutputT expand(InputT input) {
+        return fn.apply(input);
+      }
+    };
+  }
 }
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
index b91740a2c6d..35d18d9378e 100644
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
@@ -17,18 +17,29 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static org.apache.beam.sdk.values.TypeDescriptors.integers;
 import static org.hamcrest.Matchers.empty;
 import static org.junit.Assert.assertThat;
 
+import java.io.Serializable;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
 /** Tests for {@link PTransform} base class. */
 @RunWith(JUnit4.class)
-public class PTransformTest {
+public class PTransformTest implements Serializable {
+
+  @Rule public final transient TestPipeline pipeline = TestPipeline.create();
+
   @Test
   public void testPopulateDisplayDataDefaultBehavior() {
     PTransform<PCollection<String>, PCollection<String>> transform =
@@ -41,4 +52,24 @@ public void testPopulateDisplayDataDefaultBehavior() {
     DisplayData displayData = DisplayData.from(transform);
     assertThat(displayData.items(), empty());
   }
+
+  @Test
+  @Category(NeedsRunner.class)
+  public void testComposeBasicSerializableFunction() throws Exception {
+    PCollection<Integer> output =
+        pipeline
+            .apply(Create.of(1, 2, 3))
+            .apply(
+                PTransform.compose(
+                    (PCollection<Integer> numbers) -> {
+                      PCollection<Integer> inverted =
+                          numbers.apply(MapElements.into(integers()).via(input 
-> -input));
+                      return PCollectionList.of(numbers)
+                          .and(inverted)
+                          .apply(Flatten.pCollections());
+                    }));
+
+    PAssert.that(output).containsInAnyOrder(-2, -1, -3, 2, 1, 3);
+    pipeline.run();
+  }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 148250)
    Time Spent: 40m  (was: 0.5h)

> Add method for defining composite transforms as lambda expressions
> ------------------------------------------------------------------
>
>                 Key: BEAM-5413
>                 URL: https://issues.apache.org/jira/browse/BEAM-5413
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-java-core
>            Reporter: Jeff Klukas
>            Assignee: Kenneth Knowles
>            Priority: Minor
>             Fix For: 2.8.0
>
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> Defining a composite transform today requires writing a full named subclass 
> of PTransform (as [the programming guide 
> documents|https://beam.apache.org/documentation/programming-guide/#composite-transforms]
>  but there are cases where users may want to define a fairly trivial 
> composite transform using a less verbose Java 8 lambda expression.
> Consider an example where the user has defined MyDeserializationTransform 
> that attempts to deserialize byte arrays into some object, returning a 
> PCollectionTuple  with tags for successfully deserialized records (mainTag) 
> and for errors (errorTag).
> If we introduce a PTransform::compose method that takes in a 
> SerializableFunction, the user can handle errors in a small lambda expression:
>  
> {code:java}
> byteArrays
>     .apply("attempt to deserialize messages", 
>         new MyDeserializationTransform())
>     .apply("write deserialization errors",
>         PTransform.compose((PCollectionTuple input) -> {
>             input
>               .get(errorTag)
>               .apply(new MyErrorOutputTransform());
>             return input.get(mainTag);
>         })
>     .apply("more processing on the deserialized messages", 
>          new MyOtherTransform())
> {code}
> This style allows a more concise and fluent pipeline definition than is 
> currently possible.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to