-
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: replace remote proposal tracking with uncommitted log size protection #31408
storage: replace remote proposal tracking with uncommitted log size protection #31408
Conversation
60e364c
to
6f5cc54
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.
Reviewable status: complete! 0 of 0 LGTMs obtained
pkg/storage/storagepb/state.proto, line 93 at r3 (raw file):
uint64 last_index = 2; uint64 num_pending = 3; uint64 num_remote_pending = 9;
Add reserved 9
.
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.
curious whether etcd-io/etcd#10063 will break something.
Reviewed 2 of 2 files at r1, 11 of 11 files at r2, 8 of 8 files at r3, 3 of 3 files at r4.
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale)
pkg/base/config.go, line 101 at r2 (raw file):
// defaultRaftLogMaxSize specifies the upper bound that a single Range's // Raft log is limited to. defaultRaftLogMaxSize = envutil.EnvOrDefaultInt64(
Consider renaming so that it's obvious that it's the threshold after which truncation is preferred over letting the Raft log grow further due to straggler followers. Perhaps RaftLogForceTruncationThreshold
?.
pkg/base/config.go, line 107 at r2 (raw file):
// that a leader will send to followers in a single MsgApp. defaultRaftMaxSizePerMsg = envutil.EnvOrDefaultInt( "COCKROACH_RAFT_MAX_SIZE_PER_MSG", 16*1024)
/* 16 kb */
pkg/base/config.go, line 496 at r3 (raw file):
if cfg.RaftProposalQuota == 0 { // By default, set this to a fraction of RaftLogMaxSize. See comment // above for the tradeoffs of setting this higher or lower.
"See comment above" -> "See the comment on the field."
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.
LGTM for UI stuff
6f5cc54
to
7c171f8
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.
TFTRs!
Reviewable status: complete! 1 of 0 LGTMs obtained
pkg/base/config.go, line 101 at r2 (raw file):
Previously, tschottdorf (Tobias Schottdorf) wrote…
Consider renaming so that it's obvious that it's the threshold after which truncation is preferred over letting the Raft log grow further due to straggler followers. Perhaps
RaftLogForceTruncationThreshold
?.
Done.
pkg/base/config.go, line 107 at r2 (raw file):
Previously, tschottdorf (Tobias Schottdorf) wrote…
/* 16 kb */
Done.
pkg/base/config.go, line 496 at r3 (raw file):
Previously, tschottdorf (Tobias Schottdorf) wrote…
"See comment above" -> "See the comment on the field."
Done.
pkg/storage/storagepb/state.proto, line 93 at r3 (raw file):
Previously, bdarnell (Ben Darnell) wrote…
Add
reserved 9
.
Done.
Picks up etcd-io/etcd#10167. Future commits will use the new setting to replace broken logic that prevented unbounded Raft log growth. This also picks up etcd-io/etcd#10063. Release note: None
This centralizes all Raft configuration and makes it easier to configure in tests. Release note: None
…rotection This change reverts most of the non-testing code from 03b116f and f2f3fd2 and replaces it with use of the MaxUncommittedEntriesSize config. This configuration was added in etcd-io/etcd#10167 and provides protection against unbounded Raft log growth when a Raft group stops being able to commit entries. It makes proposals into Raft safer because proposers don't need to verify before the fact that the proposal isn't a duplicate that might be blowing up the size of the Raft group. By default, the configuration is set to double the Replica's proposal quota. The logic here is that the quotaPool should be responsible for throttling proposals in all cases except for unbounded Raft re-proposals because it queues efficiently instead of dropping proposals on the floor indiscriminately. Release note (bug fix): Fix a bug where Raft proposals could get stuck if forwarded to a leader who could not itself append a new entry to its log.
The reverse of 192a828. Release note: None
7c171f8
to
0ffdb68
Compare
bors r+ |
31408: storage: replace remote proposal tracking with uncommitted log size protection r=nvanbenschoten a=nvanbenschoten Closes #30064. This change reverts most of the non-testing code from 03b116f and f2f3fd2 and replaces it with use of the MaxUncommittedEntriesSize config. This configuration was added in etcd-io/etcd#10167 and provides protection against unbounded Raft log growth when a Raft group stops being able to commit entries. It makes proposals into Raft safer because proposers don't need to verify before the fact that the proposal isn't a duplicate that might be blowing up the size of the Raft group. By default, the configuration is set to double the Replica's proposal quota. The logic here is that the quotaPool should be responsible for throttling proposals in all cases except for unbounded Raft re-proposals because it queues efficiently instead of dropping proposals on the floor indiscriminately. Release note (bug fix): Fix a bug where Raft proposals could get stuck if forwarded to a leader who could not itself append a new entry to its log. This will be backported, but not to 2.1.0. The plan is to get it into 2.1.1. Co-authored-by: Nathan VanBenschoten <[email protected]>
Build succeeded |
Yes to 2.1; I'm not sure about 2.0. Unless it's a trivial backport it might not be worth the risk. |
Closes #30064.
This change reverts most of the non-testing code from 03b116f and f2f3fd2
and replaces it with use of the MaxUncommittedEntriesSize config. This
configuration was added in etcd-io/etcd#10167 and provides protection against
unbounded Raft log growth when a Raft group stops being able to commit
entries. It makes proposals into Raft safer because proposers don't need
to verify before the fact that the proposal isn't a duplicate that might
be blowing up the size of the Raft group.
By default, the configuration is set to double the Replica's proposal quota.
The logic here is that the quotaPool should be responsible for throttling
proposals in all cases except for unbounded Raft re-proposals because it
queues efficiently instead of dropping proposals on the floor indiscriminately.
Release note (bug fix): Fix a bug where Raft proposals could get
stuck if forwarded to a leader who could not itself append a new
entry to its log.
This will be backported, but not to 2.1.0. The plan is to get it into 2.1.1.