-
Notifications
You must be signed in to change notification settings - Fork 3.9k
/
Copy pathstream_ingestion_processor.go
408 lines (352 loc) · 12.5 KB
/
stream_ingestion_processor.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
// Copyright 2020 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 streamingest
import (
"context"
"sort"
"time"
"github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/ccl/streamingccl"
"github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv/bulk"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"golang.org/x/sync/errgroup"
)
var minimumFlushInterval = settings.RegisterPublicDurationSettingWithExplicitUnit(
"bulkio.stream_ingestion.minimum_flush_interval",
"the minimum timestamp between flushes; flushes may still occur if internal buffers fill up",
5*time.Second,
nil, /* validateFn */
)
var streamIngestionResultTypes = []*types.T{
types.Bytes, // jobspb.ResolvedSpans
}
type mvccKeyValues []storage.MVCCKeyValue
func (s mvccKeyValues) Len() int { return len(s) }
func (s mvccKeyValues) Swap(i, j int) { s[i], s[j] = s[j], s[i] }
func (s mvccKeyValues) Less(i, j int) bool { return s[i].Key.Less(s[j].Key) }
type streamIngestionProcessor struct {
execinfra.ProcessorBase
flowCtx *execinfra.FlowCtx
spec execinfrapb.StreamIngestionDataSpec
output execinfra.RowReceiver
// curBatch temporarily batches MVCC Keys so they can be
// sorted before ingestion.
// TODO: This doesn't yet use a buffering adder since the current
// implementation is specific to ingesting KV pairs without timestamps rather
// than MVCCKeys.
curBatch mvccKeyValues
// batcher is used to flush SSTs to the storage layer.
batcher *bulk.SSTBatcher
timer *timeutil.Timer
// client is a streaming client which provides a stream of events from a given
// address.
client streamclient.Client
// Checkpoint events may need to be buffered if they arrive within the same
// minimumFlushInterval.
bufferedCheckpoints map[streamingccl.PartitionAddress]hlc.Timestamp
// lastFlushTime keeps track of the last time that we flushed due to a
// checkpoint timestamp event.
lastFlushTime time.Time
// When the event channel closes, we should flush any events that remains to
// be buffered. The processor keeps track of if we're done seeing new events,
// and have attempted to flush them with `internalDrained`.
internalDrained bool
// ingestionErr stores any error that is returned from the worker goroutine so
// that it can be forwarded through the DistSQL flow.
ingestionErr error
// eventCh is the merged event channel of all of the partition event streams.
eventCh chan partitionEvent
}
// partitionEvent augments a normal event with the partition it came from.
type partitionEvent struct {
streamingccl.Event
partition streamingccl.PartitionAddress
}
var _ execinfra.Processor = &streamIngestionProcessor{}
var _ execinfra.RowSource = &streamIngestionProcessor{}
const streamIngestionProcessorName = "stream-ingestion-processor"
func newStreamIngestionDataProcessor(
flowCtx *execinfra.FlowCtx,
processorID int32,
spec execinfrapb.StreamIngestionDataSpec,
post *execinfrapb.PostProcessSpec,
output execinfra.RowReceiver,
) (execinfra.Processor, error) {
streamClient, err := streamclient.NewStreamClient(spec.StreamAddress)
if err != nil {
return nil, err
}
// Check if there are any interceptor methods that need to be registered with
// the stream client.
// These methods are invoked on every emitted Event.
if knobs, ok := flowCtx.Cfg.TestingKnobs.StreamIngestionTestingKnobs.(*sql.
StreamIngestionTestingKnobs); ok {
if knobs.Interceptors != nil {
if interceptable, ok := streamClient.(streamclient.InterceptableStreamClient); ok {
for _, interceptor := range knobs.Interceptors {
interceptable.RegisterInterception(interceptor)
}
}
}
}
sip := &streamIngestionProcessor{
flowCtx: flowCtx,
spec: spec,
output: output,
curBatch: make([]storage.MVCCKeyValue, 0),
client: streamClient,
bufferedCheckpoints: make(map[streamingccl.PartitionAddress]hlc.Timestamp),
timer: timeutil.NewTimer(),
}
if err := sip.Init(sip, post, streamIngestionResultTypes, flowCtx, processorID, output, nil, /* memMonitor */
execinfra.ProcStateOpts{
InputsToDrain: []execinfra.RowSource{},
TrailingMetaCallback: func(context.Context) []execinfrapb.ProducerMetadata {
sip.close()
return nil
},
},
); err != nil {
return nil, err
}
return sip, nil
}
// Start is part of the RowSource interface.
func (sip *streamIngestionProcessor) Start(ctx context.Context) {
ctx = sip.StartInternal(ctx, streamIngestionProcessorName)
evalCtx := sip.FlowCtx.EvalCtx
db := sip.FlowCtx.Cfg.DB
var err error
sip.batcher, err = bulk.MakeStreamSSTBatcher(ctx, db, evalCtx.Settings,
func() int64 { return storageccl.MaxImportBatchSize(evalCtx.Settings) })
if err != nil {
sip.MoveToDraining(errors.Wrap(err, "creating stream sst batcher"))
return
}
// Initialize the event streams.
startTime := timeutil.Unix(0 /* sec */, sip.spec.StartTime.WallTime)
eventChs := make(map[streamingccl.PartitionAddress]chan streamingccl.Event)
for _, partitionAddress := range sip.spec.PartitionAddresses {
eventCh, err := sip.client.ConsumePartition(ctx, partitionAddress, startTime)
if err != nil {
sip.MoveToDraining(errors.Wrapf(err, "consuming partition %v", partitionAddress))
return
}
eventChs[partitionAddress] = eventCh
}
sip.eventCh = sip.merge(ctx, eventChs)
}
// Next is part of the RowSource interface.
func (sip *streamIngestionProcessor) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) {
if sip.State != execinfra.StateRunning {
return nil, sip.DrainHelper()
}
progressUpdate, err := sip.consumeEvents()
if err != nil {
sip.MoveToDraining(err)
return nil, sip.DrainHelper()
}
if progressUpdate != nil {
progressBytes, err := protoutil.Marshal(progressUpdate)
if err != nil {
sip.MoveToDraining(err)
return nil, sip.DrainHelper()
}
row := rowenc.EncDatumRow{
rowenc.DatumToEncDatum(types.Bytes, tree.NewDBytes(tree.DBytes(progressBytes))),
}
return row, nil
}
if sip.ingestionErr != nil {
sip.MoveToDraining(sip.ingestionErr)
return nil, sip.DrainHelper()
}
sip.MoveToDraining(nil /* error */)
return nil, sip.DrainHelper()
}
// ConsumerClosed is part of the RowSource interface.
func (sip *streamIngestionProcessor) ConsumerClosed() {
sip.close()
}
func (sip *streamIngestionProcessor) close() {
if sip.InternalClose() {
if sip.batcher != nil {
sip.batcher.Close()
}
if sip.timer != nil {
sip.timer.Stop()
}
}
}
// merge takes events from all the streams and merges them into a single
// channel.
func (sip *streamIngestionProcessor) merge(
ctx context.Context, partitionStreams map[streamingccl.PartitionAddress]chan streamingccl.Event,
) chan partitionEvent {
merged := make(chan partitionEvent)
var g errgroup.Group
for partition, eventCh := range partitionStreams {
partition := partition
eventCh := eventCh
g.Go(func() error {
ctxDone := ctx.Done()
for {
select {
case event, ok := <-eventCh:
if !ok {
return nil
}
pe := partitionEvent{
Event: event,
partition: partition,
}
select {
case merged <- pe:
case <-ctxDone:
return ctx.Err()
}
case <-ctxDone:
return ctx.Err()
}
}
})
}
go func() {
sip.ingestionErr = g.Wait()
close(merged)
}()
return merged
}
// consumeEvents handles processing events on the merged event queue and returns
// once a checkpoint event has been emitted so that it can inform the downstream
// frontier processor to consider updating the frontier.
//
// It should only make a claim that about the resolved timestamp of a partition
// increasing after it has flushed all KV events previously received by that
// partition.
func (sip *streamIngestionProcessor) consumeEvents() (*jobspb.ResolvedSpans, error) {
// This timer is used to batch up resolved timestamp events that occur within
// a given time interval, as to not flush too often and allow the buffer to
// accumulate data.
// A flush may still occur if the in memory buffer becomes full.
sv := &sip.FlowCtx.Cfg.Settings.SV
if sip.internalDrained {
return nil, nil
}
for sip.State == execinfra.StateRunning {
select {
case event, ok := <-sip.eventCh:
if !ok {
sip.internalDrained = true
return sip.flush()
}
switch event.Type() {
case streamingccl.KVEvent:
if err := sip.bufferKV(event); err != nil {
return nil, err
}
case streamingccl.CheckpointEvent:
if err := sip.bufferCheckpoint(event); err != nil {
return nil, err
}
minFlushInterval := minimumFlushInterval.Get(sv)
if timeutil.Since(sip.lastFlushTime) < minFlushInterval {
// Not enough time has passed since the last flush. Let's set a timer
// that will trigger a flush eventually.
// TODO: This resets the timer every checkpoint event, but we only
// need to reset it once.
sip.timer.Reset(time.Until(sip.lastFlushTime.Add(minFlushInterval)))
continue
}
return sip.flush()
default:
return nil, errors.Newf("unknown streaming event type %v", event.Type())
}
case <-sip.timer.C:
sip.timer.Read = true
return sip.flush()
}
}
// No longer running, we've closed our batcher.
return nil, nil
}
func (sip *streamIngestionProcessor) bufferKV(event partitionEvent) error {
// TODO: In addition to flushing when receiving a checkpoint event, we
// should also flush when we've buffered sufficient KVs. A buffering adder
// would save us here.
kv := event.GetKV()
if kv == nil {
return errors.New("kv event expected to have kv")
}
mvccKey := storage.MVCCKey{
Key: kv.Key,
Timestamp: kv.Value.Timestamp,
}
sip.curBatch = append(sip.curBatch, storage.MVCCKeyValue{Key: mvccKey, Value: kv.Value.RawBytes})
return nil
}
func (sip *streamIngestionProcessor) bufferCheckpoint(event partitionEvent) error {
resolvedTimePtr := event.GetResolved()
if resolvedTimePtr == nil {
return errors.New("checkpoint event expected to have a resolved timestamp")
}
resolvedTime := *resolvedTimePtr
// Buffer the checkpoint.
if lastTimestamp, ok := sip.bufferedCheckpoints[event.partition]; !ok || lastTimestamp.Less(resolvedTime) {
sip.bufferedCheckpoints[event.partition] = resolvedTime
}
return nil
}
func (sip *streamIngestionProcessor) flush() (*jobspb.ResolvedSpans, error) {
flushedCheckpoints := jobspb.ResolvedSpans{ResolvedSpans: make([]jobspb.ResolvedSpan, 0)}
// Ensure that the current batch is sorted.
sort.Sort(sip.curBatch)
for _, kv := range sip.curBatch {
if err := sip.batcher.AddMVCCKey(sip.Ctx, kv.Key, kv.Value); err != nil {
return nil, errors.Wrapf(err, "adding key %+v", kv)
}
}
if err := sip.batcher.Flush(sip.Ctx); err != nil {
return nil, errors.Wrap(err, "flushing")
}
// Go through buffered checkpoint events, and put them on the channel to be
// emitted to the downstream frontier processor.
for partition, timestamp := range sip.bufferedCheckpoints {
// Each partition is represented by a span defined by the
// partition address.
spanStartKey := roachpb.Key(partition)
resolvedSpan := jobspb.ResolvedSpan{
Span: roachpb.Span{Key: spanStartKey, EndKey: spanStartKey.Next()},
Timestamp: timestamp,
}
flushedCheckpoints.ResolvedSpans = append(flushedCheckpoints.ResolvedSpans, resolvedSpan)
}
// Reset the current batch.
sip.curBatch = nil
sip.lastFlushTime = timeutil.Now()
sip.bufferedCheckpoints = make(map[streamingccl.PartitionAddress]hlc.Timestamp)
return &flushedCheckpoints, sip.batcher.Reset(sip.Ctx)
}
func init() {
rowexec.NewStreamIngestionDataProcessor = newStreamIngestionDataProcessor
}