This is an automated email from the ASF dual-hosted git repository.

damondouglas pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new ff272db75e6 [Proposal] Improve DisplayData support in PTransform API 
(#30115)
ff272db75e6 is described below

commit ff272db75e685024dcae4cf84f3a979568c8ea62
Author: Claire McGinty <[email protected]>
AuthorDate: Thu Feb 8 12:43:20 2024 -0500

    [Proposal] Improve DisplayData support in PTransform API (#30115)
    
    * Improve DisplayData support in PTransform API
    
    * checkstyle
---
 .../org/apache/beam/sdk/transforms/PTransform.java | 30 +++++++++++++++++++++-
 .../apache/beam/sdk/transforms/PTransformTest.java | 21 +++++++++++++++
 .../org/apache/beam/sdk/transforms/ParDoTest.java  |  4 +++
 3 files changed, 54 insertions(+), 1 deletion(-)

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 c0c3638b28d..939ac043f76 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
@@ -20,14 +20,17 @@ package org.apache.beam.sdk.transforms;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.DisplayData.ItemSpec;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
 import org.apache.beam.sdk.transforms.resourcehints.ResourceHints;
 import org.apache.beam.sdk.util.NameUtils;
@@ -217,6 +220,25 @@ public abstract class PTransform<InputT extends PInput, 
OutputT extends POutput>
     return resourceHints;
   }
 
+  /**
+   * Set display data for your PTransform.
+   *
+   * @param displayData a list of {@link ItemSpec} instances.
+   * @return a reference to the same transfrom instance.
+   *     <p>For example:
+   *     <pre>{@code
+   * Pipeline p = ...
+   * ...
+   * p.apply(new 
SomeTransform().setDisplayData(ImmutableList.of(DisplayData.item("userFn", 
userFn.getClass())))
+   * ...
+   *
+   * }</pre>
+   */
+  public PTransform<InputT, OutputT> setDisplayData(@NonNull List<ItemSpec<?>> 
displayData) {
+    this.displayData = displayData;
+    return this;
+  }
+
   /** Returns annotations map to provide additional hints to the runner. */
   public Map<String, byte[]> getAnnotations() {
     return annotations;
@@ -243,6 +265,8 @@ public abstract class PTransform<InputT extends PInput, 
OutputT extends POutput>
 
   protected transient @NonNull Map<String, byte @NonNull []> annotations = new 
HashMap<>();
 
+  protected transient @NonNull List<ItemSpec<?>> displayData = new 
ArrayList<>();
+
   protected PTransform() {
     this.name = null;
   }
@@ -346,7 +370,11 @@ public abstract class PTransform<InputT extends PInput, 
OutputT extends POutput>
    * provide their own display data.
    */
   @Override
-  public void populateDisplayData(DisplayData.Builder builder) {}
+  public void populateDisplayData(DisplayData.Builder builder) {
+    if (this.displayData != null) {
+      this.displayData.forEach(builder::add);
+    }
+  }
 
   /**
    * For a {@code SerializableFunction<InputT, OutputT>} {@code fn}, returns a 
{@code PTransform}
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 7196b244708..4692d16a605 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,9 +17,11 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static 
org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
 import static org.apache.beam.sdk.values.TypeDescriptors.integers;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.hasSize;
 import static org.junit.Assert.assertEquals;
 
 import java.io.Serializable;
@@ -29,6 +31,7 @@ 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.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -54,6 +57,24 @@ public class PTransformTest implements Serializable {
     assertThat(displayData.items(), empty());
   }
 
+  @Test
+  public void testSetDisplayData() {
+    PTransform<PCollection<String>, PCollection<String>> transform =
+        new PTransform<PCollection<String>, PCollection<String>>() {
+          @Override
+          public PCollection<String> expand(PCollection<String> begin) {
+            throw new IllegalArgumentException("Should never be applied");
+          }
+        };
+    transform.setDisplayData(
+        ImmutableList.of(DisplayData.item("key1", "value1"), 
DisplayData.item("key2", 2L)));
+
+    final DisplayData displayData = DisplayData.from(transform);
+    assertThat(displayData.items(), hasSize(2));
+    assertThat(displayData, hasDisplayItem("key1", "value1"));
+    assertThat(displayData, hasDisplayItem("key2", 2L));
+  }
+
   @Test
   public void testNamedCompose() {
     PTransform<PCollection<Integer>, PCollection<Integer>> composed =
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
index 0cb05ca31a3..89dcafbdf94 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
@@ -507,6 +507,10 @@ public class ParDoTest implements Serializable {
                   DisplayDataMatchers.hasValue(fn.getClass().getName()))));
 
       assertThat(displayData, includesDisplayDataFor("fn", fn));
+
+      // Test setting DisplayData through PTransform API
+      parDo.setDisplayData(ImmutableList.of(DisplayData.item("doFnMetadata", 
"baz")));
+      assertThat(DisplayData.from(parDo), hasDisplayItem("doFnMetadata", 
"baz"));
     }
 
     @Test

Reply via email to