[BEAM-920] Add support for Watermarks in the Spark runner.#1987
[BEAM-920] Add support for Watermarks in the Spark runner.#1987amitsela wants to merge 13 commits intoapache:masterfrom
Conversation
|
This PR is the first of 2-3 PRs that will complete BEAM-920 to support the Beam model for streaming. This PR adds support to |
|
R: @kennknowles |
|
CC : @jbonofre |
|
Refer to this link for build results (access rights to CI server needed): |
|
|
||
| p.run().waitUntilFinish(Duration.millis(options.getBatchIntervalMillis()).multipliedBy(3)); | ||
|
|
||
| System.out.println(WatermarksDoFn.strings); |
|
|
||
| @ProcessElement | ||
| public void processElement(ProcessContext c) { | ||
| if (GlobalWatermarkHolder.get() == null |
There was a problem hiding this comment.
This should be able to be replaced by TestStream + triggers, right?
There was a problem hiding this comment.
Yes, next PR, lining it up as we speak.
| Queue<SparkWatermarks> timesQueue = en.getValue(); | ||
|
|
||
| // current state, if exists. | ||
| Instant currentLowWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; |
There was a problem hiding this comment.
This logic seems about right. I admit that I have forgotten some of the details of our deep dive on watermarks.
There was a problem hiding this comment.
I've updated the design doc: https://docs.google.com/document/d/1bOu0-g7Z1LVG0kYEQ38GDWaFjFl45u-2aZV87RmlLKI/edit?usp=sharing
| timesQueue = new ConcurrentLinkedQueue<>(); | ||
| } | ||
| timesQueue.offer(sparkWatermarks); | ||
| sourceTimes.put(sourceId, timesQueue); |
There was a problem hiding this comment.
I see that the watermarks for unbounded sources are tracked here. What I don't quite follow is the behavior in fan-in situations where the watermark is the min of the inputs.
There was a problem hiding this comment.
So in cases such as Flatten where the WM is the min{WM1, WM2} it is up to whoever uses WMs to deduce that (such as GABW implementation).
This would be handled by SparkTimerInternals.
This WM mechanism only reflects the current-batch lo/hi WMs per source. Adding to that the information on upstream/origin sources to a PCollection is supposed to provide all necessary information to find the right WM for the computation, so for example a GABW following a Flatten would have to choose the Min{WM...}.
This is implemented in the next PR as well since it's related to triggers and generally a part of GABW implementation.
There was a problem hiding this comment.
See here in the branch I'm preparing for the next PR: https://github.com/amitsela/beam/blob/triggers-and-panes/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java#L66
|
@kennknowles that's sort of the trade-off between gradual PRing with (fairly) small changes or one huge PR. |
…ould be revisited there.
|
Rebased and removed leftover printouts. |
|
Refer to this link for build results (access rights to CI server needed): Build result: FAILURE[...truncated 12098 lines...] at hudson.remoting.UserRequest.perform(UserRequest.java:153) at hudson.remoting.UserRequest.perform(UserRequest.java:50) at hudson.remoting.Request$2.run(Request.java:332) at hudson.remoting.InterceptingExecutorService$1.call(InterceptingExecutorService.java:68) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745)Caused by: org.apache.maven.plugin.MojoFailureException: You have 1 Checkstyle violation. at org.apache.maven.plugin.checkstyle.CheckstyleViolationCheckMojo.execute(CheckstyleViolationCheckMojo.java:588) at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:134) at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:208) ... 31 more2017-02-18T21:40:39.238 [ERROR] 2017-02-18T21:40:39.238 [ERROR] Re-run Maven using the -X switch to enable full debug logging.2017-02-18T21:40:39.238 [ERROR] 2017-02-18T21:40:39.238 [ERROR] For more information about the errors and possible solutions, please read the following articles:2017-02-18T21:40:39.238 [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException2017-02-18T21:40:39.239 [ERROR] 2017-02-18T21:40:39.239 [ERROR] After correcting the problems, you can resume the build with the command2017-02-18T21:40:39.239 [ERROR] mvn -rf :beam-runners-sparkchannel stoppedSetting status of c2415a0 to FAILURE with url https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/7583/ and message: 'Build finished. 'Using context: Jenkins: Maven clean install--none-- |
|
retest this please. |
|
Refer to this link for build results (access rights to CI server needed): Failed Tests: 1beam_PreCommit_Java_MavenInstall/org.apache.beam:beam-runners-spark: 1--none-- |
|
retest this please. |
|
Refer to this link for build results (access rights to CI server needed): |
|
Test flake was weird, but I don't see a point in investing too much here since this test specifically is to be removed in the next PR where Watermarks are part of "TestStream-like" test pipelines. |
kennknowles
left a comment
There was a problem hiding this comment.
OK, LGTM. I understand that these things are going to have more changes as they develop, anyhow, so let's keep momentum. Sorry it took so long to grok this.
|
Added a fix to clear the state of singletons BEFORE pipeline execution via |
|
Refer to this link for build results (access rights to CI server needed): |
Be sure to do all of the following to help us incorporate your contribution
quickly and easily:
[BEAM-<Jira issue #>] Description of pull requestmvn clean verify. (Even better, enableTravis-CI on your fork and ensure the whole test matrix passes).
<Jira issue #>in the title with the actual Jira issuenumber, if there is one.
Individual Contributor License Agreement.