-
Notifications
You must be signed in to change notification settings - Fork 9.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
raft: separate MaxCommittedSizePerReady config from MaxSizePerMsg #10258
Merged
xiang90
merged 1 commit into
etcd-io:master
from
ajwerner:ajwerner/raft_committed_entries_size
Nov 15, 2018
Merged
raft: separate MaxCommittedSizePerReady config from MaxSizePerMsg #10258
xiang90
merged 1 commit into
etcd-io:master
from
ajwerner:ajwerner/raft_committed_entries_size
Nov 15, 2018
Conversation
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Prior to this change, MaxSizePerMsg was used both to cap the total byte size of entries in messages as well as the total byte size of entries passed through CommittedEntries in the Ready struct. This change adds a new Config parameter MaxCommittedSizePerReady which defaults to MaxSizePerMsg and contols the second of above descibed settings.
Codecov Report
@@ Coverage Diff @@
## master #10258 +/- ##
==========================================
+ Coverage 71.67% 71.69% +0.02%
==========================================
Files 390 390
Lines 36369 36371 +2
==========================================
+ Hits 26067 26078 +11
+ Misses 8488 8483 -5
+ Partials 1814 1810 -4
Continue to review full report at Codecov.
|
bdarnell
approved these changes
Nov 14, 2018
nvanbenschoten
approved these changes
Nov 14, 2018
lgtm |
ajwerner
added a commit
to cockroachdb/vendored
that referenced
this pull request
Nov 15, 2018
Pick up etcd-io/etcd#10258
ajwerner
added a commit
to ajwerner/cockroach
that referenced
this pull request
Nov 15, 2018
Before this change, the size of log committed log entries which a replica could apply at a time was bound to the same configuration as the total size of log entries which could be sent in a message (MaxSizePerMsg) which is generally kilobytes. This limit had an impact on the throughput of writes to a replica, particularly when writing large amounts of data. A new raft configuration option MaxCommittedSizePerReady was adding to etcd/raft in (etcd-io/etcd#10258) which allows these two size parameters to be decoupled. This change adopts the configuration and sets it to a default of 64MB. On the below workload which is set up to always return exactly one entry per Ready wiht the old configuration we see a massive win in both throughput and latency. ``` ./workload run kv {pgurl:1-3} \ --init --splits=10 \ --duration 60s \ --read-percent=${READ_PERCENT} \ --min-block-bytes=8193 --max-block-bytes=16385 \ --concurrency=1024 ``` ``` name old ops/s new ops/s delta KV0 483 ± 3% 2025 ± 3% +319.32% (p=0.002 n=6+6) ``` Before: ``` _elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total 60.0s 0 29570 492.8 1981.2 2281.7 5100.3 5637.1 6442.5 write 60.0s 0 28405 473.4 2074.8 2281.7 5637.1 6710.9 7516.2 write 60.0s 0 28615 476.9 2074.3 2550.1 5905.6 6442.5 8321.5 write 60.0s 0 28718 478.6 2055.4 2550.1 5100.3 6442.5 7516.2 write 60.0s 0 28567 476.1 2079.8 2684.4 4831.8 5368.7 6442.5 write 60.0s 0 29981 499.7 1975.7 1811.9 5368.7 6174.0 6979.3 write ``` After: ``` _elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total 60.0s 0 119652 1994.0 510.9 486.5 1006.6 1409.3 4295.0 write 60.0s 0 125321 2088.4 488.5 469.8 906.0 1275.1 4563.4 write 60.0s 0 119644 1993.9 505.2 469.8 1006.6 1610.6 5637.1 write 60.0s 0 119027 1983.6 511.4 469.8 1073.7 1946.2 4295.0 write 60.0s 0 121723 2028.5 500.6 469.8 1040.2 1677.7 4160.7 write 60.0s 0 123697 2061.4 494.1 469.8 1006.6 1610.6 4295.0 write ``` Fixes cockroachdb#31511 Release note: None
ajwerner
added a commit
to ajwerner/cockroach
that referenced
this pull request
Nov 15, 2018
Before this change, the size of log committed log entries which a replica could apply at a time was bound to the same configuration as the total size of log entries which could be sent in a message (MaxSizePerMsg) which is generally kilobytes. This limit had an impact on the throughput of writes to a replica, particularly when writing large amounts of data. A new raft configuration option MaxCommittedSizePerReady was adding to etcd/raft in (etcd-io/etcd#10258) which allows these two size parameters to be decoupled. This change adopts the configuration and sets it to a default of 64MB. On the below workload which is set up to always return exactly one entry per Ready with the old configuration we see a massive win in both throughput and latency. ``` ./workload run kv {pgurl:1-3} \ --init --splits=10 \ --duration 60s \ --read-percent=${READ_PERCENT} \ --min-block-bytes=8193 --max-block-bytes=16385 \ --concurrency=1024 ``` ``` name old ops/s new ops/s delta KV0 483 ± 3% 2025 ± 3% +319.32% (p=0.002 n=6+6) ``` Before: ``` _elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total 60.0s 0 29570 492.8 1981.2 2281.7 5100.3 5637.1 6442.5 write 60.0s 0 28405 473.4 2074.8 2281.7 5637.1 6710.9 7516.2 write 60.0s 0 28615 476.9 2074.3 2550.1 5905.6 6442.5 8321.5 write 60.0s 0 28718 478.6 2055.4 2550.1 5100.3 6442.5 7516.2 write 60.0s 0 28567 476.1 2079.8 2684.4 4831.8 5368.7 6442.5 write 60.0s 0 29981 499.7 1975.7 1811.9 5368.7 6174.0 6979.3 write ``` After: ``` _elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total 60.0s 0 119652 1994.0 510.9 486.5 1006.6 1409.3 4295.0 write 60.0s 0 125321 2088.4 488.5 469.8 906.0 1275.1 4563.4 write 60.0s 0 119644 1993.9 505.2 469.8 1006.6 1610.6 5637.1 write 60.0s 0 119027 1983.6 511.4 469.8 1073.7 1946.2 4295.0 write 60.0s 0 121723 2028.5 500.6 469.8 1040.2 1677.7 4160.7 write 60.0s 0 123697 2061.4 494.1 469.8 1006.6 1610.6 4295.0 write ``` Fixes cockroachdb#31511 Release note: None
craig bot
pushed a commit
to cockroachdb/cockroach
that referenced
this pull request
Nov 15, 2018
32387: storage: adopt new raft MaxCommittedSizePerReady config parameter r=ajwerner a=ajwerner Before this change, the size of log committed log entries which a replica could apply at a time was bound to the same configuration as the total size of log entries which could be sent in a message (MaxSizePerMsg) which is generally kilobytes. This limit had an impact on the throughput of writes to a replica, particularly when writing large amounts of data. A new raft configuration option MaxCommittedSizePerReady was adding to etcd/raft in (etcd-io/etcd#10258) which allows these two size parameters to be decoupled. This change adopts the configuration and sets it to a default of 64MB. On the below workload which is set up to always return exactly one entry per Ready wiht the old configuration we see a massive win in both throughput and latency. ``` ./workload run kv {pgurl:1-3} \ --init --splits=10 \ --duration 60s \ --read-percent=${READ_PERCENT} \ --min-block-bytes=8193 --max-block-bytes=16385 \ --concurrency=1024 ``` ``` name old ops/s new ops/s delta KV0 483 ± 3% 2025 ± 3% +319.32% (p=0.002 n=6+6) ``` Before: ``` _elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total 60.0s 0 29570 492.8 1981.2 2281.7 5100.3 5637.1 6442.5 write 60.0s 0 28405 473.4 2074.8 2281.7 5637.1 6710.9 7516.2 write 60.0s 0 28615 476.9 2074.3 2550.1 5905.6 6442.5 8321.5 write 60.0s 0 28718 478.6 2055.4 2550.1 5100.3 6442.5 7516.2 write 60.0s 0 28567 476.1 2079.8 2684.4 4831.8 5368.7 6442.5 write 60.0s 0 29981 499.7 1975.7 1811.9 5368.7 6174.0 6979.3 write ``` After: ``` _elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total 60.0s 0 119652 1994.0 510.9 486.5 1006.6 1409.3 4295.0 write 60.0s 0 125321 2088.4 488.5 469.8 906.0 1275.1 4563.4 write 60.0s 0 119644 1993.9 505.2 469.8 1006.6 1610.6 5637.1 write 60.0s 0 119027 1983.6 511.4 469.8 1073.7 1946.2 4295.0 write 60.0s 0 121723 2028.5 500.6 469.8 1040.2 1677.7 4160.7 write 60.0s 0 123697 2061.4 494.1 469.8 1006.6 1610.6 4295.0 write ``` Fixes #31511 Release note: None Co-authored-by: Andrew Werner <[email protected]>
ajwerner
added a commit
to ajwerner/cockroach
that referenced
this pull request
Dec 18, 2018
Before this change, the size of log committed log entries which a replica could apply at a time was bound to the same configuration as the total size of log entries which could be sent in a message (MaxSizePerMsg) which is generally kilobytes. This limit had an impact on the throughput of writes to a replica, particularly when writing large amounts of data. A new raft configuration option MaxCommittedSizePerReady was adding to etcd/raft in (etcd-io/etcd#10258) which allows these two size parameters to be decoupled. This change adopts the configuration and sets it to a default of 64MB. On the below workload which is set up to always return exactly one entry per Ready with the old configuration we see a massive win in both throughput and latency. ``` ./workload run kv {pgurl:1-3} \ --init --splits=10 \ --duration 60s \ --read-percent=${READ_PERCENT} \ --min-block-bytes=8193 --max-block-bytes=16385 \ --concurrency=1024 ``` ``` name old ops/s new ops/s delta KV0 483 ± 3% 2025 ± 3% +319.32% (p=0.002 n=6+6) ``` Before: ``` _elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total 60.0s 0 29570 492.8 1981.2 2281.7 5100.3 5637.1 6442.5 write 60.0s 0 28405 473.4 2074.8 2281.7 5637.1 6710.9 7516.2 write 60.0s 0 28615 476.9 2074.3 2550.1 5905.6 6442.5 8321.5 write 60.0s 0 28718 478.6 2055.4 2550.1 5100.3 6442.5 7516.2 write 60.0s 0 28567 476.1 2079.8 2684.4 4831.8 5368.7 6442.5 write 60.0s 0 29981 499.7 1975.7 1811.9 5368.7 6174.0 6979.3 write ``` After: ``` _elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)__total 60.0s 0 119652 1994.0 510.9 486.5 1006.6 1409.3 4295.0 write 60.0s 0 125321 2088.4 488.5 469.8 906.0 1275.1 4563.4 write 60.0s 0 119644 1993.9 505.2 469.8 1006.6 1610.6 5637.1 write 60.0s 0 119027 1983.6 511.4 469.8 1073.7 1946.2 4295.0 write 60.0s 0 121723 2028.5 500.6 469.8 1040.2 1677.7 4160.7 write 60.0s 0 123697 2061.4 494.1 469.8 1006.6 1610.6 4295.0 write ``` Fixes cockroachdb#31511 Release note: None
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Prior to this change, MaxSizePerMsg was used both to cap the total byte size of
entries in messages as well as the total byte size of entries passed through
CommittedEntries in the Ready struct. This change adds a new Config parameter
MaxCommittedSizePerReady which defaults to MaxSizePerMsg and contols the second
of above descibed settings.
This enables resolution of cockroachdb/cockroach#31511
Please read https://github.com/etcd-io/etcd/blob/master/CONTRIBUTING.md#contribution-flow.