Skip to content

Commit

Permalink
storage: adopt new raft MaxCommittedSizePerReady config parameter
Browse files Browse the repository at this point in the history
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
  • Loading branch information
ajwerner committed Nov 15, 2018
1 parent 9ef9532 commit ec36024
Show file tree
Hide file tree
Showing 4 changed files with 24 additions and 10 deletions.
4 changes: 2 additions & 2 deletions Gopkg.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

21 changes: 17 additions & 4 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,11 +102,17 @@ var (
defaultRaftLogTruncationThreshold = envutil.EnvOrDefaultInt64(
"COCKROACH_RAFT_LOG_TRUNCATION_THRESHOLD", 4<<20 /* 4 MB */)

// defaultRaftMaxSizePerMsg specifies the maximum number of Raft log entries
// that a leader will send to followers in a single MsgApp.
// defaultRaftMaxSizePerMsg specifies the maximum aggregate byte size of Raft
// log entries that a leader will send to followers in a single MsgApp.
defaultRaftMaxSizePerMsg = envutil.EnvOrDefaultInt(
"COCKROACH_RAFT_MAX_SIZE_PER_MSG", 16<<10 /* 16 KB */)

// defaultRaftMaxSizeCommittedSizePerReady specifies the maximum aggregate
// byte size of the committed log entries which a node will receive in a
// single Ready.
defaultRaftMaxCommittedSizePerReady = envutil.EnvOrDefaultInt(
"COCKROACH_RAFT_MAX_COMMITTED_SIZE_PER_READY", 64<<20 /* 64 MB */)

// defaultRaftMaxSizePerMsg specifies how many "inflight" messages a leader
// will send to a follower without hearing a response.
defaultRaftMaxInflightMsgs = envutil.EnvOrDefaultInt(
Expand Down Expand Up @@ -467,10 +473,14 @@ type RaftConfig struct {
// committed but continue to be proposed.
RaftMaxUncommittedEntriesSize uint64

// RaftMaxSizePerMsg controls how many Raft log entries the leader will send to
// followers in a single MsgApp.
// RaftMaxSizePerMsg controls the maximum aggregate byte size of Raft log
// entries the leader will send to followers in a single MsgApp.
RaftMaxSizePerMsg uint64

// RaftMaxCommittedSizePerReady controls the maximum aggregate byte size of
// committed Raft log entries a replica will receive in a single Ready.
RaftMaxCommittedSizePerReady uint64

// RaftMaxInflightMsgs controls how many "inflight" messages Raft will send
// to a follower without hearing a response. The total number of Raft log
// entries is a combination of this setting and RaftMaxSizePerMsg. The
Expand Down Expand Up @@ -512,6 +522,9 @@ func (cfg *RaftConfig) SetDefaults() {
if cfg.RaftMaxSizePerMsg == 0 {
cfg.RaftMaxSizePerMsg = uint64(defaultRaftMaxSizePerMsg)
}
if cfg.RaftMaxCommittedSizePerReady == 0 {
cfg.RaftMaxCommittedSizePerReady = uint64(defaultRaftMaxCommittedSizePerReady)
}
if cfg.RaftMaxInflightMsgs == 0 {
cfg.RaftMaxInflightMsgs = defaultRaftMaxInflightMsgs
}
Expand Down
7 changes: 4 additions & 3 deletions pkg/storage/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,9 +138,9 @@ func TestStoreConfig(clock *hlc.Clock) StoreConfig {
}
st := cluster.MakeTestingClusterSettings()
sc := StoreConfig{
Settings: st,
AmbientCtx: log.AmbientContext{Tracer: st.Tracer},
Clock: clock,
Settings: st,
AmbientCtx: log.AmbientContext{Tracer: st.Tracer},
Clock: clock,
CoalescedHeartbeatsInterval: 50 * time.Millisecond,
RaftHeartbeatIntervalTicks: 1,
ScanInterval: 10 * time.Minute,
Expand Down Expand Up @@ -171,6 +171,7 @@ func newRaftConfig(
ElectionTick: storeCfg.RaftElectionTimeoutTicks,
HeartbeatTick: storeCfg.RaftHeartbeatIntervalTicks,
MaxUncommittedEntriesSize: storeCfg.RaftMaxUncommittedEntriesSize,
MaxCommittedSizePerReady: storeCfg.RaftMaxCommittedSizePerReady,
MaxSizePerMsg: storeCfg.RaftMaxSizePerMsg,
MaxInflightMsgs: storeCfg.RaftMaxInflightMsgs,
Storage: strg,
Expand Down
2 changes: 1 addition & 1 deletion vendor

0 comments on commit ec36024

Please sign in to comment.