-
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
Conversation
|
Run Spark ValidatesRunner |
|
Unfortunately I'm stuck with some flaky tests. It looks like watermarks are not advanced in a deterministic way. @aromanenko-dev @echauchot if you have some time I'd be more than grateful for a 2nd pair of 👀 . Successful run (watermark advanced early enough, so that timer is triggered and the only element is emitted): Failed run (watermark is advanced too late, timer doesn't trigger and element is lost): |
|
Assigning reviewers. If you would like to opt out of this review, comment R: @lukecwik for label java. Available commands:
The PR bot will only process comments in the main thread (not review comments). |
fed3793 to
d6cfb9a
Compare
|
Run Spark ValidatesRunner |
2 similar comments
|
Run Spark ValidatesRunner |
|
Run Spark ValidatesRunner |
| @@ -105,6 +105,9 @@ public void testLateDataAccumulating() { | |||
| .advanceWatermarkTo(instant.plus(Duration.standardMinutes(6))) | |||
| // These elements are late but within the allowed lateness | |||
| .addElements(TimestampedValue.of(4L, instant), TimestampedValue.of(5L, instant)) | |||
| .advanceWatermarkTo(instant.plus(Duration.standardMinutes(10))) | |||
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.
@kennknowles Maybe you could answer this? I'm wondering if this is an issue of the Spark streaming runner (and how this is handled by other runners) or if it's a lack of my own understanding.
Without advancing the watermark once more the (lower) input watermark remains at 6 mins, but data in [0,5 min) won't be considered late until it passes 10 mins.
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.
Just responding to let you know I have been on vacation and I will look at this later today.
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.
Thanks a lot! I'm off as well for a bit, so no rush on this.
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.
@kennknowles Finally back to this, if you could have a look it would be great:)
d6cfb9a to
e50fec9
Compare
|
Run Spark ValidatesRunner |
|
Run Java PreCommit |
|
Reminder, please take a look at this pr: @lukecwik |
|
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment R: @kennknowles for label java. Available commands:
|
|
Reminder, please take a look at this pr: @kennknowles |
|
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment R: @lukecwik for label java. Available commands:
|
|
Reminder, please take a look at this pr: @lukecwik |
kennknowles
left a comment
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 is super useful! Thank you!
runners/spark/spark_runner.gradle
Outdated
|
|
||
| classpath = configurations.validatesRunner | ||
| testClassesDirs += files(project.sourceSets.test.output.classesDirs) | ||
| testClassesDirs += files( | ||
| project(":sdks:java:core").sourceSets.test.output.classesDirs, |
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).
| * Override factory to replace {@link Read.Unbounded} with {@link UnboundedReadFromBoundedSource} | ||
| * to force streaming mode. | ||
| */ | ||
| private static class UnboundedReadFromBoundedSourceOverrideFactory<T> |
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 seems useful as a general thing that could be in runners-core-construction FWIW.
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.
Happy to do that though I'm not entirely sure if the factory is of much value by itself. I also had to fix the outputs in a non trivial way using a visitor after the replacement, see https://github.com/apache/beam/pull/22620/files#diff-d81f49eb0330230bd03ce6cd33b5f70f59c443aac57741e877ececbada32b16bR246-R274. I couldn't find a way to achieve this in mapOutputs of the override factory itself.
Let me know what you think.
.../org/apache/beam/runners/spark/translation/streaming/SparkRunnerStreamingContextFactory.java
Outdated
Show resolved
Hide resolved
...ers/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java
Outdated
Show resolved
Hide resolved
...ers/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java
Outdated
Show resolved
Hide resolved
sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
Outdated
Show resolved
Hide resolved
|
Run Spark ValidatesRunner |
|
@kennknowles FYI, a significant number of VR tests are constantly failing here. If I run them independently they usually succeed. It looks like there's some indeterminism around watermark propagation in the runner, see #23129. |
e50fec9 to
9eaf52e
Compare
|
I don't know if anyone currently around would be familiar with SparkRunner watermark propagation. |
|
I think it is valuable to get these tests running and disable them. The test and list of disabled tests can be a real representation of the current state. That way things that are green can stay green. |
|
run spark validatesrunner |
|
Run Spark ValidatesRunner |
79c1953 to
b35dbaa
Compare
|
Run Spark ValidatesRunner |
|
I took a bit of a turn here after validating my initial approach replacing bounded sources with
The initial reason for this approach was to prevent the Spark runner from failing when streaming was forced via pipeline options in VR tests for bounded test cases: Spark refuses to start if there's no streaming workload scheduled. Unfortunately, this hides any watermark issues uncovered above as VR tests succeed. |
In the Beam model, this condition is that a GroupByKey of an unbounded PCollection in global window must have a trigger. But you can still have a bounded PCollection in streaming mode. So the summary is:
|
|
@kennknowles fine to merge this? |
|
Reminder, please take a look at this pr: @lukecwik |
|
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment R: @kennknowles for label java. Available commands:
|
|
@kennknowles kind ping, are you ok to merge it? |
Run VR tests for Spark streaming runner rather than custom tests (test are already run as part of the "normal" unit test run).
If
forceStreamingis set totrue, theTestSparkRunnerwill replaceRead.BoundedwithUnboundedReadFromBoundedSourceso tests are run in streaming mode.Additionally this PR adds support for
TestStream.Closes #22472
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
R: @username).addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, commentfixes #<ISSUE NUMBER>instead.CHANGES.mdwith noteworthy changes.See the Contributor Guide for more tips on how to make review process smoother.
To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md
GitHub Actions Tests Status (on master branch)
See CI.md for more information about GitHub Actions CI.