github.com/apache/beam/sdks/v2@v2.48.2/go/examples/snippets/09triggers.go (about) 1 // Licensed to the Apache Software Foundation (ASF) under one or more 2 // contributor license agreements. See the NOTICE file distributed with 3 // this work for additional information regarding copyright ownership. 4 // The ASF licenses this file to You under the Apache License, Version 2.0 5 // (the "License"); you may not use this file except in compliance with 6 // the License. You may obtain a copy of the License at 7 // 8 // http://www.apache.org/licenses/LICENSE-2.0 9 // 10 // Unless required by applicable law or agreed to in writing, software 11 // distributed under the License is distributed on an "AS IS" BASIS, 12 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 // See the License for the specific language governing permissions and 14 // limitations under the License. 15 16 // Package snippets contains code used in the Beam Programming Guide 17 // as examples for the Apache Beam Go SDK. These snippets are compiled 18 // and their tests run to ensure correctness. However, due to their 19 // piecemeal pedagogical use, they may not be the best example of 20 // production code. 21 // 22 // The Beam Programming Guide can be found at https://beam.apache.org/documentation/programming-guide/. 23 package snippets 24 25 import ( 26 "time" 27 28 "github.com/apache/beam/sdks/v2/go/pkg/beam" 29 "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window" 30 "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window/trigger" 31 ) 32 33 func TriggerAfterEndOfWindow(s beam.Scope, pCollection beam.PCollection) { 34 // [START after_window_trigger] 35 trigger := trigger.AfterEndOfWindow(). 36 EarlyFiring(trigger.AfterProcessingTime(). 37 PlusDelay(60 * time.Second)). 38 LateFiring(trigger.Repeat(trigger.AfterCount(1))) 39 // [END after_window_trigger] 40 beam.WindowInto(s, window.NewFixedWindows(10*time.Second), pCollection, beam.Trigger(trigger), beam.PanesDiscard()) 41 } 42 43 func TriggerAlways(s beam.Scope, pCollection beam.PCollection) { 44 // [START always_trigger] 45 beam.WindowInto(s, window.NewFixedWindows(10*time.Second), pCollection, 46 beam.Trigger(trigger.Always()), 47 beam.PanesDiscard(), 48 ) 49 // [END always_trigger] 50 } 51 52 func ComplexTriggers(s beam.Scope, pcollection beam.PCollection) { 53 // [START setting_a_trigger] 54 windowedItems := beam.WindowInto(s, 55 window.NewFixedWindows(1*time.Minute), pcollection, 56 beam.Trigger(trigger.AfterProcessingTime(). 57 PlusDelay(1*time.Minute)), 58 beam.AllowedLateness(30*time.Minute), 59 beam.PanesDiscard(), 60 ) 61 // [END setting_a_trigger] 62 63 // [START setting_allowed_lateness] 64 allowedToBeLateItems := beam.WindowInto(s, 65 window.NewFixedWindows(1*time.Minute), pcollection, 66 beam.Trigger(trigger.AfterProcessingTime(). 67 PlusDelay(1*time.Minute)), 68 beam.AllowedLateness(30*time.Minute), 69 ) 70 // [END setting_allowed_lateness] 71 72 // [START model_composite_triggers] 73 compositeTriggerItems := beam.WindowInto(s, 74 window.NewFixedWindows(1*time.Minute), pcollection, 75 beam.Trigger(trigger.AfterEndOfWindow(). 76 LateFiring(trigger.AfterProcessingTime(). 77 PlusDelay(10*time.Minute))), 78 beam.AllowedLateness(2*24*time.Hour), 79 ) 80 // [END model_composite_triggers] 81 82 // TODO(BEAM-3304) AfterAny is not yet implemented. 83 // Implement so the following compiles when no longer commented out. 84 85 // [START other_composite_trigger] 86 // beam.Trigger( 87 // trigger.TriggerAfterAny( 88 // trigger.AfterCount(100), 89 // trigger.AfterProcessingTime(). 90 // PlusDelay(1*time.Minute)), 91 // ) 92 // [END other_composite_trigger] 93 94 _ = []beam.PCollection{ 95 windowedItems, 96 allowedToBeLateItems, 97 compositeTriggerItems, 98 } 99 }