ibzib commented on a change in pull request #12157:
URL: https://github.com/apache/beam/pull/12157#discussion_r474316833



##########
File path: runners/spark/job-server/build.gradle
##########
@@ -82,20 +82,68 @@ runShadow {
     jvmArgs += 
["-Dorg.slf4j.simpleLogger.defaultLogLevel=${project.property('logLevel')}"]
 }
 
-def portableValidatesRunnerTask(String name) {
-  createPortableValidatesRunnerTask(
-    name: "validatesPortableRunner${name}",
-    jobServerDriver: "org.apache.beam.runners.spark.SparkJobServerDriver",
-    jobServerConfig: 
"--job-host=localhost,--job-port=0,--artifact-port=0,--expansion-port=0",
-    testClasspathConfiguration: configurations.validatesPortableRunner,
-    numParallelTests: 4,
-    environment: 
BeamModulePlugin.PortableValidatesRunnerConfiguration.Environment.EMBEDDED,
-    systemProperties: [
-      "beam.spark.test.reuseSparkContext": "false",
-      "spark.ui.enabled": "false",
-      "spark.ui.showConsoleProgress": "false",
-    ],
-    testCategories: {
+def portableValidatesRunnerTask(String name, Boolean streaming) {
+  def pipelineOptions = []
+  def testCategories
+  def testFilter
+
+  if (streaming) {
+    pipelineOptions += "--streaming"
+    pipelineOptions += "--streamingTimeoutMs=30000"
+
+    testCategories = {
+      includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
+      excludeCategories 
'org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders'
+      excludeCategories 'org.apache.beam.sdk.testing.LargeKeys$Above100MB'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesCustomWindowMerging'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesFailureMessage'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesGaugeMetrics'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesMapState'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesSetState'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesOrderedListState'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesTimerMap'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesKeyInParDo'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesOnWindowExpiration'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream'
+      // SplittableDoFnTests
+      excludeCategories 
'org.apache.beam.sdk.testing.UsesBoundedSplittableParDo'
+      excludeCategories 
'org.apache.beam.sdk.testing.UsesSplittableParDoWithWindowedSideInputs'
+      excludeCategories 
'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesStrictTimerOrdering'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesBundleFinalizer'
+      // Currently unsupported in portable streaming:
+      // TODO (BEAM-10712)
+      excludeCategories 'org.apache.beam.sdk.testing.UsesSideInputs'
+      // TODO (BEAM-10754)
+      excludeCategories 'org.apache.beam.sdk.testing.UsesStatefulParDo'
+      // TODO (BEAM-10755)
+      excludeCategories 'org.apache.beam.sdk.testing.UsesTimersInParDo'
+    }
+
+    testFilter = {
+      // TODO(BEAM-10094)
+      excludeTestsMatching 
'org.apache.beam.sdk.transforms.FlattenTest.testFlattenWithDifferentInputAndOutputCoders2'
+      // Currently unsupported in portable streaming:

Review comment:
       We should file a JIRA for all these (since they are conspicuously all 
windowing related).

##########
File path: 
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPortablePipelineTranslator.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.spark.translation;
+
+import java.util.Set;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
+import org.apache.beam.runners.spark.SparkPipelineOptions;
+import org.apache.spark.api.java.JavaSparkContext;
+
+/**
+ * Interface for portable Spark translators. This allows for a uniform 
invocation pattern for
+ * pipeline translation between streaming and portable runners.

Review comment:
       "streaming and portable" should be "streaming and batch."

##########
File path: 
runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java
##########
@@ -101,7 +104,9 @@ public void setUpMocks() throws Exception {
   public void sdkErrorsSurfaceOnClose() throws Exception {
     SparkExecutableStageFunction<Integer, ?> function = 
getFunction(Collections.emptyMap());
     doThrow(new Exception()).when(remoteBundle).close();
-    function.call(Collections.emptyIterator());
+    List<WindowedValue<Integer>> dummyList = new ArrayList<>();

Review comment:
       Nit: "dummy" sounds a lot like mocks or stubs, which are technical terms 
that describe the implementation of test objects. This list isn't a "dummy" in 
that sense; it's a real list, albeit one constructed for test purposes.
   
   tl;dr rename this `inputs`.

##########
File path: runners/spark/job-server/build.gradle
##########
@@ -82,20 +82,68 @@ runShadow {
     jvmArgs += 
["-Dorg.slf4j.simpleLogger.defaultLogLevel=${project.property('logLevel')}"]
 }
 
-def portableValidatesRunnerTask(String name) {
-  createPortableValidatesRunnerTask(
-    name: "validatesPortableRunner${name}",
-    jobServerDriver: "org.apache.beam.runners.spark.SparkJobServerDriver",
-    jobServerConfig: 
"--job-host=localhost,--job-port=0,--artifact-port=0,--expansion-port=0",
-    testClasspathConfiguration: configurations.validatesPortableRunner,
-    numParallelTests: 4,
-    environment: 
BeamModulePlugin.PortableValidatesRunnerConfiguration.Environment.EMBEDDED,
-    systemProperties: [
-      "beam.spark.test.reuseSparkContext": "false",
-      "spark.ui.enabled": "false",
-      "spark.ui.showConsoleProgress": "false",
-    ],
-    testCategories: {
+def portableValidatesRunnerTask(String name, Boolean streaming) {
+  def pipelineOptions = []
+  def testCategories
+  def testFilter
+
+  if (streaming) {
+    pipelineOptions += "--streaming"
+    pipelineOptions += "--streamingTimeoutMs=30000"
+
+    testCategories = {
+      includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
+      excludeCategories 
'org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders'
+      excludeCategories 'org.apache.beam.sdk.testing.LargeKeys$Above100MB'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesCustomWindowMerging'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesFailureMessage'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesGaugeMetrics'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesParDoLifecycle'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesMapState'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesSetState'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesOrderedListState'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesTimerMap'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesKeyInParDo'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesOnWindowExpiration'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream'
+      // SplittableDoFnTests

Review comment:
       This should've been there before, but the jira for this (BEAM-7222) 
should be linked.

##########
File path: 
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java
##########
@@ -118,6 +118,12 @@
 
   @Override
   public Iterator<RawUnionValue> call(Iterator<WindowedValue<InputT>> inputs) 
throws Exception {
+    // Do not call processElements if there are no inputs
+    // Otherwise, this may cause validation errors (e.g. ParDoTest)
+    if (!inputs.hasNext()) {
+      return new ConcurrentLinkedQueue<RawUnionValue>().iterator();

Review comment:
       `Collections.emptyIterator()` has the same behavior, but is more concise.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to