Skip to content

Commit

Permalink
Merge #34502
Browse files Browse the repository at this point in the history
34502: storage: ignore non-live probing follower during log truncation r=petermattis a=tbg

Need to write tests and such, but I wanted to at least post this
before the weekend.

----

In the previous code, a follower in probing status which was not
recently active (i.e. a dead node) would permanently suppress
log truncations unless the Raft log was above threshold size (but the
size tracks only what the current leaseholder has written, i.e., it
can undercount dramatically). As a result, snapshots to other nodes
would get blocked if the log was in fact large (>16mb), leading to
ranges which effectively couldn't change their set of members.

Release note (bug fix): Prevent a problem that would cause the Raft log
to grow very large which in turn could prevent replication changes.

Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
  • Loading branch information
craig[bot] and tbg committed Feb 8, 2019
2 parents 70cd045 + 328fadc commit 0173eaf
Show file tree
Hide file tree
Showing 2 changed files with 51 additions and 18 deletions.
22 changes: 17 additions & 5 deletions pkg/storage/raft_log_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -377,6 +377,13 @@ func computeTruncateDecision(input truncateDecisionInput) truncateDecision {
decision.ChosenVia = truncatableIndexChosenViaQuorumIndex

for _, progress := range input.RaftStatus.Progress {
if !progress.RecentActive {
// If a follower isn't recently active, don't lower the truncation
// index for it as the follower is likely not online at all and would
// block log truncation forever.
continue
}

// Generally we truncate to the quorum commit index when the log becomes
// too large, but we make an exception for live followers which are
// being probed (i.e. the leader doesn't know how far they've caught
Expand All @@ -396,10 +403,11 @@ func computeTruncateDecision(input truncateDecisionInput) truncateDecision {
// ranges will be split many times over, resulting in a flurry of
// snapshots with overlapping bounds that put significant stress on the
// Raft snapshot queue.
probing := (progress.RecentActive && progress.State == raft.ProgressStateProbe)
if probing && decision.NewFirstIndex > decision.Input.FirstIndex {
decision.NewFirstIndex = decision.Input.FirstIndex
decision.ChosenVia = truncatableIndexChosenViaProbingFollower
if progress.State == raft.ProgressStateProbe {
if decision.NewFirstIndex > decision.Input.FirstIndex {
decision.NewFirstIndex = decision.Input.FirstIndex
decision.ChosenVia = truncatableIndexChosenViaProbingFollower
}
} else if !input.LogTooLarge() && decision.NewFirstIndex > progress.Match {
decision.NewFirstIndex = progress.Match
decision.ChosenVia = truncatableIndexChosenViaFollowers
Expand Down Expand Up @@ -454,7 +462,11 @@ func computeTruncateDecision(input truncateDecisionInput) truncateDecision {
func getQuorumIndex(raftStatus *raft.Status) uint64 {
match := make([]uint64, 0, len(raftStatus.Progress))
for _, progress := range raftStatus.Progress {
match = append(match, progress.Match)
if progress.State == raft.ProgressStateReplicate {
match = append(match, progress.Match)
} else {
match = append(match, 0)
}
}
sort.Sort(uint64Slice(match))
quorum := computeQuorum(len(match))
Expand Down
47 changes: 34 additions & 13 deletions pkg/storage/raft_log_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,13 +92,24 @@ func TestGetQuorumIndex(t *testing.T) {
Progress: make(map[uint64]raft.Progress),
}
for j, v := range c.progress {
status.Progress[uint64(j)] = raft.Progress{Match: v}
status.Progress[uint64(j)] = raft.Progress{State: raft.ProgressStateReplicate, Match: v}
}
quorumMatchedIndex := getQuorumIndex(status)
if c.expected != quorumMatchedIndex {
t.Fatalf("%d: expected %d, but got %d", i, c.expected, quorumMatchedIndex)
}
}

// Verify that only replicating followers are taken into account (i.e. others
// are treated as Match == 0).
status := &raft.Status{
Progress: map[uint64]raft.Progress{
1: {State: raft.ProgressStateReplicate, Match: 100},
2: {State: raft.ProgressStateSnapshot, Match: 100},
3: {State: raft.ProgressStateReplicate, Match: 90},
},
}
assert.Equal(t, uint64(90), getQuorumIndex(status))
}

func TestComputeTruncateDecision(t *testing.T) {
Expand Down Expand Up @@ -205,7 +216,7 @@ func TestComputeTruncateDecision(t *testing.T) {
Progress: make(map[uint64]raft.Progress),
}
for j, v := range c.progress {
status.Progress[uint64(j)] = raft.Progress{State: raft.ProgressStateReplicate, Match: v, Next: v + 1}
status.Progress[uint64(j)] = raft.Progress{RecentActive: true, State: raft.ProgressStateReplicate, Match: v, Next: v + 1}
}
decision := computeTruncateDecision(truncateDecisionInput{
RaftStatus: status,
Expand Down Expand Up @@ -233,11 +244,11 @@ func TestComputeTruncateDecisionProgressStatusProbe(t *testing.T) {
exp := map[bool]map[bool]string{ // (tooLarge, active)
false: {
true: "should truncate: false [truncate 0 entries to first index 10 (chosen via: probing follower)]",
false: "should truncate: false [truncate 90 entries to first index 100 (chosen via: followers)]",
false: "should truncate: true [truncate 190 entries to first index 200 (chosen via: followers)]",
},
true: {
true: "should truncate: false [truncate 0 entries to first index 10 (chosen via: probing follower); log too large (2.0 KiB > 1.0 KiB)]",
false: "should truncate: true [truncate 290 entries to first index 300 (chosen via: quorum); log too large (2.0 KiB > 1.0 KiB); implies 2 Raft snapshots]",
false: "should truncate: true [truncate 290 entries to first index 300 (chosen via: quorum); log too large (2.0 KiB > 1.0 KiB); implies 1 Raft snapshot]",
},
}

Expand All @@ -246,15 +257,25 @@ func TestComputeTruncateDecisionProgressStatusProbe(t *testing.T) {
status := raft.Status{
Progress: make(map[uint64]raft.Progress),
}
for j, v := range []uint64{500, 400, 300, 200, 100} {
pr := raft.Progress{
Match: v,
RecentActive: true,
State: raft.ProgressStateReplicate,
}
if v == 300 {
pr.RecentActive = active
pr.State = raft.ProgressStateProbe
for j, v := range []uint64{100, 200, 300, 400, 500} {
var pr raft.Progress
if v == 100 {
// A probing follower is probed with some index (Next) but
// it has a zero Match (i.e. no idea how much of its log
// agrees with ours).
pr = raft.Progress{
RecentActive: active,
State: raft.ProgressStateProbe,
Match: 0,
Next: v,
}
} else { // everyone else
pr = raft.Progress{
Match: v,
Next: v + 1,
RecentActive: true,
State: raft.ProgressStateReplicate,
}
}
status.Progress[uint64(j)] = pr
}
Expand Down

0 comments on commit 0173eaf

Please sign in to comment.