Skip to content

Commit

Permalink
kvserver: do lazy map allocations in replicaFlowControl
Browse files Browse the repository at this point in the history
These methods will typically return empty maps, so we can avoid the
allocations.

Informs #111561

Epic: none

Release note: None
  • Loading branch information
sumeerbhola committed Oct 26, 2023
1 parent b13be53 commit 56b5c2f
Show file tree
Hide file tree
Showing 2 changed files with 15 additions and 4 deletions.
16 changes: 12 additions & 4 deletions pkg/kv/kvserver/flow_control_replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,12 +60,14 @@ func (rf *replicaFlowControl) getPausedFollowers() map[roachpb.ReplicaID]struct{

func (rf *replicaFlowControl) getBehindFollowers() map[roachpb.ReplicaID]struct{} {
rf.assertLocked()
behindFollowers := make(map[roachpb.ReplicaID]struct{})
var behindFollowers map[roachpb.ReplicaID]struct{}
rf.mu.internalRaftGroup.WithProgress(func(id uint64, _ raft.ProgressType, progress rafttracker.Progress) {
if progress.State == rafttracker.StateReplicate {
return
}

if behindFollowers == nil {
behindFollowers = make(map[roachpb.ReplicaID]struct{})
}
replID := roachpb.ReplicaID(id)
behindFollowers[replID] = struct{}{}

Expand All @@ -90,12 +92,15 @@ func (rf *replicaFlowControl) getBehindFollowers() map[roachpb.ReplicaID]struct{

func (rf *replicaFlowControl) getInactiveFollowers() map[roachpb.ReplicaID]struct{} {
rf.assertLocked()
inactiveFollowers := make(map[roachpb.ReplicaID]struct{})
var inactiveFollowers map[roachpb.ReplicaID]struct{}
for _, desc := range rf.getDescriptor().Replicas().Descriptors() {
if desc.ReplicaID == rf.getReplicaID() {
continue
}
if !rf.mu.lastUpdateTimes.isFollowerActiveSince(desc.ReplicaID, timeutil.Now(), rf.store.cfg.RangeLeaseDuration) {
if inactiveFollowers == nil {
inactiveFollowers = make(map[roachpb.ReplicaID]struct{})
}
inactiveFollowers[desc.ReplicaID] = struct{}{}
}
}
Expand All @@ -104,12 +109,15 @@ func (rf *replicaFlowControl) getInactiveFollowers() map[roachpb.ReplicaID]struc

func (rf *replicaFlowControl) getDisconnectedFollowers() map[roachpb.ReplicaID]struct{} {
rf.assertLocked()
disconnectedFollowers := make(map[roachpb.ReplicaID]struct{})
var disconnectedFollowers map[roachpb.ReplicaID]struct{}
for _, desc := range rf.getDescriptor().Replicas().Descriptors() {
if desc.ReplicaID == rf.getReplicaID() {
continue
}
if !rf.store.raftTransportForFlowControl.isConnectedTo(desc.StoreID) {
if disconnectedFollowers == nil {
disconnectedFollowers = make(map[roachpb.ReplicaID]struct{})
}
disconnectedFollowers[desc.ReplicaID] = struct{}{}
}
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/flow_control_replica_integration.go
Original file line number Diff line number Diff line change
Expand Up @@ -286,6 +286,9 @@ func (f *replicaFlowControlIntegrationImpl) handle() (kvflowcontrol.Handle, bool
// reconnect previously disconnected streams if we're able.
func (f *replicaFlowControlIntegrationImpl) refreshStreams(ctx context.Context, reason string) {
f.disconnectStreams(ctx, f.notActivelyReplicatingTo(), reason)
// TODO(sumeer): we call notActivelyReplicatingTo() again in tryReconnect(),
// which is wasteful, since refreshStreams is called on every raft tick.
// Simply pass the return value from the call above to the following method.
f.tryReconnect(ctx)
}

Expand Down

0 comments on commit 56b5c2f

Please sign in to comment.