Browse Source

Merge pull request #7758 from a-robinson/leak

raft: Avoid holding unneeded memory in unstable log's entries array
Xiang Li 8 years ago
parent
commit
6dd807481c
1 changed files with 20 additions and 0 deletions
  1. 20 0
      raft/log_unstable.go

+ 20 - 0
raft/log_unstable.go

@@ -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
 	}
 }