-
Notifications
You must be signed in to change notification settings - Fork 169
/
raft_flow_control_test.go
138 lines (118 loc) · 4.32 KB
/
raft_flow_control_test.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
// Copyright 2015 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package raft
import (
"testing"
"github.com/stretchr/testify/require"
pb "go.etcd.io/raft/v3/raftpb"
)
// TestMsgAppFlowControlFull ensures:
// 1. msgApp can fill the sending window until full
// 2. when the window is full, no more msgApp can be sent.
func TestMsgAppFlowControlFull(t *testing.T) {
r := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2)))
r.becomeCandidate()
r.becomeLeader()
pr2 := r.trk.Progress[2]
// force the progress to be in replicate state
pr2.BecomeReplicate()
// fill in the inflights window
for i := 0; i < r.trk.MaxInflight; i++ {
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
ms := r.readMessages()
require.Len(t, ms, 1)
require.Equal(t, pb.MsgApp, ms[0].Type)
}
// ensure 1
require.True(t, pr2.IsPaused())
// ensure 2
for i := 0; i < 10; i++ {
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
ms := r.readMessages()
require.Empty(t, ms)
}
}
// TestMsgAppFlowControlMoveForward ensures msgAppResp can move
// forward the sending window correctly:
// 1. valid msgAppResp.index moves the windows to pass all smaller or equal index.
// 2. out-of-dated msgAppResp has no effect on the sliding window.
func TestMsgAppFlowControlMoveForward(t *testing.T) {
r := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2)))
r.becomeCandidate()
r.becomeLeader()
pr2 := r.trk.Progress[2]
// force the progress to be in replicate state
pr2.BecomeReplicate()
// fill in the inflights window
for i := 0; i < r.trk.MaxInflight; i++ {
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
r.readMessages()
}
// 1 is noop, 2 is the first proposal we just sent.
// so we start with 2.
for tt := 2; tt < r.trk.MaxInflight; tt++ {
// move forward the window
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: uint64(tt)})
r.readMessages()
// fill in the inflights window again
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
ms := r.readMessages()
require.Len(t, ms, 1)
require.Equal(t, pb.MsgApp, ms[0].Type)
// ensure 1
require.True(t, pr2.IsPaused())
// ensure 2
for i := 0; i < tt; i++ {
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgAppResp, Index: uint64(i)})
require.True(t, pr2.IsPaused())
}
}
}
// TestMsgAppFlowControlRecvHeartbeat ensures a heartbeat response
// frees one slot if the window is full.
func TestMsgAppFlowControlRecvHeartbeat(t *testing.T) {
r := newTestRaft(1, 5, 1, newTestMemoryStorage(withPeers(1, 2)))
r.becomeCandidate()
r.becomeLeader()
pr2 := r.trk.Progress[2]
// force the progress to be in replicate state
pr2.BecomeReplicate()
// fill in the inflights window
for i := 0; i < r.trk.MaxInflight; i++ {
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
r.readMessages()
}
for tt := 1; tt < 5; tt++ {
// recv tt msgHeartbeatResp and expect one free slot
for i := 0; i < tt; i++ {
require.True(t, pr2.IsPaused())
// Unpauses the progress, sends an empty MsgApp, and pauses it again.
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
ms := r.readMessages()
require.Len(t, ms, 1)
require.Equal(t, pb.MsgApp, ms[0].Type)
require.Empty(t, ms[0].Entries)
}
// No more appends are sent if there are no heartbeats.
for i := 0; i < 10; i++ {
require.True(t, pr2.IsPaused())
r.Step(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("somedata")}}})
ms := r.readMessages()
require.Empty(t, ms)
}
// clear all pending messages.
r.Step(pb.Message{From: 2, To: 1, Type: pb.MsgHeartbeatResp})
r.readMessages()
}
}