-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
dist_sender_rangefeed.go
405 lines (370 loc) · 12.1 KB
/
dist_sender_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
395
396
397
398
399
400
401
402
403
404
405
// Copyright 2018 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 kvcoord
import (
"context"
"fmt"
"io"
"sync"
"time"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"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/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
)
type singleRangeInfo struct {
rs roachpb.RSpan
ts hlc.Timestamp
token rangecache.EvictionToken
}
// RangeFeed divides a RangeFeed request on range boundaries and establishes a
// RangeFeed to each of the individual ranges. It streams back results on the
// provided channel.
//
// Note that the timestamps in RangeFeedCheckpoint events that are streamed back
// may be lower than the timestamp given here.
func (ds *DistSender) RangeFeed(
ctx context.Context,
span roachpb.Span,
ts hlc.Timestamp,
withDiff bool,
eventCh chan<- *roachpb.RangeFeedEvent,
) error {
ctx = ds.AnnotateCtx(ctx)
ctx, sp := tracing.EnsureChildSpan(ctx, ds.AmbientContext.Tracer, "dist sender")
defer sp.Finish()
rs, err := keys.SpanAddr(span)
if err != nil {
return err
}
rr := &rangeFeedRegistry{
RangeFeedContext: RangeFeedContext{
CtxTags: logtags.FromContext(ctx).String(),
Span: span,
TS: ts,
WithDiff: withDiff,
},
}
ds.activeRangeFeeds.Store(rr, nil)
defer ds.activeRangeFeeds.Delete(rr)
g := ctxgroup.WithContext(ctx)
// Goroutine that processes subdivided ranges and creates a rangefeed for
// each.
rangeCh := make(chan singleRangeInfo, 16)
g.GoCtx(func(ctx context.Context) error {
for {
select {
case sri := <-rangeCh:
// Spawn a child goroutine to process this feed.
g.GoCtx(func(ctx context.Context) error {
return rr.startPartialRangeFeed(ctx, ds, sri.rs, sri.ts, sri.token, withDiff, rangeCh, eventCh)
})
case <-ctx.Done():
return ctx.Err()
}
}
})
// Kick off the initial set of ranges.
g.GoCtx(func(ctx context.Context) error {
return ds.divideAndSendRangeFeedToRanges(ctx, rs, ts, rangeCh)
})
return g.Wait()
}
// RangeFeedContext is the structure containing arguments passed to
// RangeFeed call. It functions as a kind of key for an active range feed.
type RangeFeedContext struct {
CtxTags string // context tags
// Span, timestamp and withDiff options passed to RangeFeed call.
Span roachpb.Span
TS hlc.Timestamp
WithDiff bool
}
// ActiveRangeFeed structure describes the state of currently executing range feed.
type ActiveRangeFeed struct {
Span roachpb.Span
StartTS hlc.Timestamp
NodeID roachpb.NodeID
LastEvent time.Time
}
// ActiveRangeFeedIterFn is an iterator function which is passed ActiveRangeFeed structure.
type ActiveRangeFeedIterFn func(rfCtx RangeFeedContext, feed ActiveRangeFeed) error
// ForEachActiveRangeFeed invokes provided function for each active range feed.
func (ds *DistSender) ForEachActiveRangeFeed(fn ActiveRangeFeedIterFn) (iterErr error) {
const continueIter = true
const stopIter = false
ds.activeRangeFeeds.Range(func(k, v interface{}) bool {
r := k.(*rangeFeedRegistry)
r.ranges.Range(func(k, v interface{}) bool {
active := k.(*activeRangeFeed)
active.Lock()
defer active.Unlock()
if err := fn(r.RangeFeedContext, active.ActiveRangeFeed); err != nil {
iterErr = err
return stopIter
}
return continueIter
})
return iterErr == nil
})
return
}
// activeRangeFeed is a thread safe ActiveRangeFeed.
type activeRangeFeed struct {
syncutil.Mutex
ActiveRangeFeed
}
func (a *activeRangeFeed) onRangeEvent(nodeID roachpb.NodeID) {
a.Lock()
defer a.Unlock()
a.LastEvent = timeutil.Now()
a.NodeID = nodeID
}
// rangeFeedRegistry is responsible for keeping track of currently executing
// range feeds.
type rangeFeedRegistry struct {
RangeFeedContext
// Map of ranges (activeRangeFeed* -> nil) started by this registry.
ranges sync.Map
}
// startPartialRangeFeed starts execution of partial rangefeed.
func (r *rangeFeedRegistry) startPartialRangeFeed(
ctx context.Context,
ds *DistSender,
rs roachpb.RSpan,
ts hlc.Timestamp,
token rangecache.EvictionToken,
withDiff bool,
rangeCh chan<- singleRangeInfo,
eventCh chan<- *roachpb.RangeFeedEvent,
) error {
active := &activeRangeFeed{}
active.ActiveRangeFeed = ActiveRangeFeed{
Span: rs.AsRawSpanWithNoLocals(),
StartTS: ts,
}
r.ranges.Store(active, nil)
defer r.ranges.Delete(active)
return ds.partialRangeFeed(
ctx, rs, ts, token, withDiff, rangeCh, eventCh, active.onRangeEvent)
}
func (ds *DistSender) divideAndSendRangeFeedToRanges(
ctx context.Context, rs roachpb.RSpan, ts hlc.Timestamp, rangeCh chan<- singleRangeInfo,
) error {
// As RangeIterator iterates, it can return overlapping descriptors (and
// during splits, this happens frequently), but divideAndSendRangeFeedToRanges
// intends to split up the input into non-overlapping spans aligned to range
// boundaries. So, as we go, keep track of the remaining uncovered part of
// `rs` in `nextRS`.
nextRS := rs
ri := NewRangeIterator(ds)
for ri.Seek(ctx, nextRS.Key, Ascending); ri.Valid(); ri.Next(ctx) {
desc := ri.Desc()
partialRS, err := nextRS.Intersect(desc)
if err != nil {
return err
}
nextRS.Key = partialRS.EndKey
select {
case rangeCh <- singleRangeInfo{
rs: partialRS,
ts: ts,
token: ri.Token(),
}:
case <-ctx.Done():
return ctx.Err()
}
if !ri.NeedAnother(nextRS) {
break
}
}
return ri.Error()
}
// partialRangeFeed establishes a RangeFeed to the range specified by desc. It
// manages lifecycle events of the range in order to maintain the RangeFeed
// connection; this may involve instructing higher-level functions to retry
// this rangefeed, or subdividing the range further in the event of a split.
func (ds *DistSender) partialRangeFeed(
ctx context.Context,
rs roachpb.RSpan,
ts hlc.Timestamp,
token rangecache.EvictionToken,
withDiff bool,
rangeCh chan<- singleRangeInfo,
eventCh chan<- *roachpb.RangeFeedEvent,
onRangeEvent onRangeEventCb,
) error {
// Bound the partial rangefeed to the partial span.
span := rs.AsRawSpanWithNoLocals()
// Start a retry loop for sending the batch to the range.
for r := retry.StartWithCtx(ctx, ds.rpcRetryOptions); r.Next(); {
// If we've cleared the descriptor on a send failure, re-lookup.
if !token.Valid() {
var err error
ri, err := ds.getRoutingInfo(ctx, rs.Key, rangecache.EvictionToken{}, false)
if err != nil {
log.VErrEventf(ctx, 1, "range descriptor re-lookup failed: %s", err)
if !rangecache.IsRangeLookupErrorRetryable(err) {
return err
}
continue
}
token = ri
}
// Establish a RangeFeed for a single Range.
maxTS, err := ds.singleRangeFeed(ctx, span, ts, withDiff, token.Desc(), eventCh, onRangeEvent)
// Forward the timestamp in case we end up sending it again.
ts.Forward(maxTS)
if err != nil {
if log.V(1) {
log.Infof(ctx, "RangeFeed %s disconnected with last lastEvent %s ago: %v",
span, timeutil.Since(ts.GoTime()), err)
}
switch {
case errors.HasType(err, (*roachpb.StoreNotFoundError)(nil)) ||
errors.HasType(err, (*roachpb.NodeUnavailableError)(nil)):
// These errors are likely to be unique to the replica that
// reported them, so no action is required before the next
// retry.
case IsSendError(err), errors.HasType(err, (*roachpb.RangeNotFoundError)(nil)):
// Evict the descriptor from the cache and reload on next attempt.
token.Evict(ctx)
token = rangecache.EvictionToken{}
continue
case errors.HasType(err, (*roachpb.RangeKeyMismatchError)(nil)):
// Evict the descriptor from the cache.
token.Evict(ctx)
return ds.divideAndSendRangeFeedToRanges(ctx, rs, ts, rangeCh)
case errors.HasType(err, (*roachpb.RangeFeedRetryError)(nil)):
var t *roachpb.RangeFeedRetryError
if ok := errors.As(err, &t); !ok {
return errors.AssertionFailedf("wrong error type: %T", err)
}
switch t.Reason {
case roachpb.RangeFeedRetryError_REASON_REPLICA_REMOVED,
roachpb.RangeFeedRetryError_REASON_RAFT_SNAPSHOT,
roachpb.RangeFeedRetryError_REASON_LOGICAL_OPS_MISSING,
roachpb.RangeFeedRetryError_REASON_SLOW_CONSUMER:
// Try again with same descriptor. These are transient
// errors that should not show up again.
continue
case roachpb.RangeFeedRetryError_REASON_RANGE_SPLIT,
roachpb.RangeFeedRetryError_REASON_RANGE_MERGED,
roachpb.RangeFeedRetryError_REASON_NO_LEASEHOLDER:
// Evict the descriptor from the cache.
token.Evict(ctx)
return ds.divideAndSendRangeFeedToRanges(ctx, rs, ts, rangeCh)
default:
return errors.AssertionFailedf("unrecognized retriable error type: %T", err)
}
default:
return err
}
}
}
return ctx.Err()
}
type onRangeEventCb func(nodeID roachpb.NodeID)
// singleRangeFeed gathers and rearranges the replicas, and makes a RangeFeed
// RPC call. Results will be sent on the provided channel. Returns the timestamp
// of the maximum rangefeed lastEvent seen, which can be used to re-establish
// the rangefeed with a larger starting timestamp, reflecting the fact that all
// values up to the last lastEvent have already been observed. Returns the
// request's timestamp if not checkpoints are seen.
func (ds *DistSender) singleRangeFeed(
ctx context.Context,
span roachpb.Span,
ts hlc.Timestamp,
withDiff bool,
desc *roachpb.RangeDescriptor,
eventCh chan<- *roachpb.RangeFeedEvent,
onRangeEvent onRangeEventCb,
) (hlc.Timestamp, error) {
args := roachpb.RangeFeedRequest{
Span: span,
Header: roachpb.Header{
Timestamp: ts,
RangeID: desc.RangeID,
},
WithDiff: withDiff,
}
var latencyFn LatencyFunc
if ds.rpcContext != nil {
latencyFn = ds.rpcContext.RemoteClocks.Latency
}
replicas, err := NewReplicaSlice(ctx, ds.nodeDescs, desc, nil, AllExtantReplicas)
if err != nil {
return args.Timestamp, err
}
replicas.OptimizeReplicaOrder(ds.getNodeDescriptor(), latencyFn)
// The RangeFeed is not used for system critical traffic so use a DefaultClass
// connection regardless of the range.
opts := SendOptions{class: rpc.DefaultClass}
transport, err := ds.transportFactory(opts, ds.nodeDialer, replicas)
if err != nil {
return args.Timestamp, err
}
defer transport.Release()
for {
if transport.IsExhausted() {
return args.Timestamp, newSendError(
fmt.Sprintf("sending to all %d replicas failed", len(replicas)))
}
args.Replica = transport.NextReplica()
clientCtx, client, err := transport.NextInternalClient(ctx)
if err != nil {
log.VErrEventf(ctx, 2, "RPC error: %s", err)
continue
}
log.VEventf(ctx, 3, "attempting to create a RangeFeed over replica %s", args.Replica)
stream, err := client.RangeFeed(clientCtx, &args)
if err != nil {
log.VErrEventf(ctx, 2, "RPC error: %s", err)
if grpcutil.IsAuthError(err) {
// Authentication or authorization error. Propagate.
return args.Timestamp, err
}
continue
}
for {
event, err := stream.Recv()
if err == io.EOF {
return args.Timestamp, nil
}
if err != nil {
return args.Timestamp, err
}
switch t := event.GetValue().(type) {
case *roachpb.RangeFeedCheckpoint:
if t.Span.Contains(args.Span) {
args.Timestamp.Forward(t.ResolvedTS)
}
case *roachpb.RangeFeedError:
log.VErrEventf(ctx, 2, "RangeFeedError: %s", t.Error.GoError())
return args.Timestamp, t.Error.GoError()
}
onRangeEvent(args.Replica.NodeID)
select {
case eventCh <- event:
case <-ctx.Done():
return args.Timestamp, ctx.Err()
}
}
}
}