You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/08/31 22:44:24 UTC

[GitHub] [beam] lostluck commented on a change in pull request #15430: [BEAM-3304] Helper functions for triggers

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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org