-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
replica_send.go
774 lines (708 loc) · 29.8 KB
/
replica_send.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
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
// Copyright 2019 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 kvserver
import (
"context"
"reflect"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/observedts"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
)
// Send executes a command on this range, dispatching it to the
// read-only, read-write, or admin execution path as appropriate.
// ctx should contain the log tags from the store (and up).
func (r *Replica) Send(
ctx context.Context, ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, *roachpb.Error) {
return r.sendWithRangeID(ctx, r.RangeID, &ba)
}
// sendWithRangeID takes an unused rangeID argument so that the range
// ID will be accessible in stack traces (both in panics and when
// sampling goroutines from a live server). This line is subject to
// the whims of the compiler and it can be difficult to find the right
// value, but as of this writing the following example shows a stack
// while processing range 21 (0x15) (the first occurrence of that
// number is the rangeID argument, the second is within the encoded
// BatchRequest, although we don't want to rely on that occurring
// within the portion printed in the stack trace):
//
// github.com/cockroachdb/cockroach/pkg/storage.(*Replica).sendWithRangeID(0xc420d1a000, 0x64bfb80, 0xc421564b10, 0x15, 0x153fd4634aeb0193, 0x0, 0x100000001, 0x1, 0x15, 0x0, ...)
func (r *Replica) sendWithRangeID(
ctx context.Context, rangeID roachpb.RangeID, ba *roachpb.BatchRequest,
) (*roachpb.BatchResponse, *roachpb.Error) {
var br *roachpb.BatchResponse
if r.leaseholderStats != nil && ba.Header.GatewayNodeID != 0 {
r.leaseholderStats.record(ba.Header.GatewayNodeID)
}
// Add the range log tag.
ctx = r.AnnotateCtx(ctx)
// If the internal Raft group is not initialized, create it and wake the leader.
r.maybeInitializeRaftGroup(ctx)
isReadOnly := ba.IsReadOnly()
if err := r.checkBatchRequest(ba, isReadOnly); err != nil {
return nil, roachpb.NewError(err)
}
if err := r.maybeBackpressureBatch(ctx, ba); err != nil {
return nil, roachpb.NewError(err)
}
if err := r.maybeRateLimitBatch(ctx, ba); err != nil {
return nil, roachpb.NewError(err)
}
// NB: must be performed before collecting request spans.
ba, err := maybeStripInFlightWrites(ba)
if err != nil {
return nil, roachpb.NewError(err)
}
if filter := r.store.cfg.TestingKnobs.TestingRequestFilter; filter != nil {
if pErr := filter(ctx, *ba); pErr != nil {
return nil, pErr
}
}
// Differentiate between read-write, read-only, and admin.
var pErr *roachpb.Error
if isReadOnly {
log.Event(ctx, "read-only path")
fn := (*Replica).executeReadOnlyBatch
br, pErr = r.executeBatchWithConcurrencyRetries(ctx, ba, fn)
} else if ba.IsWrite() {
log.Event(ctx, "read-write path")
fn := (*Replica).executeWriteBatch
br, pErr = r.executeBatchWithConcurrencyRetries(ctx, ba, fn)
} else if ba.IsAdmin() {
log.Event(ctx, "admin path")
br, pErr = r.executeAdminBatch(ctx, ba)
} else if len(ba.Requests) == 0 {
// empty batch; shouldn't happen (we could handle it, but it hints
// at someone doing weird things, and once we drop the key range
// from the header it won't be clear how to route those requests).
log.Fatalf(ctx, "empty batch")
} else {
log.Fatalf(ctx, "don't know how to handle command %s", ba)
}
if pErr != nil {
log.Eventf(ctx, "replica.Send got error: %s", pErr)
} else {
if filter := r.store.cfg.TestingKnobs.TestingResponseFilter; filter != nil {
pErr = filter(ctx, *ba, br)
}
}
// Return range information if it was requested. Note that we don't return it
// on errors because the code doesn't currently support returning both a br
// and a pErr here. Also, some errors (e.g. NotLeaseholderError) have custom
// ways of returning range info.
if pErr == nil {
r.maybeAddRangeInfoToResponse(ctx, ba, br)
}
r.recordImpactOnRateLimiter(ctx, br)
return br, pErr
}
func (r *Replica) maybeAddRangeInfoToResponse(
ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse,
) {
if ba.ReturnRangeInfo {
desc, lease := r.GetDescAndLease(ctx)
br.RangeInfos = []roachpb.RangeInfo{{Desc: desc, Lease: lease}}
if !r.ClusterSettings().Version.IsActive(ctx, clusterversion.ClientRangeInfosOnBatchResponse) {
// Also set the RangeInfo on the individual responses, for compatibility
// with 20.1.
for _, r := range br.Responses {
reply := r.GetInner()
header := reply.Header()
header.DeprecatedRangeInfos = br.RangeInfos
reply.SetHeader(header)
}
}
} else if ba.ClientRangeInfo != nil {
returnRangeInfoIfClientStale(ctx, br, r, *ba.ClientRangeInfo)
}
}
// returnRangeInfoIfClientStale populates br.RangeInfos if the client doesn't
// have up-to-date info about the range's descriptor and lease.
func returnRangeInfoIfClientStale(
ctx context.Context, br *roachpb.BatchResponse, r *Replica, cinfo roachpb.ClientRangeInfo,
) {
desc, lease := r.GetDescAndLease(ctx)
// Compare the client's info with the replica's info to detect if the client
// has stale knowledge. Note that the client can have more recent knowledge
// than the replica in case this is a follower.
needInfo := (cinfo.LeaseSequence < lease.Sequence) ||
(cinfo.DescriptorGeneration < desc.Generation)
if !needInfo {
return
}
log.VEventf(ctx, 3, "client had stale range info; returning an update")
br.RangeInfos = []roachpb.RangeInfo{
{
Desc: desc,
Lease: lease,
},
}
// We're going to sometimes return info on the ranges coming right before or
// right after r, if it looks like r came from a range that has recently split
// and the client doesn't know about it. After a split, the client benefits
// from learning about both resulting ranges.
if cinfo.DescriptorGeneration >= desc.Generation {
return
}
maybeAddRange := func(rr KeyRange) {
if rr.Desc().Generation != desc.Generation {
// The next range does not look like it came from a split that produced
// both r and this next range. Of course, this has false negatives (e.g.
// if either the LHS or the RHS split multiple times since the client's
// version). For best fidelity, the client could send the range's start
// and end keys and the server could use that to return all overlapping
// descriptors (like we do for RangeKeyMismatchErrors), but sending those
// keys on every RPC seems too expensive.
return
}
var rangeInfo roachpb.RangeInfo
if rep, ok := rr.(*Replica); ok {
// Note that we return the lease even if it's expired. The kvclient can
// use it as it sees fit.
rangeInfo.Desc, rangeInfo.Lease = rep.GetDescAndLease(ctx)
} else {
rangeInfo.Desc = *rr.Desc()
}
br.RangeInfos = append(br.RangeInfos, rangeInfo)
}
r.store.VisitReplicasByKey(ctx, roachpb.RKeyMin, desc.StartKey, DescendingKeyOrder, func(ctx context.Context, prevR KeyRange) bool {
if !prevR.Desc().EndKey.Equal(desc.StartKey) {
// The next range does not correspond to the range immediately preceding r.
return false
}
maybeAddRange(prevR)
return false
})
r.store.VisitReplicasByKey(ctx, desc.EndKey, roachpb.RKeyMax, AscendingKeyOrder, func(ctx context.Context, nextR KeyRange) bool {
if !nextR.Desc().StartKey.Equal(desc.EndKey) {
// The next range does not correspond to the range immediately after r.
return false
}
maybeAddRange(nextR)
return false
})
}
// batchExecutionFn is a method on Replica that is able to execute a
// BatchRequest. It is called with the batch, along with the status of
// the lease that the batch is operating under and a guard for the
// latches protecting the request.
//
// The function will return either a batch response or an error. The function
// also has the option to pass ownership of the concurrency guard back to the
// caller. However, it does not need to. Instead, it can assume responsibility
// for releasing the concurrency guard it was provided by returning nil. This is
// useful is cases where the function:
// 1. eagerly released the concurrency guard after it determined that isolation
// from conflicting requests was no longer needed.
// 2. is continuing to execute asynchronously and needs to maintain isolation
// from conflicting requests throughout the lifetime of its asynchronous
// processing. The most prominent example of asynchronous processing is
// with requests that have the "async consensus" flag set. A more subtle
// case is with requests that are acknowledged by the Raft machinery after
// their Raft entry has been committed but before it has been applied to
// the replicated state machine. In all of these cases, responsibility
// for releasing the concurrency guard is handed to Raft.
//
// However, this option is not permitted if the function returns a "server-side
// concurrency retry error" (see isConcurrencyRetryError for more details). If
// the function returns one of these errors, it must also pass ownership of the
// concurrency guard back to the caller.
type batchExecutionFn func(
*Replica, context.Context, *roachpb.BatchRequest, kvserverpb.LeaseStatus, *concurrency.Guard,
) (*roachpb.BatchResponse, *concurrency.Guard, *roachpb.Error)
var _ batchExecutionFn = (*Replica).executeWriteBatch
var _ batchExecutionFn = (*Replica).executeReadOnlyBatch
// executeBatchWithConcurrencyRetries is the entry point for client (non-admin)
// requests that execute against the range's state. The method coordinates the
// execution of requests that may require multiple retries due to interactions
// with concurrent transactions.
//
// The method acquires latches for the request, which synchronizes it with
// conflicting requests. This permits the execution function to run without
// concern of coordinating with logically conflicting operations, although it
// still needs to worry about coordinating with non-conflicting operations when
// accessing shared data structures.
//
// If the execution function hits a concurrency error like a WriteIntentError or
// a TransactionPushError it will propagate the error back to this method, which
// handles the process of retrying batch execution after addressing the error.
func (r *Replica) executeBatchWithConcurrencyRetries(
ctx context.Context, ba *roachpb.BatchRequest, fn batchExecutionFn,
) (br *roachpb.BatchResponse, pErr *roachpb.Error) {
// Try to execute command; exit retry loop on success.
var g *concurrency.Guard
var latchSpans, lockSpans *spanset.SpanSet
requestEvalKind := concurrency.PessimisticEval
defer func() {
// NB: wrapped to delay g evaluation to its value when returning.
if g != nil {
r.concMgr.FinishReq(g)
}
}()
for {
// Exit loop if context has been canceled or timed out.
if err := ctx.Err(); err != nil {
return nil, roachpb.NewError(errors.Wrap(err, "aborted during Replica.Send"))
}
// Determine the lease under which to evaluate the request.
var status kvserverpb.LeaseStatus
if !ba.ReadConsistency.RequiresReadLease() {
// Get a clock reading for checkExecutionCanProceed.
status.Timestamp = r.Clock().Now()
} else if ba.IsSingleSkipLeaseCheckRequest() {
// For lease commands, use the provided previous lease for verification.
status.Lease = ba.GetPrevLeaseForLeaseRequest()
status.Timestamp = r.Clock().Now()
} else {
// If the request is a write or a consistent read, it requires the
// range lease or permission to serve via follower reads.
if status, pErr = r.redirectOnOrAcquireLease(ctx); pErr != nil {
if nErr := r.canServeFollowerRead(ctx, ba, pErr); nErr != nil {
return nil, nErr
}
}
}
// Limit the transaction's maximum timestamp using observed timestamps.
ba.Txn = observedts.LimitTxnMaxTimestamp(ctx, ba.Txn, status)
// Determine the maximal set of key spans that the batch will operate on.
// We only need to do this once and we make sure to do so after we have
// limited the transaction's maximum timestamp. This will be done only in
// the first iteration of the for loop, which means requestEvalKind can be
// set to OptimisticEval only in the first iteration.
if latchSpans == nil {
var err error
latchSpans, lockSpans, requestEvalKind, err = r.collectSpans(ba)
if err != nil {
return nil, roachpb.NewError(err)
}
// Handle load-based splitting.
r.recordBatchForLoadBasedSplitting(ctx, ba, latchSpans)
}
// SequenceReq may acquire latches, if not already held, to prevent
// overlapping requests from executing until this request completes. After
// latching, if not doing optimistic evaluation, it will wait on any
// conflicting locks to ensure that the request has full isolation during
// evaluation. This returns a request guard that must be eventually
// released. For optimistic evaluation, Guard.CheckOptimisticNoConflicts
// must be called immediately after successful evaluation.
var resp []roachpb.ResponseUnion
g, resp, pErr = r.concMgr.SequenceReq(ctx, g, concurrency.Request{
Txn: ba.Txn,
Timestamp: ba.Timestamp,
Priority: ba.UserPriority,
ReadConsistency: ba.ReadConsistency,
WaitPolicy: ba.WaitPolicy,
Requests: ba.Requests,
LatchSpans: latchSpans,
LockSpans: lockSpans,
EvalKind: requestEvalKind,
})
if pErr != nil {
return nil, pErr
} else if resp != nil {
br = new(roachpb.BatchResponse)
br.Responses = resp
return br, nil
}
if filter := r.store.cfg.TestingKnobs.TestingLatchFilter; filter != nil {
if pErr := filter(ctx, *ba); pErr != nil {
return nil, pErr
}
}
br, g, pErr = fn(r, ctx, ba, status, g)
if pErr == nil {
// Success.
return br, nil
} else if !isConcurrencyRetryError(pErr) {
// Propagate error.
return nil, pErr
}
// The batch execution func returned a server-side concurrency retry
// error. It must have also handed back ownership of the concurrency
// guard without having already released the guard's latches.
g.AssertLatches()
if filter := r.store.cfg.TestingKnobs.TestingConcurrencyRetryFilter; filter != nil {
filter(ctx, *ba, pErr)
}
// Typically, retries are marked PessimisticEval. The one exception is a
// pessimistic retry immediately after an optimistic eval which failed
// when checking for conflicts, which is handled below. Note that an
// optimistic eval failure for any other reason will also retry as
// PessimisticEval.
requestEvalKind = concurrency.PessimisticEval
switch t := pErr.GetDetail().(type) {
case *roachpb.WriteIntentError:
// Drop latches, but retain lock wait-queues.
if g, pErr = r.handleWriteIntentError(ctx, ba, g, status.Lease, pErr, t); pErr != nil {
return nil, pErr
}
case *roachpb.TransactionPushError:
// Drop latches, but retain lock wait-queues.
if g, pErr = r.handleTransactionPushError(ctx, ba, g, pErr, t); pErr != nil {
return nil, pErr
}
case *roachpb.IndeterminateCommitError:
// Drop latches and lock wait-queues.
r.concMgr.FinishReq(g)
g = nil
// Then launch a task to handle the indeterminate commit error.
if pErr = r.handleIndeterminateCommitError(ctx, ba, pErr, t); pErr != nil {
return nil, pErr
}
case *roachpb.MergeInProgressError:
// Drop latches and lock wait-queues.
r.concMgr.FinishReq(g)
g = nil
// Then listen for the merge to complete.
if pErr = r.handleMergeInProgressError(ctx, ba, pErr, t); pErr != nil {
return nil, pErr
}
case *roachpb.OptimisticEvalConflictsError:
// We are deliberately not dropping latches. The next iteration will
// pessimistically check for locks while holding these latches, and will
// find them again and queue up, and then release latches.
requestEvalKind = concurrency.PessimisticAfterFailedOptimisticEval
default:
log.Fatalf(ctx, "unexpected concurrency retry error %T", t)
}
// Retry...
}
}
// isConcurrencyRetryError returns whether or not the provided error is a
// "concurrency retry error" that will be captured and retried by
// executeBatchWithConcurrencyRetries. Most concurrency retry errors are
// handled by dropping a request's latches, waiting for and/or ensuring that
// the condition which caused the error is handled, re-sequencing through the
// concurrency manager, and executing the request again. The one exception is
// OptimisticEvalConflictsError, where there is no need to drop latches, and
// the request can immediately proceed to retrying pessimistically.
func isConcurrencyRetryError(pErr *roachpb.Error) bool {
switch pErr.GetDetail().(type) {
case *roachpb.WriteIntentError:
// If a request hits a WriteIntentError, it adds the conflicting intent
// to the lockTable through a process called "lock discovery". It then
// waits in the lock's wait-queue during its next sequencing pass.
case *roachpb.TransactionPushError:
// If a PushTxn request hits a TransactionPushError, it attempted to
// push another transactions record but did not succeed. It enqueues the
// pushee transaction in the txnWaitQueue and waits on the record to
// change or expire during its next sequencing pass.
case *roachpb.IndeterminateCommitError:
// If a PushTxn hits a IndeterminateCommitError, it attempted to push an
// expired transaction record in the STAGING state. It's unclear whether
// the pushee is aborted or committed, so the request must kick off the
// "transaction recovery procedure" to resolve this ambiguity before
// retrying.
case *roachpb.MergeInProgressError:
// If a request hits a MergeInProgressError, the replica it is being
// evaluated against is in the process of being merged into its left-hand
// neighbor. The request cannot proceed until the range merge completes,
// either successfully or unsuccessfully, so it waits before retrying.
// If the merge does complete successfully, the retry will be rejected
// with an error that will propagate back to the client.
case *roachpb.OptimisticEvalConflictsError:
// Optimistic evaluation encountered a conflict. The request will
// immediately retry pessimistically.
default:
return false
}
return true
}
func (r *Replica) handleWriteIntentError(
ctx context.Context,
ba *roachpb.BatchRequest,
g *concurrency.Guard,
lease roachpb.Lease,
pErr *roachpb.Error,
t *roachpb.WriteIntentError,
) (*concurrency.Guard, *roachpb.Error) {
if r.store.cfg.TestingKnobs.DontPushOnWriteIntentError {
return g, pErr
}
// g's latches will be dropped, but it retains its spot in lock wait-queues.
return r.concMgr.HandleWriterIntentError(ctx, g, lease.Sequence, t)
}
func (r *Replica) handleTransactionPushError(
ctx context.Context,
ba *roachpb.BatchRequest,
g *concurrency.Guard,
pErr *roachpb.Error,
t *roachpb.TransactionPushError,
) (*concurrency.Guard, *roachpb.Error) {
// On a transaction push error, retry immediately if doing so will enqueue
// into the txnWaitQueue in order to await further updates to the unpushed
// txn's status. We check ShouldPushImmediately to avoid retrying
// non-queueable PushTxnRequests (see #18191).
dontRetry := r.store.cfg.TestingKnobs.DontRetryPushTxnFailures
if !dontRetry && ba.IsSinglePushTxnRequest() {
pushReq := ba.Requests[0].GetInner().(*roachpb.PushTxnRequest)
dontRetry = txnwait.ShouldPushImmediately(pushReq)
}
if dontRetry {
return g, pErr
}
// g's latches will be dropped, but it retains its spot in lock wait-queues
// (though a PushTxn shouldn't be in any lock wait-queues).
return r.concMgr.HandleTransactionPushError(ctx, g, t), nil
}
func (r *Replica) handleIndeterminateCommitError(
ctx context.Context,
ba *roachpb.BatchRequest,
pErr *roachpb.Error,
t *roachpb.IndeterminateCommitError,
) *roachpb.Error {
if r.store.cfg.TestingKnobs.DontRecoverIndeterminateCommits {
return pErr
}
// On an indeterminate commit error, attempt to recover and finalize the
// stuck transaction. Retry immediately if successful.
if _, err := r.store.recoveryMgr.ResolveIndeterminateCommit(ctx, t); err != nil {
// Do not propagate ambiguous results; assume success and retry original op.
if errors.HasType(err, (*roachpb.AmbiguousResultError)(nil)) {
return nil
}
// Propagate new error. Preserve the error index.
newPErr := roachpb.NewError(err)
newPErr.Index = pErr.Index
return newPErr
}
// We've recovered the transaction that blocked the push; retry command.
return nil
}
func (r *Replica) handleMergeInProgressError(
ctx context.Context,
ba *roachpb.BatchRequest,
pErr *roachpb.Error,
t *roachpb.MergeInProgressError,
) *roachpb.Error {
// A merge was in progress. We need to retry the command after the merge
// completes, as signaled by the closing of the replica's mergeComplete
// channel. Note that the merge may have already completed, in which case
// its mergeComplete channel will be nil.
mergeCompleteCh := r.getMergeCompleteCh()
if mergeCompleteCh == nil {
// Merge no longer in progress. Retry the command.
return nil
}
log.Event(ctx, "waiting on in-progress merge")
select {
case <-mergeCompleteCh:
// Merge complete. Retry the command.
return nil
case <-ctx.Done():
return roachpb.NewError(errors.Wrap(ctx.Err(), "aborted during merge"))
case <-r.store.stopper.ShouldQuiesce():
return roachpb.NewError(&roachpb.NodeUnavailableError{})
}
}
// executeAdminBatch executes the command directly. There is no interaction
// with the spanlatch manager or the timestamp cache, as admin commands
// are not meant to consistently access or modify the underlying data.
// Admin commands must run on the lease holder replica. Batch support here is
// limited to single-element batches; everything else catches an error.
func (r *Replica) executeAdminBatch(
ctx context.Context, ba *roachpb.BatchRequest,
) (*roachpb.BatchResponse, *roachpb.Error) {
if len(ba.Requests) != 1 {
return nil, roachpb.NewErrorf("only single-element admin batches allowed")
}
args := ba.Requests[0].GetInner()
if sp := tracing.SpanFromContext(ctx); sp != nil {
sp.SetOperationName(reflect.TypeOf(args).String())
}
// Admin commands always require the range lease.
status, pErr := r.redirectOnOrAcquireLease(ctx)
if pErr != nil {
return nil, pErr
}
// Note there is no need to limit transaction max timestamp on admin requests.
// Verify that the batch can be executed.
// NB: we pass nil for the spanlatch guard because we haven't acquired
// latches yet. This is ok because each individual request that the admin
// request sends will acquire latches.
if _, err := r.checkExecutionCanProceed(ctx, ba, nil /* g */, &status); err != nil {
return nil, roachpb.NewError(err)
}
var resp roachpb.Response
switch tArgs := args.(type) {
case *roachpb.AdminSplitRequest:
var reply roachpb.AdminSplitResponse
reply, pErr = r.AdminSplit(ctx, *tArgs, "manual")
resp = &reply
case *roachpb.AdminUnsplitRequest:
var reply roachpb.AdminUnsplitResponse
reply, pErr = r.AdminUnsplit(ctx, *tArgs, "manual")
resp = &reply
case *roachpb.AdminMergeRequest:
var reply roachpb.AdminMergeResponse
reply, pErr = r.AdminMerge(ctx, *tArgs, "manual")
resp = &reply
case *roachpb.AdminTransferLeaseRequest:
pErr = roachpb.NewError(r.AdminTransferLease(ctx, tArgs.Target))
resp = &roachpb.AdminTransferLeaseResponse{}
case *roachpb.AdminChangeReplicasRequest:
chgs := tArgs.Changes()
desc, err := r.ChangeReplicas(ctx, &tArgs.ExpDesc, SnapshotRequest_REBALANCE, kvserverpb.ReasonAdminRequest, "", chgs)
pErr = roachpb.NewError(err)
if pErr != nil {
resp = &roachpb.AdminChangeReplicasResponse{}
} else {
resp = &roachpb.AdminChangeReplicasResponse{
Desc: *desc,
}
}
case *roachpb.AdminRelocateRangeRequest:
err := r.store.AdminRelocateRange(ctx, *r.Desc(), tArgs.VoterTargets, tArgs.NonVoterTargets)
pErr = roachpb.NewError(err)
resp = &roachpb.AdminRelocateRangeResponse{}
case *roachpb.CheckConsistencyRequest:
var reply roachpb.CheckConsistencyResponse
reply, pErr = r.CheckConsistency(ctx, *tArgs)
resp = &reply
case *roachpb.ImportRequest:
cArgs := batcheval.CommandArgs{
EvalCtx: NewReplicaEvalContext(r, todoSpanSet),
Header: ba.Header,
Args: args,
}
var err error
resp, err = importCmdFn(ctx, cArgs)
pErr = roachpb.NewError(err)
case *roachpb.AdminScatterRequest:
reply, err := r.adminScatter(ctx, *tArgs)
pErr = roachpb.NewError(err)
resp = &reply
case *roachpb.AdminVerifyProtectedTimestampRequest:
reply, err := r.adminVerifyProtectedTimestamp(ctx, *tArgs)
pErr = roachpb.NewError(err)
resp = &reply
default:
return nil, roachpb.NewErrorf("unrecognized admin command: %T", args)
}
if pErr != nil {
return nil, pErr
}
br := &roachpb.BatchResponse{}
br.Add(resp)
br.Txn = resp.Header().Txn
return br, nil
}
// checkBatchRequest verifies BatchRequest validity requirements. In particular,
// the batch must have an assigned timestamp, and either all requests must be
// read-only, or none.
//
// TODO(tschottdorf): should check that request is contained in range and that
// EndTxn only occurs at the very end.
func (r *Replica) checkBatchRequest(ba *roachpb.BatchRequest, isReadOnly bool) error {
if ba.Timestamp.IsEmpty() {
// For transactional requests, Store.Send sets the timestamp. For non-
// transactional requests, the client sets the timestamp. Either way, we
// need to have a timestamp at this point.
return errors.New("Replica.checkBatchRequest: batch does not have timestamp assigned")
}
consistent := ba.ReadConsistency == roachpb.CONSISTENT
if isReadOnly {
if !consistent && ba.Txn != nil {
// Disallow any inconsistent reads within txns.
return errors.Errorf("cannot allow %v reads within a transaction", ba.ReadConsistency)
}
} else if !consistent {
return errors.Errorf("%v mode is only available to reads", ba.ReadConsistency)
}
return nil
}
func (r *Replica) collectSpans(
ba *roachpb.BatchRequest,
) (latchSpans, lockSpans *spanset.SpanSet, requestEvalKind concurrency.RequestEvalKind, _ error) {
latchSpans, lockSpans = new(spanset.SpanSet), new(spanset.SpanSet)
isReadOnly := ba.IsReadOnly()
r.mu.RLock()
desc := r.descRLocked()
liveCount := r.mu.state.Stats.LiveCount
r.mu.RUnlock()
// TODO(bdarnell): need to make this less global when local
// latches are used more heavily. For example, a split will
// have a large read-only span but also a write (see #10084).
// Currently local spans are the exception, so preallocate for the
// common case in which all are global. We rarely mix read and
// write commands, so preallocate for writes if there are any
// writes present in the batch.
//
// TODO(bdarnell): revisit as the local portion gets its appropriate
// use.
if ba.IsLocking() {
latchGuess := len(ba.Requests)
if et, ok := ba.GetArg(roachpb.EndTxn); ok {
// EndTxn declares a global write for each of its lock spans.
latchGuess += len(et.(*roachpb.EndTxnRequest).LockSpans) - 1
}
latchSpans.Reserve(spanset.SpanReadWrite, spanset.SpanGlobal, latchGuess)
lockSpans.Reserve(spanset.SpanReadWrite, spanset.SpanGlobal, len(ba.Requests))
} else {
latchSpans.Reserve(spanset.SpanReadOnly, spanset.SpanGlobal, len(ba.Requests))
lockSpans.Reserve(spanset.SpanReadOnly, spanset.SpanGlobal, len(ba.Requests))
}
// For non-local, MVCC spans we annotate them with the request timestamp
// during declaration. This is the timestamp used during latch acquisitions.
// For read requests this works as expected, reads are performed at the same
// timestamp. During writes however, we may encounter a versioned value newer
// than the request timestamp, and may have to retry at a higher timestamp.
// This is still safe as we're only ever writing at timestamps higher than the
// timestamp any write latch would be declared at.
batcheval.DeclareKeysForBatch(desc, ba.Header, latchSpans)
for _, union := range ba.Requests {
inner := union.GetInner()
if cmd, ok := batcheval.LookupCommand(inner.Method()); ok {
cmd.DeclareKeys(desc, ba.Header, inner, latchSpans, lockSpans)
} else {
return nil, nil, concurrency.PessimisticEval, errors.Errorf("unrecognized command %s", inner.Method())
}
}
// Commands may create a large number of duplicate spans. De-duplicate
// them to reduce the number of spans we pass to the spanlatch manager.
for _, s := range [...]*spanset.SpanSet{latchSpans, lockSpans} {
s.SortAndDedup()
// If any command gave us spans that are invalid, bail out early
// (before passing them to the spanlatch manager, which may panic).
if err := s.Validate(); err != nil {
return nil, nil, concurrency.PessimisticEval, err
}
}
if isReadOnly {
// Evaluate batches optimistically if they have a key limit which is less
// than the number of live keys on the Range. Ignoring write latches and
// locks can be beneficial because it can help avoid waiting on writes to
// keys that the batch will never actually need to read due to the
// overestimate of its key bounds. Only after it is clear exactly what
// spans were read do we verify whether there were any conflicts with
// concurrent writes.
//
// This case is not uncommon; for example, a Scan which requests the entire
// range but has a limit of 1 result. We want to avoid allowing overly broad
// spans from backing up the latch manager, or encountering too much contention
// in the lock table.
//
// The heuristic is limit < k * liveCount, where k <= 1. The use of k=1
// below is an un-tuned setting.
limit := ba.Header.MaxSpanRequestKeys
if limit > 0 && limit < liveCount {
requestEvalKind = concurrency.OptimisticEval
}
}
return latchSpans, lockSpans, requestEvalKind, nil
}