You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by dh...@apache.org on 2016/06/27 19:32:44 UTC
[2/2] incubator-beam git commit: DisplayDataEvaluator: Add support
for source transforms
DisplayDataEvaluator: Add support for source transforms
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/8fdf434f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8fdf434f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8fdf434f
Branch: refs/heads/master
Commit: 8fdf434f18f00f5aa76c8cc9993e29f7fee3c33a
Parents: 9abd092
Author: Vikas Kedigehalli <vi...@google.com>
Authored: Mon Jun 27 11:14:11 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Jun 27 12:32:23 2016 -0700
----------------------------------------------------------------------
.../display/DisplayDataEvaluator.java | 29 +++++++++++++++++---
.../display/DisplayDataEvaluatorTest.java | 14 ++++++++++
2 files changed, 39 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8fdf434f/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
index a17e06f..a78a4ad 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
@@ -25,6 +25,7 @@ import org.apache.beam.sdk.runners.TransformTreeNode;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.POutput;
@@ -79,8 +80,8 @@ public class DisplayDataEvaluator {
* @return the set of {@link DisplayData} for primitive {@link PTransform PTransforms}.
*/
public <InputT> Set<DisplayData> displayDataForPrimitiveTransforms(
- final PTransform<? super PCollection<InputT>, ? extends POutput> root,
- Coder<InputT> inputCoder) {
+ final PTransform<? super PCollection<InputT>, ? extends POutput> root,
+ Coder<InputT> inputCoder) {
Create.Values<InputT> input = Create.of();
if (inputCoder != null) {
@@ -89,9 +90,29 @@ public class DisplayDataEvaluator {
Pipeline pipeline = Pipeline.create(options);
pipeline
- .apply(input)
- .apply(root);
+ .apply(input)
+ .apply(root);
+ return displayDataForPipeline(pipeline, root);
+ }
+
+ /**
+ * Traverse the specified source {@link PTransform}, collecting {@link DisplayData} registered
+ * on the inner primitive {@link PTransform PTransforms}.
+ *
+ * @param root The source root {@link PTransform} to traverse
+ * @return the set of {@link DisplayData} for primitive source {@link PTransform PTransforms}.
+ */
+ public Set<DisplayData> displayDataForPrimitiveSourceTransforms(
+ final PTransform<? super PBegin, ? extends POutput> root) {
+ Pipeline pipeline = Pipeline.create(options);
+ pipeline
+ .apply(root);
+
+ return displayDataForPipeline(pipeline, root);
+ }
+
+ private static Set<DisplayData> displayDataForPipeline(Pipeline pipeline, PTransform root) {
PrimitiveDisplayDataPTransformVisitor visitor = new PrimitiveDisplayDataPTransformVisitor(root);
pipeline.traverseTopologically(visitor);
return visitor.getPrimitivesDisplayData();
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8fdf434f/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java
index 318c116..ce32b7d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java
@@ -23,9 +23,11 @@ import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.not;
import static org.junit.Assert.assertThat;
+import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.POutput;
@@ -92,4 +94,16 @@ public class DisplayDataEvaluatorTest implements Serializable {
assertThat(displayData, hasItem(hasDisplayItem("foo")));
}
+
+ @Test
+ public void testSourceTransform() {
+ PTransform<? super PBegin, ? extends POutput> myTransform = TextIO.Read
+ .from("foo.*")
+ .withoutValidation();
+
+ DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
+ Set<DisplayData> displayData = evaluator.displayDataForPrimitiveSourceTransforms(myTransform);
+
+ assertThat(displayData, hasItem(hasDisplayItem("filePattern", "foo.*")));
+ }
}