[
https://issues.apache.org/jira/browse/BEAM-3304?focusedWorklogId=680854&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-680854
]
ASF GitHub Bot logged work on BEAM-3304:
----------------------------------------
Author: ASF GitHub Bot
Created on: 12/Nov/21 15:18
Start Date: 12/Nov/21 15:18
Worklog Time Spent: 10m
Work Description: riteshghorse commented on a change in pull request
#15952:
URL: https://github.com/apache/beam/pull/15952#discussion_r748369532
##########
File path: sdks/go/pkg/beam/core/graph/window/trigger/trigger.go
##########
@@ -138,49 +143,114 @@ func (tr Trigger) AlignedTo(period time.Duration, offset
time.Time) Trigger {
// TODO: Change to call UnixMilli() once we move to only
supporting a go version > 1.17.
offsetMillis = offset.Unix()*1e3 +
int64(offset.Nanosecond())/1e6
}
- tr.TimestampTransforms = append(tr.TimestampTransforms,
AlignToTransform{
+ tr.timestampTransforms = append(tr.timestampTransforms,
AlignToTransform{
Period: int64(period / time.Millisecond),
Offset: offsetMillis,
})
return tr
}
+// RepeatTrigger fires a sub-trigger repeatedly.
+type RepeatTrigger struct {
+ subTrigger Trigger
+}
+
+func (t RepeatTrigger) trigger() {}
+
+// SubTrigger returns the trigger to be repeated.
+func (t *RepeatTrigger) SubTrigger() Trigger {
+ return t.subTrigger
+}
+
// Repeat constructs a trigger that fires a trigger repeatedly
// once the condition has been met.
//
// Ex: trigger.Repeat(trigger.AfterCount(1)) is same as trigger.Always().
func Repeat(tr Trigger) Trigger {
- return Trigger{Kind: RepeatTrigger, SubTriggers: []Trigger{tr}}
+ return &RepeatTrigger{subTrigger: tr}
+}
+
+// AfterEndOfWindowTrigger provides option to set triggers for early and late
firing.
+type AfterEndOfWindowTrigger struct {
+ earlyFiring Trigger
+ lateFiring Trigger
+}
+
+func (t AfterEndOfWindowTrigger) trigger() {}
+
+// EarlyTrigger returns the Early Firing trigger for AfterEndOfWindowTrigger.
+func (t *AfterEndOfWindowTrigger) EarlyTrigger() Trigger {
Review comment:
Makes sense.
##########
File path: sdks/go/pkg/beam/core/graph/window/trigger/trigger.go
##########
@@ -65,59 +119,13 @@ type AlignToTransform struct {
func (AlignToTransform) timestampTransform() {}
-const (
- DefaultTrigger string = "Trigger_Default_"
- AlwaysTrigger string = "Trigger_Always_"
- AfterAnyTrigger string = "Trigger_AfterAny_"
- AfterAllTrigger string = "Trigger_AfterAll_"
- AfterProcessingTimeTrigger string =
"Trigger_AfterProcessing_Time_"
- ElementCountTrigger string = "Trigger_ElementCount_"
- AfterEndOfWindowTrigger string =
"Trigger_AfterEndOfWindow_"
- RepeatTrigger string = "Trigger_Repeat_"
- OrFinallyTrigger string = "Trigger_OrFinally_"
- NeverTrigger string = "Trigger_Never_"
- AfterSynchronizedProcessingTimeTrigger string =
"Trigger_AfterSynchronizedProcessingTime_"
-)
-
-// Default constructs a default trigger that fires once after the end of
window.
-// Late Data is discarded.
-func Default() Trigger {
- return Trigger{Kind: DefaultTrigger}
-}
-
-// Always constructs a trigger that fires immediately
-// whenever an element is received.
-//
-// Equivalent to trigger.Repeat(trigger.AfterCount(1))
-func Always() Trigger {
- return Trigger{Kind: AlwaysTrigger}
-}
-
-// AfterCount constructs a trigger that fires after
-// at least `count` number of elements are processed.
-func AfterCount(count int32) Trigger {
- return Trigger{Kind: ElementCountTrigger, ElementCount: count}
-}
-
-// AfterProcessingTime constructs a trigger that fires relative to
-// when input first arrives.
-//
-// Must be configured with calls to PlusDelay, or AlignedTo. May be
-// configured with additional delay.
-func AfterProcessingTime() Trigger {
- return Trigger{Kind: AfterProcessingTimeTrigger}
-}
-
// PlusDelay configures an AfterProcessingTime trigger to fire after a
specified delay,
// no smaller than a millisecond.
-func (tr Trigger) PlusDelay(delay time.Duration) Trigger {
- if tr.Kind != AfterProcessingTimeTrigger {
- panic(fmt.Errorf("can't apply processing delay to %s, want:
AfterProcessingTimeTrigger", tr.Kind))
- }
+func (tr *AfterProcessingTimeTrigger) PlusDelay(delay time.Duration) Trigger {
Review comment:
Got it.
--
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: 680854)
Time Spent: 14h 10m (was: 14h)
> 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
> Priority: P3
> Time Spent: 14h 10m
> 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.20.1#820001)