-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
batching_sink.go
456 lines (389 loc) · 11.2 KB
/
batching_sink.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
// Copyright 2023 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt
package changefeedccl
import (
"context"
"hash"
"sync"
"time"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent"
"github.com/cockroachdb/cockroach/pkg/util/admission"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/intsets"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
)
// SinkClient is an interface to an external sink, where messages are written
// into batches as they arrive and once ready are flushed out.
type SinkClient interface {
MakeResolvedPayload(body []byte, topic string) (SinkPayload, error)
MakeBatchBuffer() BatchBuffer
Flush(context.Context, SinkPayload) error
Close() error
}
// BatchBuffer is an interface to aggregate KVs into a payload that can be sent
// to the sink.
type BatchBuffer interface {
Append(key []byte, value []byte, topic string)
ShouldFlush() bool
Close() (SinkPayload, error)
}
// SinkPayload is an interface representing a sink-specific representation of a
// batch of messages that is ready to be emitted by its EmitRow method.
type SinkPayload interface{}
// batchingSink wraps a SinkClient to provide a Sink implementation that calls
// the SinkClient methods to form batches and flushes those batches across
// multiple parallel IO workers.
type batchingSink struct {
client SinkClient
topicNamer *TopicNamer
concreteType sinkType
ioWorkers int
minFlushFrequency time.Duration
retryOpts retry.Options
ts timeutil.TimeSource
metrics metricsRecorder
knobs batchingSinkKnobs
// eventCh is the channel used to send requests from the Sink caller routines
// to the batching routine. Messages can either be a flushReq or a kvEvent.
eventCh chan interface{}
termErr error
wg ctxgroup.Group
hasher hash.Hash32
pacer *admission.Pacer
doneCh chan struct{}
}
type batchingSinkKnobs struct {
OnAppend func(*rowEvent)
}
type flushReq struct {
waiter chan struct{}
}
type rowEvent struct {
key []byte
val []byte
topic string
alloc kvevent.Alloc
mvcc hlc.Timestamp
}
// Flush implements the Sink interface, returning the first error that has
// occured in the past EmitRow calls.
func (s *batchingSink) Flush(ctx context.Context) error {
flushWaiter := make(chan struct{})
select {
case <-ctx.Done():
case <-s.doneCh:
case s.eventCh <- flushReq{waiter: flushWaiter}:
}
select {
case <-ctx.Done():
return ctx.Err()
case <-s.doneCh:
return nil
case <-flushWaiter:
return s.termErr
}
}
var _ Sink = (*batchingSink)(nil)
// Event structs and batch structs which are transferred across routines (and
// therefore escape to the heap) can both be incredibly frequent (every event
// may be its own batch) and temporary, so to avoid GC thrashing they are both
// claimed and freed from object pools.
var eventPool sync.Pool = sync.Pool{
New: func() interface{} {
return new(rowEvent)
},
}
func newRowEvent() *rowEvent {
return eventPool.Get().(*rowEvent)
}
func freeRowEvent(e *rowEvent) {
*e = rowEvent{}
eventPool.Put(e)
}
var batchPool sync.Pool = sync.Pool{
New: func() interface{} {
return new(sinkBatch)
},
}
func newSinkBatch() *sinkBatch {
return batchPool.Get().(*sinkBatch)
}
func freeSinkBatchEvent(b *sinkBatch) {
*b = sinkBatch{}
batchPool.Put(b)
}
// EmitRow implements the Sink interface.
func (s *batchingSink) EmitRow(
ctx context.Context,
topic TopicDescriptor,
key, value []byte,
updated, mvcc hlc.Timestamp,
alloc kvevent.Alloc,
) error {
s.metrics.recordMessageSize(int64(len(key) + len(value)))
payload := newRowEvent()
payload.key = key
payload.val = value
payload.topic = "" // unimplemented for now
payload.mvcc = mvcc
payload.alloc = alloc
select {
case <-ctx.Done():
return ctx.Err()
case s.eventCh <- payload:
case <-s.doneCh:
}
return nil
}
// EmitResolvedTimestamp implements the Sink interface.
func (s *batchingSink) EmitResolvedTimestamp(
ctx context.Context, encoder Encoder, resolved hlc.Timestamp,
) error {
data, err := encoder.EncodeResolvedTimestamp(ctx, "", resolved)
if err != nil {
return err
}
payload, err := s.client.MakeResolvedPayload(data, "")
if err != nil {
return err
}
if err = s.Flush(ctx); err != nil {
return err
}
return retry.WithMaxAttempts(ctx, s.retryOpts, s.retryOpts.MaxRetries+1, func() error {
defer s.metrics.recordFlushRequestCallback()()
return s.client.Flush(ctx, payload)
})
}
// Close implements the Sink interface.
func (s *batchingSink) Close() error {
close(s.doneCh)
_ = s.wg.Wait()
if s.pacer != nil {
s.pacer.Close()
}
return s.client.Close()
}
// Dial implements the Sink interface.
func (s *batchingSink) Dial() error {
return nil
}
// getConcreteType implements the Sink interface.
func (s *batchingSink) getConcreteType() sinkType {
return s.concreteType
}
// sinkBatch stores an in-progress/complete batch of messages, along with
// metadata related to the batch.
type sinkBatch struct {
buffer BatchBuffer
payload SinkPayload // payload is nil until FinalizePayload has been called
numMessages int
numKVBytes int // the total amount of uncompressed kv data in the batch
keys intsets.Fast // An intset of the keys within the batch to provide to parallelIO
bufferTime time.Time // The earliest time a message was inserted into the batch
mvcc hlc.Timestamp
alloc kvevent.Alloc
hasher hash.Hash32
}
// FinalizePayload closes the writer to produce a payload that is ready to be
// Flushed by the SinkClient.
func (sb *sinkBatch) FinalizePayload() error {
payload, err := sb.buffer.Close()
if err != nil {
return err
}
sb.payload = payload
return nil
}
// Keys implements the IORequest interface.
func (sb *sinkBatch) Keys() intsets.Fast {
return sb.keys
}
func (sb *sinkBatch) isEmpty() bool {
return sb.numMessages == 0
}
func hashToInt(h hash.Hash32, buf []byte) int {
h.Reset()
h.Write(buf)
return int(h.Sum32())
}
// Append adds the contents of a kvEvent to the batch, merging its alloc pool.
func (sb *sinkBatch) Append(e *rowEvent) {
if sb.isEmpty() {
sb.bufferTime = timeutil.Now()
}
sb.buffer.Append(e.key, e.val, e.topic)
sb.keys.Add(hashToInt(sb.hasher, e.key))
sb.numMessages += 1
sb.numKVBytes += len(e.key) + len(e.val)
if sb.mvcc.IsEmpty() || e.mvcc.Less(sb.mvcc) {
sb.mvcc = e.mvcc
}
sb.alloc.Merge(&e.alloc)
}
func (s *batchingSink) handleError(err error) {
if s.termErr == nil {
s.termErr = err
}
}
func (s *batchingSink) newBatchBuffer() *sinkBatch {
batch := newSinkBatch()
batch.buffer = s.client.MakeBatchBuffer()
batch.hasher = s.hasher
return batch
}
// runBatchingWorker combines 1 or more KV events into batches, sending the IO
// requests out either once the batch is full or a flush request arrives.
func (s *batchingSink) runBatchingWorker(ctx context.Context) {
batchBuffer := s.newBatchBuffer()
// Once finalized, batches are sent to a parallelIO struct which handles
// performing multiple Flushes in parallel while maintaining Keys() ordering.
ioHandler := func(ctx context.Context, req IORequest) error {
defer s.metrics.recordFlushRequestCallback()()
batch, _ := req.(*sinkBatch)
return s.client.Flush(ctx, batch.payload)
}
ioEmitter := newParallelIO(ctx, s.retryOpts, s.ioWorkers, ioHandler, s.metrics)
defer ioEmitter.Close()
var handleResult func(result *ioResult)
tryFlushBatch := func() {
if batchBuffer.isEmpty() {
return
}
toFlush := batchBuffer
batchBuffer = s.newBatchBuffer()
if err := toFlush.FinalizePayload(); err != nil {
s.handleError(err)
return
}
// Emitting needs to also handle any incoming results to avoid a deadlock
// with trying to emit while the emitter is blocked on returning a result.
for {
select {
case <-ctx.Done():
case ioEmitter.requestCh <- toFlush:
case result := <-ioEmitter.resultCh:
handleResult(result)
continue
case <-s.doneCh:
}
break
}
}
// Flushing requires tracking the number of inflight messages and confirming
// completion to the requester once the counter reaches 0.
inflight := 0
var sinkFlushWaiter chan struct{}
handleResult = func(result *ioResult) {
batch, _ := result.request.(*sinkBatch)
if result.err != nil {
s.handleError(result.err)
} else {
s.metrics.recordEmittedBatch(
batch.bufferTime, batch.numMessages, batch.mvcc, batch.numKVBytes, sinkDoesNotCompress,
)
}
inflight -= batch.numMessages
if (result.err != nil || inflight == 0) && sinkFlushWaiter != nil {
close(sinkFlushWaiter)
sinkFlushWaiter = nil
}
freeIOResult(result)
batch.alloc.Release(ctx)
freeSinkBatchEvent(batch)
}
flushTimer := s.ts.NewTimer()
defer flushTimer.Stop()
for {
if s.pacer != nil {
if err := s.pacer.Pace(ctx); err != nil {
if pacerLogEvery.ShouldLog() {
log.Errorf(ctx, "automatic sink batcher pacing: %v", err)
}
}
}
select {
case req := <-s.eventCh:
if flush, isFlush := req.(flushReq); isFlush {
if inflight == 0 {
close(flush.waiter)
} else {
sinkFlushWaiter = flush.waiter
tryFlushBatch()
}
} else if event, isKV := req.(*rowEvent); isKV {
inflight += 1
// If we're about to append to an empty batch, start the timer to
// guarantee the messages do not stay buffered longer than the
// configured frequency.
if batchBuffer.isEmpty() && s.minFlushFrequency > 0 {
flushTimer.Reset(s.minFlushFrequency)
}
batchBuffer.Append(event)
if s.knobs.OnAppend != nil {
s.knobs.OnAppend(event)
}
// The event struct can be freed as the contents are expected to be
// managed by the batch instead.
freeRowEvent(event)
if batchBuffer.buffer.ShouldFlush() {
s.metrics.recordSizeBasedFlush()
tryFlushBatch()
}
}
case result := <-ioEmitter.resultCh:
handleResult(result)
case <-flushTimer.Ch():
flushTimer.MarkRead()
tryFlushBatch()
case <-ctx.Done():
return
case <-s.doneCh:
return
}
if s.termErr != nil {
return
}
}
}
func makeBatchingSink(
ctx context.Context,
concreteType sinkType,
client SinkClient,
minFlushFrequency time.Duration,
retryOpts retry.Options,
numWorkers int,
topicNamer *TopicNamer,
pacer *admission.Pacer,
timeSource timeutil.TimeSource,
metrics metricsRecorder,
) Sink {
sink := &batchingSink{
client: client,
topicNamer: topicNamer,
concreteType: concreteType,
minFlushFrequency: minFlushFrequency,
ioWorkers: numWorkers,
retryOpts: retryOpts,
ts: timeSource,
metrics: metrics,
eventCh: make(chan interface{}, flushQueueDepth),
wg: ctxgroup.WithContext(ctx),
hasher: makeHasher(),
pacer: pacer,
doneCh: make(chan struct{}),
}
sink.wg.GoCtx(func(ctx context.Context) error {
sink.runBatchingWorker(ctx)
return nil
})
return sink
}