blob: a4581143d1e929a86574f5050649141958822fb1 [file] [log] [blame]
Takahiro Suzukid7bf8202020-12-17 20:21:59 +09001// Copyright 2019 The etcd Authors
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7// http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15package tracker
16
17import (
18 "fmt"
19 "sort"
20 "strings"
21
22 "go.etcd.io/etcd/raft/quorum"
23 pb "go.etcd.io/etcd/raft/raftpb"
24)
25
26// Config reflects the configuration tracked in a ProgressTracker.
27type Config struct {
28 Voters quorum.JointConfig
29 // AutoLeave is true if the configuration is joint and a transition to the
30 // incoming configuration should be carried out automatically by Raft when
31 // this is possible. If false, the configuration will be joint until the
32 // application initiates the transition manually.
33 AutoLeave bool
34 // Learners is a set of IDs corresponding to the learners active in the
35 // current configuration.
36 //
37 // Invariant: Learners and Voters does not intersect, i.e. if a peer is in
38 // either half of the joint config, it can't be a learner; if it is a
39 // learner it can't be in either half of the joint config. This invariant
40 // simplifies the implementation since it allows peers to have clarity about
41 // its current role without taking into account joint consensus.
42 Learners map[uint64]struct{}
43 // When we turn a voter into a learner during a joint consensus transition,
44 // we cannot add the learner directly when entering the joint state. This is
45 // because this would violate the invariant that the intersection of
46 // voters and learners is empty. For example, assume a Voter is removed and
47 // immediately re-added as a learner (or in other words, it is demoted):
48 //
49 // Initially, the configuration will be
50 //
51 // voters: {1 2 3}
52 // learners: {}
53 //
54 // and we want to demote 3. Entering the joint configuration, we naively get
55 //
56 // voters: {1 2} & {1 2 3}
57 // learners: {3}
58 //
59 // but this violates the invariant (3 is both voter and learner). Instead,
60 // we get
61 //
62 // voters: {1 2} & {1 2 3}
63 // learners: {}
64 // next_learners: {3}
65 //
66 // Where 3 is now still purely a voter, but we are remembering the intention
67 // to make it a learner upon transitioning into the final configuration:
68 //
69 // voters: {1 2}
70 // learners: {3}
71 // next_learners: {}
72 //
73 // Note that next_learners is not used while adding a learner that is not
74 // also a voter in the joint config. In this case, the learner is added
75 // right away when entering the joint configuration, so that it is caught up
76 // as soon as possible.
77 LearnersNext map[uint64]struct{}
78}
79
80func (c Config) String() string {
81 var buf strings.Builder
82 fmt.Fprintf(&buf, "voters=%s", c.Voters)
83 if c.Learners != nil {
84 fmt.Fprintf(&buf, " learners=%s", quorum.MajorityConfig(c.Learners).String())
85 }
86 if c.LearnersNext != nil {
87 fmt.Fprintf(&buf, " learners_next=%s", quorum.MajorityConfig(c.LearnersNext).String())
88 }
89 if c.AutoLeave {
90 fmt.Fprintf(&buf, " autoleave")
91 }
92 return buf.String()
93}
94
95// Clone returns a copy of the Config that shares no memory with the original.
96func (c *Config) Clone() Config {
97 clone := func(m map[uint64]struct{}) map[uint64]struct{} {
98 if m == nil {
99 return nil
100 }
101 mm := make(map[uint64]struct{}, len(m))
102 for k := range m {
103 mm[k] = struct{}{}
104 }
105 return mm
106 }
107 return Config{
108 Voters: quorum.JointConfig{clone(c.Voters[0]), clone(c.Voters[1])},
109 Learners: clone(c.Learners),
110 LearnersNext: clone(c.LearnersNext),
111 }
112}
113
114// ProgressTracker tracks the currently active configuration and the information
115// known about the nodes and learners in it. In particular, it tracks the match
116// index for each peer which in turn allows reasoning about the committed index.
117type ProgressTracker struct {
118 Config
119
120 Progress ProgressMap
121
122 Votes map[uint64]bool
123
124 MaxInflight int
125}
126
127// MakeProgressTracker initializes a ProgressTracker.
128func MakeProgressTracker(maxInflight int) ProgressTracker {
129 p := ProgressTracker{
130 MaxInflight: maxInflight,
131 Config: Config{
132 Voters: quorum.JointConfig{
133 quorum.MajorityConfig{},
134 nil, // only populated when used
135 },
136 Learners: nil, // only populated when used
137 LearnersNext: nil, // only populated when used
138 },
139 Votes: map[uint64]bool{},
140 Progress: map[uint64]*Progress{},
141 }
142 return p
143}
144
145// ConfState returns a ConfState representing the active configuration.
146func (p *ProgressTracker) ConfState() pb.ConfState {
147 return pb.ConfState{
148 Voters: p.Voters[0].Slice(),
149 VotersOutgoing: p.Voters[1].Slice(),
150 Learners: quorum.MajorityConfig(p.Learners).Slice(),
151 LearnersNext: quorum.MajorityConfig(p.LearnersNext).Slice(),
152 AutoLeave: p.AutoLeave,
153 }
154}
155
156// IsSingleton returns true if (and only if) there is only one voting member
157// (i.e. the leader) in the current configuration.
158func (p *ProgressTracker) IsSingleton() bool {
159 return len(p.Voters[0]) == 1 && len(p.Voters[1]) == 0
160}
161
162type matchAckIndexer map[uint64]*Progress
163
164var _ quorum.AckedIndexer = matchAckIndexer(nil)
165
166// AckedIndex implements IndexLookuper.
167func (l matchAckIndexer) AckedIndex(id uint64) (quorum.Index, bool) {
168 pr, ok := l[id]
169 if !ok {
170 return 0, false
171 }
172 return quorum.Index(pr.Match), true
173}
174
175// Committed returns the largest log index known to be committed based on what
176// the voting members of the group have acknowledged.
177func (p *ProgressTracker) Committed() uint64 {
178 return uint64(p.Voters.CommittedIndex(matchAckIndexer(p.Progress)))
179}
180
181func insertionSort(sl []uint64) {
182 a, b := 0, len(sl)
183 for i := a + 1; i < b; i++ {
184 for j := i; j > a && sl[j] < sl[j-1]; j-- {
185 sl[j], sl[j-1] = sl[j-1], sl[j]
186 }
187 }
188}
189
190// Visit invokes the supplied closure for all tracked progresses in stable order.
191func (p *ProgressTracker) Visit(f func(id uint64, pr *Progress)) {
192 n := len(p.Progress)
193 // We need to sort the IDs and don't want to allocate since this is hot code.
194 // The optimization here mirrors that in `(MajorityConfig).CommittedIndex`,
195 // see there for details.
196 var sl [7]uint64
197 ids := sl[:]
198 if len(sl) >= n {
199 ids = sl[:n]
200 } else {
201 ids = make([]uint64, n)
202 }
203 for id := range p.Progress {
204 n--
205 ids[n] = id
206 }
207 insertionSort(ids)
208 for _, id := range ids {
209 f(id, p.Progress[id])
210 }
211}
212
213// QuorumActive returns true if the quorum is active from the view of the local
214// raft state machine. Otherwise, it returns false.
215func (p *ProgressTracker) QuorumActive() bool {
216 votes := map[uint64]bool{}
217 p.Visit(func(id uint64, pr *Progress) {
218 if pr.IsLearner {
219 return
220 }
221 votes[id] = pr.RecentActive
222 })
223
224 return p.Voters.VoteResult(votes) == quorum.VoteWon
225}
226
227// VoterNodes returns a sorted slice of voters.
228func (p *ProgressTracker) VoterNodes() []uint64 {
229 m := p.Voters.IDs()
230 nodes := make([]uint64, 0, len(m))
231 for id := range m {
232 nodes = append(nodes, id)
233 }
234 sort.Slice(nodes, func(i, j int) bool { return nodes[i] < nodes[j] })
235 return nodes
236}
237
238// LearnerNodes returns a sorted slice of learners.
239func (p *ProgressTracker) LearnerNodes() []uint64 {
240 if len(p.Learners) == 0 {
241 return nil
242 }
243 nodes := make([]uint64, 0, len(p.Learners))
244 for id := range p.Learners {
245 nodes = append(nodes, id)
246 }
247 sort.Slice(nodes, func(i, j int) bool { return nodes[i] < nodes[j] })
248 return nodes
249}
250
251// ResetVotes prepares for a new round of vote counting via recordVote.
252func (p *ProgressTracker) ResetVotes() {
253 p.Votes = map[uint64]bool{}
254}
255
256// RecordVote records that the node with the given id voted for this Raft
257// instance if v == true (and declined it otherwise).
258func (p *ProgressTracker) RecordVote(id uint64, v bool) {
259 _, ok := p.Votes[id]
260 if !ok {
261 p.Votes[id] = v
262 }
263}
264
265// TallyVotes returns the number of granted and rejected Votes, and whether the
266// election outcome is known.
267func (p *ProgressTracker) TallyVotes() (granted int, rejected int, _ quorum.VoteResult) {
268 // Make sure to populate granted/rejected correctly even if the Votes slice
269 // contains members no longer part of the configuration. This doesn't really
270 // matter in the way the numbers are used (they're informational), but might
271 // as well get it right.
272 for id, pr := range p.Progress {
273 if pr.IsLearner {
274 continue
275 }
276 v, voted := p.Votes[id]
277 if !voted {
278 continue
279 }
280 if v {
281 granted++
282 } else {
283 rejected++
284 }
285 }
286 result := p.Voters.VoteResult(p.Votes)
287 return granted, rejected, result
288}