beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-4176) Java: Portable batch runner passes all ValidatesRunner tests that non-portable runner passes
Date Tue, 18 Sep 2018 22:16:00 GMT

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

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

                Author: ASF GitHub Bot
            Created on: 18/Sep/18 22:15
            Start Date: 18/Sep/18 22:15
    Worklog Time Spent: 10m 
      Work Description: lukecwik closed pull request #6328: [BEAM-4176] enumerate primitive
transforms in portable construction
URL: https://github.com/apache/beam/pull/6328
 
 
   

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/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/NativeTransforms.java
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/NativeTransforms.java
new file mode 100644
index 00000000000..730a5ca561f
--- /dev/null
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/NativeTransforms.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core.construction;
+
+import com.google.auto.service.AutoService;
+import java.util.Iterator;
+import java.util.ServiceLoader;
+import java.util.function.Predicate;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+
+/**
+ * An extension point for users to define their own native transforms for usage with specific
+ * runners. This extension point enables shared libraries within the Apache Beam codebase
to treat
+ * the native transform as a primitive transforms that the runner implicitly understands.
+ *
+ * <p><b>Warning:</b>Usage of native transforms within pipelines will prevent
users from migrating
+ * between runners as there is no expectation that the transform will be understood by all
runners.
+ * Note that for some use cases this can be a way to test out a new type of transform on
a limited
+ * set of runners and promote its adoption as a primitive within the Apache Beam model.
+ *
+ * <p>Note that users are required to ensure that translation and execution for the
native transform
+ * is supported by their runner.
+ *
+ * <p>Automatic registration occurs by creating a {@link ServiceLoader} entry and a
concrete
+ * implementation of the {@link IsNativeTransform} interface. It is optional but recommended
to use
+ * one of the many build time tools such as {@link AutoService} to generate the necessary
META-INF
+ * files automatically.
+ */
+public class NativeTransforms {
+  /**
+   * Returns true if an only if the Runner understands this transform and can handle it directly.
+   */
+  public static boolean isNative(RunnerApi.PTransform pTransform) {
+    Iterator<IsNativeTransform> matchers = ServiceLoader.load(IsNativeTransform.class).iterator();
+    while (matchers.hasNext()) {
+      if (matchers.next().test(pTransform)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** A predicate which returns true if and only if the transform is a native transform.
*/
+  public interface IsNativeTransform extends Predicate<RunnerApi.PTransform> {
+    @Override
+    boolean test(RunnerApi.PTransform pTransform);
+  }
+}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java
index 7be45388695..0401b572dda 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java
@@ -67,7 +67,14 @@
       getUrn(StandardPTransforms.Primitives.ASSIGN_WINDOWS);
   public static final String TEST_STREAM_TRANSFORM_URN =
       getUrn(StandardPTransforms.Primitives.TEST_STREAM);
+  public static final String MAP_WINDOWS_TRANSFORM_URN =
+      getUrn(StandardPTransforms.Primitives.MAP_WINDOWS);
 
+  /**
+   * @deprecated SDKs should move away from creating `Read` transforms and migrate to using
Impulse
+   *     + SplittableDoFns.
+   */
+  @Deprecated
   public static final String READ_TRANSFORM_URN =
       getUrn(StandardPTransforms.DeprecatedPrimitives.READ);
   /**
@@ -85,6 +92,12 @@
   public static final String RESHUFFLE_URN = getUrn(StandardPTransforms.Composites.RESHUFFLE);
   public static final String WRITE_FILES_TRANSFORM_URN =
       getUrn(StandardPTransforms.Composites.WRITE_FILES);
+
+  // SplittableParDoComponents
+  public static final String SPLITTABLE_PAIR_WITH_RESTRICTION_URN =
+      getUrn(SplittableParDoComponents.PAIR_WITH_RESTRICTION);
+  public static final String SPLITTABLE_SPLIT_RESTRICTION_URN =
+      getUrn(SplittableParDoComponents.SPLIT_RESTRICTION);
   public static final String SPLITTABLE_PROCESS_KEYED_URN =
       getUrn(SplittableParDoComponents.PROCESS_KEYED_ELEMENTS);
   public static final String SPLITTABLE_PROCESS_ELEMENTS_URN =
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
index 20a3176a3af..3eed941f5a6 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
@@ -19,8 +19,20 @@
 package org.apache.beam.runners.core.construction.graph;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.ASSIGN_WINDOWS_TRANSFORM_URN;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.CREATE_VIEW_TRANSFORM_URN;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.FLATTEN_TRANSFORM_URN;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.IMPULSE_TRANSFORM_URN;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.MAP_WINDOWS_TRANSFORM_URN;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.READ_TRANSFORM_URN;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_PROCESS_ELEMENTS_URN;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.SPLITTABLE_PROCESS_KEYED_URN;
+import static org.apache.beam.runners.core.construction.PTransformTranslation.TEST_STREAM_TRANSFORM_URN;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import com.google.common.graph.MutableNetwork;
@@ -31,6 +43,7 @@
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
@@ -44,6 +57,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
 import org.apache.beam.runners.core.construction.Environments;
+import org.apache.beam.runners.core.construction.NativeTransforms;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
@@ -113,15 +127,35 @@ private QueryablePipeline(Collection<String> transformIds, Components
components
       PTransform transform = transformEntry.getValue();
       boolean isPrimitive = isPrimitiveTransform(transform);
       if (isPrimitive) {
-        ids.add(transformEntry.getKey());
+        List<String> subtransforms = transform.getSubtransformsList();
+        if (subtransforms.isEmpty()) {
+          ids.add(transformEntry.getKey());
+        } else {
+          ids.addAll(subtransforms);
+        }
       }
     }
     return ids;
   }
 
-  /** Returns true if the provided transform is a primitive. A primitive has no subtransforms.
*/
+  private static final Set<String> PRIMITIVE_URNS =
+      ImmutableSet.of(
+          PAR_DO_TRANSFORM_URN,
+          FLATTEN_TRANSFORM_URN,
+          GROUP_BY_KEY_TRANSFORM_URN,
+          IMPULSE_TRANSFORM_URN,
+          ASSIGN_WINDOWS_TRANSFORM_URN,
+          TEST_STREAM_TRANSFORM_URN,
+          MAP_WINDOWS_TRANSFORM_URN,
+          READ_TRANSFORM_URN,
+          CREATE_VIEW_TRANSFORM_URN,
+          SPLITTABLE_PROCESS_KEYED_URN,
+          SPLITTABLE_PROCESS_ELEMENTS_URN);
+
+  /** Returns true if the provided transform is a primitive. */
   private static boolean isPrimitiveTransform(PTransform transform) {
-    return transform.getSubtransformsCount() == 0;
+    String urn = PTransformTranslation.urnForTransformOrNull(transform);
+    return PRIMITIVE_URNS.contains(urn) || NativeTransforms.isNative(transform);
   }
 
   private MutableNetwork<PipelineNode, PipelineEdge> buildNetwork(
@@ -342,7 +376,7 @@ public Components getComponents() {
   }
 
   private Set<String> getLocalSideInputNames(PTransform transform) {
-    if (PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn()))
{
+    if (PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) {
       try {
         return ParDoPayload.parseFrom(transform.getSpec().getPayload()).getSideInputsMap().keySet();
       } catch (InvalidProtocolBufferException e) {
@@ -354,7 +388,7 @@ public Components getComponents() {
   }
 
   private Set<String> getLocalUserStateNames(PTransform transform) {
-    if (PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn()))
{
+    if (PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) {
       try {
         return ParDoPayload.parseFrom(transform.getSpec().getPayload()).getStateSpecsMap().keySet();
       } catch (InvalidProtocolBufferException e) {
@@ -366,7 +400,7 @@ public Components getComponents() {
   }
 
   private Set<String> getLocalTimerNames(PTransform transform) {
-    if (PTransformTranslation.PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn()))
{
+    if (PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) {
       try {
         return ParDoPayload.parseFrom(transform.getSpec().getPayload()).getTimerSpecsMap().keySet();
       } catch (InvalidProtocolBufferException e) {
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/NativeTransformsTest.java
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/NativeTransformsTest.java
new file mode 100644
index 00000000000..ec7bde6d73c
--- /dev/null
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/NativeTransformsTest.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core.construction;
+
+import com.google.auto.service.AutoService;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.core.construction.NativeTransforms.IsNativeTransform;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link NativeTransforms}. */
+@RunWith(JUnit4.class)
+public class NativeTransformsTest {
+  /** A test implementation of a {@link IsNativeTransform}. */
+  @AutoService(IsNativeTransform.class)
+  public static class TestNativeTransform implements IsNativeTransform {
+
+    @Override
+    public boolean test(RunnerApi.PTransform pTransform) {
+      return "test".equals(PTransformTranslation.urnForTransformOrNull(pTransform));
+    }
+  }
+
+  @Test
+  public void testMatch() {
+    Assert.assertTrue(
+        NativeTransforms.isNative(
+            RunnerApi.PTransform.newBuilder()
+                .setSpec(RunnerApi.FunctionSpec.newBuilder().setUrn("test").build())
+                .build()));
+  }
+
+  @Test
+  public void testNoMatch() {
+    Assert.assertFalse(NativeTransforms.isNative(RunnerApi.PTransform.getDefaultInstance()));
+  }
+}
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
index c47f743013f..06a2c6f3610 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
@@ -178,26 +178,30 @@ public void singleEnvironmentBecomesASingleStage() {
    * becomes all runner-executed
    */
   @Test
-  public void unknownTransformsNoEnvironmentBecomeRunnerExecuted() {
+  public void transformsWithNoEnvironmentBecomeRunnerExecuted() {
     Components components =
         partialComponents
             .toBuilder()
             .putTransforms(
                 "mystery",
                 PTransform.newBuilder()
+                    .setSpec(
+                        FunctionSpec.newBuilder()
+                            .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN))
                     .setUniqueName("Mystery")
                     .putInputs("input", "impulse.out")
                     .putOutputs("output", "mystery.out")
-                    .setSpec(FunctionSpec.newBuilder().setUrn("beam:transform:mystery:v1.4"))
                     .build())
             .putPcollections("mystery.out", pc("mystery.out"))
             .putTransforms(
                 "enigma",
                 PTransform.newBuilder()
+                    .setSpec(
+                        FunctionSpec.newBuilder()
+                            .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN))
                     .setUniqueName("Enigma")
                     .putInputs("input", "impulse.out")
                     .putOutputs("output", "enigma.out")
-                    .setSpec(FunctionSpec.newBuilder().setUrn("beam:transform:enigma:v1"))
                     .build())
             .putPcollections("enigma.out", pc("enigma.out"))
             .build();
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
index a1175afcffb..fad88a996ab 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
@@ -104,6 +104,10 @@ public void differentEnvironmentsThrows() {
                 .putTransforms(
                     "read",
                     PTransform.newBuilder()
+                        .setSpec(
+                            FunctionSpec.newBuilder()
+                                .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                                .build())
                         .putInputs("input", "impulse.out")
                         .putOutputs("output", "read.out")
                         .build())
@@ -839,6 +843,9 @@ public void materializesWithSideInputConsumer() {
                 .putTransforms(
                     "side_read",
                     PTransform.newBuilder()
+                        .setSpec(
+                            FunctionSpec.newBuilder()
+                                .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN))
                         .putInputs("input", "impulse.out")
                         .putOutputs("output", "side_read.out")
                         .build())
@@ -939,6 +946,9 @@ public void sideInputIncludedInStage() {
                     "side_read",
                     PTransform.newBuilder()
                         .setUniqueName("side_read")
+                        .setSpec(
+                            FunctionSpec.newBuilder()
+                                .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN))
                         .putInputs("input", "impulse.out")
                         .putOutputs("output", "side_read.out")
                         .build())
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java
index 098b29acad2..cef3fbe8d7c 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java
@@ -36,8 +36,10 @@
 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.Environment;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
+import org.apache.beam.runners.core.construction.PTransformTranslation;
 import org.apache.beam.runners.core.construction.graph.OutputDeduplicator.DeduplicationResult;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
@@ -59,22 +61,41 @@ public void unchangedWithNoDuplicates() {
      *              \-> two -> .out /
      */
     PCollection redOut = PCollection.newBuilder().setUniqueName("red.out").build();
-    PTransform red = PTransform.newBuilder().putOutputs("out", redOut.getUniqueName()).build();
+    PTransform red =
+        PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
+            .putOutputs("out", redOut.getUniqueName())
+            .build();
     PCollection oneOut = PCollection.newBuilder().setUniqueName("one.out").build();
     PTransform one =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", redOut.getUniqueName())
             .putOutputs("out", oneOut.getUniqueName())
             .build();
     PCollection twoOut = PCollection.newBuilder().setUniqueName("two.out").build();
     PTransform two =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", redOut.getUniqueName())
             .putOutputs("out", twoOut.getUniqueName())
             .build();
     PCollection blueOut = PCollection.newBuilder().setUniqueName("blue.out").build();
     PTransform blue =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("one", oneOut.getUniqueName())
             .putInputs("two", twoOut.getUniqueName())
             .putOutputs("out", blueOut.getUniqueName())
@@ -145,22 +166,41 @@ public void duplicateOverStages() {
      *             --> [two -> .out -> shared ->] .out:1 /
      */
     PCollection redOut = PCollection.newBuilder().setUniqueName("red.out").build();
-    PTransform red = PTransform.newBuilder().putOutputs("out", redOut.getUniqueName()).build();
+    PTransform red =
+        PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
+            .putOutputs("out", redOut.getUniqueName())
+            .build();
     PCollection oneOut = PCollection.newBuilder().setUniqueName("one.out").build();
     PTransform one =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", redOut.getUniqueName())
             .putOutputs("out", oneOut.getUniqueName())
             .build();
     PCollection twoOut = PCollection.newBuilder().setUniqueName("two.out").build();
     PTransform two =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", redOut.getUniqueName())
             .putOutputs("out", twoOut.getUniqueName())
             .build();
     PCollection sharedOut = PCollection.newBuilder().setUniqueName("shared.out").build();
     PTransform shared =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("one", oneOut.getUniqueName())
             .putInputs("two", twoOut.getUniqueName())
             .putOutputs("shared", sharedOut.getUniqueName())
@@ -168,6 +208,10 @@ public void duplicateOverStages() {
     PCollection blueOut = PCollection.newBuilder().setUniqueName("blue.out").build();
     PTransform blue =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", sharedOut.getUniqueName())
             .putOutputs("out", blueOut.getUniqueName())
             .build();
@@ -267,16 +311,31 @@ public void duplicateOverStagesAndTransforms() {
      *             -----------------> shared:0 -> .out:1 /
      */
     PCollection redOut = PCollection.newBuilder().setUniqueName("red.out").build();
-    PTransform red = PTransform.newBuilder().putOutputs("out", redOut.getUniqueName()).build();
+    PTransform red =
+        PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
+            .putOutputs("out", redOut.getUniqueName())
+            .build();
     PCollection oneOut = PCollection.newBuilder().setUniqueName("one.out").build();
     PTransform one =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", redOut.getUniqueName())
             .putOutputs("out", oneOut.getUniqueName())
             .build();
     PCollection sharedOut = PCollection.newBuilder().setUniqueName("shared.out").build();
     PTransform shared =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("one", oneOut.getUniqueName())
             .putInputs("red", redOut.getUniqueName())
             .putOutputs("shared", sharedOut.getUniqueName())
@@ -284,6 +343,10 @@ public void duplicateOverStagesAndTransforms() {
     PCollection blueOut = PCollection.newBuilder().setUniqueName("blue.out").build();
     PTransform blue =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", sharedOut.getUniqueName())
             .putOutputs("out", blueOut.getUniqueName())
             .build();
@@ -384,28 +447,51 @@ public void multipleDuplicatesInStages() {
      *           [-> three -> .out -> otherShared -> .out:1] ---/
      */
     PCollection redOut = PCollection.newBuilder().setUniqueName("red.out").build();
-    PTransform red = PTransform.newBuilder().putOutputs("out", redOut.getUniqueName()).build();
+    PTransform red =
+        PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
+            .putOutputs("out", redOut.getUniqueName())
+            .build();
     PCollection threeOut = PCollection.newBuilder().setUniqueName("three.out").build();
     PTransform three =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", redOut.getUniqueName())
             .putOutputs("out", threeOut.getUniqueName())
             .build();
     PCollection oneOut = PCollection.newBuilder().setUniqueName("one.out").build();
     PTransform one =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", redOut.getUniqueName())
             .putOutputs("out", oneOut.getUniqueName())
             .build();
     PCollection twoOut = PCollection.newBuilder().setUniqueName("two.out").build();
     PTransform two =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", redOut.getUniqueName())
             .putOutputs("out", twoOut.getUniqueName())
             .build();
     PCollection sharedOut = PCollection.newBuilder().setUniqueName("shared.out").build();
     PTransform shared =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("one", oneOut.getUniqueName())
             .putInputs("two", twoOut.getUniqueName())
             .putOutputs("shared", sharedOut.getUniqueName())
@@ -413,6 +499,10 @@ public void multipleDuplicatesInStages() {
     PCollection otherSharedOut = PCollection.newBuilder().setUniqueName("shared.out2").build();
     PTransform otherShared =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("multi", threeOut.getUniqueName())
             .putInputs("two", twoOut.getUniqueName())
             .putOutputs("out", otherSharedOut.getUniqueName())
@@ -420,6 +510,10 @@ public void multipleDuplicatesInStages() {
     PCollection blueOut = PCollection.newBuilder().setUniqueName("blue.out").build();
     PTransform blue =
         PTransform.newBuilder()
+            .setSpec(
+                FunctionSpec.newBuilder()
+                    .setUrn(PTransformTranslation.PAR_DO_TRANSFORM_URN)
+                    .build())
             .putInputs("in", sharedOut.getUniqueName())
             .putOutputs("out", blueOut.getUniqueName())
             .build();
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/QueryablePipelineTest.java
b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/QueryablePipelineTest.java
index 691e768c7ce..93a907d2f80 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/QueryablePipelineTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/QueryablePipelineTest.java
@@ -92,11 +92,18 @@ public void fromComponentsWithMalformedComponents() {
     Components components =
         Components.newBuilder()
             .putTransforms(
-                "root", PTransform.newBuilder().putOutputs("output", "output.out").build())
+                "root",
+                PTransform.newBuilder()
+                    .setSpec(
+                        FunctionSpec.newBuilder()
+                            .setUrn(PTransformTranslation.IMPULSE_TRANSFORM_URN)
+                            .build())
+                    .putOutputs("output", "output.out")
+                    .build())
             .build();
 
     thrown.expect(IllegalArgumentException.class);
-    QueryablePipeline.forPrimitivesIn(components);
+    QueryablePipeline.forPrimitivesIn(components).getComponents();
   }
 
   @Test
@@ -236,7 +243,15 @@ public void transformWithSameSideAndMainInput() {
         Components.newBuilder()
             .putPcollections("read_pc", RunnerApi.PCollection.getDefaultInstance())
             .putPcollections("pardo_out", RunnerApi.PCollection.getDefaultInstance())
-            .putTransforms("root", PTransform.newBuilder().putOutputs("out", "read_pc").build())
+            .putTransforms(
+                "root",
+                PTransform.newBuilder()
+                    .setSpec(
+                        FunctionSpec.newBuilder()
+                            .setUrn(PTransformTranslation.IMPULSE_TRANSFORM_URN)
+                            .build())
+                    .putOutputs("out", "read_pc")
+                    .build())
             .putTransforms(
                 "multiConsumer",
                 PTransform.newBuilder()
diff --git a/runners/flink/job-server/build.gradle b/runners/flink/job-server/build.gradle
index 5aa69e63ef3..4f8405a8f01 100644
--- a/runners/flink/job-server/build.gradle
+++ b/runners/flink/job-server/build.gradle
@@ -73,6 +73,6 @@ runShadow {
 
 createPortableValidatesRunnerTask(
         jobServerDriver: "org.apache.beam.runners.flink.FlinkJobServerDriver",
-        jobServerConfig: "--clean-artifacts-per-job",
+        jobServerConfig: "--clean-artifacts-per-job,--job-host=localhost",
         testClasspathConfiguration: configurations.validatesPortableRunner
 )
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
index 57395d33a72..93b38f5cc4c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
@@ -285,7 +285,7 @@ public void testStableUniqueNameError() {
   /** Tests that Pipeline supports a pass-through identity function. */
   @Test
   @Category(ValidatesRunner.class)
-  public void testIdentityTransform() throws Exception {
+  public void testIdentityTransform() {
 
     PCollection<Integer> output =
         pipeline.apply(Create.of(1, 2, 3, 4)).apply("IdentityTransform", new IdentityTransform<>());


 

----------------------------------------------------------------
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:
users@infra.apache.org


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

    Worklog Id:     (was: 145506)
    Time Spent: 22h  (was: 21h 50m)

> Java: Portable batch runner passes all ValidatesRunner tests that non-portable runner
passes
> --------------------------------------------------------------------------------------------
>
>                 Key: BEAM-4176
>                 URL: https://issues.apache.org/jira/browse/BEAM-4176
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-flink
>            Reporter: Ben Sidhom
>            Priority: Major
>         Attachments: Screen Shot 2018-08-14 at 4.18.31 PM.png, Screen Shot 2018-09-03
at 11.07.38 AM.png
>
>          Time Spent: 22h
>  Remaining Estimate: 0h
>
> We need this as a sanity check that runner execution is correct.



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

Mime
View raw message