-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
Copy pathreplica.go
2453 lines (2240 loc) · 104 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 kvserver
import (
"context"
"sort"
"sync/atomic"
"time"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/docs"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/load"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/rangefeed"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/split"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate"
"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"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/grunning"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"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/tracing"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
"github.com/kr/pretty"
"go.etcd.io/raft/v3"
"go.etcd.io/raft/v3/raftpb"
"go.etcd.io/raft/v3/tracker"
)
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
// Transaction names and operations used for range changes.
// Note that those names are used by tests to perform request filtering
// in absence of better criteria. If names are changed, tests should be
// updated accordingly to avoid flakiness.
replicaChangeTxnName = "change-replica"
splitTxnName = "split"
mergeTxnName = "merge"
replicaChangeTxnGetDescOpName = "change-replica-get-desc"
replicaChangeTxnUpdateDescOpName = "change-replica-update-desc"
defaultReplicaRaftMuWarnThreshold = 500 * time.Millisecond
)
// StrictGCEnforcement controls whether requests are rejected based on the GC
// threshold and the current GC TTL (true) or just based on the GC threshold
// (false).
var StrictGCEnforcement = settings.RegisterBoolSetting(
settings.ApplicationLevel,
"kv.gc_ttl.strict_enforcement.enabled",
"if true, fail to serve requests at timestamps below the TTL even if the data still exists",
true,
)
type atomicDescInfo struct {
full redact.RedactableString
fullUnredacted string // `full.StripMarkers()` for use in String() without extra allocs
idOnly string // "<RangeID>/<ReplicaID>" only
}
type atomicDescString struct {
v atomic.Value // *atomicDescInfo
}
// store atomically updates d.strPtr with the string representation of desc.
func (d *atomicDescString) store(replicaID roachpb.ReplicaID, desc *roachpb.RangeDescriptor) {
printRid := func(w redact.SafePrinter) {
w.Printf("%d/", desc.RangeID)
if replicaID == 0 {
w.SafeString("?")
} else {
w.Printf("%d", replicaID)
}
}
str := redact.Sprintfn(func(w redact.SafePrinter) {
printRid(w)
w.SafeString(":")
if !desc.IsInitialized() {
w.SafeString("{-}")
} else {
const maxRangeChars = 30
rngStr := keys.PrettyPrintRange(roachpb.Key(desc.StartKey), roachpb.Key(desc.EndKey), maxRangeChars)
w.UnsafeString(rngStr)
}
})
ridOnly := redact.Sprintfn(func(w redact.SafePrinter) {
printRid(w)
}).StripMarkers()
d.v.Store(&atomicDescInfo{
full: str,
fullUnredacted: str.StripMarkers(),
idOnly: ridOnly,
})
}
// 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 d.get().fullUnredacted
}
// ID returns `rX/Y`, i.e. omits the key range portion.
func (d *atomicDescString) ID() string {
return d.get().idOnly
}
// SafeFormat renders the string safely.
func (d *atomicDescString) SafeFormat(w redact.SafePrinter, _ rune) {
w.Print(d.get().full)
}
// Get returns the string representation of the range; since we are not
// using a lock, the copy might be inconsistent.
func (d *atomicDescString) get() *atomicDescInfo {
return d.v.Load().(*atomicDescInfo)
}
// 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))
}
// raftSparseStatus is a variant of raft.Status without Config and
// Progress.Inflights, which are expensive to copy.
type raftSparseStatus struct {
raft.BasicStatus
Progress map[uint64]tracker.Progress
}
// ReplicaMutex is an RWMutex. It has its own type to make it easier to look for
// usages specific to the replica mutex.
type ReplicaMutex syncutil.RWMutex
func (mu *ReplicaMutex) Lock() {
(*syncutil.RWMutex)(mu).Lock()
}
func (mu *ReplicaMutex) Unlock() {
(*syncutil.RWMutex)(mu).Unlock()
}
func (mu *ReplicaMutex) RLock() {
(*syncutil.RWMutex)(mu).RLock()
}
func (mu *ReplicaMutex) AssertHeld() {
(*syncutil.RWMutex)(mu).AssertHeld()
}
func (mu *ReplicaMutex) AssertRHeld() {
(*syncutil.RWMutex)(mu).AssertRHeld()
}
func (mu *ReplicaMutex) RUnlock() {
(*syncutil.RWMutex)(mu).RUnlock()
}
// 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 {
// A replica's AmbientCtx includes the log tags from the parent node and
// store.
log.AmbientContext
RangeID roachpb.RangeID // Only set by the constructor
// The ID of the replica within the Raft group. Only set by the constructor,
// so it will not change over the lifetime of this replica. If addressed
// under a newer replicaID, the replica immediately replicaGCs itself to
// make way for the newer incarnation.
replicaID roachpb.ReplicaID
// The start key of a Range remains constant throughout its lifetime (it does
// not change through splits or merges). This field carries a copy of
// r.mu.state.Desc.StartKey (and nil if the replica is not initialized). The
// copy is maintained to allow inserting locked Replicas into
// Store.mu.replicasByKey (keyed on start key) without the risk of deadlock.
// The synchronization for this field works as follows:
//
// - the field must not be accessed for uninitialized replicas, except:
// - when setting the field (i.e. when initializing the replica), under `mu`.
//
// Due to the first rule, any access to the field is preceded by an
// acquisition of `mu` (Replica.IsInitialized) which serializes the write and
// any subsequent reads of the field.
//
// The writes to this key happen in Replica.setStartKeyLocked.
startKey roachpb.RKey
// creationTime is the time that the Replica struct was initially constructed.
creationTime time.Time
store *Store
abortSpan *abortspan.AbortSpan // Avoids anomalous reads after abort
// loadStats tracks a sliding window of throughput on this replica.
// Multiple types of throughput are accounted for. Where the localities of
// requests are tracked in order in addition to the aggregate, in order to
// inform load based lease and replica rebalancing decisions.
loadStats *load.ReplicaLoad
// Held in read mode during read-only commands. Held in exclusive mode to
// prevent read-only commands from executing. Acquired before the embedded
// RWMutex.
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
// isInitialized is true if we know the metadata of this replica's range,
// either because we created it or we have received an initial snapshot from
// another node. It is false when a replica has been created in response to an
// incoming message but we are waiting for our initial snapshot.
// The field can be accessed atomically without needing to acquire the
// replica.mu lock. All updates to state.Desc should be duplicated here.
isInitialized syncutil.AtomicBool
// connectionClass controls the ConnectionClass used to send raft messages.
connectionClass atomicConnectionClass
// raftCtx is the Context to use for below-Raft work on this replica. The
// context is pre-determined in order to save on allocations for annotating
// with the replica ID. The Raft contexts that raftCtx replaces don't have
// anything interesting in them, so the operations using this raftCtx don't
// miss out on anything.
raftCtx context.Context
// breaker is a per-Replica circuit breaker. Its purpose is to avoid incurring
// large (infinite) latencies on client requests when the Replica is unable to
// serve commands. This circuit breaker does *not* recruit the occasional
// request to determine whether it is safe to heal the breaker. Instead, it
// has its own probe that is executed asynchronously and determines when the
// Replica is healthy again.
//
// See replica_circuit_breaker.go for details.
breaker *replicaCircuitBreaker
// 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. Always non-nil.
// TODO(pavelkalinnikov): remove sideloaded == nil checks.
sideloaded logstore.SideloadStorage
// stateMachine is used to apply committed raft entries.
stateMachine replicaStateMachine
// decoder is used to decode committed raft entries.
decoder replicaDecoder
}
// localMsgs contains a collection of raftpb.Message that target the local
// RawNode. They are to be delivered on the next iteration of handleRaftReady.
//
// Locking notes:
// - Replica.localMsgs must be held to append messages to active.
// - Replica.raftMu and Replica.localMsgs must both be held to switch slices.
// - Replica.raftMu < Replica.localMsgs
localMsgs struct {
syncutil.Mutex
active, recycled []raftpb.Message
}
// 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.sendRaftMessageRaftMuLocked).
//
// 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.
//
// Locking notes: Replica.raftMu < Replica.mu < Replica.lastSeenReplicas
lastSeenReplicas struct {
syncutil.Mutex
to, from roachpb.ReplicaDescriptor
}
// Contains the lease history when enabled.
leaseHistory *leaseHistory
// concMgr sequences incoming requests and provides isolation between
// requests that intend to perform conflicting operations. It is the
// centerpiece of transaction contention handling.
concMgr concurrency.Manager
// tenantLimiter rate limits requests on a per-tenant basis and accumulates
// metrics about it. This is determined by the start key of the Replica,
// once initialized.
//
// The lifecycle of this is tricky. Because we can't reliably bar requests
// from accessing this even when the replica is destroyed[^1], this will
// stick around on a destroyed replica and can be accessed. The quota pool
// will be closed, however, so it will not accept any writes.
//
// See tenantrate.TestUseAfterRelease.
//
// [^1]: TODO(pavelkalinnikov): we can but it'd be a larger refactor.
tenantLimiter tenantrate.Limiter
// tenantMetricsRef is a metrics reference indicating the tenant under
// which to track the range's contributions. This is determined by the
// start key of the Replica, once initialized.
// Its purpose is to help track down missing/extraneous release operations
// that would not be apparent or easy to resolve when refcounting at the store
// level only.
tenantMetricsRef *tenantMetricsRef
// sideTransportClosedTimestamp encapsulates state related to the closed
// timestamp's information about the range. Note that the
// sideTransportClosedTimestamp does not incorporate the closed timestamp
// information carried by Raft commands. That can be found in
// r.mu.state.RaftClosedTimestamp. Generally, the Raft state should be queried
// in parallel with the side transport state to determine an up to date closed
// timestamp (i.e. the maximum across the two). For a given LAI, the side
// transport closed timestamp will always lead the Raft closed timestamp.
// Across LAIs, the larger LAI will always include the larger closed
// timestamp, independent of the source.
sideTransportClosedTimestamp sidetransportAccess
mu struct {
// Protects all fields in the mu struct.
ReplicaMutex
// 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 and
// readOnlyCmdMu.
//
// When this replica is being removed, the destroyStatus is updated and
// RangeTombstone is written in the same raftMu critical section.
destroyStatus
// Is the range quiescent? Quiescent ranges are not Tick()'d and unquiesce
// whenever a Raft operation is performed.
//
// Replica objects always begin life in a quiescent state, as the field is
// set to true in the Replica constructor newUnloadedReplica. They unquiesce
// and set the field to false in either maybeUnquiesceAndWakeLeaderLocked or
// maybeUnquiesceWithOptionsLocked, which are called in response to Raft
// traffic.
//
// Only initialized replicas that have a non-nil internalRaftGroup are
// allowed to unquiesce and be Tick()'d. See canUnquiesceRLocked for an
// explanation of these conditions.
quiescent bool
// laggingFollowersOnQuiesce is the set of dead replicas that are not
// up-to-date with the rest of the quiescent Raft group. Nil if !quiescent.
laggingFollowersOnQuiesce laggingReplicaSet
// 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{}
// mergeTxnID contains the ID of the in-progress merge transaction, if a
// merge is currently in progress. Otherwise, the ID is empty.
mergeTxnID uuid.UUID
// The state of the Raft state machine. Updated only when raftMu and mu are
// both held.
state kvserverpb.ReplicaState
// Last index/term written to the raft log (not necessarily durable locally
// or committed by the group). Note that lastTermNotDurable may be 0 (and
// thus invalid) even when lastIndexNotDurable is known, in which case the
// term will have to be retrieved from the Raft log entry. Use the
// invalidLastTerm constant for this case.
lastIndexNotDurable kvpb.RaftIndex
lastTermNotDurable kvpb.RaftTerm
// A map of raft log index of pending 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.
//
// Note that both raftLogSize and raftLogSizeTrusted do not include the
// effect of pending log truncations (see Replica.pendingLogTruncations).
// Hence, they are fine for metrics etc., but not for deciding whether we
// should create another pending truncation. For the latter, we compute
// the post-pending-truncation size using pendingLogTruncations.
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.ClockTimestamp
// minValidObservedTimestamp is the minimum timestamp from an external
// transaction that the leaseholder will respect. This protects the case
// where a store becomes the leaseholder for data that it didn't previously
// own. In the case where no leases or data ever move, the store uses the
// observed timestamp on transactions to minimize the size of the
// uncertainty window for transactions that hit the same store multiple
// times. This prevents uncertainty restarts and generally helps
// performance. The problem occurs if a store transfers either its lease or
// data to a different store. Since the clocks are different, the strong
// guarantee of the local limit is violated, and stale reads can occur. By
// setting this value as part of any data movement, and checking this when
// determining whether to perform an uncertainty restart, this violation is
// prevented.
//
// For more, see pkg/kv/kvserver/uncertainty/doc.go.
minValidObservedTimestamp hlc.ClockTimestamp
// The span config for this replica.
conf roachpb.SpanConfig
// spanConfigExplicitlySet tracks whether a span config was explicitly set
// on this replica (as opposed to it having initialized with the default
// span config).
spanConfigExplicitlySet bool
// 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.
//
// The propBuf is the one closing timestamps, so evaluating writes must be
// registered with the propBuf through TrackEvaluatingRequest before their
// write timestamp is decided.
//
// Access to proposalBuf must occur *without* holding the mutex.
// Instead, the buffer internally holds a reference to mu and will use
// it appropriately.
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. A proposal is "pending" until it is "finalized",
// meaning that `finishApplication` has been invoked on the proposal (which
// informs the client that the proposal has now been applied, optionally
// with an error, which may be an AmbiguousResultError).
//
// 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. There are two kinds of reproposals:
//
// (1) the exact same entry is handed to raft (possibly despite already being
// present in the log), usually after a timeout[^1].
//
// (2) an existing proposal is updated with a new MaxLeaseIndex and handed to
// raft, i.e. we're intentionally creating a duplicate. This exists because
// for pipelined proposals, the client's goroutine returns without waiting
// for the proposal to apply.[^2][^3] When (2) is carried out, the existing
// copies of the proposal in the log will be "Superseded", see below. Note
// that (2) will only be invoked for proposals that aren't currently in the
// proposals map any more because they're in the middle of being applied;
// as part of (2), they are re-added to the map.
//
// To understand reproposals, we need a broad overview of entry application,
// which is batched (i.e. may process multiple log entries to be applied in
// a batched fashion). In entry application, the following steps are taken:
//
// 1. retrieve all local proposals: iterate through the entries in order,
// and look them up in the proposals map. For each "local" entry (i.e.
// tracked in the map), remove it from the map (unless the proposal
// is not superseded, see below) and attach the value to the entry.
// 2. for each entry:
// - stage written and in-memory effects of the entry (some may apply as no-ops
// if they fail below-raft checks such as the MaxLeaseIndex check)
// - Assuming the MaxLeaseIndex is violated and additional constraints are
// satisfied, carry out (2) from above. On success, we know now that there
// will be a reproposal in the log that can successfully apply. We unbind
// the local proposal (so we don't signal it) and apply the current entry
// as a no-op.
// 3. carry out additional side effects of the entire batch (stats updates etc).
//
// A prerequisite for (2) is that there currently aren't any copies of the proposal
// in the log that may ultimately apply, or we risk doubly applying commands - a
// correctness bug. After (2), any copies of the entry present in the log will have
// a MaxLeaseIndex strictly less than that of the in-memory command, and will be
// Superseded() by it.
//
// We can always safely create an identical copy (i.e. (1)) because of the
// replay protection conferred by the MaxLeaseIndex - all but the first
// proposal (that reach consensus) will be rejected (i.e. apply as a no-op).
//
// Naively, one might hope that by invoking (2) upon applying an entry for
// a command that is rejected due to a MaxLeaseIndex one could achieve the
// invariant that there is only ever one unapplied copy of the entry in the
// log, and then the in-memory proposal could reflect the MaxLeaseIndex
// assigned to this unapplied copy at all times.
//
// Unfortunately, for various reasons, this invariant does not hold:
// - entry application isn't durable, so upon a restart, we might roll
// back to a log position that yet has to catch up over multiple previous
// incarnations of (2), i.e. we will see the same entry multiple times at
// various MaxLeaseIndex values.
// (This technically not a problem, since we're losing the in-memory proposal
// during the restart anyway, but should be kept in mind anyway).
// - Raft proposal forwarding due to (1)-type reproposals could "in
// principle" lead to an old copy of the entry appearing again in the
// unapplied log, at least if we make the reasonable assumption that
// forwarded proposals may arrive at the leader with arbitrary delays.
//
// As a result, we can't "just" invoke (2) when seeing a rejected command,
// we additionally have to verify that there isn't a more recent reproposal
// underway that could apply successfully and supersedes the one we're
// currently looking at.
// So we carry out (2) only if the MaxLeaseIndex of the in-mem proposal matches
// that of the current entry, and update the in-mem MaxLeaseIndex with the result
// of (2) if it did.
//
// An example follows. Consider the following situation (where N is some base
// index not relevant to the example) in which we have one inflight proposal which
// has been triplicated in the log (due to [^1]):
//
// proposals[id] = p{Cmd{MaxLeaseIndex: 100, ...}}
//
// ... (unrelated entries)
// raftlog[N] = Cmd{MaxLeaseIndex: 100, ...}
// ... (unrelated entries)
// raftlog[N+12] = (same as N)
// ... (unrelated entries)
// raftlog[N+15] = (same as N)
//
// where we assume that the `MaxLeaseIndex` 100 is invalid, i.e. when we see
// the first copy of the command being applied, we've already applied some
// command with equal or higher `MaxLeaseIndex`. In a world without
// mechanism (2), `N` would be rejected, and would finalize the proposal
// (i.e. signal the client with an error and remove the entry from
// `proposals`). Later, `N+12` and `N+15` would similarly be rejected (but
// they wouldn't even be regarded as local proposals any more due to not
// being present in `proposals`).
//
// However, (2) exists and it will engage during application of `N`: realizing
// that the current copies of the entry are all going to be rejected, it will
// modify the proposal by assigning a new `MaxLeaseIndex` to it, and handing
// it to `(*Replica).propose` again (which hands it to the proposal buffer,
// which will at some point flush it, leading to re-insertion into the raft
// log and the `proposals` map). The result will be this picture:
//
// proposals[id] = p{Cmd{MaxLeaseIndex: 192, ...}} <-- modified
//
// ... (unrelated entries)
// raftlog[N] = Cmd{MaxLeaseIndex: 100, ...} <-- applied (as no-op)
// ... (unrelated entries)
// raftlog[N+12] = (same as N) (superseded)
// ... (unrelated entries)
// raftlog[N+15] = (same as N) (superseded)
// ... (unrelated entries)
// raftlog[N+18] = Cmd{MaxLeaseIndex: 192, ...} <-- modified
//
// `N+18` might (in fact, is likely to) apply successfully. As a result, when
// we consider `N+12` or `N+15` for application, we must *not* carry out (2)
// again, or we break replay protection. In other words, the `MaxLeaseIndex`
// of the command being applied must be compared with the `MaxLeaseIndex` of
// the command in the proposals map; only if they match do we know that this
// is the most recent (in MaxLeaseIndex order) copy of the command, and only
// then can (2) engage. In addition, an entry that doesn't pass this equality
// check must not signal the proposer and/or unlink from the proposals map (as a
// newer reproposal which might succeed is likely in the log)[^4].
//
// Another way of framing the above is that `proposals[id].Cmd.MaxLeaseIndex`
// actually tracks the maximum `MaxLeaseIndex` of all copies that may be present in
// the log.
//
// If (2) results in an error (for example, since the proposal now fails to
// respect the closed timestamp), that error will finalize the proposal and
// is returned to the client.
//
// [^1]: https://github.com/cockroachdb/cockroach/blob/59ce13b6052a99a0318e3dfe017908ff5630db30/pkg/kv/kvserver/replica_raft.go#L1224
// [^2]: https://github.com/cockroachdb/cockroach/blob/59ce13b6052a99a0318e3dfe017908ff5630db30/pkg/kv/kvserver/replica_application_result.go#L148
// [^3]: it's debatable how useful this below-raft reproposal mechanism is.
// It was introduced in https://github.com/cockroachdb/cockroach/pull/35261,
// and perhaps could be phased out again if we also did
// https://github.com/cockroachdb/cockroach/issues/21849. Historical
// evidence points to https://github.com/cockroachdb/cockroach/issues/28876
// as the motivation for introducing this mechanism, i.e. it was about
// reducing failure rates early in the life of a cluster when raft
// leaderships were being determined. Perhaps we could "simply" disable
// async writes unless leadership was stable instead, by blocking on the
// proposal anyway.
// [^4]: https://github.com/cockroachdb/cockroach/blob/ab6a8650621ae798377f12bbfc1eee2fbec95480/pkg/kv/kvserver/replica_application_decoder.go#L100-L114
proposals map[kvserverbase.CmdIDKey]*ProposalData
// Indicates that the replica is in the process of applying log entries.
// Updated to true in handleRaftReady before entries are removed from
// the proposals map and set to false after they are applied. Useful in
// conjunction with len(proposals) to check for any in-flight proposals
// whose effects have not yet taken hold without synchronizing with
// raftMu and the entire handleRaftReady loop. Not needed if raftMu is
// already held.
applyingEntries bool
// The replica's Raft group "node". Can be nil for destroyed replicas
// (destroyReasonRemoved) and in some tests, otherwise is never nil.
//
// TODO(erikgrinaker): make this never be nil.
internalRaftGroup *raft.RawNode
// The ID of the leader replica within the Raft group. NB: this is updated
// in a separate critical section from the Raft group, and can therefore
// briefly be out of sync with the Raft status.
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
// 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.
//
// 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).
lastUpdateTimes lastUpdateTimesMap
// 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 kvpb.RaftIndex
// 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. At that point ownership of the quota is
// transferred from r.mu.proposals to this queue.
// We'll release the respective quota once all replicas have persisted the
// corresponding entry into their logs (or once we give up waiting on some
// replica because it looks like it's dead).
quotaReleaseQueue []*quotapool.IntAlloc
// Counts calls to Replica.tick()
ticks int
// lastProposalAtTicks tracks the time of the last proposal, in ticks.
lastProposalAtTicks 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
// cachedProtectedTS provides the state of the protected timestamp
// subsystem as used on the request serving path to determine the effective
// gc threshold given the current TTL when using strict GC enforcement.
//
// It would be too expensive to go read from the protected timestamp cache
// for every request. Instead, if clients want to ensure that their request
// will see the effect of a protected timestamp record, they need to verify
// the request. See the comment on the struct for more details.
cachedProtectedTS cachedProtectedTimestampState
// largestPreviousMaxRangeSizeBytes tracks a previous conf.RangeMaxBytes
// which exceeded the current conf.RangeMaxBytes to help defeat the range
// backpressure mechanism in cases where a user reduces the configured range
// size. It is set when the span config changes to a smaller value and the
// current range size exceeds the new value. It is cleared after the range's
// size drops below its current conf.MaxRangeBytes or if the
// conf.MaxRangeBytes increases to surpass the current value.
largestPreviousMaxRangeSizeBytes int64
// failureToGossipSystemConfig is set to true when the leaseholder of the
// range containing the system config span fails to gossip due to an
// outstanding intent (see MaybeGossipSystemConfig). It is reset when the
// system config is successfully gossiped or when the Replica loses the
// lease. It is read when handling a MaybeGossipSystemConfigIfHaveFailure
// local result trigger. That trigger is set when an EndTransaction with an
// ABORTED status is evaluated on a range containing the system config span.
//
// While the gossipping of the system config span is best-effort, the sql
// schema leasing mechanism degrades dramatically if changes are not
// gossiped. This degradation is due to the fact that schema changes, after
// writing intents, often need to ensure that there aren't outstanding
// leases on old versions and if there are, roll back and wait until there
// are not. The problem is that this waiting may take a long time if the
// current leaseholders are not notified. We deal with this by detecting the
// abort of a transaction which might have blocked the system config from
// being gossiped and attempting to gossip again.
failureToGossipSystemConfig bool
tenantID roachpb.TenantID // Set when first initialized, not modified after
// Historical information about the command that set the closed timestamp.
closedTimestampSetter closedTimestampSetterInfo
// Followers to which replication traffic is currently dropped.
//
// Never mutated in place (always replaced wholesale), so can be leaked
// outside the surrounding mutex.
pausedFollowers map[roachpb.ReplicaID]struct{}
slowProposalCount int64 // updated in refreshProposalsLocked
// replicaFlowControlIntegration is used to interface with replication flow
// control. It's backed by the node-level kvflowcontrol.Controller that
// manages flow tokens for on a per <tenant,work class> basis, which it
// interfaces through a replica-level kvflowcontrol.Handle. It's
// actively used on replicas initiating replication traffic, i.e. are
// both the leaseholder and raft leader.
//
// Accessing it requires Replica.mu to be held, exclusively.
replicaFlowControlIntegration replicaFlowControlIntegration
}
// The raft log truncations that are pending. Access is protected by its own
// mutex. All implementation details should be considered hidden except to
// the code in raft_log_truncator.go. External code should only use the
// computePostTrunc* methods.
pendingLogTruncations pendingLogTruncations
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
// opFilter is a best-effort filter that informs the raft processing
// goroutine of which logical operations the rangefeed processor is
// interested in based on the processor's current registrations.
//
// The filter is allowed to return false positives, but not false
// negatives. False negatives are avoided by updating (expanding) the
// filter while holding the Replica.raftMu when adding new registrations
// after flushing the rangefeed.Processor event channel. This ensures
// that no events that were filtered before the new registration was
// added will be observed by the new registration and all events after
// the new registration will respect the updated filter.
//
// Requires Replica.rangefeedMu be held when mutating the pointer.
opFilter *rangefeed.Filter
}
// 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 contains a set of remote ReplicaIDs that are to be reported
// as unreachable on the next raft tick.
unreachablesMu struct {
syncutil.Mutex
remotes map[roachpb.ReplicaID]struct{}
}
// r.mu < r.protectedTimestampMu
protectedTimestampMu struct {
syncutil.Mutex
// minStateReadTimestamp is a lower bound on the timestamp of the cached
// protected timestamp state which may be used when updating
// pendingGCThreshold. This field acts to eliminate races between
// verification of protected timestamp records and the setting of a new
// GC threshold
minStateReadTimestamp hlc.Timestamp
// pendingGCThreshold holds a timestamp which is being proposed as a new
// GC threshold for the range.
pendingGCThreshold hlc.Timestamp
}
}
// 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 redact.StringWithoutMarkers(r)
}
// SafeFormat implements the redact.SafeFormatter interface.
func (r *Replica) SafeFormat(w redact.SafePrinter, _ rune) {
w.Printf("[n%d,s%d,r%s]",
r.store.Ident.NodeID, r.store.Ident.StoreID, &r.rangeStr)
}
// ReplicaID returns the ID for the Replica. This value is fixed for the
// lifetime of the Replica.
func (r *Replica) ReplicaID() roachpb.ReplicaID {
return r.replicaID
}
// ID returns the FullReplicaID for the Replica.
func (r *Replica) ID() storage.FullReplicaID {
return storage.FullReplicaID{RangeID: r.RangeID, ReplicaID: r.replicaID}
}
// cleanupFailedProposal cleans up after a proposal that has failed. It
// clears any references to the proposal and releases associated quota.
// It requires that Replica.mu is exclusively held.
func (r *Replica) cleanupFailedProposalLocked(p *ProposalData) {
r.mu.AssertHeld()
delete(r.mu.proposals, p.idKey)
p.releaseQuota()
}
// GetMinBytes gets the replica's minimum byte threshold.
func (r *Replica) GetMinBytes(_ context.Context) int64 {
r.mu.RLock()
defer r.mu.RUnlock()
return r.mu.conf.RangeMinBytes
}
// GetMaxBytes gets the replica's maximum byte threshold.
func (r *Replica) GetMaxBytes(_ context.Context) int64 {
r.mu.RLock()
defer r.mu.RUnlock()
return r.mu.conf.RangeMaxBytes
}
// SetSpanConfig sets the replica's span config. It returns whether the change
// to the span config was "significant". For significant changes, the caller
// should queue up the span to all the relevant queues since they may not decide
// to process this replica.
func (r *Replica) SetSpanConfig(conf roachpb.SpanConfig) bool {
r.mu.Lock()
defer r.mu.Unlock()
oldConf := r.mu.conf
if r.IsInitialized() && !r.mu.conf.IsEmpty() && !conf.IsEmpty() {
total := r.mu.state.Stats.Total()
// Set largestPreviousMaxRangeSizeBytes if the current range size is
// greater than the new limit, if the limit has decreased from what we
// last remember, and we don't already have a larger value.
if total > conf.RangeMaxBytes && conf.RangeMaxBytes < r.mu.conf.RangeMaxBytes &&
r.mu.largestPreviousMaxRangeSizeBytes < r.mu.conf.RangeMaxBytes &&
// We also want to make sure that we're replacing a real span config.
// If we didn't have this check, the default value would prevent
// backpressure until the range got larger than it.
r.mu.spanConfigExplicitlySet {
r.mu.largestPreviousMaxRangeSizeBytes = r.mu.conf.RangeMaxBytes
} else if r.mu.largestPreviousMaxRangeSizeBytes > 0 &&
r.mu.largestPreviousMaxRangeSizeBytes < conf.RangeMaxBytes {
// Reset it if the new limit is larger than the largest we were
// aware of.
r.mu.largestPreviousMaxRangeSizeBytes = 0
}
}
if knobs := r.store.TestingKnobs(); knobs != nil && knobs.SetSpanConfigInterceptor != nil {
conf = knobs.SetSpanConfigInterceptor(r.descRLocked(), conf)
}
r.mu.conf, r.mu.spanConfigExplicitlySet = conf, true
return oldConf.HasConfigurationChange(conf)
}
// MaybeQueue attempts to check and queue against the subset of that are
// impacted by changes to the SpanConfig. This should be called after any
// changes to the span configs.