[
https://issues.apache.org/jira/browse/BEAM-3304?focusedWorklogId=644979&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-644979
]
ASF GitHub Bot logged work on BEAM-3304:
----------------------------------------
Author: ASF GitHub Bot
Created on: 01/Sep/21 09:23
Start Date: 01/Sep/21 09:23
Worklog Time Spent: 10m
Work Description: lostluck commented on a change in pull request #15430:
URL: https://github.com/apache/beam/pull/15430#discussion_r699718391
##########
File path: sdks/go/pkg/beam/core/graph/window/trigger.go
##########
@@ -35,3 +35,38 @@ const (
NeverTrigger string = "Trigger_Never_"
AfterSynchronizedProcessingTimeTrigger string =
"Trigger_AfterSynchronizedProcessingTime_"
)
+
+// TriggerDefault constructs a default trigger that fires after the end of
window.
+// No provision for late arriving data.
Review comment:
Does this mean it discards late data, or is it unspecified? Does that
mean it fires once?
##########
File path: sdks/go/test/integration/primitives/windowinto.go
##########
@@ -121,7 +128,21 @@ func TriggerAlways(s beam.Scope) {
col := teststream.Create(s, con)
windowSize := 10 * time.Second
- validate(s.Scope("Fixed"), window.NewFixedWindows(windowSize), col,
window.Trigger{Kind: window.AlwaysTrigger}, window.Discarding, 1.0, 2.0, 3.0)
+ validateEquals(s.Scope("Fixed"), window.NewFixedWindows(windowSize),
col, window.TriggerAlways(), window.Discarding, 1.0, 2.0, 3.0)
+}
+
+func validateCount(s beam.Scope, wfn *window.Fn, in beam.PCollection, tr
window.Trigger, m window.AccumulationMode, expected int) {
+ var accumulationMode beam.AccumulationMode
+ switch m {
+ case window.Accumulating:
+ accumulationMode = beam.PanesAccumulate()
+ case window.Discarding:
+ accumulationMode = beam.PanesDiscard()
+ }
Review comment:
Same comment here.
##########
File path: sdks/go/test/integration/primitives/windowinto.go
##########
@@ -110,7 +117,7 @@ func TriggerDefault(s beam.Scope) {
col := teststream.Create(s, con)
windowSize := 10 * time.Second
- validate(s.Scope("Fixed"), window.NewFixedWindows(windowSize), col,
window.Trigger{Kind: window.DefaultTrigger}, window.Accumulating, 6.0, 9.0)
+ validateEquals(s.Scope("Fixed"), window.NewFixedWindows(windowSize),
col, window.TriggerDefault(), window.Discarding, 6.0, 9.0)
Review comment:
Why did the accumulation mode change from Accumulating to Discarding
here?
##########
File path: sdks/go/test/integration/primitives/windowinto.go
##########
@@ -93,8 +93,15 @@ func WindowSums_Lifted(s beam.Scope) {
WindowSums(s.Scope("Lifted"), stats.SumPerKey)
}
-func validate(s beam.Scope, wfn *window.Fn, in beam.PCollection, tr
window.Trigger, m window.AccumulationMode, expected ...interface{}) {
- windowed := beam.WindowInto(s, wfn, in, beam.WindowTrigger{Name: tr},
beam.AccumulationMode{Mode: m})
+func validateEquals(s beam.Scope, wfn *window.Fn, in beam.PCollection, tr
window.Trigger, m window.AccumulationMode, expected ...interface{}) {
+ var accumulationMode beam.AccumulationMode
+ switch m {
+ case window.Accumulating:
+ accumulationMode = beam.PanesAccumulate()
+ case window.Discarding:
+ accumulationMode = beam.PanesDiscard()
+ }
Review comment:
Consider instead changing `m` to be `beam.AccumulationMode` and just
passing in the argument as before, rather than delegating to a switch, etc.
##########
File path: sdks/go/pkg/beam/core/graph/window/trigger.go
##########
@@ -35,3 +35,38 @@ const (
NeverTrigger string = "Trigger_Never_"
AfterSynchronizedProcessingTimeTrigger string =
"Trigger_AfterSynchronizedProcessingTime_"
)
+
+// TriggerDefault constructs a default trigger that fires after the end of
window.
+// No provision for late arriving data.
+func TriggerDefault() Trigger {
+ return Trigger{Kind: DefaultTrigger}
+}
+
+// TriggerAlways constructs an always trigger that keeps firing immediately
after an element is processed.
+func TriggerAlways() Trigger {
+ return Trigger{Kind: AlwaysTrigger}
+}
+
+// TriggerAfterCount constructs an element count trigger that fires after
atleast `count` number of elements are processed.
+func TriggerAfterCount(count int32) Trigger {
+ return Trigger{Kind: ElementCountTrigger, ElementCount: count}
+}
+
+// TriggerAfterProcessingTime constructs a after processing time trigger that
fires after 'delay' milliseconds of processing time has passed.
Review comment:
```suggestion
// TriggerAfterProcessingTime constructs an after processing time trigger
that fires after 'delay' milliseconds of processing time have passed.
```
##########
File path: sdks/go/pkg/beam/core/graph/window/trigger.go
##########
@@ -35,3 +35,38 @@ const (
NeverTrigger string = "Trigger_Never_"
AfterSynchronizedProcessingTimeTrigger string =
"Trigger_AfterSynchronizedProcessingTime_"
)
+
+// TriggerDefault constructs a default trigger that fires after the end of
window.
+// No provision for late arriving data.
+func TriggerDefault() Trigger {
+ return Trigger{Kind: DefaultTrigger}
+}
+
+// TriggerAlways constructs an always trigger that keeps firing immediately
after an element is processed.
Review comment:
Consider clarifying that this is equivalent to
`TriggerRepeat(TriggerAfterCount(1))`. Does anything change here WRT late data
or not?
##########
File path: sdks/go/pkg/beam/core/graph/window/trigger.go
##########
@@ -35,3 +35,38 @@ const (
NeverTrigger string = "Trigger_Never_"
AfterSynchronizedProcessingTimeTrigger string =
"Trigger_AfterSynchronizedProcessingTime_"
)
+
+// TriggerDefault constructs a default trigger that fires after the end of
window.
+// No provision for late arriving data.
+func TriggerDefault() Trigger {
+ return Trigger{Kind: DefaultTrigger}
+}
+
+// TriggerAlways constructs an always trigger that keeps firing immediately
after an element is processed.
+func TriggerAlways() Trigger {
+ return Trigger{Kind: AlwaysTrigger}
+}
+
+// TriggerAfterCount constructs an element count trigger that fires after
atleast `count` number of elements are processed.
+func TriggerAfterCount(count int32) Trigger {
+ return Trigger{Kind: ElementCountTrigger, ElementCount: count}
+}
+
+// TriggerAfterProcessingTime constructs a after processing time trigger that
fires after 'delay' milliseconds of processing time has passed.
+func TriggerAfterProcessingTime(delay int64) Trigger {
+ return Trigger{Kind: AfterProcessingTimeTrigger, Delay: delay}
+}
+
+// TriggerRepeat constructs a repeat trigger that fires a trigger repeatedly
once the condition has been met.
+// Ex: window.TriggerRepeat(window.TriggerAfterCount(1)) is same as
window.TriggerAlways().
+func TriggerRepeat(tr Trigger) Trigger {
+ return Trigger{Kind: RepeatTrigger, SubTriggers: []Trigger{tr}}
+}
+
+// TriggerAfterEndOfWindow constructs an end of window trigger that is
configurable for early firing trigger(before the end of window)
+// and late firing trigger(after the end of window).
+// As of now, the values of Early firing is set to TriggerDefault and Late
firing is set to TriggerAlways.
+func TriggerAfterEndOfWindow() Trigger {
+ // TODO(BEAM-3304): modify it to take parameters for early and late
firing trigger
+ return Trigger{Kind: AfterEndOfWindowTrigger}
+}
Review comment:
If we're going to change the parameters we shouldn't be putting this in
yet. Adding parameters is a breaking incompatible change to users. We could
however, have this explicit behavior and call the method:
`TriggerAfterEndOfWindowAlways()`, so that way we can maintain the behavior for
any users, and can later add the more general `TriggerAfterEndOfWindow()`
function that takes parameters.
Separately, the proposed configuration, does it mean that there won't be any
early firings?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 644979)
Time Spent: 8.5h (was: 8h 20m)
> Go triggering support
> ---------------------
>
> Key: BEAM-3304
> URL: https://issues.apache.org/jira/browse/BEAM-3304
> Project: Beam
> Issue Type: Improvement
> Components: sdk-go
> Reporter: Henning Rohde
> Assignee: Ritesh Ghorse
> Priority: P3
> Time Spent: 8.5h
> Remaining Estimate: 0h
>
> `Add support for triggers.
> [https://beam.apache.org/documentation/programming-guide/#triggers]
> Triggers are special runner side behavior indicating how to handle data WRT
> the watermark and window. Commonly configuring the processing for “late data”
> and similar.
> These are not currently implemented for user use in the Go SDK. Reshuffle
> configures triggers, but it’s not accessible. A correct trigger
> implementation can at least re-implement Reshuffle in a user pipeline, rather
> than handled specially within the framework.
> * Requires extending the window package to be able to configure the various
> triggers.
> * Specifically being able to compose triggers as also permitted by the proto.
> **
> [https://github.com/apache/beam/blob/6e7b1c44bc7275ee047afc059fd610cd3f4e5bee/model/pipeline/src/main/proto/beam_runner_api.proto#L1111]
>
> * Requires updating the graphx package translate.go to marshal (and
> unmarshal?) the triggers to and from Beam PipelineProto Windowing strategies.
> * Requires supporting triggers with the beam.WindowInto transform for user
> pipeline use as well as complete documentation on its use from the user side.
> **
> [https://github.com/apache/beam/blob/6e7b1c44bc7275ee047afc059fd610cd3f4e5bee/sdks/go/pkg/beam/windowing.go]
>
> * Panes need to be decoded, otherwise triggering will cause runtime errors:
> [https://lists.apache.org/thread.html/r94c42d2d116f6464cd6b689543e5e578edf8310bf7c6e48a0958a56c%40%3Cdev.beam.apache.org%3E]
>
> * Handle pane propagation and observation in the exec package, and in user
> dofns.
> ** Panes indicate whether data was on time or not, and similar facets which
> may be relevant for processing.
> ** Might simply extend the existing window interface.
>
> Similar to windowing, many of the same places as
> https://issues.apache.org/jira/browse/BEAM-11100 need to be modified.
> At simplest though, it's mostly a runner side construction, with less concern
> on the exec side, and generally much simpler.
> Appropriate integration tests against portable runners must be implemented:
> [https://github.com/apache/beam/tree/master/sdks/go/test/integration/primitives]
>
> And optionally add support for the configurable triggers to the the Go Direct
> Runner. However, the results must be compared and validated against a
> semantically correct runner like the python portable runner first. At
> minimum, the Go Direct Runner should be made aware of triggers and produce a
> coherent error whenever there's a trigger it can't deal with.
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)