Skip to content

Commit d58741b

Browse files
craig[bot]pav-kv
andcommitted
Merge #143736
143736: kvserver: move a couple of fields to shMu r=tbg a=pav-kv This allows reading the fields outside `Replica.mu` sections. Related to #140235 Co-authored-by: Pavel Kalinnikov <[email protected]>
2 parents 54966d5 + dd1944a commit d58741b

File tree

5 files changed

+26
-28
lines changed

5 files changed

+26
-28
lines changed

pkg/kv/kvserver/replica.go

Lines changed: 8 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -558,6 +558,14 @@ type Replica struct {
558558
// log was checked for truncation or at the time of the last Raft log
559559
// truncation.
560560
raftLogLastCheckSize int64
561+
562+
// leaderID is the ID of the leader replica within the Raft group.
563+
// NB: this is updated in a separate critical section from the Raft group,
564+
// and can therefore briefly be out of sync with the Raft status.
565+
leaderID roachpb.ReplicaID
566+
// currentRACv2Mode is always in-sync with RawNode.
567+
// MsgAppPull <=> LazyReplication.
568+
currentRACv2Mode rac2.RaftMsgAppMode
561569
}
562570

563571
mu struct {
@@ -859,10 +867,6 @@ type Replica struct {
859867
// TODO(erikgrinaker): make this never be nil.
860868
internalRaftGroup *raft.RawNode
861869

862-
// The ID of the leader replica within the Raft group. NB: this is updated
863-
// in a separate critical section from the Raft group, and can therefore
864-
// briefly be out of sync with the Raft status.
865-
leaderID roachpb.ReplicaID
866870
// The most recently added replica for the range and when it was added.
867871
// Used to determine whether a replica is new enough that we shouldn't
868872
// penalize it for being slightly behind. These field gets cleared out once
@@ -990,11 +994,6 @@ type Replica struct {
990994
// implementation.
991995
replicaFlowControlIntegration replicaFlowControlIntegration
992996

993-
// The currentRACv2Mode is always in-sync with RawNode.
994-
// MsgAppPull <=> LazyReplication.
995-
// Updated with both raftMu and mu held.
996-
currentRACv2Mode rac2.RaftMsgAppMode
997-
998997
// raftTracer is used to trace raft messages that are sent with a
999998
// tracing context.
1000999
raftTracer rafttrace.RaftTracer

pkg/kv/kvserver/replica_init.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -246,7 +246,7 @@ func newUninitializedReplicaWithoutRaftGroup(
246246
store.cfg.Settings, store.stopper, r.AmbientContext, r, onTrip, onReset,
247247
)
248248
r.LeaderlessWatcher = NewLeaderlessWatcher(r)
249-
r.mu.currentRACv2Mode = r.replicationAdmissionControlModeToUse(context.TODO())
249+
r.shMu.currentRACv2Mode = r.replicationAdmissionControlModeToUse(context.TODO())
250250
r.raftMu.flowControlLevel = kvflowcontrol.GetV2EnabledWhenLeaderLevel(
251251
r.raftCtx, store.ClusterSettings(), store.TestingKnobs().FlowControlTestingKnobs)
252252
if r.raftMu.flowControlLevel > kvflowcontrol.V2NotEnabledWhenLeader {
@@ -373,7 +373,7 @@ func (r *Replica) initRaftGroupRaftMuLockedReplicaMuLocked() error {
373373
raftpb.PeerID(r.replicaID),
374374
r.shMu.state.RaftAppliedIndex,
375375
r.store.cfg,
376-
r.mu.currentRACv2Mode == rac2.MsgAppPull,
376+
r.shMu.currentRACv2Mode == rac2.MsgAppPull,
377377
&raftLogger{ctx: ctx},
378378
(*replicaRLockedStoreLiveness)(r),
379379
r.store.raftMetrics,

pkg/kv/kvserver/replica_proposal_quota.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -145,8 +145,8 @@ func (r *Replica) updateProposalQuotaRaftMuLocked(
145145
// need for a setting change callback, as handleRaftReady (the caller), will
146146
// be called at least at the tick interval for a non-quiescent range.
147147
shouldInitQuotaPool := false
148-
if r.mu.leaderID != lastLeaderID {
149-
if r.replicaID == r.mu.leaderID {
148+
if r.shMu.leaderID != lastLeaderID {
149+
if r.replicaID == r.shMu.leaderID {
150150
r.mu.lastUpdateTimes = make(map[roachpb.ReplicaID]time.Time)
151151
r.mu.lastUpdateTimes.updateOnBecomeLeader(r.shMu.state.Desc.Replicas().Descriptors(), now)
152152
r.mu.replicaFlowControlIntegration.onBecameLeader(ctx)
@@ -176,10 +176,10 @@ func (r *Replica) updateProposalQuotaRaftMuLocked(
176176
}
177177
return
178178
}
179-
} else if r.replicaID == r.mu.leaderID && r.mu.proposalQuota == nil && enabled {
179+
} else if r.replicaID == r.shMu.leaderID && r.mu.proposalQuota == nil && enabled {
180180
r.mu.lastProposalAtTicks = r.mu.ticks // delay imminent quiescence
181181
shouldInitQuotaPool = true
182-
} else if r.replicaID != r.mu.leaderID {
182+
} else if r.replicaID != r.shMu.leaderID {
183183
// We're a follower and have been since the last update. Nothing to do.
184184
return
185185
}

pkg/kv/kvserver/replica_raft.go

Lines changed: 11 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -651,7 +651,7 @@ func (r *Replica) ticksSinceLastMessageRLocked() int64 {
651651
// status where necessary.
652652
func (r *Replica) isRaftLeaderRLocked() bool {
653653
// Defensively check replicaID != 0.
654-
return r.replicaID != 0 && r.replicaID == r.mu.leaderID
654+
return r.replicaID != 0 && r.replicaID == r.shMu.leaderID
655655
}
656656

657657
var errRemoved = errors.New("replica removed")
@@ -955,10 +955,11 @@ func (r *Replica) handleRaftReadyRaftMuLocked(
955955
var raftNodeBasicState replica_rac2.RaftNodeBasicState
956956
var logSnapshot raft.LogSnapshot
957957

958-
r.mu.Lock()
959-
rac2ModeForReady := r.mu.currentRACv2Mode
960-
leaderID := r.mu.leaderID
958+
rac2ModeForReady := r.shMu.currentRACv2Mode
959+
leaderID := r.shMu.leaderID
961960
lastLeaderID := leaderID
961+
962+
r.mu.Lock()
962963
err := r.withRaftGroupLocked(func(raftGroup *raft.RawNode) (bool, error) {
963964
r.deliverLocalRaftMsgsRaftMuLockedReplicaMuLocked(ctx, raftGroup)
964965

@@ -976,7 +977,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked(
976977
// Ready, and switch to pull mode after that.
977978
switchToPullModeAfterReady = true
978979
}
979-
r.mu.currentRACv2Mode = rac2ModeToUse
980+
r.shMu.currentRACv2Mode = rac2ModeToUse
980981
}
981982
logSnapshot = raftGroup.LogSnapshot()
982983
if hasReady = raftGroup.HasReady(); hasReady {
@@ -1256,11 +1257,11 @@ func (r *Replica) handleRaftReadyRaftMuLocked(
12561257
r.mu.Lock()
12571258
r.asLogStorage().updateStateRaftMuLockedMuLocked(state)
12581259
var becameLeader bool
1259-
if r.mu.leaderID != leaderID {
1260-
r.mu.leaderID = leaderID
1260+
if r.shMu.leaderID != leaderID {
1261+
r.shMu.leaderID = leaderID
12611262
// Clear the remote proposal set. Would have been nil already if not
12621263
// previously the leader.
1263-
becameLeader = r.mu.leaderID == r.replicaID
1264+
becameLeader = leaderID == r.replicaID
12641265
}
12651266
r.mu.Unlock()
12661267

@@ -1570,9 +1571,7 @@ func (r *Replica) processRACv2PiggybackedAdmitted(ctx context.Context) {
15701571
func (r *Replica) processRACv2RangeController(ctx context.Context) {
15711572
r.raftMu.Lock()
15721573
defer r.raftMu.Unlock()
1573-
// Can read Replica.mu.currentRACv2Mode since updates require both raftMu
1574-
// and Replica.mu.
1575-
mode := r.mu.currentRACv2Mode
1574+
mode := r.shMu.currentRACv2Mode
15761575
var logSnapshot raft.LogSnapshot
15771576
if mode == rac2.MsgAppPull {
15781577
err := r.withRaftGroup(func(raftGroup *raft.RawNode) (bool, error) {
@@ -1585,7 +1584,7 @@ func (r *Replica) processRACv2RangeController(ctx context.Context) {
15851584
}
15861585
}
15871586
r.flowControlV2.ProcessSchedulerEventRaftMuLocked(
1588-
ctx, r.mu.currentRACv2Mode, logSnapshot)
1587+
ctx, r.shMu.currentRACv2Mode, logSnapshot)
15891588
}
15901589

15911590
// SendMsgApp implements rac2.MsgAppSender.

pkg/kv/kvserver/store_raft.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -852,7 +852,7 @@ func (s *Store) supportWithdrawnCallback(supportWithdrawnForStoreIDs map[roachpb
852852
if !r.mu.asleep {
853853
return false
854854
}
855-
leader, err := r.getReplicaDescriptorByIDRLocked(r.mu.leaderID, roachpb.ReplicaDescriptor{})
855+
leader, err := r.getReplicaDescriptorByIDRLocked(r.shMu.leaderID, roachpb.ReplicaDescriptor{})
856856
// If we found the replica's leader's store, and it doesn't match any of
857857
// the stores in supportWithdrawnForStoreIDs, the replica shouldn't wake
858858
// up. In all other cases, the replica should wake up.

0 commit comments

Comments
 (0)