-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Support VR test including TestStream for Spark runner in streaming mode #22620
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
690b75a
Support VR test including TestStream for Spark runner in streaming mo…
9eaf52e
Review feedback
480560c
Detect batch/streaming mode based on Pipeline in TestSparkRunner
b35dbaa
Don't force streaming by replacing bounded sources with unbounded rea…
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
142 changes: 142 additions & 0 deletions
142
.../spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,142 @@ | ||
| /* | ||
| * 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 @Nullable List<TestStream.Event<T>> events; | ||
|
|
||
| private int currentEventIndex = 0; | ||
|
|
||
| private boolean insertEmptyBatch = false; | ||
|
|
||
| private long lastValidTimeMs = 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) { | ||
| TestStream.Event<T> event = insertEmptyBatch ? null : nextEvent(); | ||
mosche marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| if (event == null) { | ||
| insertEmptyBatch = false; | ||
| waitForLastBatch(validTime); | ||
| return Option.apply(emptyRDD()); | ||
|
|
||
| } else if (event instanceof ElementEvent) { | ||
| waitForLastBatch(validTime); | ||
| return Option.apply(buildRdd((ElementEvent<T>) event)); | ||
|
|
||
| } else if (event instanceof WatermarkEvent) { | ||
| waitForLastBatch(validTime); | ||
| addWatermark(validTime, (WatermarkEvent<T>) event); | ||
| // insert an additional empty batch so watermark can propagate | ||
| insertEmptyBatch = true; | ||
| return Option.apply(emptyRDD()); | ||
|
|
||
| } else if (event instanceof ProcessingTimeEvent) { | ||
| throw new UnsupportedOperationException( | ||
| "Advancing Processing time is not supported by the Spark Runner."); | ||
| } else { | ||
| throw new IllegalStateException("Unknown event type " + event); | ||
| } | ||
| } | ||
|
|
||
| private void waitForLastBatch(Time validTime) { | ||
| while (GlobalWatermarkHolder.getLastWatermarkedBatchTime() < lastValidTimeMs) { | ||
| Uninterruptibles.sleepUninterruptibly(10, TimeUnit.MILLISECONDS); | ||
| } | ||
| lastValidTimeMs = validTime.milliseconds(); | ||
| } | ||
|
|
||
| private @Nullable TestStream.Event<T> nextEvent() { | ||
| List<TestStream.Event<T>> events = Preconditions.checkStateNotNull(this.events); | ||
| return events.size() > currentEventIndex ? events.get(currentEventIndex++) : null; | ||
| } | ||
|
|
||
| private void addWatermark(Time time, WatermarkEvent<T> event) { | ||
| SparkWatermarks watermarks = | ||
| new SparkWatermarks(lastWatermark, event.getWatermark(), new Instant(time.milliseconds())); | ||
| lastWatermark = event.getWatermark(); | ||
| GlobalWatermarkHolder.add(id(), watermarks); | ||
| } | ||
|
|
||
| @Override | ||
| public void start() {} | ||
|
|
||
| @Override | ||
| public void stop() {} | ||
|
|
||
| private RDD<WindowedValue<T>> emptyRDD() { | ||
| return ssc().sparkContext().emptyRDD(classTag()); | ||
| } | ||
|
|
||
| private RDD<WindowedValue<T>> buildRdd(ElementEvent<T> event) { | ||
| List<byte[]> binaryData = new ArrayList<>(); | ||
| for (TimestampedValue<T> elem : event.getElements()) { | ||
| WindowedValue<T> wv = | ||
| WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp()); | ||
| binaryData.add(CoderHelpers.toByteArray(wv, coder)); | ||
| } | ||
|
|
||
| return new JavaSparkContext(ssc().sparkContext()) | ||
| .parallelize(binaryData) | ||
| .map(CoderHelpers.fromByteFunction(coder)) | ||
| .rdd(); | ||
| } | ||
|
|
||
| private static <T> ClassTag<WindowedValue<T>> classTag() { | ||
| return JavaSparkContext$.MODULE$.fakeClassTag(); | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This change makes sense. I don't understand how it worked before. Was it not actually running the VR tests?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, in fact VR test were never run/supported for Spark in streaming mode. I just stumbled on this accidentally when I started looking into bugs related to onWindowExpiration :(
Instead there was some custom tests in the module that try to mimic the VR test, but they only cover a very small part (and also run as unit tests).