-
Notifications
You must be signed in to change notification settings - Fork 3.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
storage: prevent unbounded raft log growth without quorum #27774
storage: prevent unbounded raft log growth without quorum #27774
Conversation
Extra logging in |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice. Now to figure out a good fix.
Reviewable status: complete! 0 of 0 LGTMs obtained
c1a0dca
to
184f08f
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Well gosh. That sure is a simple test.
Reviewed 1 of 1 files at r2.
Reviewable status: complete! 0 of 0 LGTMs obtained
pkg/storage/client_raft_test.go, line 1168 at r2 (raw file):
putRes := make(chan *roachpb.Error) go func() { putArgs := putArgs([]byte("b"), make([]byte, 8<<10))
/* 8KiB */
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's get either your solution in #27772 or Ben's proposed solution (don't refresh proposals on the leader for reasonTicks
) into this PR. I have a preference for Ben's approach because we don't have to check raft.Status
on every proposal.
Reviewable status: complete! 0 of 0 LGTMs obtained
pkg/storage/client_raft_test.go, line 1174 at r2 (raw file):
// Wait for a bit and watch for Raft log growth. wait := time.After(30 * time.Second)
This makes the growth obvious, but does the test need to run this long to ensure that growth is not happening? I would think with a small enough raft tick interval we can run the test for a second or two and see unexpected Raft log growth.
Yeah, I didn't mean to mark this PR as "approved" but apparently Reviewable has a mind of its own these days. |
Is that guaranteed to work fine? We could have put a proposal in when we weren't (Raft) leader, but then we get leadership and will never repropose them even though they're not going to pop out of the log. Also, I agree that it'd be nice not to have to call Also, why don't we have that problem when commands are stuck on the follower? Wouldn't they still send a forwarded proposal to the leader every time, where it would get stuck in the log? This would be rare with 3x replication (if you have a follower you're committing anyway) but could happen with higher replication factors. It doesn't solve the problem completely, but #27054 could help by making the proposals that get put into the Raft log repeatedly very small |
I think the idea is that when getting leadership we would repropose the commands with a
Yes, that's a good point and a serious issue with the approach I proposed in #27772 (comment).
Agreed, this would all still break down if we were a follower in 5x replication group where all other followers went down but the leader stayed up. We would just keep forwarding the proposal to the leader and keep appending to its log. #27054 does sound like it would help improve the general situation with @tschottdorf I'm curious what your take on @bdarnell's mention of #21849 is. Finding all the scenerios where |
You're right. Doing anything based on leadership status only works for 3x replication. And my other proposal, to skip reasonTicks when we don't have a quorum of known-live nodes, is likely subject to problems when a reproposal is needed to unblock the liveness range itself. Handling ErrProposalDropped and taking control of proposal forwarding definitely looks like the right long-term fix, but we couldn't backport it to 2.0. How about this: Leaders skip reasonTicks reproposals, and drop incoming MsgProps if the uncommitted portion of the raft log is too large. |
I read that as "drop incoming MsgProps if the raft log is too large". Basing this off the size of the uncommitted portion of the raft log won't have any of the issues I raised. I think I'm coming around to the idea of looking at node liveness in 2.0 and fixing this properly in 2.1. We could ignore the node liveness check on the node liveness range itself if we are afraid of creating a dependency cycle between node liveness proposals and the node liveness state itself. |
The leader will continue to send MsgApps to followers when and if they are healthy, eventually committing the outstanding proposals and making room for new ones. This all happens at the raft level independently of the lease. A lease will be needed to truncate the raft log, but this should proceed normally once quorum is reestablished. |
184f08f
to
c8d6794
Compare
Ok, this is ready for a review. I went a slightly different route with the approach to preventing duplicate forwarded proposals (see PR description), but it's actually pretty similar to @bdarnell's final comment. Completely avoiding duplicate forwarded proposals by tracking them on the leader makes this easier to test and feels a little more safe to me. I don't see any real issues with the other approach though. |
f958580
to
7306d9f
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have to take another look at this tomorrow. One quick comment for tonight.
Reviewable status: complete! 0 of 0 LGTMs obtained
pkg/storage/replica.go, line 390 at r3 (raw file):
// The set allows leaders to detect duplicate forwarded commands and // avoid re-proposing the same forwarded command multiple times. proposalsForwarded map[storagebase.CmdIDKey]struct{}
proposalsForwarded
sounds like it would be maintained on the follower. I would have named this receivedProposals
or remoteProposals
. We could also rename the proposals
field to localProposals
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Release note: This prevents out-of-memory crashes, not "loss of quorum situations".
Reviewed 4 of 6 files at r3, 2 of 2 files at r4.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale)
pkg/storage/replica.go, line 3776 at r4 (raw file):
} case raftpb.EntryConfChange: allSeen = false
Comment that we could peek into the EntryConfChange to find the command ID, but we don't expect follower-initiated conf changes.
pkg/storage/replica.go, line 3896 at r4 (raw file):
// Clear the forwarded proposal set. No-op if not previously the leader. for cmdID := range r.mu.proposalsForwarded { delete(r.mu.proposalsForwarded, cmdID)
Reset the map to nil instead of deleting all the entries. We're unlikely to need the map again soon.
pkg/storage/replica_test.go, line 8132 at r4 (raw file):
// Enable so that pending commands are reproposed even though the replica // will become the Raft leader. cfg.RefreshPendingCommandsWhenLeader = true
I don't think it makes sense to add a new option so that this test can go on working like it was. This test is no longer verifying the behavior of reasonTicks refreshes as seen in production, so it should either be updated to match the new behavior (that is, by making its proposals through a follower so the reasonTicks reproposal is necessary) or be deleted.
7306d9f
to
1c4ea0c
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Release note: This prevents out-of-memory crashes, not "loss of quorum situations".
Hah, I wish. I think you missed the next line of the release notes. "Prevent loss of quorum situations from allowing unbounded growth of a Range's Raft log."
Handling ErrProposalDropped and taking control of proposal forwarding definitely looks like the right long-term fix, but we couldn't backport it to 2.0.
I'm actually not so sure about this anymore. Never refreshing with reasonTicks
on the leader is a solid improvement on its own and tracking successfully forwarded proposals gets us the other half of the way there. Correct me if I'm wrong, but I believe the primary difference between "handling ErrProposalDropped" and what we do here is that we're tracking whether forwarded proposals succesfully made it into the Raft log on the leader here instead of on the proposer/follower like we do in the other approach. In the other approach, we would still need to re-propose and re-forward whenever the leader changed so we would still need similar bookkeeping. I guess the main benefit of the other approach is avoiding the continuous forwarding (that now becomes a no-op) under a steady leader when nothing is being committed.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale)
pkg/storage/client_raft_test.go, line 1168 at r2 (raw file):
Previously, benesch (Nikhil Benesch) wrote…
/* 8KiB */
Done.
pkg/storage/client_raft_test.go, line 1174 at r2 (raw file):
Previously, petermattis (Peter Mattis) wrote…
This makes the growth obvious, but does the test need to run this long to ensure that growth is not happening? I would think with a small enough raft tick interval we can run the test for a second or two and see unexpected Raft log growth.
Done.
pkg/storage/replica.go, line 390 at r3 (raw file):
Previously, petermattis (Peter Mattis) wrote…
proposalsForwarded
sounds like it would be maintained on the follower. I would have named thisreceivedProposals
orremoteProposals
. We could also rename theproposals
field tolocalProposals
.
Done. Went with remoteProposals
and renamed proposals
to localProposals
.
pkg/storage/replica.go, line 3776 at r4 (raw file):
Previously, bdarnell (Ben Darnell) wrote…
Comment that we could peek into the EntryConfChange to find the command ID, but we don't expect follower-initiated conf changes.
Done.
pkg/storage/replica.go, line 3896 at r4 (raw file):
Previously, bdarnell (Ben Darnell) wrote…
Reset the map to nil instead of deleting all the entries. We're unlikely to need the map again soon.
Done.
pkg/storage/replica_test.go, line 8132 at r4 (raw file):
Previously, bdarnell (Ben Darnell) wrote…
I don't think it makes sense to add a new option so that this test can go on working like it was. This test is no longer verifying the behavior of reasonTicks refreshes as seen in production, so it should either be updated to match the new behavior (that is, by making its proposals through a follower so the reasonTicks reproposal is necessary) or be deleted.
Done.
508815e
to
99615e1
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hah, I wish. I think you missed the next line of the release notes. "Prevent loss of quorum situations from allowing unbounded growth of a Range's Raft log."
I saw that, I just think that OOM crashes better captures the user-observable behavior here. It doesn't necessarily lead to loss of quorum (if the other nodes decide to go on without the former leader that is filling up its log, the quorum is fine).
I guess the main benefit of the other approach is avoiding the continuous forwarding (that now becomes a no-op) under a steady leader when nothing is being committed.
Right. Changing the way proposals are forwarded would be a cleanup and allow us to get rid of reasonTicks
completely, but it wouldn't have any advantage in terms of raft log growth over what you're doing here.
Reviewed 2 of 5 files at r5, 3 of 3 files at r6.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale)
pkg/storage/client_raft_test.go, line 1151 at r6 (raw file):
// Refresh pending commands on every Raft group tick instead of // every RaftElectionTimeoutTicks. sc.TestingKnobs.RefreshReasonTicksPeriodicity = 3
Shouldn't this be 1 to match the comment? (or the comment should be updated)
pkg/storage/store.go, line 774 at r6 (raw file):
// periodically. DisableRefreshReasonTicks bool // RefreshReasonTicksPeriodicity overrides the default periodicity in which
s/icity//g (or s/periodicity/interval/g)
99615e1
to
cad5106
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I saw that, I just think that OOM crashes better captures the user-observable behavior here. It doesn't necessarily lead to loss of quorum (if the other nodes decide to go on without the former leader that is filling up its log, the quorum is fine).
I still don't think we're seeing eye to eye. I'm not saying that this was ever causing a loss of quorum, I'm saying that a loss of quorum would previously cause unbounded log growth and that this prevents this adverse effect. Maybe the release note is confusingly worded?
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale)
pkg/storage/client_raft_test.go, line 1151 at r6 (raw file):
Previously, bdarnell (Ben Darnell) wrote…
Shouldn't this be 1 to match the comment? (or the comment should be updated)
Done. Good catch, I was playing with this to make sure it was working as expected and forgot to bring it back down to 1.
pkg/storage/store.go, line 774 at r6 (raw file):
Previously, bdarnell (Ben Darnell) wrote…
s/icity//g (or s/periodicity/interval/g)
Done.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, I see. I parse that sentence as "(Prevent loss of quorum situations) from [caused by] allowing unbounded growth of a Range's Raft log" but you meant "Prevent (loss of quorum situations from allowing unbounded growth of a Range's Raft lo)". I think it would be clearer as "Prevent unbounded growth of the raft log caused by a loss of quorum".
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale)
Fixes cockroachdb#27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in cockroachdb#27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent unbounded growth of the raft log caused by a loss of quorum.
Release note: None
cad5106
to
192a828
Compare
Done. Glad neither of us was going crazy. bors r+ |
👎 Rejected by code reviews |
bors r+ |
27774: storage: prevent unbounded raft log growth without quorum r=nvanbenschoten a=nvanbenschoten Fixes #27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in #27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent loss of quorum situations from allowing unbounded growth of a Range's Raft log. Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Build succeeded |
27868: backport-2.0: storage: prevent unbounded raft log growth without quorum r=nvanbenschoten a=nvanbenschoten Backport 2/2 commits from #27774. /cc @cockroachdb/release --- Fixes #27772. This change adds safeguards to prevent cases where a raft log would grow without bound during loss of quorum scenarios. It also adds a new test that demonstrates that the raft log does not grow without bound in these cases. There are two cases that need to be handled to prevent the unbounded raft log growth observed in #27772. 1. When the leader proposes a command and cannot establish a quorum. In this case, we know the leader has the entry in its log, so there's no need to refresh it with `reasonTicks`. To avoid this, we no longer use `refreshTicks` as a leader. 2. When a follower proposes a command that is forwarded to the leader who cannot establish a quorum. In this case, the follower can't be sure (currently) that the leader got the proposal, so it needs to refresh using `reasonTicks`. However, the leader now detects duplicate forwarded proposals and avoids appending redundant entries to its log. It does so by maintaining a set of in-flight forwarded proposals that it has received during its term as leader. This set is reset after every leadership change. Both of these cases are tested against in the new TestLogGrowthWhenRefreshingPendingCommands. Without both of the safeguards introduced in this commit, the test fails. Release note (bug fix): Prevent loss of quorum situations from allowing unbounded growth of a Range's Raft log. 28225: release-2.0: importccl: Preserve '\r\n' during CSV import r=dt a=dt Backport 1/1 commits from #28181. /cc @cockroachdb/release --- See #25344. Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com> Co-authored-by: neeral <neeral@users.noreply.github.com> Co-authored-by: David Taylor <tinystatemachine@gmail.com>
@@ -1126,6 +1127,144 @@ func TestRefreshPendingCommands(t *testing.T) { | |||
} | |||
} | |||
|
|||
// Test that when a Raft group is not able to establish a quorum, its Raft log | |||
// does not grow without bound. It tests two different scenerios where this used |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
scenerios 💩
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Way ahead of you d41111d#diff-b1bd51088f60beeacafe950f5bc700f8R1131 😄
Fixes #27772.
This change adds safeguards to prevent cases where a raft log
would grow without bound during loss of quorum scenarios. It
also adds a new test that demonstrates that the raft log does
not grow without bound in these cases.
There are two cases that need to be handled to prevent the
unbounded raft log growth observed in #27772.
quorum. In this case, we know the leader has the entry in
its log, so there's no need to refresh it with
reasonTicks
.To avoid this, we no longer use
refreshTicks
as a leader.leader who cannot establish a quorum. In this case, the
follower can't be sure (currently) that the leader got the
proposal, so it needs to refresh using
reasonTicks
. However,the leader now detects duplicate forwarded proposals and
avoids appending redundant entries to its log. It does so
by maintaining a set of in-flight forwarded proposals that
it has received during its term as leader. This set is reset
after every leadership change.
Both of these cases are tested against in the new
TestLogGrowthWhenRefreshingPendingCommands. Without both of
the safeguards introduced in this commit, the test fails.
Release note (bug fix): Prevent loss of quorum situations from
allowing unbounded growth of a Range's Raft log.