@@ -184,6 +184,8 @@ type Config struct {
184184 MaxBundleSize int
185185 // Whether to use real-time clock as processing time
186186 EnableRTC bool
187+ // Whether to process the data in a streaming mode
188+ StreamingMode bool
187189}
188190
189191// ElementManager handles elements, watermarks, and related errata to determine
@@ -1296,6 +1298,43 @@ func (ss *stageState) AddPending(em *ElementManager, newPending []element) int {
12961298 return ss .kind .addPending (ss , em , newPending )
12971299}
12981300
1301+ func (ss * stageState ) injectTriggeredBundlesIfReady (em * ElementManager , window typex.Window , key string ) int {
1302+ // Check on triggers for this key.
1303+ // We use an empty linkID as the key into state for aggregations.
1304+ count := 0
1305+ if ss .state == nil {
1306+ ss .state = make (map [LinkID ]map [typex.Window ]map [string ]StateData )
1307+ }
1308+ lv , ok := ss .state [LinkID {}]
1309+ if ! ok {
1310+ lv = make (map [typex.Window ]map [string ]StateData )
1311+ ss .state [LinkID {}] = lv
1312+ }
1313+ wv , ok := lv [window ]
1314+ if ! ok {
1315+ wv = make (map [string ]StateData )
1316+ lv [window ] = wv
1317+ }
1318+ state := wv [key ]
1319+ endOfWindowReached := window .MaxTimestamp () < ss .input
1320+ ready := ss .strat .IsTriggerReady (triggerInput {
1321+ newElementCount : 1 ,
1322+ endOfWindowReached : endOfWindowReached ,
1323+ }, & state )
1324+
1325+ if ready {
1326+ state .Pane = computeNextTriggeredPane (state .Pane , endOfWindowReached )
1327+ }
1328+ // Store the state as triggers may have changed it.
1329+ ss .state [LinkID {}][window ][key ] = state
1330+
1331+ // If we're ready, it's time to fire!
1332+ if ready {
1333+ count += ss .buildTriggeredBundle (em , key , window )
1334+ }
1335+ return count
1336+ }
1337+
12991338// addPending for aggregate stages behaves likes stateful stages, but don't need to handle timers or a separate window
13001339// expiration condition.
13011340func (* aggregateStageKind ) addPending (ss * stageState , em * ElementManager , newPending []element ) int {
@@ -1315,6 +1354,13 @@ func (*aggregateStageKind) addPending(ss *stageState, em *ElementManager, newPen
13151354 if ss .pendingByKeys == nil {
13161355 ss .pendingByKeys = map [string ]* dataAndTimers {}
13171356 }
1357+
1358+ type windowKey struct {
1359+ window typex.Window
1360+ key string
1361+ }
1362+ pendingWindowKeys := set [windowKey ]{}
1363+
13181364 count := 0
13191365 for _ , e := range newPending {
13201366 count ++
@@ -1327,37 +1373,18 @@ func (*aggregateStageKind) addPending(ss *stageState, em *ElementManager, newPen
13271373 ss .pendingByKeys [string (e .keyBytes )] = dnt
13281374 }
13291375 heap .Push (& dnt .elements , e )
1330- // Check on triggers for this key.
1331- // We use an empty linkID as the key into state for aggregations.
1332- if ss .state == nil {
1333- ss .state = make (map [LinkID ]map [typex.Window ]map [string ]StateData )
1334- }
1335- lv , ok := ss .state [LinkID {}]
1336- if ! ok {
1337- lv = make (map [typex.Window ]map [string ]StateData )
1338- ss .state [LinkID {}] = lv
1339- }
1340- wv , ok := lv [e .window ]
1341- if ! ok {
1342- wv = make (map [string ]StateData )
1343- lv [e .window ] = wv
1344- }
1345- state := wv [string (e .keyBytes )]
1346- endOfWindowReached := e .window .MaxTimestamp () < ss .input
1347- ready := ss .strat .IsTriggerReady (triggerInput {
1348- newElementCount : 1 ,
1349- endOfWindowReached : endOfWindowReached ,
1350- }, & state )
13511376
1352- if ready {
1353- state .Pane = computeNextTriggeredPane (state .Pane , endOfWindowReached )
1377+ if em .config .StreamingMode {
1378+ // In streaming mode, we check trigger readiness on each element
1379+ count += ss .injectTriggeredBundlesIfReady (em , e .window , string (e .keyBytes ))
1380+ } else {
1381+ // In batch mode, we store key + window pairs here and check trigger readiness for each of them later.
1382+ pendingWindowKeys .insert (windowKey {window : e .window , key : string (e .keyBytes )})
13541383 }
1355- // Store the state as triggers may have changed it.
1356- ss .state [LinkID {}][e .window ][string (e .keyBytes )] = state
1357-
1358- // If we're ready, it's time to fire!
1359- if ready {
1360- count += ss .buildTriggeredBundle (em , e .keyBytes , e .window )
1384+ }
1385+ if ! em .config .StreamingMode {
1386+ for wk := range pendingWindowKeys {
1387+ count += ss .injectTriggeredBundlesIfReady (em , wk .window , wk .key )
13611388 }
13621389 }
13631390 return count
@@ -1493,9 +1520,9 @@ func (ss *stageState) savePanes(bundID string, panesInBundle []bundlePane) {
14931520// buildTriggeredBundle must be called with the stage.mu lock held.
14941521// When in discarding mode, returns 0.
14951522// When in accumulating mode, returns the number of fired elements to maintain a correct pending count.
1496- func (ss * stageState ) buildTriggeredBundle (em * ElementManager , key [] byte , win typex.Window ) int {
1523+ func (ss * stageState ) buildTriggeredBundle (em * ElementManager , key string , win typex.Window ) int {
14971524 var toProcess []element
1498- dnt := ss .pendingByKeys [string ( key ) ]
1525+ dnt := ss .pendingByKeys [key ]
14991526 var notYet []element
15001527
15011528 rb := RunBundle {StageID : ss .ID , BundleID : "agg-" + em .nextBundID (), Watermark : ss .input }
@@ -1524,7 +1551,7 @@ func (ss *stageState) buildTriggeredBundle(em *ElementManager, key []byte, win t
15241551 }
15251552 dnt .elements = append (dnt .elements , notYet ... )
15261553 if dnt .elements .Len () == 0 {
1527- delete (ss .pendingByKeys , string ( key ) )
1554+ delete (ss .pendingByKeys , key )
15281555 } else {
15291556 // Ensure the heap invariants are maintained.
15301557 heap .Init (& dnt .elements )
@@ -1537,15 +1564,15 @@ func (ss *stageState) buildTriggeredBundle(em *ElementManager, key []byte, win t
15371564 {
15381565 win : win ,
15391566 key : string (key ),
1540- pane : ss .state [LinkID {}][win ][string ( key ) ].Pane ,
1567+ pane : ss .state [LinkID {}][win ][key ].Pane ,
15411568 },
15421569 }
15431570
15441571 ss .makeInProgressBundle (
15451572 func () string { return rb .BundleID },
15461573 toProcess ,
15471574 ss .input ,
1548- singleSet (string ( key ) ),
1575+ singleSet (key ),
15491576 nil ,
15501577 panesInBundle ,
15511578 )
0 commit comments