-
Notifications
You must be signed in to change notification settings - Fork 3.7k
/
flow_control_replica.go
140 lines (122 loc) · 4.4 KB
/
flow_control_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
// Copyright 2023 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"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowcontrolpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"go.etcd.io/raft/v3"
rafttracker "go.etcd.io/raft/v3/tracker"
)
// replicaFlowControl is a concrete implementation of the replicaForFlowControl
// interface.
type replicaFlowControl Replica
var _ replicaForFlowControl = &replicaFlowControl{}
func (rf *replicaFlowControl) assertLocked() {
rf.mu.AssertHeld()
}
func (rf *replicaFlowControl) annotateCtx(ctx context.Context) context.Context {
return rf.AnnotateCtx(ctx)
}
func (rf *replicaFlowControl) getTenantID() roachpb.TenantID {
rf.assertLocked()
return rf.mu.tenantID
}
func (rf *replicaFlowControl) getReplicaID() roachpb.ReplicaID {
return rf.replicaID
}
func (rf *replicaFlowControl) getRangeID() roachpb.RangeID {
return rf.RangeID
}
func (rf *replicaFlowControl) getDescriptor() *roachpb.RangeDescriptor {
rf.assertLocked()
r := (*Replica)(rf)
return r.descRLocked()
}
func (rf *replicaFlowControl) getPausedFollowers() map[roachpb.ReplicaID]struct{} {
rf.assertLocked()
return rf.mu.pausedFollowers
}
func (rf *replicaFlowControl) getBehindFollowers() map[roachpb.ReplicaID]struct{} {
rf.assertLocked()
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{}{}
// TODO(irfansharif): Integrating with these other progress fields
// from raft. For replicas exiting rafttracker.StateProbe, perhaps
// compare progress.Match against status.Commit to make sure it's
// sufficiently caught up with respect to its raft log before we
// start deducting tokens for it (lest we run into I3a from
// kvflowcontrol/doc.go). To play well with the replica-level
// proposal quota pool, maybe we also factor its base index?
// Replicas that crashed and came back could come back in
// StateReplicate but be behind on their logs. If we're deducting
// tokens right away for subsequent proposals, it would take some
// time for it to catch up and then later return those tokens to us.
// This is I3a again; do it as part of #95563.
_ = progress.RecentActive
_ = progress.MsgAppFlowPaused
_ = progress.Match
})
return behindFollowers
}
func (rf *replicaFlowControl) getInactiveFollowers() map[roachpb.ReplicaID]struct{} {
rf.assertLocked()
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{}{}
}
}
return inactiveFollowers
}
func (rf *replicaFlowControl) getDisconnectedFollowers() map[roachpb.ReplicaID]struct{} {
rf.assertLocked()
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{}{}
}
}
return disconnectedFollowers
}
func (rf *replicaFlowControl) getAppliedLogPosition() kvflowcontrolpb.RaftLogPosition {
rf.assertLocked()
status := rf.mu.internalRaftGroup.BasicStatus()
return kvflowcontrolpb.RaftLogPosition{
Term: status.Term,
Index: status.Applied,
}
}
func (rf *replicaFlowControl) isScratchRange() bool {
rf.assertLocked()
r := (*Replica)(rf)
return r.isScratchRangeRLocked()
}