@@ -379,15 +379,15 @@ type Replica struct {
379
379
minLeaseProposedTS hlc.Timestamp
380
380
// A pointer to the zone config for this replica.
381
381
zone * config.ZoneConfig
382
- // localProposals stores the Raft in-flight commands which originated at
382
+ // proposals stores the Raft in-flight commands which originated at
383
383
// this Replica, i.e. all commands for which propose has been called,
384
384
// but which have not yet applied.
385
385
//
386
386
// The *ProposalData in the map are "owned" by it. Elements from the
387
387
// map must only be referenced while Replica.mu is held, except if the
388
388
// element is removed from the map first. The notable exception is the
389
389
// contained RaftCommand, which we treat as immutable.
390
- localProposals map [storagebase.CmdIDKey ]* ProposalData
390
+ proposals map [storagebase.CmdIDKey ]* ProposalData
391
391
internalRaftGroup * raft.RawNode
392
392
// The ID of the replica within the Raft group. May be 0 if the replica has
393
393
// been created from a preemptive snapshot (i.e. before being added to the
@@ -703,7 +703,7 @@ func (r *Replica) initRaftMuLockedReplicaMuLocked(
703
703
r .cmdQMu .queues [spanset .SpanLocal ] = NewCommandQueue (false /* optimizeOverlap */ )
704
704
r .cmdQMu .Unlock ()
705
705
706
- r .mu .localProposals = map [storagebase.CmdIDKey ]* ProposalData {}
706
+ r .mu .proposals = map [storagebase.CmdIDKey ]* ProposalData {}
707
707
r .mu .checksums = map [uuid.UUID ]ReplicaChecksum {}
708
708
// Clear the internal raft group in case we're being reset. Since we're
709
709
// reloading the raft state below, it isn't safe to use the existing raft
@@ -874,7 +874,7 @@ func (r *Replica) cancelPendingCommandsLocked() {
874
874
Err : roachpb .NewError (roachpb .NewAmbiguousResultError ("removing replica" )),
875
875
ProposalRetry : proposalRangeNoLongerExists ,
876
876
}
877
- for _ , p := range r .mu .localProposals {
877
+ for _ , p := range r .mu .proposals {
878
878
r .cleanupFailedProposalLocked (p )
879
879
p .finishApplication (pr )
880
880
}
@@ -885,7 +885,7 @@ func (r *Replica) cancelPendingCommandsLocked() {
885
885
func (r * Replica ) cleanupFailedProposalLocked (p * ProposalData ) {
886
886
// Clear the proposal from the proposals map. May be a no-op if the
887
887
// proposal has not yet been inserted into the map.
888
- delete (r .mu .localProposals , p .idKey )
888
+ delete (r .mu .proposals , p .idKey )
889
889
// Release associated quota pool resources if we have been tracking
890
890
// this command.
891
891
//
@@ -1896,7 +1896,7 @@ func (r *Replica) State() storagepb.RangeInfo {
1896
1896
var ri storagepb.RangeInfo
1897
1897
ri .ReplicaState = * (protoutil .Clone (& r .mu .state )).(* storagepb.ReplicaState )
1898
1898
ri .LastIndex = r .mu .lastIndex
1899
- ri .NumPending = uint64 (len (r .mu .localProposals ))
1899
+ ri .NumPending = uint64 (len (r .mu .proposals ))
1900
1900
ri .RaftLogSize = r .mu .raftLogSize
1901
1901
ri .NumDropped = uint64 (r .mu .droppedMessages )
1902
1902
if r .mu .proposalQuota != nil {
@@ -3584,11 +3584,11 @@ func (r *Replica) insertProposalLocked(
3584
3584
proposal .idKey , proposal .command .MaxLeaseIndex )
3585
3585
}
3586
3586
3587
- if _ , ok := r .mu .localProposals [proposal .idKey ]; ok {
3587
+ if _ , ok := r .mu .proposals [proposal .idKey ]; ok {
3588
3588
ctx := r .AnnotateCtx (context .TODO ())
3589
3589
log .Fatalf (ctx , "pending command already exists for %s" , proposal .idKey )
3590
3590
}
3591
- r .mu .localProposals [proposal .idKey ] = proposal
3591
+ r .mu .proposals [proposal .idKey ] = proposal
3592
3592
if isLease {
3593
3593
// For lease requests, we return zero because no real MaxLeaseIndex is assigned.
3594
3594
// We could also return the lastAssignedIndex but this invites confusion.
@@ -3812,7 +3812,7 @@ func (r *Replica) propose(
3812
3812
}
3813
3813
3814
3814
// Must not use `proposal` in the closure below as a proposal which is not
3815
- // present in r.mu.localProposals is no longer protected by the mutex. Abandoning
3815
+ // present in r.mu.proposals is no longer protected by the mutex. Abandoning
3816
3816
// a command only abandons the associated context. As soon as we propose a
3817
3817
// command to Raft, ownership passes to the "below Raft" machinery. In
3818
3818
// particular, endCmds will be invoked when the command is applied. There are
@@ -3821,7 +3821,7 @@ func (r *Replica) propose(
3821
3821
// range.
3822
3822
tryAbandon := func () bool {
3823
3823
r .mu .Lock ()
3824
- p , ok := r .mu .localProposals [idKey ]
3824
+ p , ok := r .mu .proposals [idKey ]
3825
3825
if ok {
3826
3826
// TODO(radu): Should this context be created via tracer.ForkCtxSpan?
3827
3827
// We'd need to make sure the span is finished eventually.
@@ -3833,7 +3833,7 @@ func (r *Replica) propose(
3833
3833
return proposalCh , tryAbandon , maxLeaseIndex , nil
3834
3834
}
3835
3835
3836
- // submitProposalLocked proposes or re-proposes a command in r.mu.localProposals .
3836
+ // submitProposalLocked proposes or re-proposes a command in r.mu.proposals .
3837
3837
// The replica lock must be held.
3838
3838
func (r * Replica ) submitProposalLocked (p * ProposalData ) error {
3839
3839
p .proposedAtTicks = r .mu .ticks
@@ -3948,9 +3948,9 @@ func (r *Replica) quiesce() bool {
3948
3948
3949
3949
func (r * Replica ) quiesceLocked () bool {
3950
3950
ctx := r .AnnotateCtx (context .TODO ())
3951
- if len (r .mu .localProposals ) != 0 {
3951
+ if len (r .mu .proposals ) != 0 {
3952
3952
if log .V (3 ) {
3953
- log .Infof (ctx , "not quiescing: %d pending commands" , len (r .mu .localProposals ))
3953
+ log .Infof (ctx , "not quiescing: %d pending commands" , len (r .mu .proposals ))
3954
3954
}
3955
3955
return false
3956
3956
}
@@ -4577,7 +4577,7 @@ func (r *Replica) tick(livenessMap IsLiveMap) (bool, error) {
4577
4577
// correctness issues.
4578
4578
func (r * Replica ) maybeQuiesceLocked (livenessMap IsLiveMap ) bool {
4579
4579
ctx := r .AnnotateCtx (context .TODO ())
4580
- status , ok := shouldReplicaQuiesce (ctx , r , r .store .Clock ().Now (), len (r .mu .localProposals ), livenessMap )
4580
+ status , ok := shouldReplicaQuiesce (ctx , r , r .store .Clock ().Now (), len (r .mu .proposals ), livenessMap )
4581
4581
if ! ok {
4582
4582
return false
4583
4583
}
@@ -4828,7 +4828,7 @@ func (r *Replica) refreshProposalsLocked(refreshAtDelta int, reason refreshRaftR
4828
4828
4829
4829
numShouldRetry := 0
4830
4830
var reproposals pendingCmdSlice
4831
- for _ , p := range r .mu .localProposals {
4831
+ for _ , p := range r .mu .proposals {
4832
4832
if p .command .MaxLeaseIndex == 0 {
4833
4833
// Commands without a MaxLeaseIndex cannot be reproposed, as they might
4834
4834
// apply twice. We also don't want to ask the proposer to retry these
@@ -4843,7 +4843,7 @@ func (r *Replica) refreshProposalsLocked(refreshAtDelta int, reason refreshRaftR
4843
4843
} else if cannotApplyAnyMore := ! p .command .ReplicatedEvalResult .IsLeaseRequest &&
4844
4844
p .command .MaxLeaseIndex <= r .mu .state .LeaseAppliedIndex ; cannotApplyAnyMore {
4845
4845
// The command's designated lease index slot was filled up. We got to
4846
- // LeaseAppliedIndex and p is still pending in r.mu.localProposals ; generally
4846
+ // LeaseAppliedIndex and p is still pending in r.mu.proposals ; generally
4847
4847
// this means that proposal p didn't commit, and it will be sent back to
4848
4848
// the proposer for a retry - the request needs to be re-evaluated and the
4849
4849
// command re-proposed with a new MaxLeaseIndex. Note that this branch is not
@@ -4854,7 +4854,7 @@ func (r *Replica) refreshProposalsLocked(refreshAtDelta int, reason refreshRaftR
4854
4854
// reasonSnapshotApplied - in that case we don't know if p or some other
4855
4855
// command filled the p.command.MaxLeaseIndex slot (i.e. p might have been
4856
4856
// applied, but we're not watching for every proposal when applying a
4857
- // snapshot, so nobody removed p from r.mu.localProposals ). In this
4857
+ // snapshot, so nobody removed p from r.mu.proposals ). In this
4858
4858
// ambiguous case, we'll also send the command back to the proposer for a
4859
4859
// retry, but the proposer needs to be aware that, if the retry fails, an
4860
4860
// AmbiguousResultError needs to be returned to the higher layers.
@@ -4915,7 +4915,7 @@ func (r *Replica) refreshProposalsLocked(refreshAtDelta int, reason refreshRaftR
4915
4915
// that they can make it in the right place. Reproposing in order is
4916
4916
// definitely required, however.
4917
4917
//
4918
- // TODO(tschottdorf): evaluate whether `r.mu.localProposals ` should
4918
+ // TODO(tschottdorf): evaluate whether `r.mu.proposals ` should
4919
4919
// be a list/slice.
4920
4920
sort .Sort (reproposals )
4921
4921
for _ , p := range reproposals {
@@ -5295,14 +5295,14 @@ func (r *Replica) processRaftCommand(
5295
5295
}
5296
5296
5297
5297
r .mu .Lock ()
5298
- proposal , proposedLocally := r .mu .localProposals [idKey ]
5298
+ proposal , proposedLocally := r .mu .proposals [idKey ]
5299
5299
5300
5300
// TODO(tschottdorf): consider the Trace situation here.
5301
5301
if proposedLocally {
5302
5302
// We initiated this command, so use the caller-supplied context.
5303
5303
ctx = proposal .ctx
5304
5304
proposal .ctx = nil // avoid confusion
5305
- delete (r .mu .localProposals , idKey )
5305
+ delete (r .mu .proposals , idKey )
5306
5306
}
5307
5307
5308
5308
leaseIndex , proposalRetry , forcedErr := r .checkForcedErrLocked (ctx , idKey , raftCmd , proposal , proposedLocally )
0 commit comments