-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
replica.go
1734 lines (1564 loc) · 65.7 KB
/
replica.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
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2014 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 storage
import (
"context"
"fmt"
"reflect"
"strings"
"sync/atomic"
"time"
"unsafe"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/abortspan"
"github.com/cockroachdb/cockroach/pkg/storage/batcheval"
"github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb"
"github.com/cockroachdb/cockroach/pkg/storage/engine"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/rangefeed"
"github.com/cockroachdb/cockroach/pkg/storage/spanlatch"
"github.com/cockroachdb/cockroach/pkg/storage/spanset"
"github.com/cockroachdb/cockroach/pkg/storage/split"
"github.com/cockroachdb/cockroach/pkg/storage/stateloader"
"github.com/cockroachdb/cockroach/pkg/storage/storagebase"
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
"github.com/cockroachdb/cockroach/pkg/storage/txnwait"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/google/btree"
"github.com/kr/pretty"
opentracing "github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
"go.etcd.io/etcd/raft"
)
const (
// configGossipTTL is the time-to-live for configuration maps.
// optimizePutThreshold is the minimum length of a contiguous run
// of batched puts or conditional puts, after which the constituent
// put operations will possibly be optimized by determining whether
// the key space being written is starting out empty.
optimizePutThreshold = 10
replicaChangeTxnName = "change-replica"
splitTxnName = "split"
mergeTxnName = "merge"
defaultReplicaRaftMuWarnThreshold = 500 * time.Millisecond
)
var testingDisableQuiescence = envutil.EnvOrDefaultBool("COCKROACH_DISABLE_QUIESCENCE", false)
var disableSyncRaftLog = settings.RegisterBoolSetting(
"kv.raft_log.disable_synchronization_unsafe",
"set to true to disable synchronization on Raft log writes to persistent storage. "+
"Setting to true risks data loss or data corruption on server crashes. "+
"The setting is meant for internal testing only and SHOULD NOT be used in production.",
false,
)
// UseAtomicReplicationChanges determines whether to issue atomic replication changes.
// This has no effect until the cluster version is 19.2 or higher.
var UseAtomicReplicationChanges = settings.RegisterBoolSetting(
"kv.atomic_replication_changes.enabled",
"use atomic replication changes",
true,
)
// MaxCommandSizeFloor is the minimum allowed value for the MaxCommandSize
// cluster setting.
const MaxCommandSizeFloor = 4 << 20 // 4MB
// MaxCommandSize wraps "kv.raft.command.max_size".
var MaxCommandSize = settings.RegisterValidatedByteSizeSetting(
"kv.raft.command.max_size",
"maximum size of a raft command",
64<<20,
func(size int64) error {
if size < MaxCommandSizeFloor {
return fmt.Errorf("max_size must be greater than %s", humanizeutil.IBytes(MaxCommandSizeFloor))
}
return nil
},
)
type proposalReevaluationReason int
const (
proposalNoReevaluation proposalReevaluationReason = iota
// proposalIllegalLeaseIndex indicates the proposal failed to apply at
// a Lease index it was not legal for. The command should be re-evaluated.
proposalIllegalLeaseIndex
)
type atomicDescString struct {
strPtr unsafe.Pointer
}
// store atomically updates d.strPtr with the string representation of desc.
func (d *atomicDescString) store(replicaID roachpb.ReplicaID, desc *roachpb.RangeDescriptor) {
var buf strings.Builder
fmt.Fprintf(&buf, "%d/", desc.RangeID)
if replicaID == 0 {
fmt.Fprintf(&buf, "?:")
} else {
fmt.Fprintf(&buf, "%d:", replicaID)
}
if !desc.IsInitialized() {
buf.WriteString("{-}")
} else {
const maxRangeChars = 30
rngStr := keys.PrettyPrintRange(roachpb.Key(desc.StartKey), roachpb.Key(desc.EndKey), maxRangeChars)
buf.WriteString(rngStr)
}
str := buf.String()
atomic.StorePointer(&d.strPtr, unsafe.Pointer(&str))
}
// String returns the string representation of the range; since we are not
// using a lock, the copy might be inconsistent.
func (d *atomicDescString) String() string {
return *(*string)(atomic.LoadPointer(&d.strPtr))
}
// atomicConnectionClass stores an rpc.ConnectionClass atomically.
type atomicConnectionClass uint32
// get reads the current value of the ConnectionClass.
func (c *atomicConnectionClass) get() rpc.ConnectionClass {
return rpc.ConnectionClass(atomic.LoadUint32((*uint32)(c)))
}
// set updates the current value of the ConnectionClass.
func (c *atomicConnectionClass) set(cc rpc.ConnectionClass) {
atomic.StoreUint32((*uint32)(c), uint32(cc))
}
// A Replica is a contiguous keyspace with writes managed via an
// instance of the Raft consensus algorithm. Many ranges may exist
// in a store and they are unlikely to be contiguous. Ranges are
// independent units and are responsible for maintaining their own
// integrity by replacing failed replicas, splitting and merging
// as appropriate.
type Replica struct {
log.AmbientContext
// TODO(tschottdorf): Duplicates r.mu.state.desc.RangeID; revisit that.
RangeID roachpb.RangeID // Only set by the constructor
store *Store
abortSpan *abortspan.AbortSpan // Avoids anomalous reads after abort
txnWaitQueue *txnwait.Queue // Queues push txn attempts by txn ID
// leaseholderStats tracks all incoming BatchRequests to the replica and which
// localities they come from in order to aid in lease rebalancing decisions.
leaseholderStats *replicaStats
// writeStats tracks the number of keys written by applied raft commands
// in order to aid in replica rebalancing decisions.
writeStats *replicaStats
// creatingReplica is set when a replica is created as uninitialized
// via a raft message.
creatingReplica *roachpb.ReplicaDescriptor
// Held in read mode during read-only commands. Held in exclusive mode to
// prevent read-only commands from executing. Acquired before the embedded
// RWMutex.
//
// This mutex ensures proper interleaving of splits with concurrent reads.
// Splits register an MVCC write span latch, but reads at lower timestamps
// aren't held up by this latch, which could result in reads on the RHS
// executed through the LHS after this is valid. For more detail, see:
// https://github.com/cockroachdb/cockroach/issues/32583.
readOnlyCmdMu syncutil.RWMutex
// rangeStr is a string representation of a RangeDescriptor that can be
// atomically read and updated without needing to acquire the replica.mu lock.
// All updates to state.Desc should be duplicated here.
rangeStr atomicDescString
// connectionClass controls the ConnectionClass used to send raft messages.
connectionClass atomicConnectionClass
// raftMu protects Raft processing the replica.
//
// Locking notes: Replica.raftMu < Replica.mu
raftMu struct {
syncutil.Mutex
// Note that there are two StateLoaders, in raftMu and mu,
// depending on which lock is being held.
stateLoader stateloader.StateLoader
// on-disk storage for sideloaded SSTables. nil when there's no ReplicaID.
sideloaded SideloadStorage
// stateMachine is used to apply committed raft entries.
stateMachine replicaStateMachine
// decoder is used to decode committed raft entries.
decoder replicaDecoder
}
// Contains the lease history when enabled.
leaseHistory *leaseHistory
// Enforces at most one command is running per key(s) within each span
// scope. The globally-scoped component tracks user writes (i.e. all
// keys for which keys.Addr is the identity), the locally-scoped component
// the rest (e.g. RangeDescriptor, transaction record, Lease, ...).
latchMgr spanlatch.Manager
mu struct {
// Protects all fields in the mu struct.
syncutil.RWMutex
// The destroyed status of a replica indicating if it's alive, corrupt,
// scheduled for destruction or has been GCed.
// destroyStatus should only be set while also holding the raftMu.
destroyStatus
// Is the range quiescent? Quiescent ranges are not Tick()'d and unquiesce
// whenever a Raft operation is performed.
quiescent bool
// mergeComplete is non-nil if a merge is in-progress, in which case any
// requests should be held until the completion of the merge is signaled by
// the closing of the channel.
mergeComplete chan struct{}
// The state of the Raft state machine.
state storagepb.ReplicaState
// Last index/term persisted to the raft log (not necessarily
// committed). Note that lastTerm may be 0 (and thus invalid) even when
// lastIndex is known, in which case the term will have to be retrieved
// from the Raft log entry. Use the invalidLastTerm constant for this
// case.
lastIndex, lastTerm uint64
// A map of raft log index of pending preemptive snapshots to deadlines.
// Used to prohibit raft log truncations that would leave a gap between
// the snapshot and the new first index. The map entry has a zero
// deadline while the snapshot is being sent and turns nonzero when the
// snapshot has completed, preventing truncation for a grace period
// (since there is a race between the snapshot completing and its being
// reflected in the raft status used to make truncation decisions).
//
// NB: If we kept only one value, we could end up in situations in which
// we're either giving some snapshots no grace period, or keep an
// already finished snapshot "pending" for extended periods of time
// (preventing log truncation).
snapshotLogTruncationConstraints map[uuid.UUID]snapTruncationInfo
// raftLogSize is the approximate size in bytes of the persisted raft
// log, including sideloaded entries' payloads. The value itself is not
// persisted and is computed lazily, paced by the raft log truncation
// queue which will recompute the log size when it finds it
// uninitialized. This recomputation mechanism isn't relevant for ranges
// which see regular write activity (for those the log size will deviate
// from zero quickly, and so it won't be recomputed but will undercount
// until the first truncation is carried out), but it prevents a large
// dormant Raft log from sitting around forever, which has caused problems
// in the past.
raftLogSize int64
// If raftLogSizeTrusted is false, don't trust the above raftLogSize until
// it has been recomputed.
raftLogSizeTrusted bool
// raftLogLastCheckSize is the value of raftLogSize the last time the Raft
// log was checked for truncation or at the time of the last Raft log
// truncation.
raftLogLastCheckSize int64
// pendingLeaseRequest is used to coalesce RequestLease requests.
pendingLeaseRequest pendingLeaseRequest
// minLeaseProposedTS is the minimum acceptable lease.ProposedTS; only
// leases proposed after this timestamp can be used for proposing commands.
// This is used to protect against several hazards:
// - leases held (or even proposed) before a restart cannot be used after a
// restart. This is because:
// a) the spanlatch manager is wiped during the restart; there might be
// writes in flight that do not have the latches they held reflected. So,
// we need to synchronize all new reads with those old in-flight writes.
// Forcing acquisition of a new lease essentially flushes all the
// previous raft commands.
// b) a lease transfer might have been in progress at the time of the
// restart. Using the existing lease after the restart would break the
// transfer proposer's promise to not use the existing lease.
// - a lease cannot be used after a transfer is initiated. Moreover, even
// lease extension that were in flight at the time of the transfer cannot be
// used, if they eventually apply.
minLeaseProposedTS hlc.Timestamp
// A pointer to the zone config for this replica.
zone *config.ZoneConfig
// proposalBuf buffers Raft commands as they are passed to the Raft
// replication subsystem. The buffer is populated by requests after
// evaluation and is consumed by the Raft processing thread. Once
// consumed, commands are proposed through Raft and moved to the
// proposals map.
proposalBuf propBuf
// proposals stores the Raft in-flight commands which originated at
// this Replica, i.e. all commands for which propose has been called,
// but which have not yet applied.
//
// The *ProposalData in the map are "owned" by it. Elements from the
// map must only be referenced while the Replica.mu is held, except
// if the element is removed from the map first. Modifying the proposal
// itself may require holding the raftMu as fields can be accessed
// underneath raft. See comments on ProposalData fields for synchronization
// requirements.
//
// Due to Raft reproposals, multiple in-flight Raft entries can have
// the same CmdIDKey, all corresponding to the same KV request. However,
// not all Raft entries with a given command ID will correspond directly
// to the *RaftCommand contained in its associated *ProposalData. This
// is because the *RaftCommand can be mutated during reproposals by
// Replica.tryReproposeWithNewLeaseIndex.
//
// TODO(ajwerner): move the proposal map and ProposalData entirely under
// the raftMu.
proposals map[storagebase.CmdIDKey]*ProposalData
internalRaftGroup *raft.RawNode
// The ID of the replica within the Raft group. May be 0 if the replica has
// been created from a preemptive snapshot (i.e. before being added to the
// Raft group). The replica ID will be non-zero whenever the replica is
// part of a Raft group.
replicaID roachpb.ReplicaID
// The minimum allowed ID for this replica. Initialized from
// RaftTombstone.NextReplicaID.
minReplicaID roachpb.ReplicaID
// The ID of the leader replica within the Raft group. Used to determine
// when the leadership changes.
leaderID roachpb.ReplicaID
// The most recently added replica for the range and when it was added.
// Used to determine whether a replica is new enough that we shouldn't
// penalize it for being slightly behind. These field gets cleared out once
// we know that the replica has caught up.
lastReplicaAdded roachpb.ReplicaID
lastReplicaAddedTime time.Time
// initialMaxClosed is the initial maxClosed timestamp for the replica as known
// from its left-hand-side upon creation.
initialMaxClosed hlc.Timestamp
// The most recently updated time for each follower of this range. This is updated
// every time a Raft message is received from a peer.
// Note that superficially it seems that similar information is contained in the
// Progress of a RaftStatus, which has a RecentActive field. However, that field
// is always true unless CheckQuorum is active, which at the time of writing in
// CockroachDB is not the case.
//
// The lastUpdateTimes map is also updated when a leaseholder steps up
// (making the assumption that all followers are live at that point),
// and when the range unquiesces (marking all replicating followers as
// live).
//
// TODO(tschottdorf): keeping a map on each replica seems to be
// overdoing it. We should map the replicaID to a NodeID and then use
// node liveness (or any sensible measure of the peer being around).
// The danger in doing so is that a single stuck replica on an otherwise
// functioning node could fill up the quota pool. We are already taking
// this kind of risk though: a replica that gets stuck on an otherwise
// live node will not lose leaseholdership.
lastUpdateTimes lastUpdateTimesMap
// The last seen replica descriptors from incoming Raft messages. These are
// stored so that the replica still knows the replica descriptors for itself
// and for its message recipients in the circumstances when its RangeDescriptor
// is out of date.
//
// Normally, a replica knows about the other replica descriptors for a
// range via the RangeDescriptor stored in Replica.mu.state.Desc. But that
// descriptor is only updated during a Split or ChangeReplicas operation.
// There are periods during a Replica's lifetime when that information is
// out of date:
//
// 1. When a replica is being newly created as the result of an incoming
// Raft message for it. This is the common case for ChangeReplicas and an
// uncommon case for Splits. The leader will be sending the replica
// messages and the replica needs to be able to respond before it can
// receive an updated range descriptor (via a snapshot,
// changeReplicasTrigger, or splitTrigger).
//
// 2. If the node containing a replica is partitioned or down while the
// replicas for the range are updated. When the node comes back up, other
// replicas may begin communicating with it and it needs to be able to
// respond. Unlike 1 where there is no range descriptor, in this situation
// the replica has a range descriptor but it is out of date. Note that a
// replica being removed from a node and then quickly re-added before the
// replica has been GC'd will also use the last seen descriptors. In
// effect, this is another path for which the replica's local range
// descriptor is out of date.
//
// The last seen replica descriptors are updated on receipt of every raft
// message via Replica.setLastReplicaDescriptors (see
// Store.HandleRaftRequest). These last seen descriptors are used when
// the replica's RangeDescriptor contains missing or out of date descriptors
// for a replica (see Replica.sendRaftMessage).
//
// Removing a replica from Store.mu.replicas is not a problem because
// when a replica is completely removed, it won't be recreated until
// there is another event that will repopulate the replicas map in the
// range descriptor. When it is temporarily dropped and recreated, the
// newly recreated replica will have a complete range descriptor.
lastToReplica, lastFromReplica roachpb.ReplicaDescriptor
// Computed checksum at a snapshot UUID.
checksums map[uuid.UUID]ReplicaChecksum
// proposalQuota is the quota pool maintained by the lease holder where
// incoming writes acquire quota from a fixed quota pool before going
// through. If there is no quota available, the write is throttled
// until quota is made available to the pool.
// Acquired quota for a given command is only released when all the
// replicas have persisted the corresponding entry into their logs.
proposalQuota *quotapool.IntPool
// The base index is the index up to (including) which quota was already
// released. That is, the first element in quotaReleaseQueue below is
// released as the base index moves up by one, etc.
proposalQuotaBaseIndex uint64
// Once the leader observes a proposal come 'out of Raft', we add the
// size of the associated command to a queue of quotas we have yet to
// release back to the quota pool. We only do so when all replicas have
// persisted the corresponding entry into their logs.
quotaReleaseQueue []*quotapool.IntAlloc
// Counts calls to Replica.tick()
ticks int
// Counts Raft messages refused due to queue congestion.
droppedMessages int
// Note that there are two replicaStateLoaders, in raftMu and mu,
// depending on which lock is being held.
stateLoader stateloader.StateLoader
// draining specifies whether this replica is draining. Raft leadership
// transfers due to a lease change will be attempted even if the target does
// not have all the log entries.
draining bool
}
rangefeedMu struct {
syncutil.RWMutex
// proc is an instance of a rangefeed Processor that is capable of
// routing rangefeed events to a set of subscribers. Will be nil if no
// subscribers are registered.
//
// Requires Replica.rangefeedMu be held when mutating the pointer.
// Requires Replica.raftMu be held when providing logical ops and
// informing the processor of closed timestamp updates. This properly
// synchronizes updates that are linearized and driven by the Raft log.
proc *rangefeed.Processor
}
// Throttle how often we offer this Replica to the split and merge queues.
// We have triggers downstream of Raft that do so based on limited
// information and without explicit throttling some replicas will offer once
// per applied Raft command, which is silly and also clogs up the queues'
// semaphores.
splitQueueThrottle, mergeQueueThrottle util.EveryN
// loadBasedSplitter keeps information about load-based splitting.
loadBasedSplitter split.Decider
unreachablesMu struct {
syncutil.Mutex
remotes map[roachpb.ReplicaID]struct{}
}
}
var _ batcheval.EvalContext = &Replica{}
// KeyRange is an interface type for the replicasByKey BTree, to compare
// Replica and ReplicaPlaceholder.
type KeyRange interface {
Desc() *roachpb.RangeDescriptor
rangeKeyItem
btree.Item
fmt.Stringer
}
var _ KeyRange = &Replica{}
var _ client.Sender = &Replica{}
// NewReplica initializes the replica using the given metadata. If the
// replica is initialized (i.e. desc contains more than a RangeID),
// replicaID should be 0 and the replicaID will be discovered from the
// descriptor.
func NewReplica(
desc *roachpb.RangeDescriptor, store *Store, replicaID roachpb.ReplicaID,
) (*Replica, error) {
r := newReplica(desc.RangeID, store)
return r, r.init(desc, store.Clock(), replicaID)
}
// 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)
ctx, cleanup := tracing.EnsureContext(ctx, r.AmbientContext.Tracer, "replica send")
defer cleanup()
// If the internal Raft group is not initialized, create it and wake the leader.
r.maybeInitializeRaftGroup(ctx)
isReadOnly := ba.IsReadOnly()
useRaft := !isReadOnly && ba.IsWrite()
if isReadOnly && r.store.Clock().MaxOffset() == timeutil.ClocklessMaxOffset {
// Clockless reads mode: reads go through Raft.
useRaft = true
}
if err := r.checkBatchRequest(&ba, isReadOnly); err != nil {
return nil, roachpb.NewError(err)
}
if filter := r.store.cfg.TestingKnobs.TestingRequestFilter; filter != nil {
if pErr := filter(ba); pErr != nil {
return nil, pErr
}
}
// Differentiate between admin, read-only and write.
var pErr *roachpb.Error
if useRaft {
log.Event(ctx, "read-write path")
br, pErr = r.executeWriteBatch(ctx, &ba)
} else if isReadOnly {
log.Event(ctx, "read-only path")
br, pErr = r.executeReadOnlyBatch(ctx, &ba)
} 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 {
if _, ok := pErr.GetDetail().(*roachpb.RaftGroupDeletedError); ok {
// This error needs to be converted appropriately so that
// clients will retry.
pErr = roachpb.NewError(roachpb.NewRangeNotFoundError(r.RangeID, r.store.StoreID()))
}
log.Eventf(ctx, "replica.Send got error: %s", pErr)
} else {
if filter := r.store.cfg.TestingKnobs.TestingResponseFilter; filter != nil {
pErr = filter(ba, br)
}
}
return br, pErr
}
// String returns the string representation of the replica using an
// inconsistent copy of the range descriptor. Therefore, String does not
// require a lock and its output may not be atomic with other ongoing work in
// the replica. This is done to prevent deadlocks in logging sites.
func (r *Replica) String() string {
return fmt.Sprintf("[n%d,s%d,r%s]", r.store.Ident.NodeID, r.store.Ident.StoreID, &r.rangeStr)
}
// ReplicaID returns the ID for the Replica. It may be zero if the replica does
// not know its ID. Once a Replica has a non-zero ReplicaID it will never change.
func (r *Replica) ReplicaID() roachpb.ReplicaID {
r.mu.RLock()
defer r.mu.RUnlock()
return r.mu.replicaID
}
// cleanupFailedProposal cleans up after a proposal that has failed. It
// clears any references to the proposal and releases associated quota.
// It requires that both Replica.mu and Replica.raftMu are exclusively held.
func (r *Replica) cleanupFailedProposalLocked(p *ProposalData) {
r.raftMu.AssertHeld()
r.mu.AssertHeld()
delete(r.mu.proposals, p.idKey)
p.releaseQuota()
}
// GetMinBytes gets the replica's minimum byte threshold.
func (r *Replica) GetMinBytes() int64 {
r.mu.RLock()
defer r.mu.RUnlock()
return *r.mu.zone.RangeMinBytes
}
// GetMaxBytes gets the replica's maximum byte threshold.
func (r *Replica) GetMaxBytes() int64 {
r.mu.RLock()
defer r.mu.RUnlock()
return *r.mu.zone.RangeMaxBytes
}
// SetZoneConfig sets the replica's zone config.
func (r *Replica) SetZoneConfig(zone *config.ZoneConfig) {
r.mu.Lock()
defer r.mu.Unlock()
r.mu.zone = zone
}
// IsFirstRange returns true if this is the first range.
func (r *Replica) IsFirstRange() bool {
return r.RangeID == 1
}
// IsDestroyed returns a non-nil error if the replica has been destroyed
// and the reason if it has.
func (r *Replica) IsDestroyed() (DestroyReason, error) {
r.mu.RLock()
defer r.mu.RUnlock()
return r.isDestroyedRLocked()
}
func (r *Replica) isDestroyedRLocked() (DestroyReason, error) {
return r.mu.destroyStatus.reason, r.mu.destroyStatus.err
}
// DescAndZone returns the authoritative range descriptor as well
// as the zone config for the replica.
func (r *Replica) DescAndZone() (*roachpb.RangeDescriptor, *config.ZoneConfig) {
r.mu.RLock()
defer r.mu.RUnlock()
return r.mu.state.Desc, r.mu.zone
}
// Desc returns the authoritative range descriptor, acquiring a replica lock in
// the process.
func (r *Replica) Desc() *roachpb.RangeDescriptor {
r.mu.RLock()
defer r.mu.RUnlock()
return r.mu.state.Desc
}
func (r *Replica) descRLocked() *roachpb.RangeDescriptor {
r.mu.AssertRHeld()
return r.mu.state.Desc
}
// NodeID returns the ID of the node this replica belongs to.
func (r *Replica) NodeID() roachpb.NodeID {
return r.store.nodeDesc.NodeID
}
// GetNodeLocality returns the locality of the node this replica belongs to.
func (r *Replica) GetNodeLocality() roachpb.Locality {
return r.store.nodeDesc.Locality
}
// ClusterSettings returns the node's ClusterSettings.
func (r *Replica) ClusterSettings() *cluster.Settings {
return r.store.cfg.Settings
}
// StoreID returns the Replica's StoreID.
func (r *Replica) StoreID() roachpb.StoreID {
return r.store.StoreID()
}
// EvalKnobs returns the EvalContext's Knobs.
func (r *Replica) EvalKnobs() storagebase.BatchEvalTestingKnobs {
return r.store.cfg.TestingKnobs.EvalKnobs
}
// Clock returns the hlc clock shared by this replica.
func (r *Replica) Clock() *hlc.Clock {
return r.store.Clock()
}
// DB returns the Replica's client DB.
func (r *Replica) DB() *client.DB {
return r.store.DB()
}
// Engine returns the Replica's underlying Engine. In most cases the
// evaluation Batch should be used instead.
func (r *Replica) Engine() engine.Engine {
return r.store.Engine()
}
// AbortSpan returns the Replica's AbortSpan.
func (r *Replica) AbortSpan() *abortspan.AbortSpan {
// Despite its name, the AbortSpan doesn't hold on-disk data in
// memory. It just provides methods that take a Batch, so SpanSet
// declarations are enforced there.
return r.abortSpan
}
// GetLimiters returns the Replica's limiters.
func (r *Replica) GetLimiters() *batcheval.Limiters {
return &r.store.limiters
}
// GetTxnWaitQueue returns the Replica's txnwait.Queue.
func (r *Replica) GetTxnWaitQueue() *txnwait.Queue {
return r.txnWaitQueue
}
// GetTerm returns the term of the given index in the raft log.
func (r *Replica) GetTerm(i uint64) (uint64, error) {
r.mu.RLock()
defer r.mu.RUnlock()
return r.raftTermRLocked(i)
}
// GetRangeID returns the Range ID.
func (r *Replica) GetRangeID() roachpb.RangeID {
return r.RangeID
}
// GetGCThreshold returns the GC threshold.
func (r *Replica) GetGCThreshold() hlc.Timestamp {
r.mu.RLock()
defer r.mu.RUnlock()
return *r.mu.state.GCThreshold
}
// maxReplicaIDOfAny returns the maximum ReplicaID of any replica, including
// voters and learners.
func maxReplicaIDOfAny(desc *roachpb.RangeDescriptor) roachpb.ReplicaID {
if desc == nil || !desc.IsInitialized() {
return 0
}
var maxID roachpb.ReplicaID
for _, repl := range desc.Replicas().All() {
if repl.ReplicaID > maxID {
maxID = repl.ReplicaID
}
}
return maxID
}
// LastReplicaAdded returns the ID of the most recently added replica and the
// time at which it was added.
func (r *Replica) LastReplicaAdded() (roachpb.ReplicaID, time.Time) {
r.mu.RLock()
defer r.mu.RUnlock()
return r.mu.lastReplicaAdded, r.mu.lastReplicaAddedTime
}
// GetReplicaDescriptor returns the replica for this range from the range
// descriptor. Returns a *RangeNotFoundError if the replica is not found.
// No other errors are returned.
func (r *Replica) GetReplicaDescriptor() (roachpb.ReplicaDescriptor, error) {
r.mu.RLock()
defer r.mu.RUnlock()
return r.getReplicaDescriptorRLocked()
}
// getReplicaDescriptorRLocked is like getReplicaDescriptor, but assumes that
// r.mu is held for either reading or writing.
func (r *Replica) getReplicaDescriptorRLocked() (roachpb.ReplicaDescriptor, error) {
repDesc, ok := r.mu.state.Desc.GetReplicaDescriptor(r.store.StoreID())
if ok {
return repDesc, nil
}
return roachpb.ReplicaDescriptor{}, roachpb.NewRangeNotFoundError(r.RangeID, r.store.StoreID())
}
func (r *Replica) getMergeCompleteCh() chan struct{} {
r.mu.RLock()
mergeCompleteCh := r.mu.mergeComplete
r.mu.RUnlock()
return mergeCompleteCh
}
// setLastReplicaDescriptors sets the the most recently seen replica
// descriptors to those contained in the *RaftMessageRequest, acquiring r.mu
// to do so.
func (r *Replica) setLastReplicaDescriptors(req *RaftMessageRequest) {
r.mu.Lock()
r.mu.lastFromReplica = req.FromReplica
r.mu.lastToReplica = req.ToReplica
r.mu.Unlock()
}
// GetMVCCStats returns a copy of the MVCC stats object for this range.
// This accessor is thread-safe, but provides no guarantees about its
// synchronization with any concurrent writes.
func (r *Replica) GetMVCCStats() enginepb.MVCCStats {
r.mu.RLock()
defer r.mu.RUnlock()
return *r.mu.state.Stats
}
// GetSplitQPS returns the Replica's queries/s request rate.
//
// NOTE: This should only be used for load based splitting, only
// works when the load based splitting cluster setting is enabled.
//
// Use QueriesPerSecond() for current QPS stats for all other purposes.
func (r *Replica) GetSplitQPS() float64 {
return r.loadBasedSplitter.LastQPS(timeutil.Now())
}
// ContainsKey returns whether this range contains the specified key.
//
// TODO(bdarnell): This is not the same as RangeDescriptor.ContainsKey.
func (r *Replica) ContainsKey(key roachpb.Key) bool {
return storagebase.ContainsKey(*r.Desc(), key)
}
// ContainsKeyRange returns whether this range contains the specified
// key range from start to end.
func (r *Replica) ContainsKeyRange(start, end roachpb.Key) bool {
return storagebase.ContainsKeyRange(*r.Desc(), start, end)
}
// GetLastReplicaGCTimestamp reads the timestamp at which the replica was
// last checked for removal by the replica gc queue.
func (r *Replica) GetLastReplicaGCTimestamp(ctx context.Context) (hlc.Timestamp, error) {
key := keys.RangeLastReplicaGCTimestampKey(r.RangeID)
var timestamp hlc.Timestamp
_, err := engine.MVCCGetProto(ctx, r.store.Engine(), key, hlc.Timestamp{}, ×tamp,
engine.MVCCGetOptions{})
if err != nil {
return hlc.Timestamp{}, err
}
return timestamp, nil
}
func (r *Replica) setLastReplicaGCTimestamp(ctx context.Context, timestamp hlc.Timestamp) error {
key := keys.RangeLastReplicaGCTimestampKey(r.RangeID)
return engine.MVCCPutProto(ctx, r.store.Engine(), nil, key, hlc.Timestamp{}, nil, ×tamp)
}
// getQueueLastProcessed returns the last processed timestamp for the
// specified queue, or the zero timestamp if not available.
func (r *Replica) getQueueLastProcessed(ctx context.Context, queue string) (hlc.Timestamp, error) {
key := keys.QueueLastProcessedKey(r.Desc().StartKey, queue)
var timestamp hlc.Timestamp
if r.store != nil {
_, err := engine.MVCCGetProto(ctx, r.store.Engine(), key, hlc.Timestamp{}, ×tamp,
engine.MVCCGetOptions{})
if err != nil {
log.VErrEventf(ctx, 2, "last processed timestamp unavailable: %s", err)
return hlc.Timestamp{}, err
}
}
log.VEventf(ctx, 2, "last processed timestamp: %s", timestamp)
return timestamp, nil
}
// setQueueLastProcessed writes the last processed timestamp for the
// specified queue.
func (r *Replica) setQueueLastProcessed(
ctx context.Context, queue string, timestamp hlc.Timestamp,
) error {
key := keys.QueueLastProcessedKey(r.Desc().StartKey, queue)
return r.store.DB().PutInline(ctx, key, ×tamp)
}
// RaftStatus returns the current raft status of the replica. It returns nil
// if the Raft group has not been initialized yet.
func (r *Replica) RaftStatus() *raft.Status {
r.mu.RLock()
defer r.mu.RUnlock()
return r.raftStatusRLocked()
}
func (r *Replica) raftStatusRLocked() *raft.Status {
if rg := r.mu.internalRaftGroup; rg != nil {
s := rg.Status()
return &s
}
return nil
}
// State returns a copy of the internal state of the Replica, along with some
// auxiliary information.
func (r *Replica) State() storagepb.RangeInfo {
var ri storagepb.RangeInfo
// NB: this acquires an RLock(). Reentrant RLocks are deadlock prone, so do
// this first before RLocking below. Performance of this extra lock
// acquisition is not a concern.
ri.ActiveClosedTimestamp = r.maxClosed(context.Background())
// NB: numRangefeedRegistrations doesn't require Replica.mu to be locked.
// However, it does require coordination between multiple goroutines, so
// it's best to keep it out of the Replica.mu critical section.
ri.RangefeedRegistrations = int64(r.numRangefeedRegistrations())
r.mu.RLock()
defer r.mu.RUnlock()
ri.ReplicaState = *(protoutil.Clone(&r.mu.state)).(*storagepb.ReplicaState)
ri.LastIndex = r.mu.lastIndex
ri.NumPending = uint64(r.numPendingProposalsRLocked())
ri.RaftLogSize = r.mu.raftLogSize
ri.RaftLogSizeTrusted = r.mu.raftLogSizeTrusted
ri.NumDropped = uint64(r.mu.droppedMessages)
if r.mu.proposalQuota != nil {
ri.ApproximateProposalQuota = int64(r.mu.proposalQuota.ApproximateQuota())
ri.ProposalQuotaBaseIndex = int64(r.mu.proposalQuotaBaseIndex)
ri.ProposalQuotaReleaseQueue = make([]int64, len(r.mu.quotaReleaseQueue))
for i, a := range r.mu.quotaReleaseQueue {
if a != nil {
ri.ProposalQuotaReleaseQueue[i] = int64(a.Acquired())
}
}
}
ri.RangeMaxBytes = *r.mu.zone.RangeMaxBytes
if desc := ri.ReplicaState.Desc; desc != nil {
// Learner replicas don't serve follower reads, but they still receive
// closed timestamp updates, so include them here.
for _, replDesc := range desc.Replicas().All() {
r.store.cfg.ClosedTimestamp.Storage.VisitDescending(replDesc.NodeID, func(e ctpb.Entry) (done bool) {
mlai, found := e.MLAI[r.RangeID]
if !found {
return false // not done
}
if ri.NewestClosedTimestamp.ClosedTimestamp.Less(e.ClosedTimestamp) {
ri.NewestClosedTimestamp.NodeID = replDesc.NodeID
ri.NewestClosedTimestamp.MLAI = int64(mlai)
ri.NewestClosedTimestamp.ClosedTimestamp = e.ClosedTimestamp
}
return true // done
})
}
}
return ri
}
// assertStateLocked can be called from the Raft goroutine to check that the
// in-memory and on-disk states of the Replica are congruent.
// Requires that both r.raftMu and r.mu are held.
//
// TODO(tschottdorf): Consider future removal (for example, when #7224 is resolved).
func (r *Replica) assertStateLocked(ctx context.Context, reader engine.Reader) {
diskState, err := r.mu.stateLoader.Load(ctx, reader, r.mu.state.Desc)
if err != nil {
log.Fatal(ctx, err)
}
if !diskState.Equal(r.mu.state) {
// The roundabout way of printing here is to expose this information in sentry.io.
//
// TODO(dt): expose properly once #15892 is addressed.
log.Errorf(ctx, "on-disk and in-memory state diverged:\n%s", pretty.Diff(diskState, r.mu.state))
r.mu.state.Desc, diskState.Desc = nil, nil
log.Fatal(ctx, log.Safe(
fmt.Sprintf("on-disk and in-memory state diverged: %s",
pretty.Diff(diskState, r.mu.state)),
))
}
}
// requestCanProceed returns an error if a request (identified by its
// key span and timestamp) can proceed. It may be called multiple