blob: a2638f5f01e523cb8ccebde71ac083b7bf64d49b [file] [log] [blame]
Abhilash S.L3b494632019-07-16 15:51:09 +05301// 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"
Devmalya Paulfb990a52019-07-09 10:01:49 -040020 "strings"
Abhilash S.L3b494632019-07-16 15:51:09 +053021
22 "go.etcd.io/etcd/raft/quorum"
23)
24
25// Config reflects the configuration tracked in a ProgressTracker.
26type Config struct {
27 Voters quorum.JointConfig
28 // Learners is a set of IDs corresponding to the learners active in the
29 // current configuration.
30 //
31 // Invariant: Learners and Voters does not intersect, i.e. if a peer is in
32 // either half of the joint config, it can't be a learner; if it is a
33 // learner it can't be in either half of the joint config. This invariant
34 // simplifies the implementation since it allows peers to have clarity about
35 // its current role without taking into account joint consensus.
36 Learners map[uint64]struct{}
Devmalya Paulfb990a52019-07-09 10:01:49 -040037 // When we turn a voter into a learner during a joint consensus transition,
38 // we cannot add the learner directly when entering the joint state. This is
39 // because this would violate the invariant that the intersection of
40 // voters and learners is empty. For example, assume a Voter is removed and
41 // immediately re-added as a learner (or in other words, it is demoted):
Abhilash S.L3b494632019-07-16 15:51:09 +053042 //
43 // Initially, the configuration will be
44 //
45 // voters: {1 2 3}
46 // learners: {}
47 //
48 // and we want to demote 3. Entering the joint configuration, we naively get
49 //
50 // voters: {1 2} & {1 2 3}
51 // learners: {3}
52 //
53 // but this violates the invariant (3 is both voter and learner). Instead,
Devmalya Paulfb990a52019-07-09 10:01:49 -040054 // we get
Abhilash S.L3b494632019-07-16 15:51:09 +053055 //
56 // voters: {1 2} & {1 2 3}
57 // learners: {}
58 // next_learners: {3}
59 //
60 // Where 3 is now still purely a voter, but we are remembering the intention
61 // to make it a learner upon transitioning into the final configuration:
62 //
63 // voters: {1 2}
64 // learners: {3}
65 // next_learners: {}
66 //
67 // Note that next_learners is not used while adding a learner that is not
68 // also a voter in the joint config. In this case, the learner is added
Devmalya Paulfb990a52019-07-09 10:01:49 -040069 // right away when entering the joint configuration, so that it is caught up
70 // as soon as possible.
71 LearnersNext map[uint64]struct{}
Abhilash S.L3b494632019-07-16 15:51:09 +053072}
73
Devmalya Paulfb990a52019-07-09 10:01:49 -040074func (c Config) String() string {
75 var buf strings.Builder
76 fmt.Fprintf(&buf, "voters=%s", c.Voters)
77 if c.Learners != nil {
78 fmt.Fprintf(&buf, " learners=%s", quorum.MajorityConfig(c.Learners).String())
Abhilash S.L3b494632019-07-16 15:51:09 +053079 }
Devmalya Paulfb990a52019-07-09 10:01:49 -040080 if c.LearnersNext != nil {
81 fmt.Fprintf(&buf, " learners_next=%s", quorum.MajorityConfig(c.LearnersNext).String())
82 }
83 return buf.String()
84}
85
86// Clone returns a copy of the Config that shares no memory with the original.
87func (c *Config) Clone() Config {
88 clone := func(m map[uint64]struct{}) map[uint64]struct{} {
89 if m == nil {
90 return nil
91 }
92 mm := make(map[uint64]struct{}, len(m))
93 for k := range m {
94 mm[k] = struct{}{}
95 }
96 return mm
97 }
98 return Config{
99 Voters: quorum.JointConfig{clone(c.Voters[0]), clone(c.Voters[1])},
100 Learners: clone(c.Learners),
101 LearnersNext: clone(c.LearnersNext),
102 }
Abhilash S.L3b494632019-07-16 15:51:09 +0530103}
104
105// ProgressTracker tracks the currently active configuration and the information
106// known about the nodes and learners in it. In particular, it tracks the match
107// index for each peer which in turn allows reasoning about the committed index.
108type ProgressTracker struct {
109 Config
110
Devmalya Paulfb990a52019-07-09 10:01:49 -0400111 Progress ProgressMap
Abhilash S.L3b494632019-07-16 15:51:09 +0530112
113 Votes map[uint64]bool
114
115 MaxInflight int
116}
117
118// MakeProgressTracker initializes a ProgressTracker.
119func MakeProgressTracker(maxInflight int) ProgressTracker {
120 p := ProgressTracker{
121 MaxInflight: maxInflight,
122 Config: Config{
123 Voters: quorum.JointConfig{
124 quorum.MajorityConfig{},
Devmalya Paulfb990a52019-07-09 10:01:49 -0400125 nil, // only populated when used
Abhilash S.L3b494632019-07-16 15:51:09 +0530126 },
Devmalya Paulfb990a52019-07-09 10:01:49 -0400127 Learners: nil, // only populated when used
128 LearnersNext: nil, // only populated when used
Abhilash S.L3b494632019-07-16 15:51:09 +0530129 },
130 Votes: map[uint64]bool{},
131 Progress: map[uint64]*Progress{},
132 }
133 return p
134}
135
136// IsSingleton returns true if (and only if) there is only one voting member
137// (i.e. the leader) in the current configuration.
138func (p *ProgressTracker) IsSingleton() bool {
139 return len(p.Voters[0]) == 1 && len(p.Voters[1]) == 0
140}
141
142type matchAckIndexer map[uint64]*Progress
143
144var _ quorum.AckedIndexer = matchAckIndexer(nil)
145
146// AckedIndex implements IndexLookuper.
147func (l matchAckIndexer) AckedIndex(id uint64) (quorum.Index, bool) {
148 pr, ok := l[id]
149 if !ok {
150 return 0, false
151 }
152 return quorum.Index(pr.Match), true
153}
154
155// Committed returns the largest log index known to be committed based on what
156// the voting members of the group have acknowledged.
157func (p *ProgressTracker) Committed() uint64 {
158 return uint64(p.Voters.CommittedIndex(matchAckIndexer(p.Progress)))
159}
160
Abhilash S.L3b494632019-07-16 15:51:09 +0530161// Visit invokes the supplied closure for all tracked progresses.
162func (p *ProgressTracker) Visit(f func(id uint64, pr *Progress)) {
163 for id, pr := range p.Progress {
164 f(id, pr)
165 }
166}
167
168// QuorumActive returns true if the quorum is active from the view of the local
169// raft state machine. Otherwise, it returns false.
170func (p *ProgressTracker) QuorumActive() bool {
171 votes := map[uint64]bool{}
172 p.Visit(func(id uint64, pr *Progress) {
173 if pr.IsLearner {
174 return
175 }
176 votes[id] = pr.RecentActive
177 })
178
179 return p.Voters.VoteResult(votes) == quorum.VoteWon
180}
181
182// VoterNodes returns a sorted slice of voters.
183func (p *ProgressTracker) VoterNodes() []uint64 {
184 m := p.Voters.IDs()
185 nodes := make([]uint64, 0, len(m))
186 for id := range m {
187 nodes = append(nodes, id)
188 }
189 sort.Slice(nodes, func(i, j int) bool { return nodes[i] < nodes[j] })
190 return nodes
191}
192
193// LearnerNodes returns a sorted slice of learners.
194func (p *ProgressTracker) LearnerNodes() []uint64 {
195 nodes := make([]uint64, 0, len(p.Learners))
196 for id := range p.Learners {
197 nodes = append(nodes, id)
198 }
199 sort.Slice(nodes, func(i, j int) bool { return nodes[i] < nodes[j] })
200 return nodes
201}
202
203// ResetVotes prepares for a new round of vote counting via recordVote.
204func (p *ProgressTracker) ResetVotes() {
205 p.Votes = map[uint64]bool{}
206}
207
208// RecordVote records that the node with the given id voted for this Raft
209// instance if v == true (and declined it otherwise).
210func (p *ProgressTracker) RecordVote(id uint64, v bool) {
211 _, ok := p.Votes[id]
212 if !ok {
213 p.Votes[id] = v
214 }
215}
216
217// TallyVotes returns the number of granted and rejected Votes, and whether the
218// election outcome is known.
219func (p *ProgressTracker) TallyVotes() (granted int, rejected int, _ quorum.VoteResult) {
220 // Make sure to populate granted/rejected correctly even if the Votes slice
221 // contains members no longer part of the configuration. This doesn't really
222 // matter in the way the numbers are used (they're informational), but might
223 // as well get it right.
224 for id, pr := range p.Progress {
225 if pr.IsLearner {
226 continue
227 }
228 v, voted := p.Votes[id]
229 if !voted {
230 continue
231 }
232 if v {
233 granted++
234 } else {
235 rejected++
236 }
237 }
238 result := p.Voters.VoteResult(p.Votes)
239 return granted, rejected, result
240}