raft: Avoid holding unneeded memory in unstable log's entries array

Accumulation of old entries in the underlying array backing the
entries slice has been found to cause massive memory growth in
CockroachDB for workloads that do large (1MB) writes
(https://github.com/cockroachdb/cockroach/issues/14776)

This doesn't appear to have much consistent effect on the raft
benchmarks, although it's worth noting that they vary quite a bit
between runs so it's kind of tough to draw strong conclusions from them.
Let me know if there are any different benchmarks you'd like me to run!

Fixes #7746

benchmark              old ns/op     new ns/op     delta
BenchmarkOneNode-8     3283          3125          -4.81%

benchmark              old allocs     new allocs     delta
BenchmarkOneNode-8     6              6              +0.00%

benchmark              old bytes     new bytes     delta
BenchmarkOneNode-8     796           727           -8.67%

benchmark                     old ns/op     new ns/op     delta
BenchmarkProposal3Nodes-8     4269          4337          +1.59%

benchmark                     old allocs     new allocs     delta
BenchmarkProposal3Nodes-8     15             13             -13.33%

benchmark                     old bytes     new bytes     delta
BenchmarkProposal3Nodes-8     5839          4544          -22.18%
This commit is contained in:
Alex Robinson 2017-04-17 23:20:39 -04:00
parent 0d52598fc1
commit 45406d8486

View File

@ -85,6 +85,26 @@ func (u *unstable) stableTo(i, t uint64) {
if gt == t && i >= u.offset {
u.entries = u.entries[i+1-u.offset:]
u.offset = i + 1
u.shrinkEntriesArray()
}
}
// shrinkEntriesArray discards the underlying array used by the entries slice
// if most of it isn't being used. This avoids holding references to a bunch of
// potentially large entries that aren't needed anymore. Simply clearing the
// entries wouldn't be safe because clients might still be using them.
func (u *unstable) shrinkEntriesArray() {
// We replace the array if we're using less than half of the space in
// it. This number is fairly arbitrary, chosen as an attempt to balance
// memory usage vs number of allocations. It could probably be improved
// with some focused tuning.
const lenMultiple = 2
if len(u.entries) == 0 {
u.entries = nil
} else if len(u.entries)*lenMultiple < cap(u.entries) {
newEntries := make([]pb.Entry, len(u.entries))
copy(newEntries, u.entries)
u.entries = newEntries
}
}