Skip to content
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.

Commit 3b7c2ed

Browse files
committedMay 28, 2024·
tracker: rename MsgAppFlowPaused
Epic: none Release note: none
1 parent ea9103e commit 3b7c2ed

8 files changed

+53
-53
lines changed
 

‎pkg/kv/kvserver/flow_control_replica.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -85,7 +85,7 @@ func (rf *replicaFlowControl) getBehindFollowers() map[roachpb.ReplicaID]struct{
8585
// time for it to catch up and then later return those tokens to us.
8686
// This is I3a again; do it as part of #95563.
8787
_ = progress.RecentActive
88-
_ = progress.MsgAppFlowPaused
88+
_ = progress.MsgAppProbesPaused
8989
_ = progress.Match
9090
})
9191
return behindFollowers

‎pkg/kv/kvserver/flow_control_replica_integration_test.go

+7-7
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,7 @@ import (
5252
// follows: progress=(replid@match:<state>:<active>:<paused>,...).
5353
// <state> is one of {probe,replicate,snapshot}, <active> is
5454
// {active,!inactive}, and <paused> is {paused,!paused}. The latter controls
55-
// MsgAppFlowPaused in the raft library, not the CRDB-level follower
55+
// MsgAppProbesPaused in the raft library, not the CRDB-level follower
5656
// pausing.
5757
//
5858
// B. For the raft transport, we can specify the set of replica IDs we're
@@ -169,12 +169,12 @@ func TestFlowControlReplicaIntegration(t *testing.T) {
169169
paused := parts[3] == "paused"
170170

171171
progress[replID] = tracker.Progress{
172-
Match: uint64(index),
173-
State: state,
174-
RecentActive: active,
175-
MsgAppFlowPaused: paused,
176-
Inflights: tracker.NewInflights(1, 0), // avoid NPE
177-
IsLearner: false,
172+
Match: uint64(index),
173+
State: state,
174+
RecentActive: active,
175+
MsgAppProbesPaused: paused,
176+
Inflights: tracker.NewInflights(1, 0), // avoid NPE
177+
IsLearner: false,
178178
}
179179

180180
case "descriptor", "paused", "inactive":

‎pkg/kv/kvserver/split_delay_helper_test.go

+4-4
Original file line numberDiff line numberDiff line change
@@ -139,10 +139,10 @@ func TestSplitDelayToAvoidSnapshot(t *testing.T) {
139139
st := statusWithState(raft.StateLeader)
140140
st.Progress = map[uint64]tracker.Progress{
141141
2: {
142-
State: state,
143-
RecentActive: true,
144-
MsgAppFlowPaused: true, // Unifies string output below.
145-
Inflights: &tracker.Inflights{},
142+
State: state,
143+
RecentActive: true,
144+
MsgAppProbesPaused: true, // Unifies string output below.
145+
Inflights: &tracker.Inflights{},
146146
},
147147
// Healthy follower just for kicks.
148148
3: {State: tracker.StateReplicate},

‎pkg/raft/raft.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -1514,7 +1514,7 @@ func stepLeader(r *raft, m pb.Message) error {
15141514
// If snapshot finish, wait for the MsgAppResp from the remote node before sending
15151515
// out the next MsgApp.
15161516
// If snapshot failure, wait for a heartbeat interval before next try
1517-
pr.MsgAppFlowPaused = true
1517+
pr.MsgAppProbesPaused = true
15181518
case pb.MsgUnreachable:
15191519
// During optimistic replication, if the remote becomes unreachable,
15201520
// there is huge probability that a MsgApp is lost.

‎pkg/raft/raft_snap_test.go

+4-4
Original file line numberDiff line numberDiff line change
@@ -86,8 +86,8 @@ func TestSnapshotFailure(t *testing.T) {
8686
if sm.trk.Progress[2].Next != 1 {
8787
t.Fatalf("Next = %d, want 1", sm.trk.Progress[2].Next)
8888
}
89-
if !sm.trk.Progress[2].MsgAppFlowPaused {
90-
t.Errorf("MsgAppFlowPaused = %v, want true", sm.trk.Progress[2].MsgAppFlowPaused)
89+
if !sm.trk.Progress[2].MsgAppProbesPaused {
90+
t.Errorf("MsgAppProbesPaused = %v, want true", sm.trk.Progress[2].MsgAppProbesPaused)
9191
}
9292
}
9393

@@ -109,8 +109,8 @@ func TestSnapshotSucceed(t *testing.T) {
109109
if sm.trk.Progress[2].Next != 12 {
110110
t.Fatalf("Next = %d, want 12", sm.trk.Progress[2].Next)
111111
}
112-
if !sm.trk.Progress[2].MsgAppFlowPaused {
113-
t.Errorf("MsgAppFlowPaused = %v, want true", sm.trk.Progress[2].MsgAppFlowPaused)
112+
if !sm.trk.Progress[2].MsgAppProbesPaused {
113+
t.Errorf("MsgAppProbesPaused = %v, want true", sm.trk.Progress[2].MsgAppProbesPaused)
114114
}
115115
}
116116

‎pkg/raft/raft_test.go

+8-8
Original file line numberDiff line numberDiff line change
@@ -125,16 +125,16 @@ func TestProgressResumeByHeartbeatResp(t *testing.T) {
125125
r.becomeCandidate()
126126
r.becomeLeader()
127127

128-
r.trk.Progress[2].MsgAppFlowPaused = true
128+
r.trk.Progress[2].MsgAppProbesPaused = true
129129

130130
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
131-
assert.True(t, r.trk.Progress[2].MsgAppFlowPaused)
131+
assert.True(t, r.trk.Progress[2].MsgAppProbesPaused)
132132

133133
r.trk.Progress[2].BecomeReplicate()
134-
assert.False(t, r.trk.Progress[2].MsgAppFlowPaused)
135-
r.trk.Progress[2].MsgAppFlowPaused = true
134+
assert.False(t, r.trk.Progress[2].MsgAppProbesPaused)
135+
r.trk.Progress[2].MsgAppProbesPaused = true
136136
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
137-
assert.True(t, r.trk.Progress[2].MsgAppFlowPaused)
137+
assert.True(t, r.trk.Progress[2].MsgAppProbesPaused)
138138
}
139139

140140
func TestProgressPaused(t *testing.T) {
@@ -2076,7 +2076,7 @@ func TestSendAppendForProgressProbe(t *testing.T) {
20762076
assert.Zero(t, msg[0].Index)
20772077
}
20782078

2079-
assert.True(t, r.trk.Progress[2].MsgAppFlowPaused)
2079+
assert.True(t, r.trk.Progress[2].MsgAppProbesPaused)
20802080
for j := 0; j < 10; j++ {
20812081
mustAppendEntry(r, pb.Entry{Data: []byte("somedata")})
20822082
r.maybeSendAppend(2)
@@ -2087,7 +2087,7 @@ func TestSendAppendForProgressProbe(t *testing.T) {
20872087
for j := 0; j < r.heartbeatTimeout; j++ {
20882088
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgBeat})
20892089
}
2090-
assert.True(t, r.trk.Progress[2].MsgAppFlowPaused)
2090+
assert.True(t, r.trk.Progress[2].MsgAppProbesPaused)
20912091

20922092
// consume the heartbeat
20932093
msg := r.readMessages()
@@ -2100,7 +2100,7 @@ func TestSendAppendForProgressProbe(t *testing.T) {
21002100
msg := r.readMessages()
21012101
assert.Len(t, msg, 1)
21022102
assert.Zero(t, msg[0].Index)
2103-
assert.True(t, r.trk.Progress[2].MsgAppFlowPaused)
2103+
assert.True(t, r.trk.Progress[2].MsgAppProbesPaused)
21042104
}
21052105

21062106
func TestSendAppendForProgressReplicate(t *testing.T) {

‎pkg/raft/tracker/progress.go

+11-11
Original file line numberDiff line numberDiff line change
@@ -96,13 +96,13 @@ type Progress struct {
9696
// This is always true on the leader.
9797
RecentActive bool
9898

99-
// MsgAppFlowPaused is used when the MsgApp flow to a node is throttled. This
99+
// MsgAppProbesPaused is used when the MsgApp flow to a node is throttled. This
100100
// happens in StateProbe, or StateReplicate with saturated Inflights. In both
101101
// cases, we need to continue sending MsgApp once in a while to guarantee
102-
// progress, but we only do so when MsgAppFlowPaused is false (it is reset on
102+
// progress, but we only do so when MsgAppProbesPaused is false (it is reset on
103103
// receiving a heartbeat response), to not overflow the receiver. See
104-
// IsPaused().
105-
MsgAppFlowPaused bool
104+
// IsPaused() and ShouldSendMsgApp().
105+
MsgAppProbesPaused bool
106106

107107
// Inflights is a sliding window for the inflight messages.
108108
// Each inflight message contains one or more log entries.
@@ -122,7 +122,7 @@ type Progress struct {
122122
IsLearner bool
123123
}
124124

125-
// ResetState moves the Progress into the specified State, resetting MsgAppFlowPaused,
125+
// ResetState moves the Progress into the specified State, resetting MsgAppProbesPaused,
126126
// PendingSnapshot, and Inflights.
127127
func (pr *Progress) ResetState(state StateType) {
128128
pr.PauseMsgAppProbes(false)
@@ -179,7 +179,7 @@ func (pr *Progress) SentEntries(entries int, bytes uint64) {
179179
// PauseMsgAppProbes pauses or unpauses empty MsgApp messages flow, depending on
180180
// the passed-in bool.
181181
func (pr *Progress) PauseMsgAppProbes(pause bool) {
182-
pr.MsgAppFlowPaused = pause
182+
pr.MsgAppProbesPaused = pause
183183
}
184184

185185
// CanSendEntries returns true if the flow control state allows sending at least
@@ -267,9 +267,9 @@ func (pr *Progress) MaybeDecrTo(rejected, matchHint uint64) bool {
267267
func (pr *Progress) IsPaused() bool {
268268
switch pr.State {
269269
case StateProbe:
270-
return pr.MsgAppFlowPaused
270+
return pr.MsgAppProbesPaused
271271
case StateReplicate:
272-
return pr.MsgAppFlowPaused && pr.Inflights.Full()
272+
return pr.MsgAppProbesPaused && pr.Inflights.Full()
273273
case StateSnapshot:
274274
return true
275275
default:
@@ -299,7 +299,7 @@ func (pr *Progress) IsPaused() bool {
299299
func (pr *Progress) ShouldSendMsgApp(last, commit uint64) bool {
300300
switch pr.State {
301301
case StateProbe:
302-
return !pr.MsgAppFlowPaused
302+
return !pr.MsgAppProbesPaused
303303
case StateReplicate:
304304
// Send a MsgApp containing the latest commit index if:
305305
// - our commit index exceeds the in-flight commit index, and
@@ -312,14 +312,14 @@ func (pr *Progress) ShouldSendMsgApp(last, commit uint64) bool {
312312
return false
313313
}
314314
// Don't send a MsgApp if we are in a throttled replication state, i.e.
315-
// pr.Inflights.Full() && pr.MsgAppFlowPaused.
315+
// pr.Inflights.Full() && pr.MsgAppProbesPaused.
316316
if pr.IsPaused() {
317317
return false
318318
}
319319
// We are here if the follower's log is not up-to-date, and the flow is not
320320
// paused We can always send a MsgApp, except when everything is already
321321
// in-flight, and the last MsgApp was recent.
322-
return pr.Next <= last || !pr.MsgAppFlowPaused
322+
return pr.Next <= last || !pr.MsgAppProbesPaused
323323

324324
case StateSnapshot:
325325
return false

‎pkg/raft/tracker/progress_test.go

+17-17
Original file line numberDiff line numberDiff line change
@@ -24,14 +24,14 @@ func TestProgressString(t *testing.T) {
2424
ins := NewInflights(1, 0)
2525
ins.Add(123, 1)
2626
pr := &Progress{
27-
Match: 1,
28-
Next: 2,
29-
State: StateSnapshot,
30-
PendingSnapshot: 123,
31-
RecentActive: false,
32-
MsgAppFlowPaused: true,
33-
IsLearner: true,
34-
Inflights: ins,
27+
Match: 1,
28+
Next: 2,
29+
State: StateSnapshot,
30+
PendingSnapshot: 123,
31+
RecentActive: false,
32+
MsgAppProbesPaused: true,
33+
IsLearner: true,
34+
Inflights: ins,
3535
}
3636
const exp = `StateSnapshot match=1 next=2 learner paused pendingSnap=123 inactive inflight=1[full]`
3737
assert.Equal(t, exp, pr.String())
@@ -53,29 +53,29 @@ func TestProgressIsPaused(t *testing.T) {
5353
}
5454
for i, tt := range tests {
5555
p := &Progress{
56-
State: tt.state,
57-
MsgAppFlowPaused: tt.paused,
58-
Inflights: NewInflights(256, 0),
56+
State: tt.state,
57+
MsgAppProbesPaused: tt.paused,
58+
Inflights: NewInflights(256, 0),
5959
}
6060
assert.Equal(t, tt.w, p.IsPaused(), i)
6161
}
6262
}
6363

64-
// TestProgressResume ensures that MaybeDecrTo resets MsgAppFlowPaused, and
64+
// TestProgressResume ensures that MaybeDecrTo resets MsgAppProbesPaused, and
6565
// MaybeUpdate does not.
6666
//
6767
// TODO(pav-kv): there is little sense in testing these micro-behaviours in the
6868
// struct. We should test the visible behaviour instead.
6969
func TestProgressResume(t *testing.T) {
7070
p := &Progress{
71-
Next: 2,
72-
MsgAppFlowPaused: true,
71+
Next: 2,
72+
MsgAppProbesPaused: true,
7373
}
7474
p.MaybeDecrTo(1, 1)
75-
assert.False(t, p.MsgAppFlowPaused)
76-
p.MsgAppFlowPaused = true
75+
assert.False(t, p.MsgAppProbesPaused)
76+
p.MsgAppProbesPaused = true
7777
p.MaybeUpdate(2)
78-
assert.True(t, p.MsgAppFlowPaused)
78+
assert.True(t, p.MsgAppProbesPaused)
7979
}
8080

8181
func TestProgressBecomeProbe(t *testing.T) {

0 commit comments

Comments
 (0)
Please sign in to comment.