-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[Prism] Support AfterProcessingTime triggers - part 2 #36333
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
[Prism] Support AfterProcessingTime triggers - part 2 #36333
Conversation
| // This is used for processing time timers to ensure the loop re-evaluates | ||
| // stages when a processing time timer is expected to fire. | ||
| func (em *ElementManager) wakeUpAt(t mtime.Time) { | ||
| if em.testStreamHandler == nil && em.config.EnableRTC { |
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.
Addressed https://github.com/apache/beam/pull/36069/files#r2342213066.
We should also only be creating this Goroutine when the Real Time Clock is being used, and testStreamHandler is nil. Otherwise behavior will be additionally wonky.
| }) | ||
| } | ||
|
|
||
| func (*ordinaryStageKind) buildProcessingTimeBundle(ss *stageState, em *ElementManager, emNow mtime.Time) (elementHeap, mtime.Time, set[string], map[mtime.Time]int, bool) { |
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.
Addressed https://github.com/apache/beam/pull/36069/files#r2342012446
Feel free to drop the named return values, since we aren't using them here...
| slog.Debug("started a processing time bundle", "stageID", ss.ID, "bundleID", bundID, "size", len(toProcess), "emNow", emNow) | ||
| return bundID, true, stillSchedulable | ||
| func (*statefulStageKind) buildProcessingTimeBundle(ss *stageState, em *ElementManager, emNow mtime.Time) (elementHeap, mtime.Time, set[string], map[mtime.Time]int, bool) { | ||
| return handleProcessingTimeTimer(ss, em, emNow, func(e element, toProcess []element, holdsInBundle map[mtime.Time]int) { |
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.
Addressed https://github.com/apache/beam/pull/36069/files#r2342029890
Correct me if I'm wrong, I think this is the only difference preventing us from a single shared method between the general stateful handler and the aggregation handler?
(That is, a single method on ss.state that both kinds call)
…kinds into one code path.
fe01a3d to
b3cd953
Compare
|
Assigning reviewers: R: @jrmccluskey for label go. Note: If you would like to opt out of this review, comment Available commands:
The PR bot will only process comments in the main thread (not review comments). |
lostluck
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.
Are you leaving non-Go tests to another PR?
Are there any Java conformance tests that start passing now that this is in?
Yes. I tried some commented Java tests and it seems that they need AfterSynchronizedProcessingTime trigger to run. For Python ones, we will need to clear up some tests later. |
|
Well, the goodnews is I'm not sure there's any "work" to be done here for AfterSynchronizedProcessingTime, since it depends on all the workers to have caught up to the first element's arrival. But in principle, prism is always just a single worker (there can be multiple SDK stacks, but they're not workers per-se), but the "notion" of processing time across workers is probably a no-op in the current generation of Prism as a single Node local runner. |
So in the current setting of prism, it sounds like the AfterSynchronizedProcessingTime is a AftetCount(1)? I see it is a subclass of OnceTrigger in the Java doc, so it is only fired once. |
|
I think so yes. It's a trigger only implemented in Java (and Go i guess...) IIRC and labeled as an implementation detail too. It's just unfortunately externalized too much so the internality isn't maintained well. Beam's cross worker communication doesn't report a machine's local wall time, so at present there's no way to do this. To implement it properly requires the runner to manage its own machines/worker side communications better, like how Dataflow has runnerv2 for worker work acquisition. |
8c0e5ca to
d6ccb90
Compare
|
Thanks! The fact that some test pipelines are stuck if there is no watermark advancing is truly a problem. I am looking into another approach to solve that. Basically, we will change PTAL. I am running tests to see if the idea works. If it works, we will no longer need TestStream.advanceWatermark calls in our processing-time trigger tests. |
| // ss.bundlesToInject = append(ss.bundlesToInject, rb) | ||
| // Bundle is marked in progress here to prevent a race condition. | ||
| em.refreshCond.L.Lock() | ||
| em.injectedBundles = append(em.injectedBundles, rb) |
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.
I use em.injectedBundles here, because the stage-wise bundlesToInject is causing the pipeline stuck. This needs to be consolidated in the future PR.
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.
If using this mechanism also add "em.inprogressBundles.insert(rb.BundleID)" to avoid "losing" the bundle. injectedBundles are considered "inprogress" even when they aren't yet executing. This is to avoid incorrect premature terminations.
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.
Very odd that there's a stuckness going on with bundlesToInject. Not sure why that would be offhand.
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.
If using this mechanism also add "em.inprogressBundles.insert(rb.BundleID)" to avoid "losing" the bundle. injectedBundles are considered "inprogress" even when they aren't yet executing. This is to avoid incorrect premature terminations.
Yes, the line you mentioned is already there.
| if isOrdinaryStage && len(ss.sides) == 0 { | ||
| // For ordinary stage with no side inputs, we use whether there are pending elements to determine | ||
| // whether a bundle is ready or not. | ||
| if len(ss.pending) == 0 { |
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.
Use watermark to determine whether to create a bundle for aggregation and stateful stages, but not stateless stages without sideinputs.
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.
Codecov Report❌ Patch coverage is Additional details and impacted files@@ Coverage Diff @@
## master #36333 +/- ##
============================================
- Coverage 56.84% 56.84% -0.01%
Complexity 3386 3386
============================================
Files 1220 1220
Lines 185898 186056 +158
Branches 3523 3523
============================================
+ Hits 105672 105757 +85
- Misses 76885 76942 +57
- Partials 3341 3357 +16
Flags with carried forward coverage won't be shown. Click here to find out more. ☔ View full report in Codecov by Sentry. 🚀 New features to boost your workflow:
|
923dea8 to
28b2caf
Compare
lostluck
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.
LGTM, thanks for all the detailed work!
| // ss.bundlesToInject = append(ss.bundlesToInject, rb) | ||
| // Bundle is marked in progress here to prevent a race condition. | ||
| em.refreshCond.L.Lock() | ||
| em.injectedBundles = append(em.injectedBundles, rb) |
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.
If using this mechanism also add "em.inprogressBundles.insert(rb.BundleID)" to avoid "losing" the bundle. injectedBundles are considered "inprogress" even when they aren't yet executing. This is to avoid incorrect premature terminations.
| // ss.bundlesToInject = append(ss.bundlesToInject, rb) | ||
| // Bundle is marked in progress here to prevent a race condition. | ||
| em.refreshCond.L.Lock() | ||
| em.injectedBundles = append(em.injectedBundles, rb) |
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.
Very odd that there's a stuckness going on with bundlesToInject. Not sure why that would be offhand.
| // TriggerAfterProcessingTimeNotTriggered tests the AfterProcessingTime Trigger. It won't fire because 't' processing time is not reached | ||
| // Not yet supported by the flink runner: | ||
| // java.lang.UnsupportedOperationException: Advancing Processing time is not supported by the Flink Runner. | ||
| func TriggerAfterProcessingTimeNotTriggered(s beam.Scope) { |
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.
Also add a Test wrapper to
So this can get executed in the other runner suites, even if it's currently getting filtered out, it's important to not "miss" it. That's also where the "test name" for it gets added so it's visible in the lists at
https://github.com/apache/beam/blob/master/sdks/go/test/integration/integration.go#L166
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.
Why do we skip the trigger tests in prism?
beam/sdks/go/test/integration/integration.go
Line 150 in 7c88161
| "TestTrigger.*", |
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.
It was easier/preferred to add them to the prism test suite (in the no longer correctly named unimplemented_test.go) than to unblock this. I think we can probably remove this skip soon, after your work here is done.
Something something re-org.
sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go
Outdated
Show resolved
Hide resolved
sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go
Outdated
Show resolved
Hide resolved
|
FYI this PR is causing ML tests to start failing - https://github.com/apache/beam/actions/workflows/beam_PreCommit_Python_ML.yml?query=branch%3Amaster The test is wrong, and I have #36377 to fix it |
This PR is a follow-up to #36126. The majority of this PR has been reviewed in #36069, and I've added comments to the code to indicate where the feedback has been addressed.
Part of the work to support AfterProcessingTime trigger (#31438)