diff --git a/log_unstable.go b/log_unstable.go index 16cbdeff..acaecccc 100644 --- a/log_unstable.go +++ b/log_unstable.go @@ -152,30 +152,52 @@ func (u *unstable) stableTo(i, t uint64) { "entry at (%d,%d) in unstable log; ignoring", i, t, i, gt) return } - num := int(i + 1 - u.offset) - u.entries = u.entries[num:] + nextUnstableEntryIndex := int(i + 1 - u.offset) + u.shrinkEntriesSlice(nextUnstableEntryIndex) u.offset = i + 1 u.offsetInProgress = max(u.offsetInProgress, u.offset) - u.shrinkEntriesArray() } -// shrinkEntriesArray discards the underlying array used by the entries slice +// shrinkEntriesSlice 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 { +func (u *unstable) shrinkEntriesSlice(nextUnstableEntryIndex int) { + if nextUnstableEntryIndex == len(u.entries) { //all log entries have been successfully stored 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 + } else if u.doEntriesNeedCompaction(nextUnstableEntryIndex) { + u.compactEntriesSlice(nextUnstableEntryIndex) + } else { + u.entries = u.entries[nextUnstableEntryIndex:] + } +} + +func (u *unstable) doEntriesNeedCompaction(nextUnstableEntryIndex int) bool { + //Eligible for compaction if the stable entries occupy more than (1/lenMultiple) times of the: + // a) underlying-array indexes, OR + // b) total memory occupied by entries (both stable and unstable) + const lenMultiple = 2 + + countStableEntries := nextUnstableEntryIndex + if countStableEntries > cap(u.entries)/lenMultiple { + return true } + + var totalSize, stableEntriesSize int + for ind, val := range u.entries { + if ind < nextUnstableEntryIndex { + stableEntriesSize += val.Size() + } + totalSize += val.Size() + } + return (stableEntriesSize > totalSize/lenMultiple) +} + +func (u *unstable) compactEntriesSlice(nextUnstableEntryIndex int) { + countUnstableEntries := (len(u.entries) - nextUnstableEntryIndex) + unstableEntries := make([]pb.Entry, countUnstableEntries) + copy(unstableEntries, u.entries[nextUnstableEntryIndex:]) + u.entries = unstableEntries } func (u *unstable) stableSnapTo(i uint64) {