beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From k...@apache.org
Subject [1/3] beam git commit: Unit test to repro NPE in PTransformTranslation
Date Thu, 19 Oct 2017 21:55:34 GMT
Repository: beam
Updated Branches:
  refs/heads/master 0f096b12e -> 1039f5b96


Unit test to repro NPE in PTransformTranslation


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

Branch: refs/heads/master
Commit: acbbf1f5f41a6afc73fe129882de4474db58df2a
Parents: 3360b1f
Author: Kenneth Knowles <kenn@apache.org>
Authored: Thu Oct 19 13:49:31 2017 -0700
Committer: Kenneth Knowles <kenn@apache.org>
Committed: Thu Oct 19 13:49:42 2017 -0700

----------------------------------------------------------------------
 .../construction/PTransformTranslationTest.java | 35 ++++++++++++++++++--
 1 file changed, 33 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/acbbf1f5/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java
----------------------------------------------------------------------
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java
index 2ec96e8..36f912c 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java
@@ -31,9 +31,9 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import javax.annotation.Nullable;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.CountingSource;
 import org.apache.beam.sdk.io.GenerateSequence;
@@ -42,6 +42,7 @@ import org.apache.beam.sdk.runners.AppliedPTransform;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 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.transforms.View;
 import org.apache.beam.sdk.values.KV;
@@ -49,6 +50,7 @@ import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
@@ -69,17 +71,24 @@ public class PTransformTranslationTest {
     // This pipeline exists for construction, not to run any test.
     // TODO: Leaf node with understood payload - i.e. validate payloads
     ToAndFromProtoSpec readLeaf = ToAndFromProtoSpec.leaf(read(TestPipeline.create()));
+
     ToAndFromProtoSpec readMultipleInAndOut =
         ToAndFromProtoSpec.leaf(multiMultiParDo(TestPipeline.create()));
+
     TestPipeline compositeReadPipeline = TestPipeline.create();
     ToAndFromProtoSpec compositeRead =
         ToAndFromProtoSpec.composite(
             generateSequence(compositeReadPipeline),
             ToAndFromProtoSpec.leaf(read(compositeReadPipeline)));
+
+    ToAndFromProtoSpec rawLeafNullSpec =
+        ToAndFromProtoSpec.leaf(rawPTransformWithNullSpec(TestPipeline.create()));
+
     return ImmutableList.<ToAndFromProtoSpec>builder()
         .add(readLeaf)
         .add(readMultipleInAndOut)
         .add(compositeRead)
+        .add(rawLeafNullSpec)
         // TODO: Composite with multiple children
         // TODO: Composite with a composite child
         .build();
@@ -139,7 +148,7 @@ public class PTransformTranslationTest {
       // Sanity call
       components.getExistingPTransformId(child.getTransform());
     }
-    PTransform convert = PTransformTranslation
+    RunnerApi.PTransform convert = PTransformTranslation
         .toProto(spec.getTransform(), childTransforms, components);
     // Make sure the converted transform is registered. Convert it independently, but if
this is a
     // child spec, the child must be in the components.
@@ -166,6 +175,28 @@ public class PTransformTranslationTest {
         "ReadTheCount", pipeline.begin().expand(), pcollection.expand(), transform, pipeline);
   }
 
+  private static AppliedPTransform<?, ?, ?> rawPTransformWithNullSpec(Pipeline pipeline)
{
+    PTransformTranslation.RawPTransform<PBegin, PDone> rawPTransform =
+        new PTransformTranslation.RawPTransform<PBegin, PDone>() {
+          @Override
+          public String getUrn() {
+            return "fake/urn";
+          }
+
+          @Nullable
+          @Override
+          public RunnerApi.FunctionSpec getSpec() {
+            return null;
+          }
+        };
+    return AppliedPTransform.<PBegin, PDone, PTransform<PBegin, PDone>>of(
+        "RawPTransformWithNoSpec",
+        pipeline.begin().expand(),
+        PDone.in(pipeline).expand(),
+        rawPTransform,
+        pipeline);
+  }
+
   private static AppliedPTransform<?, ?, ?> multiMultiParDo(Pipeline pipeline) {
     PCollectionView<String> view =
         pipeline.apply(Create.of("foo")).apply(View.<String>asSingleton());


Mime
View raw message