[Prism] Fix an issue on pane info being overwritten by different bundles.#36188
Conversation
|
Also ran Java validatesRunner locally and all tests passed. |
|
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). |
|
For a given post aggregation stage, only one bundle should process a given key+window at a time. So one pane should be enough... |
|
Ah right but for triggers we have a global state... Hmmm. Todo to unify but otherwise this should be fine. |
|
What do you prefer to unify that? Alternatively, we can put bundlePanes into stateData. beam/sdks/go/pkg/beam/runners/prism/internal/engine/data.go Lines 38 to 44 in 06ed09a Given we already had that nested structure in stageState |
Thinking this through... A trigger fires, and we compute a pane, and we start a bundle for it. Bundle contains that key so we only have that bundle running with that key. But since this is post aggregation (Aggregations which is inherently stateful, but also never done SDK side), there's no real "stateful" restriction for the key+window on the output here, unless the post-GBK transform is stateful itself. The pane needs to be preserved downstream of these sequences since the GBK is implemented as an outside of EM transform, so we can propagate it down to the post GBK transform... But at no point, are we "forking" the pane lineage, since as a state, the pane is pretty moot. Your implementation's idea is correct then I think. |
| state map[LinkID]map[typex.Window]map[string]StateData // state data for this stage, from {tid, stateID} -> window -> userKey | ||
| stateTypeLen map[LinkID]func([]byte) int // map from state to a function that will produce the total length of a single value in bytes. | ||
| bundlesToInject []RunBundle // bundlesToInject are triggered bundles that will be injected by the watermark loop to avoid premature pipeline termination. | ||
| bundlePanes map[string]map[typex.Window]map[string]typex.PaneInfo // PanInfo snapshot for bundles, from BundleID -> window -> userKey |
There was a problem hiding this comment.
| bundlePanes map[string]map[typex.Window]map[string]typex.PaneInfo // PanInfo snapshot for bundles, from BundleID -> window -> userKey | |
| bundlePanes map[string]map[typex.Window]map[string]typex.PaneInfo // PaneInfo snapshot for bundles, from BundleID -> window -> userKey |
| if _, ok := ss.kind.(*aggregateStageKind); ok { | ||
| // For aggregate stage, buildEventTimeBundle may have saved bundle panes with empty string as key. | ||
| // Move it under the correct BundleID now. | ||
| ss.bundlePanes[bundID] = ss.bundlePanes[""] |
There was a problem hiding this comment.
We could probably avoid the sneaky empty-string handling by returning a panesInBundle map from buildEventTimeBundle, like we are doing for holdsInBundle. Similar "refused bequests" to non-stateful stages needing to return an empty holds map or empty newKeys.
In prism, each stage state only stores the latest pane info.
beam/sdks/go/pkg/beam/runners/prism/internal/engine/data.go
Lines 38 to 44 in 117042c
In the some scenarios, that's not enough. For example, when we have multiple injected bundles, and each call
computeNextTriggeredPane, the latter pane will overwrite the previous one.beam/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go
Lines 1335 to 1345 in 117042c
Here is the code to reproduce:
Running on the current code, we can see both pane info are the same for these two elements.
The correct output should be:
This PR fixes the above issue by saving pane info for bundles if needed.