Fix Write transform primitive display data

Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/ea8019ec
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/ea8019ec
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/ea8019ec

Branch: refs/heads/master
Commit: ea8019eced55023e060fd6472b8d69d15bf73839
Parents: 007526f
Author: Scott Wegner <sweg...@google.com>
Authored: Fri May 6 11:40:47 2016 -0700
Committer: bchambers <bchamb...@google.com>
Committed: Tue May 10 16:21:27 2016 -0700

----------------------------------------------------------------------
 .../runners/dataflow/io/DataflowTextIOTest.java | 23 +++++++++++++++++++-
 .../main/java/org/apache/beam/sdk/io/Write.java |  5 +++++
 2 files changed, 27 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ea8019ec/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
----------------------------------------------------------------------
diff --git 
a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
 
b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
index bfc99e8..8ff7d0e 100644
--- 
a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
+++ 
b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowTextIOTest.java
@@ -17,11 +17,21 @@
  */
 package org.apache.beam.runners.dataflow.io;
 
+import static 
org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+import static 
org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue;
+
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.startsWith;
+import static org.junit.Assert.assertThat;
+
 import org.apache.beam.runners.dataflow.DataflowPipelineRunner;
 import org.apache.beam.runners.dataflow.testing.TestDataflowPipelineOptions;
+import 
org.apache.beam.runners.dataflow.transforms.DataflowDisplayDataEvaluator;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
 import org.apache.beam.sdk.util.GcsUtil;
 import org.apache.beam.sdk.util.TestCredential;
 import org.apache.beam.sdk.util.gcsfs.GcsPath;
@@ -41,13 +51,13 @@ import java.nio.channels.SeekableByteChannel;
 import java.nio.file.Files;
 import java.nio.file.StandardOpenOption;
 import java.util.List;
+import java.util.Set;
 
 /**
  * {@link DataflowPipelineRunner} specific tests for TextIO Read and Write 
transforms.
  */
 @RunWith(JUnit4.class)
 public class DataflowTextIOTest {
-
   private TestDataflowPipelineOptions buildTestPipelineOptions() {
     TestDataflowPipelineOptions options =
         PipelineOptionsFactory.as(TestDataflowPipelineOptions.class);
@@ -116,4 +126,15 @@ public class DataflowTextIOTest {
   private void applyRead(Pipeline pipeline, String path) {
     pipeline.apply("Read(" + path + ")", TextIO.Read.from(path));
   }
+
+  @Test
+  public void testPrimitiveDisplayData() {
+    DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();
+
+    TextIO.Write.Bound<?> write = TextIO.Write.to("foobar");
+
+    Set<DisplayData> displayData = 
evaluator.displayDataForPrimitiveTransforms(write);
+    assertThat("TextIO.Write should include the file prefix in its primitive 
display data",
+        displayData, hasItem(hasDisplayItem(hasValue(startsWith("foobar")))));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ea8019ec/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java
index 9cb026a..0f2dbf8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java
@@ -204,6 +204,11 @@ public class Write {
                 c.output(result);
               }
             }
+
+            @Override
+            public void populateDisplayData(DisplayData.Builder builder) {
+              Write.Bound.this.populateDisplayData(builder);
+            }
           }).withSideInputs(writeOperationView))
           .setCoder(writeOperation.getWriterResultCoder());
 

Reply via email to