-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
rangefeed.go
394 lines (345 loc) · 12.2 KB
/
rangefeed.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
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package rangefeed
import (
"context"
"fmt"
"runtime/pprof"
"sync"
"sync/atomic"
"time"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/span"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/cockroachdb/redact"
)
//go:generate mockgen -destination=mocks_generated_test.go --package=rangefeed . DB
// TODO(ajwerner): Expose hooks for metrics.
// TODO(ajwerner): Expose access to checkpoints and the frontier.
// TODO(ajwerner): Expose better control over how the exponential backoff gets
// reset when the feed has been running successfully for a while.
// TODO(yevgeniy): Instead of rolling our own logic to parallelize scans, we should
// use streamer API instead (https://github.com/cockroachdb/cockroach/pull/68430)
// DB is an adapter to the underlying KV store.
type DB interface {
// RangeFeed runs a rangefeed on a given span with the given arguments.
// It encapsulates the RangeFeed method on roachpb.Internal.
RangeFeed(
ctx context.Context,
spans []roachpb.Span,
startFrom hlc.Timestamp,
eventC chan<- kvcoord.RangeFeedMessage,
opts ...kvcoord.RangeFeedOption,
) error
// Scan encapsulates scanning a key span at a given point in time. The method
// deals with pagination, calling the caller back for each row. Note that
// the API does not require that the rows be ordered to allow for future
// parallelism.
Scan(
ctx context.Context,
spans []roachpb.Span,
asOf hlc.Timestamp,
rowFn func(value roachpb.KeyValue),
cfg scanConfig,
) error
}
// Factory is used to construct RangeFeeds.
type Factory struct {
stopper *stop.Stopper
client DB
knobs *TestingKnobs
}
// TestingKnobs is used to inject behavior into a rangefeed for testing.
type TestingKnobs struct {
// OnRangefeedRestart is called when a rangefeed restarts.
OnRangefeedRestart func()
// IgnoreOnDeleteRangeError will ignore any errors where a DeleteRange event
// is emitted without an OnDeleteRange handler. This can be used e.g. with
// StoreTestingKnobs.GlobalMVCCRangeTombstone, to prevent the global tombstone
// causing rangefeed errors for consumers who don't expect it.
IgnoreOnDeleteRangeError bool
}
// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
func (*TestingKnobs) ModuleTestingKnobs() {}
var _ base.ModuleTestingKnobs = (*TestingKnobs)(nil)
// NewFactory constructs a new Factory.
func NewFactory(
stopper *stop.Stopper, db *kv.DB, st *cluster.Settings, knobs *TestingKnobs,
) (*Factory, error) {
kvDB, err := newDBAdapter(db, st)
if err != nil {
return nil, err
}
return newFactory(stopper, kvDB, knobs), nil
}
func newFactory(stopper *stop.Stopper, client DB, knobs *TestingKnobs) *Factory {
return &Factory{
stopper: stopper,
client: client,
knobs: knobs,
}
}
// RangeFeed constructs a new rangefeed and runs it in an async task.
//
// The rangefeed can be stopped via Close(); otherwise, it will stop when the
// server shuts down. The only error which can be returned will indicate that
// the server is being shut down.
//
// Rangefeeds do not support inline (unversioned) values, and may omit them or
// error on them. Similarly, rangefeeds will error if MVCC history is mutated
// via e.g. ClearRange. Do not use rangefeeds across such key spans.
//
// NB: for the rangefeed itself, initialTimestamp is exclusive, i.e. the first
// possible event emitted by the server (including the catchup scan) is at
// initialTimestamp.Next(). This follows from the gRPC API semantics. However,
// the initial scan (if any) is run at initialTimestamp.
func (f *Factory) RangeFeed(
ctx context.Context,
name string,
spans []roachpb.Span,
initialTimestamp hlc.Timestamp,
onValue OnValue,
options ...Option,
) (_ *RangeFeed, err error) {
r := f.New(name, initialTimestamp, onValue, options...)
if err := r.Start(ctx, spans); err != nil {
return nil, err
}
return r, nil
}
// New constructs a new RangeFeed (without running it).
func (f *Factory) New(
name string, initialTimestamp hlc.Timestamp, onValue OnValue, options ...Option,
) *RangeFeed {
r := RangeFeed{
client: f.client,
stopper: f.stopper,
knobs: f.knobs,
initialTimestamp: initialTimestamp,
name: name,
onValue: onValue,
}
initConfig(&r.config, options)
return &r
}
// OnValue is called for each rangefeed value.
type OnValue func(ctx context.Context, value *kvpb.RangeFeedValue)
// RangeFeed represents a running RangeFeed.
type RangeFeed struct {
config
name string
client DB
stopper *stop.Stopper
knobs *TestingKnobs
initialTimestamp hlc.Timestamp
spans []roachpb.Span
spansDebugStr string // Debug string describing spans
onValue OnValue
cancel context.CancelFunc
running sync.WaitGroup
started int32 // accessed atomically
}
// Start kicks off the rangefeed in an async task, it can only be invoked once.
// All the installed callbacks (OnValue, OnCheckpoint, OnFrontierAdvance,
// OnInitialScanDone) are called in said async task in a single thread.
func (f *RangeFeed) Start(ctx context.Context, spans []roachpb.Span) error {
if len(spans) == 0 {
return errors.AssertionFailedf("expected at least 1 span, got none")
}
if !atomic.CompareAndSwapInt32(&f.started, 0, 1) {
return errors.AssertionFailedf("rangefeed already started")
}
// Maintain a frontier in order to resume at a reasonable timestamp.
// TODO(ajwerner): Consider exposing the frontier through a RangeFeed method.
// Doing so would require some synchronization.
frontier, err := span.MakeFrontier(spans...)
if err != nil {
return err
}
for _, sp := range spans {
if _, err := frontier.Forward(sp, f.initialTimestamp); err != nil {
return err
}
}
// Frontier merges and de-dups passed in spans. So, use frontier to initialize
// sorted list of spans.
frontier.Entries(func(sp roachpb.Span, _ hlc.Timestamp) (done span.OpResult) {
f.spans = append(f.spans, sp)
return span.ContinueMatch
})
runWithFrontier := func(ctx context.Context) {
// pprof.Do function does exactly what we do here, but it also results in
// pprof.Do function showing up in the stack traces -- so, just set and reset
// labels manually.
defer pprof.SetGoroutineLabels(ctx)
ctx = pprof.WithLabels(ctx, pprof.Labels(append(f.extraPProfLabels, "rangefeed", f.name)...))
pprof.SetGoroutineLabels(ctx)
f.running.Add(1)
f.run(ctx, frontier)
}
f.spansDebugStr = func() string {
n := len(spans)
if n == 1 {
return spans[0].String()
}
return fmt.Sprintf("{%s}", frontier.String())
}()
ctx = logtags.AddTag(ctx, "rangefeed", f.name)
ctx, f.cancel = f.stopper.WithCancelOnQuiesce(ctx)
if err := f.stopper.RunAsyncTask(ctx, "rangefeed", runWithFrontier); err != nil {
f.cancel()
return err
}
return nil
}
// Close closes the RangeFeed and waits for it to shut down; it does so
// idempotently. It waits for the currently running handler, if any, to complete
// and guarantees that no future handlers will be invoked after this point.
func (f *RangeFeed) Close() {
f.cancel()
f.running.Wait()
}
// Run the rangefeed in a loop in the case of failure, likely due to node
// failures or general unavailability. If the rangefeed runs successfully for at
// least this long, then after subsequent failures we would like to reset the
// exponential backoff to experience long delays between retry attempts.
// This is the threshold of successful running after which the backoff state
// will be reset.
const resetThreshold = 30 * time.Second
var useMuxRangeFeed = util.ConstantWithMetamorphicTestBool("use-mux-rangefeed", false)
// run will run the RangeFeed until the context is canceled or if the client
// indicates that an initial scan error is non-recoverable.
func (f *RangeFeed) run(ctx context.Context, frontier *span.Frontier) {
defer f.running.Done()
r := retry.StartWithCtx(ctx, f.retryOptions)
restartLogEvery := log.Every(10 * time.Second)
if f.withInitialScan {
if done := f.runInitialScan(ctx, &restartLogEvery, &r); done {
return
}
}
// Check the context before kicking off a rangefeed.
if ctx.Err() != nil {
return
}
// TODO(ajwerner): Consider adding event buffering. Doing so would require
// draining when the rangefeed fails.
eventCh := make(chan kvcoord.RangeFeedMessage)
var rangefeedOpts []kvcoord.RangeFeedOption
if f.scanConfig.overSystemTable {
rangefeedOpts = append(rangefeedOpts, kvcoord.WithSystemTablePriority())
}
if f.useMuxRangefeed {
rangefeedOpts = append(rangefeedOpts, kvcoord.WithMuxRangeFeed())
}
if f.withDiff {
rangefeedOpts = append(rangefeedOpts, kvcoord.WithDiff())
}
for i := 0; r.Next(); i++ {
ts := frontier.Frontier()
if log.ExpensiveLogEnabled(ctx, 1) {
log.Eventf(ctx, "starting rangefeed from %v on %v", ts, f.spansDebugStr)
}
start := timeutil.Now()
rangeFeedTask := func(ctx context.Context) error {
return f.client.RangeFeed(ctx, f.spans, ts, eventCh, rangefeedOpts...)
}
processEventsTask := func(ctx context.Context) error {
return f.processEvents(ctx, frontier, eventCh)
}
err := ctxgroup.GoAndWait(ctx, rangeFeedTask, processEventsTask)
if errors.HasType(err, &kvpb.BatchTimestampBeforeGCError{}) ||
errors.HasType(err, &kvpb.MVCCHistoryMutationError{}) {
if errCallback := f.onUnrecoverableError; errCallback != nil {
errCallback(ctx, err)
}
log.VEventf(ctx, 1, "exiting rangefeed due to internal error: %v", err)
return
}
if err != nil && ctx.Err() == nil && restartLogEvery.ShouldLog() {
log.Warningf(ctx, "rangefeed failed %d times, restarting: %v",
redact.Safe(i), err)
}
if ctx.Err() != nil {
log.VEventf(ctx, 1, "exiting rangefeed")
return
}
ranFor := timeutil.Since(start)
log.VEventf(ctx, 1, "restarting rangefeed for %v after %v",
f.spansDebugStr, ranFor)
if f.knobs != nil && f.knobs.OnRangefeedRestart != nil {
f.knobs.OnRangefeedRestart()
}
// If the rangefeed ran successfully for long enough, reset the retry
// state so that the exponential backoff begins from its minimum value.
if ranFor > resetThreshold {
i = 1
r.Reset()
}
}
}
// processEvents processes events sent by the rangefeed on the eventCh.
func (f *RangeFeed) processEvents(
ctx context.Context, frontier *span.Frontier, eventCh <-chan kvcoord.RangeFeedMessage,
) error {
for {
select {
case ev := <-eventCh:
switch {
case ev.Val != nil:
f.onValue(ctx, ev.Val)
case ev.Checkpoint != nil:
advanced, err := frontier.Forward(ev.Checkpoint.Span, ev.Checkpoint.ResolvedTS)
if err != nil {
return err
}
if f.onCheckpoint != nil {
f.onCheckpoint(ctx, ev.Checkpoint)
}
if advanced && f.onFrontierAdvance != nil {
f.onFrontierAdvance(ctx, frontier.Frontier())
}
case ev.SST != nil:
if f.onSSTable == nil {
return errors.AssertionFailedf(
"received unexpected rangefeed SST event with no OnSSTable handler")
}
f.onSSTable(ctx, ev.SST, ev.RegisteredSpan)
case ev.DeleteRange != nil:
if f.onDeleteRange == nil {
if f.knobs != nil && f.knobs.IgnoreOnDeleteRangeError {
continue
}
return errors.AssertionFailedf(
"received unexpected rangefeed DeleteRange event with no OnDeleteRange handler: %s", ev)
}
f.onDeleteRange(ctx, ev.DeleteRange)
case ev.Error != nil:
// Intentionally do nothing, we'll get an error returned from the
// call to RangeFeed.
}
case <-ctx.Done():
return ctx.Err()
}
}
}