github.com/Jeffail/benthos/v3@v3.65.0/internal/impl/generic/buffer_system_window.go (about) 1 package generic 2 3 import ( 4 "context" 5 "errors" 6 "fmt" 7 "sync" 8 "time" 9 10 "github.com/Jeffail/benthos/v3/internal/batch" 11 "github.com/Jeffail/benthos/v3/internal/bloblang/query" 12 "github.com/Jeffail/benthos/v3/public/bloblang" 13 "github.com/Jeffail/benthos/v3/public/service" 14 ) 15 16 func tumblingWindowBufferConfig() *service.ConfigSpec { 17 return service.NewConfigSpec(). 18 // Stable(). TODO 19 Version("3.53.0"). 20 Categories("Windowing"). 21 Summary("Chops a stream of messages into tumbling or sliding windows of fixed temporal size, following the system clock."). 22 Description(` 23 A window is a grouping of messages that fit within a discrete measure of time following the system clock. Messages are allocated to a window either by the processing time (the time at which they're ingested) or by the event time, and this is controlled via the `+"[`timestamp_mapping` field](#timestamp_mapping)"+`. 24 25 In tumbling mode (default) the beginning of a window immediately follows the end of a prior window. When the buffer is initialized the first window to be created and populated is aligned against the zeroth minute of the zeroth hour of the day by default, and may therefore be open for a shorter period than the specified size. 26 27 A window is flushed only once the system clock surpasses its scheduled end. If an `+"[`allowed_lateness`](#allowed_lateness)"+` is specified then the window will not be flushed until the scheduled end plus that length of time. 28 29 When a message is added to a window it has a metadata field `+"`window_end_timestamp`"+` added to it containing the timestamp of the end of the window as an RFC3339 string. 30 31 ## Sliding Windows 32 33 Sliding windows begin from an offset of the prior windows' beginning rather than its end, and therefore messages may belong to multiple windows. In order to produce sliding windows specify a `+"[`slide` duration](#slide)"+`. 34 35 ## Back Pressure 36 37 If back pressure is applied to this buffer either due to output services being unavailable or resources being saturated, windows older than the current and last according to the system clock will be dropped in order to prevent unbounded resource usage. This means you should ensure that under the worst case scenario you have enough system memory to store two windows' worth of data at a given time (plus extra for redundancy and other services). 38 39 If messages could potentially arrive with event timestamps in the future (according to the system clock) then you should also factor in these extra messages in memory usage estimates. 40 41 ## Delivery Guarantees 42 43 This buffer honours the transaction model within Benthos in order to ensure that messages are not acknowledged until they are either intentionally dropped or successfully delivered to outputs. However, since messages belonging to an expired window are intentionally dropped there are circumstances where not all messages entering the system will be delivered. 44 45 When this buffer is configured with a slide duration it is possible for messages to belong to multiple windows, and therefore be delivered multiple times. In this case the first time the message is delivered it will be acked (or nacked) and subsequent deliveries of the same message will be a "best attempt". 46 47 During graceful termination if the current window is partially populated with messages they will be nacked such that they are re-consumed the next time the service starts. 48 `). 49 Field(service.NewBloblangField("timestamp_mapping"). 50 Description(` 51 A [Bloblang mapping](/docs/guides/bloblang/about) applied to each message during ingestion that provides the timestamp to use for allocating it a window. By default the function `+"`now()`"+` is used in order to generate a fresh timestamp at the time of ingestion (the processing time), whereas this mapping can instead extract a timestamp from the message itself (the event time). 52 53 The timestamp value assigned to `+"`root`"+` must either be a numerical unix time in seconds (with up to nanosecond precision via decimals), or a string in ISO 8601 format. If the mapping fails or provides an invalid result the message will be dropped (with logging to describe the problem). 54 `). 55 Default("root = now()"). 56 Example("root = this.created_at").Example(`root = meta("kafka_timestamp_unix").number()`)). 57 Field(service.NewStringField("size"). 58 Description("A duration string describing the size of each window. By default windows are aligned to the zeroth minute and zeroth hour on the UTC clock, meaning windows of 1 hour duration will match the turn of each hour in the day, this can be adjusted with the `offset` field."). 59 Example("30s").Example("10m")). 60 Field(service.NewStringField("slide"). 61 Description("An optional duration string describing by how much time the beginning of each window should be offset from the beginning of the previous, and therefore creates sliding windows instead of tumbling. When specified this duration must be smaller than the `size` of the window."). 62 Default(""). 63 Example("30s").Example("10m")). 64 Field(service.NewStringField("offset"). 65 Description("An optional duration string to offset the beginning of each window by, otherwise they are aligned to the zeroth minute and zeroth hour on the UTC clock. The offset cannot be a larger or equal measure to the window size or the slide."). 66 Default(""). 67 Example("-6h").Example("30m")). 68 Field(service.NewStringField("allowed_lateness"). 69 Description("An optional duration string describing the length of time to wait after a window has ended before flushing it, allowing late arrivals to be included. Since this windowing buffer uses the system clock an allowed lateness can improve the matching of messages when using event time."). 70 Default(""). 71 Example("10s").Example("1m")). 72 Example("Counting Passengers at Traffic", `Given a stream of messages relating to cars passing through various traffic lights of the form: 73 74 `+"```json"+` 75 { 76 "traffic_light": "cbf2eafc-806e-4067-9211-97be7e42cee3", 77 "created_at": "2021-08-07T09:49:35Z", 78 "registration_plate": "AB1C DEF", 79 "passengers": 3 80 } 81 `+"```"+` 82 83 We can use a window buffer in order to create periodic messages summarising the traffic for a period of time of this form: 84 85 `+"```json"+` 86 { 87 "traffic_light": "cbf2eafc-806e-4067-9211-97be7e42cee3", 88 "created_at": "2021-08-07T10:00:00Z", 89 "total_cars": 15, 90 "passengers": 43 91 } 92 `+"```"+` 93 94 With the following config:`, 95 ` 96 buffer: 97 system_window: 98 timestamp_mapping: root = this.created_at 99 size: 1h 100 101 pipeline: 102 processors: 103 # Group messages of the window into batches of common traffic light IDs 104 - group_by_value: 105 value: '${! json("traffic_light") }' 106 107 # Reduce each batch to a single message by deleting indexes > 0, and 108 # aggregate the car and passenger counts. 109 - bloblang: | 110 root = if batch_index() == 0 { 111 { 112 "traffic_light": this.traffic_light, 113 "created_at": meta("window_end_timestamp"), 114 "total_cars": json("registration_plate").from_all().unique().length(), 115 "passengers": json("passengers").from_all().sum(), 116 } 117 } else { deleted() } 118 `, 119 ) 120 121 } 122 123 func getDuration(conf *service.ParsedConfig, required bool, name string) (time.Duration, error) { 124 periodStr, err := conf.FieldString(name) 125 if err != nil { 126 return 0, err 127 } 128 if !required && periodStr == "" { 129 return 0, nil 130 } 131 period, err := time.ParseDuration(periodStr) 132 if err != nil { 133 return 0, fmt.Errorf("failed to parse field '%v' as duration: %w", name, err) 134 } 135 return period, nil 136 } 137 138 func init() { 139 err := service.RegisterBatchBuffer( 140 "system_window", tumblingWindowBufferConfig(), 141 func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchBuffer, error) { 142 size, err := getDuration(conf, true, "size") 143 if err != nil { 144 return nil, err 145 } 146 slide, err := getDuration(conf, false, "slide") 147 if err != nil { 148 return nil, err 149 } 150 if slide >= size { 151 return nil, fmt.Errorf("invalid window slide '%v' must be lower than the size '%v'", slide, size) 152 } 153 offset, err := getDuration(conf, false, "offset") 154 if err != nil { 155 return nil, err 156 } 157 if offset >= size { 158 return nil, fmt.Errorf("invalid offset '%v' must be lower than the size '%v'", offset, size) 159 } 160 if slide > 0 && offset >= slide { 161 return nil, fmt.Errorf("invalid offset '%v' must be lower than the slide '%v'", offset, slide) 162 } 163 allowedLateness, err := getDuration(conf, false, "allowed_lateness") 164 if err != nil { 165 return nil, err 166 } 167 if allowedLateness >= size { 168 return nil, fmt.Errorf("invalid allowed_lateness '%v' must be lower than the size '%v'", allowedLateness, size) 169 } 170 tsMapping, err := conf.FieldBloblang("timestamp_mapping") 171 if err != nil { 172 return nil, err 173 } 174 return newSystemWindowBuffer(tsMapping, func() time.Time { 175 return time.Now().UTC() 176 }, size, slide, offset, allowedLateness, mgr.Logger()) 177 }) 178 179 if err != nil { 180 panic(err) 181 } 182 } 183 184 //------------------------------------------------------------------------------ 185 186 type tsMessage struct { 187 ts time.Time 188 m *service.Message 189 ackFn service.AckFunc 190 } 191 192 type utcNowProvider func() time.Time 193 194 type systemWindowBuffer struct { 195 logger *service.Logger 196 197 tsMapping *bloblang.Executor 198 clock utcNowProvider 199 size, slide, offset, allowedLateness time.Duration 200 201 latestFlushedWindowEnd time.Time 202 oldestTS time.Time 203 pending []*tsMessage 204 pendingMut sync.Mutex 205 206 closedTimerChan <-chan time.Time 207 208 endOfInputChan chan struct{} 209 closeEndOfInputOnce sync.Once 210 } 211 212 func newSystemWindowBuffer( 213 tsMapping *bloblang.Executor, 214 clock utcNowProvider, 215 size, slide, offset, allowedLateness time.Duration, 216 logger *service.Logger, 217 ) (*systemWindowBuffer, error) { 218 w := &systemWindowBuffer{ 219 tsMapping: tsMapping, 220 clock: clock, 221 size: size, 222 slide: slide, 223 allowedLateness: allowedLateness, 224 offset: offset, 225 logger: logger, 226 oldestTS: clock(), 227 endOfInputChan: make(chan struct{}), 228 } 229 230 tmpTimerChan := make(chan time.Time) 231 close(tmpTimerChan) 232 w.closedTimerChan = tmpTimerChan 233 return w, nil 234 } 235 236 func (w *systemWindowBuffer) nextSystemWindow() (prevStart, prevEnd, start, end time.Time) { 237 now := w.clock() 238 239 windowEpoch := w.size 240 if w.slide > 0 { 241 windowEpoch = w.slide 242 } 243 244 // The start is now, rounded by our window epoch to the UTC clock, and with 245 // our offset (plus one to avoid overlapping with the previous window) 246 // added. 247 // 248 // If the result is after now then we rounded upwards, so we roll it back by 249 // the window epoch. 250 if start = w.clock().Round(windowEpoch).Add(1 + w.offset); start.After(now) { 251 start = start.Add(-windowEpoch) 252 } 253 254 // The result is the start of the newest active window. In the case of 255 // sliding windows this is not the "next" window to be flushed, so we need 256 // to roll back further. 257 if w.slide > 0 { 258 start = start.Add(w.slide - w.size) 259 } 260 261 // The end is our start plus the window size (minus the nanosecond added to 262 // the start). 263 end = start.Add(w.size - 1) 264 265 // Calculate the previous window as well 266 prevStart, prevEnd = start.Add(-windowEpoch), end.Add(-windowEpoch) 267 return 268 } 269 270 func (w *systemWindowBuffer) getTimestamp(i int, batch service.MessageBatch) (ts time.Time, err error) { 271 var tsValueMsg *service.Message 272 if tsValueMsg, err = batch.BloblangQuery(i, w.tsMapping); err != nil { 273 w.logger.Errorf("Timestamp mapping failed for message: %v", err) 274 err = fmt.Errorf("timestamp mapping failed: %w", err) 275 return 276 } 277 278 var tsValue interface{} 279 if tsValue, err = tsValueMsg.AsStructured(); err != nil { 280 if tsBytes, _ := tsValueMsg.AsBytes(); len(tsBytes) > 0 { 281 tsValue = string(tsBytes) 282 err = nil 283 } 284 } 285 if err != nil { 286 w.logger.Errorf("Timestamp mapping failed for message: unable to parse result as structured value: %v", err) 287 err = fmt.Errorf("unable to parse result of timestamp mapping as structured value: %w", err) 288 return 289 } 290 291 if ts, err = query.IGetTimestamp(tsValue); err != nil { 292 w.logger.Errorf("Timestamp mapping failed for message: %v", err) 293 err = fmt.Errorf("unable to parse result of timestamp mapping as timestamp: %w", err) 294 } 295 return 296 } 297 298 func (w *systemWindowBuffer) WriteBatch(ctx context.Context, msgBatch service.MessageBatch, aFn service.AckFunc) error { 299 w.pendingMut.Lock() 300 defer w.pendingMut.Unlock() 301 302 // If our output is blocked and therefore we haven't flushed more than the 303 // last two windows we purge messages that wouldn't fit within them. 304 prevStart, _, _, _ := w.nextSystemWindow() 305 if w.latestFlushedWindowEnd.Before(prevStart) && w.oldestTS.Before(prevStart) { 306 newOldestTS := w.clock() 307 newPending := make([]*tsMessage, 0, len(w.pending)) 308 for _, pending := range w.pending { 309 if pending.ts.Before(prevStart) { 310 // Reject messages too old to fit into a window by acknowledging 311 // them. 312 _ = pending.ackFn(ctx, nil) 313 continue 314 } 315 newPending = append(newPending, pending) 316 if pending.ts.Before(newOldestTS) { 317 newOldestTS = pending.ts 318 } 319 } 320 w.pending = newPending 321 } 322 323 messageAdded := false 324 aggregatedAck := batch.NewCombinedAcker(batch.AckFunc(aFn)) 325 326 // And now add new messages. 327 for i, msg := range msgBatch { 328 ts, err := w.getTimestamp(i, msgBatch) 329 if err != nil { 330 return err 331 } 332 333 // Don't add messages older than our current window start. 334 if !ts.After(w.latestFlushedWindowEnd) { 335 continue 336 } 337 338 messageAdded = true 339 w.pending = append(w.pending, &tsMessage{ 340 ts: ts, m: msg, ackFn: service.AckFunc(aggregatedAck.Derive()), 341 }) 342 if ts.Before(w.oldestTS) { 343 w.oldestTS = ts 344 } 345 } 346 347 if !messageAdded { 348 // If none of the messages have fit into a window we reject them by 349 // acknowledging the batch. 350 _ = aFn(ctx, nil) 351 } 352 return nil 353 } 354 355 func (w *systemWindowBuffer) flushWindow(ctx context.Context, start, end time.Time) (service.MessageBatch, service.AckFunc, error) { 356 w.pendingMut.Lock() 357 defer w.pendingMut.Unlock() 358 359 // Calculate the next start and purge everything older as we flush. 360 nextStart := start.Add(w.size) 361 if w.slide > 0 { 362 nextStart = start.Add(w.slide) 363 } 364 365 var flushBatch service.MessageBatch 366 var flushAcks []service.AckFunc 367 368 newPending := make([]*tsMessage, 0, len(w.pending)) 369 newOldest := w.clock() 370 for _, pending := range w.pending { 371 flush := !pending.ts.Before(start) && !pending.ts.After(end) 372 preserve := !pending.ts.Before(nextStart) 373 374 if flush { 375 tmpMsg := pending.m.Copy() 376 tmpMsg.MetaSet("window_end_timestamp", end.Format(time.RFC3339Nano)) 377 flushBatch = append(flushBatch, tmpMsg) 378 flushAcks = append(flushAcks, pending.ackFn) 379 } 380 if preserve { 381 if pending.ts.Before(newOldest) { 382 newOldest = pending.ts 383 } 384 newPending = append(newPending, pending) 385 } 386 if !flush && !preserve { 387 _ = pending.ackFn(ctx, nil) 388 } 389 } 390 391 w.pending = newPending 392 w.latestFlushedWindowEnd = end 393 w.oldestTS = newOldest 394 395 return flushBatch, func(ctx context.Context, err error) error { 396 for _, aFn := range flushAcks { 397 _ = aFn(ctx, err) 398 } 399 return nil 400 }, nil 401 } 402 403 var errWindowClosed = errors.New("message rejected as window did not complete") 404 405 func (w *systemWindowBuffer) ReadBatch(ctx context.Context) (service.MessageBatch, service.AckFunc, error) { 406 prevStart, prevEnd, nextStart, nextEnd := w.nextSystemWindow() 407 408 // We haven't been read since the previous window ended, so create that one 409 // instead in an attempt to back fill. 410 // 411 // Note that we do not need to lock around latestFlushWindowEnd because it's 412 // only written from the reader, and we only expect one active reader at a 413 // given time. If this assumption changes we would need to lock around this 414 // also. 415 if w.latestFlushedWindowEnd.Before(prevStart) { 416 if msgBatch, aFn, err := w.flushWindow(ctx, prevStart, prevEnd); len(msgBatch) > 0 || err != nil { 417 return msgBatch, aFn, err 418 } 419 } 420 421 for { 422 nextEndChan := w.closedTimerChan 423 if waitFor := nextEnd.Sub(w.clock()) + w.allowedLateness; waitFor > 0 { 424 nextEndChan = time.After(waitFor) 425 } 426 427 select { 428 case <-nextEndChan: 429 case <-ctx.Done(): 430 return nil, nil, ctx.Err() 431 case <-w.endOfInputChan: 432 // Nack all pending messages so that we re-consume them on the next 433 // start up. TODO: Eventually allow users to customize this as they 434 // may wish to flush partial windows instead. 435 w.pendingMut.Lock() 436 for _, pending := range w.pending { 437 _ = pending.ackFn(ctx, errWindowClosed) 438 } 439 w.pending = nil 440 w.pendingMut.Unlock() 441 return nil, nil, service.ErrEndOfBuffer 442 } 443 if msgBatch, aFn, err := w.flushWindow(ctx, nextStart, nextEnd); len(msgBatch) > 0 || err != nil { 444 return msgBatch, aFn, err 445 } 446 447 // Window did not contain any messages, so move onto next. 448 _, _, nextStart, nextEnd = w.nextSystemWindow() 449 } 450 } 451 452 func (w *systemWindowBuffer) EndOfInput() { 453 w.closeEndOfInputOnce.Do(func() { 454 close(w.endOfInputChan) 455 }) 456 } 457 458 func (w *systemWindowBuffer) Close(ctx context.Context) error { 459 return nil 460 }