You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/09/12 15:30:00 UTC

[GitHub] [beam] kennknowles commented on a diff in pull request #22620: Support VR test including TestStream for Spark runner in streaming mode

kennknowles commented on code in PR #22620:
URL: https://github.com/apache/beam/pull/22620#discussion_r968558666


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java:
##########
@@ -153,4 +192,84 @@ private static void awaitWatermarksOrTimeout(
     } while ((timeoutMillis -= batchDurationMillis) > 0
         && globalWatermark.isBefore(stopPipelineWatermark));
   }
+
+  /**
+   * Override factory to replace {@link Read.Unbounded} with {@link UnboundedReadFromBoundedSource}
+   * to force streaming mode.
+   */
+  private static class UnboundedReadFromBoundedSourceOverrideFactory<T>

Review Comment:
   This seems useful as a general thing that could be in `runners-core-construction` FWIW.



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.streaming;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.spark.coders.CoderHelpers;
+import org.apache.beam.runners.spark.util.GlobalWatermarkHolder;
+import org.apache.beam.runners.spark.util.GlobalWatermarkHolder.SparkWatermarks;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.testing.TestStream.ElementEvent;
+import org.apache.beam.sdk.testing.TestStream.ProcessingTimeEvent;
+import org.apache.beam.sdk.testing.TestStream.WatermarkEvent;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.Preconditions;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.JavaSparkContext$;
+import org.apache.spark.rdd.RDD;
+import org.apache.spark.streaming.StreamingContext;
+import org.apache.spark.streaming.Time;
+import org.apache.spark.streaming.dstream.InputDStream;
+import org.joda.time.Instant;
+import scala.Option;
+import scala.reflect.ClassTag;
+
+public class TestDStream<T> extends InputDStream<WindowedValue<T>> {
+  private final Coder<WindowedValue<T>> coder;
+
+  @SuppressFBWarnings("SE_TRANSIENT_FIELD_NOT_RESTORED") // not intended for use after serialization
+  private final transient List<TestStream.Event<T>> events;

Review Comment:
   I think all transient fields should be `@Nullable`. Is checkerframework disabled at the whole package level?



##########
runners/spark/spark_runner.gradle:
##########
@@ -260,21 +265,60 @@ def validatesRunnerStreaming = tasks.register("validatesRunnerStreaming", Test)
   group = "Verification"
   // Disable gradle cache
   outputs.upToDateWhen { false }
-  def pipelineOptions = JsonOutput.toJson([
-    "--runner=TestSparkRunner",
-    "--forceStreaming=true",
-    "--enableSparkMetricSinks=true",
-  ])
-  systemProperty "beamTestPipelineOptions", pipelineOptions
+  systemProperties sparkTestProperties(["--enableSparkMetricSinks": "true", "--forceStreaming": "true"])
 
   classpath = configurations.validatesRunner
-  testClassesDirs += files(project.sourceSets.test.output.classesDirs)
+  testClassesDirs += files(
+    project(":sdks:java:core").sourceSets.test.output.classesDirs,

Review Comment:
   This change makes sense. I don't understand how it worked before. Was it not actually running the VR tests?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java:
##########
@@ -85,7 +85,14 @@ public JavaStreamingContext call() throws Exception {
     EvaluationContext ctxt = new EvaluationContext(jsc, pipeline, options, jssc);
     // update cache candidates
     SparkRunner.updateCacheCandidates(pipeline, translator, ctxt);
-    pipeline.traverseTopologically(new SparkRunner.Evaluator(translator, ctxt));
+    try {
+      pipeline.traverseTopologically(new SparkRunner.Evaluator(translator, ctxt));
+    } catch (RuntimeException e) {
+      jssc.stop(false, false);

Review Comment:
   Identifiers like `jsc` and `jssc` are very unreadable to me who is not doing Spark stuff every day.



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.streaming;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.spark.coders.CoderHelpers;
+import org.apache.beam.runners.spark.util.GlobalWatermarkHolder;
+import org.apache.beam.runners.spark.util.GlobalWatermarkHolder.SparkWatermarks;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.testing.TestStream.ElementEvent;
+import org.apache.beam.sdk.testing.TestStream.ProcessingTimeEvent;
+import org.apache.beam.sdk.testing.TestStream.WatermarkEvent;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.util.Preconditions;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.JavaSparkContext$;
+import org.apache.spark.rdd.RDD;
+import org.apache.spark.streaming.StreamingContext;
+import org.apache.spark.streaming.Time;
+import org.apache.spark.streaming.dstream.InputDStream;
+import org.joda.time.Instant;
+import scala.Option;
+import scala.reflect.ClassTag;
+
+public class TestDStream<T> extends InputDStream<WindowedValue<T>> {
+  private final Coder<WindowedValue<T>> coder;
+
+  @SuppressFBWarnings("SE_TRANSIENT_FIELD_NOT_RESTORED") // not intended for use after serialization
+  private final transient List<TestStream.Event<T>> events;
+
+  private int event = 0;
+
+  private boolean insertEmptyBatch = false;
+
+  private long lastMillis = 0;
+
+  private Instant lastWatermark = Instant.EPOCH;
+
+  public TestDStream(TestStream<T> test, StreamingContext ssc) {
+    super(ssc, classTag());
+    this.coder = WindowedValue.getFullCoder(test.getValueCoder(), GlobalWindow.Coder.INSTANCE);
+    this.events = test.getEvents();
+  }
+
+  @Override
+  public Option<RDD<WindowedValue<T>>> compute(Time validTime) {
+    Preconditions.checkStateNotNull(events);
+
+    TestStream.Event<T> event = insertEmptyBatch ? null : nextEvent();

Review Comment:
   Confusing with `this.event`. I would rename `this.event` to `this.currentEventIndex` or something.



-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org