@@ -261,8 +261,10 @@ func (em *ElementManager) AddStage(ID string, inputIDs, outputIDs []string, side
261
261
262
262
// StageAggregates marks the given stage as an aggregation, which
263
263
// means elements will only be processed based on windowing strategies.
264
- func (em * ElementManager ) StageAggregates (ID string ) {
265
- em .stages [ID ].aggregate = true
264
+ func (em * ElementManager ) StageAggregates (ID string , strat WinStrat ) {
265
+ ss := em .stages [ID ]
266
+ ss .aggregate = true
267
+ ss .strat = strat
266
268
}
267
269
268
270
// StageStateful marks the given stage as stateful, which means elements are
@@ -1095,7 +1097,7 @@ type stageState struct {
1095
1097
// Special handling bits
1096
1098
stateful bool // whether this stage uses state or timers, and needs keyed processing.
1097
1099
aggregate bool // whether this stage needs to block for aggregation.
1098
- strat winStrat // Windowing Strategy for aggregation fireings.
1100
+ strat WinStrat // Windowing Strategy for aggregation fireings.
1099
1101
processingTimeTimersFamilies map [string ]bool // Indicates which timer families use the processing time domain.
1100
1102
1101
1103
// onWindowExpiration management
@@ -1154,7 +1156,6 @@ func makeStageState(ID string, inputIDs, outputIDs []string, sides []LinkID) *st
1154
1156
ID : ID ,
1155
1157
outputIDs : outputIDs ,
1156
1158
sides : sides ,
1157
- strat : defaultStrat {},
1158
1159
state : map [LinkID ]map [typex.Window ]map [string ]StateData {},
1159
1160
watermarkHolds : newHoldTracker (),
1160
1161
@@ -1179,18 +1180,21 @@ func makeStageState(ID string, inputIDs, outputIDs []string, sides []LinkID) *st
1179
1180
func (ss * stageState ) AddPending (newPending []element ) int {
1180
1181
ss .mu .Lock ()
1181
1182
defer ss .mu .Unlock ()
1182
- // TODO(#https://github.com/apache/beam/issues/31438):
1183
- // Adjust with AllowedLateness
1184
- // Data that arrives after the *output* watermark is late.
1185
- threshold := ss .output
1186
- origPending := make ([]element , 0 , ss .pending .Len ())
1187
- for _ , e := range newPending {
1188
- if e .window .MaxTimestamp () < threshold {
1189
- continue
1183
+ if ss .aggregate {
1184
+ // Late Data is data that has arrived after that window has expired.
1185
+ // We only need to drop late data before aggregations.
1186
+ // TODO - handle for side inputs too.
1187
+ threshold := ss .output
1188
+ origPending := make ([]element , 0 , ss .pending .Len ())
1189
+ for _ , e := range newPending {
1190
+ if ss .strat .EarliestCompletion (e .window ) < threshold {
1191
+ // TODO: figure out Pane and trigger firings.
1192
+ continue
1193
+ }
1194
+ origPending = append (origPending , e )
1190
1195
}
1191
- origPending = append ( origPending , e )
1196
+ newPending = origPending
1192
1197
}
1193
- newPending = origPending
1194
1198
if ss .stateful {
1195
1199
if ss .pendingByKeys == nil {
1196
1200
ss .pendingByKeys = map [string ]* dataAndTimers {}
@@ -1626,10 +1630,8 @@ func (ss *stageState) updateWatermarks(em *ElementManager) set[string] {
1626
1630
// They'll never be read in again.
1627
1631
for _ , wins := range ss .sideInputs {
1628
1632
for win := range wins {
1629
- // TODO(#https://github.com/apache/beam/issues/31438):
1630
- // Adjust with AllowedLateness
1631
1633
// Clear out anything we've already used.
1632
- if win . MaxTimestamp ( ) < newOut {
1634
+ if ss . strat . EarliestCompletion ( win ) < newOut {
1633
1635
// If the expiry is in progress, skip this window.
1634
1636
if ss .inProgressExpiredWindows [win ] > 0 {
1635
1637
continue
@@ -1640,9 +1642,7 @@ func (ss *stageState) updateWatermarks(em *ElementManager) set[string] {
1640
1642
}
1641
1643
for _ , wins := range ss .state {
1642
1644
for win := range wins {
1643
- // TODO(#https://github.com/apache/beam/issues/31438):
1644
- // Adjust with AllowedLateness
1645
- if win .MaxTimestamp () < newOut {
1645
+ if ss .strat .EarliestCompletion (win ) < newOut {
1646
1646
// If the expiry is in progress, skip collecting this window.
1647
1647
if ss .inProgressExpiredWindows [win ] > 0 {
1648
1648
continue
@@ -1685,9 +1685,7 @@ func (ss *stageState) createOnWindowExpirationBundles(newOut mtime.Time, em *Ele
1685
1685
var preventDownstreamUpdate bool
1686
1686
for win , keys := range ss .keysToExpireByWindow {
1687
1687
// Check if the window has expired.
1688
- // TODO(#https://github.com/apache/beam/issues/31438):
1689
- // Adjust with AllowedLateness
1690
- if win .MaxTimestamp () >= newOut {
1688
+ if ss .strat .EarliestCompletion (win ) >= newOut {
1691
1689
continue
1692
1690
}
1693
1691
// We can't advance the output watermark if there's garbage to collect.
0 commit comments