diff --git a/internal/manifest/l0_sublevels.go b/internal/manifest/l0_sublevels.go index 15d25f6d9d..0c73bcd0ea 100644 --- a/internal/manifest/l0_sublevels.go +++ b/internal/manifest/l0_sublevels.go @@ -7,6 +7,7 @@ package manifest import ( "bytes" "fmt" + "math" "sort" "strings" @@ -14,8 +15,11 @@ import ( "github.com/cockroachdb/pebble/internal/base" ) -// TODO(bilal): work items: -// - Integration with Pebble +// TODO(bilal): +// - Integrate compaction picking logic with the rest of pebble. +// - Refactor away slicing and indexing for simplicity and stronger correctness +// guarantees, especially in extendCandidateToRectangle and +// Pick{Base,IntraL0}Compactions. // Intervals are of the form [start, end) with no gap between intervals. Each // file overlaps perfectly with a sequence of intervals. This perfect overlap @@ -26,7 +30,7 @@ import ( // isLargest bool to remind the code about this fact. This is used for // comparisons in the following manner: // - intervalKey{k, false} < intervalKey{k, true} -// - k1 < k2 => intervalKey{k1, _} < intervalKey{k2, _}. +// - k1 < k2 -> intervalKey{k1, _} < intervalKey{k2, _}. // // For example, consider three files with bounds [a,e], [b,g], and [e,j]. The // interval keys produced would be intervalKey{a, false}, intervalKey{b, false}, @@ -107,7 +111,7 @@ type fileInterval struct { startKey intervalKey // True iff some file in this interval is compacting to base. Such intervals - // cannot have any files participate in L0 => Lbase compactions. + // cannot have any files participate in L0 -> Lbase compactions. isBaseCompacting bool // The min and max intervals index across all the files that overlap with this @@ -140,15 +144,10 @@ type fileInterval struct { fileCount int compactingFileCount int - // The number of consecutive files starting from the the top of the stack - // in this range that are not compacting. Note that any intra-L0 - // compaction can only choose from these files. Additionally after some - // subset of files starting from the top are disqualified because of being - // too young (earliestUnflushedSeqNum), any files picked are the next - // ones. - topOfStackNonCompactingFileCount int - // In increasing sublevel order. - subLevelAndFileList []subLevelAndFile + // All files in this interval, in increasing sublevel order. Indexes map + // to s.filesByAge. Note that the indexes are not sublevel indexes; the + // sublevel lives at s.filesByAge[files[i]].subLevel. + files []int // Interpolated from files in this interval. For files spanning multiple // intervals, we assume an equal distribution of bytes across all those @@ -156,6 +155,30 @@ type fileInterval struct { estimatedBytes uint64 } +// Helper type for any cases requiring a bool slice. +type bitSet []bool + +func newBitSet(n int) bitSet { + return make([]bool, n) +} + +func (b *bitSet) markBit(i int) { + (*b)[i] = true +} + +func (b *bitSet) markBits(start, end int) { + for i := start; i < end; i++ { + (*b)[i] = true + } +} + +func (b *bitSet) clearAllBits() { + for i := range (*b) { + (*b)[i] = false + } +} + + // L0SubLevels represents a sublevel view of SSTables in L0. Tables in one // sublevel are non-overlapping in key ranges, and keys in higher-indexed // sublevels shadow older versions in lower-indexed sublevels. These invariants @@ -255,14 +278,13 @@ func NewL0SubLevels( // Update state in every fileInterval for this file. for i := f.minIntervalIndex; i <= f.maxIntervalIndex; i++ { interval := &s.orderedIntervals[i] - if len(interval.subLevelAndFileList) > 0 && - subLevel <= interval.subLevelAndFileList[len(interval.subLevelAndFileList)-1].subLevel { - subLevel = interval.subLevelAndFileList[len(interval.subLevelAndFileList)-1].subLevel + 1 + if len(interval.files) > 0 && + subLevel <= s.filesByAge[interval.files[len(interval.files)-1]].subLevel { + subLevel = s.filesByAge[interval.files[len(interval.files)-1]].subLevel + 1 } s.orderedIntervals[i].fileCount++ if f.Compacting { interval.compactingFileCount++ - interval.topOfStackNonCompactingFileCount = 0 if !f.IsIntraL0Compacting { // If f.Compacting && !f.IsIntraL0Compacting, this file is // being compacted to Lbase. @@ -270,8 +292,6 @@ func NewL0SubLevels( } } else if f.IsIntraL0Compacting { return nil, errors.Errorf("file %s not marked as compacting but marked as intra-L0 compacting", f.FileNum) - } else { - interval.topOfStackNonCompactingFileCount++ } interval.estimatedBytes += interpolatedBytes if f.minIntervalIndex < interval.filesMinIntervalIndex { @@ -283,8 +303,7 @@ func NewL0SubLevels( } for i := f.minIntervalIndex; i <= f.maxIntervalIndex; i++ { interval := &s.orderedIntervals[i] - interval.subLevelAndFileList = append(interval.subLevelAndFileList, - subLevelAndFile{subLevel: subLevel, fileIndex: fileIndex}) + interval.files = append(interval.files, fileIndex) } f.subLevel = subLevel if subLevel > len(s.Files) { @@ -310,7 +329,8 @@ func NewL0SubLevels( s.orderedIntervals[j].intervalRangeIsBaseCompacting = true } } - if cumulativeBytes > flushSplitMaxBytes && (len(s.flushSplitUserKeys) == 0 || + if flushSplitMaxBytes > 0 && cumulativeBytes > flushSplitMaxBytes && + (len(s.flushSplitUserKeys) == 0 || !bytes.Equal(interval.startKey.key, s.flushSplitUserKeys[len(s.flushSplitUserKeys)-1])) { s.flushSplitUserKeys = append(s.flushSplitUserKeys, interval.startKey.key) cumulativeBytes = 0 @@ -356,25 +376,27 @@ func (s *L0SubLevels) describe(verbose bool) string { fmt.Fprintf(&buf, "0.%d: file count: %d, bytes: %d, width (mean, max): %0.1f, %d, interval range: [%d, %d]\n", i, len(s.Files[i]), totalBytes, float64(sumIntervals)/float64(len(s.Files[i])), maxIntervals, s.Files[i][0].minIntervalIndex, s.Files[i][len(s.Files[i])-1].maxIntervalIndex) - if verbose { - for _, f := range s.Files[i] { - intervals := f.maxIntervalIndex - f.minIntervalIndex + 1 + for _, f := range s.Files[i] { + intervals := f.maxIntervalIndex - f.minIntervalIndex + 1 + if verbose { fmt.Fprintf(&buf, "\t%s\n", f) - if len(s.filesByAge) > 50 && intervals*3 > len(s.orderedIntervals) { - var intervalsBytes uint64 - for k := f.minIntervalIndex; k <= f.maxIntervalIndex; k++ { - intervalsBytes += s.orderedIntervals[k].estimatedBytes - } - fmt.Fprintf(&buf, "wide file: %d, [%d, %d], byte fraction: %f\n", - f.FileNum, f.minIntervalIndex, f.maxIntervalIndex, - float64(intervalsBytes)/float64(s.fileBytes)) + } + if len(s.filesByAge) > 50 && intervals*3 > len(s.orderedIntervals) { + var intervalsBytes uint64 + for k := f.minIntervalIndex; k <= f.maxIntervalIndex; k++ { + intervalsBytes += s.orderedIntervals[k].estimatedBytes } + fmt.Fprintf(&buf, "wide file: %d, [%d, %d], byte fraction: %f\n", + f.FileNum, f.minIntervalIndex, f.maxIntervalIndex, + float64(intervalsBytes)/float64(s.fileBytes)) } } } + lastCompactingIntervalStart := -1 fmt.Fprintf(&buf, "compacting file count: %d, base compacting intervals: ", numCompactingFiles) i := 0 + foundBaseCompactingIntervals := false for ; i < len(s.orderedIntervals); i++ { interval := &s.orderedIntervals[i] if interval.fileCount == 0 { @@ -382,7 +404,11 @@ func (s *L0SubLevels) describe(verbose bool) string { } if !interval.isBaseCompacting { if lastCompactingIntervalStart != -1 { - fmt.Fprintf(&buf, "[%d, %d], ", lastCompactingIntervalStart, i-1) + if foundBaseCompactingIntervals { + buf.WriteString(", ") + } + fmt.Fprintf(&buf, "[%d, %d]", lastCompactingIntervalStart, i-1) + foundBaseCompactingIntervals = true } lastCompactingIntervalStart = -1 } else { @@ -392,7 +418,12 @@ func (s *L0SubLevels) describe(verbose bool) string { } } if lastCompactingIntervalStart != -1 { - fmt.Fprintf(&buf, "[%d, %d], ", lastCompactingIntervalStart, i-1) + if foundBaseCompactingIntervals { + buf.WriteString(", ") + } + fmt.Fprintf(&buf, "[%d, %d]", lastCompactingIntervalStart, i-1) + } else if !foundBaseCompactingIntervals { + fmt.Fprintf(&buf, "none") } fmt.Fprintln(&buf, "") return buf.String() @@ -427,7 +458,7 @@ func (s *L0SubLevels) FlushSplitKeys() [][]byte { // sublevels after all ongoing compactions run to completion. Used by compaction // picker to decide compaction score for L0. There is no scoring for intra-L0 // compactions -- they only run if L0 score is high but we're unable to pick an -// L0 => Lbase compaction. +// L0 -> Lbase compaction. func (s *L0SubLevels) MaxDepthAfterOngoingCompactions() int { depth := 0 for i := range s.orderedIntervals { @@ -439,3 +470,1010 @@ func (s *L0SubLevels) MaxDepthAfterOngoingCompactions() int { } return depth } + +// Only for temporary debugging in the absence of proper tests. +// +// TODO(bilal): Simplify away the debugging statements in this method, and make +// this a pure sanity checker. +func (s *L0SubLevels) checkCompaction(c *L0CompactionFiles) error { + includedFiles := newBitSet(len(s.filesByAge)) + fileIntervalsByLevel := make([]struct { + min int + max int + }, len(s.Files)) + for i := range fileIntervalsByLevel { + fileIntervalsByLevel[i].min = math.MaxInt32 + fileIntervalsByLevel[i].max = 0 + } + var topLevel int + var increment int + var limitReached func(int) bool + if c.isIntraL0 { + topLevel = len(s.Files) - 1 + increment = +1 + limitReached = func(level int) bool { + return level == len(s.Files) + } + } else { + topLevel = 0 + increment = -1 + limitReached = func(level int) bool { + return level < 0 + } + } + for _, f := range c.Files { + if fileIntervalsByLevel[f.subLevel].min > f.minIntervalIndex { + fileIntervalsByLevel[f.subLevel].min = f.minIntervalIndex + } + if fileIntervalsByLevel[f.subLevel].max < f.maxIntervalIndex { + fileIntervalsByLevel[f.subLevel].max = f.maxIntervalIndex + } + includedFiles.markBit(f.l0Index) + if c.isIntraL0 { + if topLevel > f.subLevel { + topLevel = f.subLevel + } + } else { + if topLevel < f.subLevel { + topLevel = f.subLevel + } + } + } + min := fileIntervalsByLevel[topLevel].min + max := fileIntervalsByLevel[topLevel].max + for level := topLevel; !limitReached(level); level += increment { + if fileIntervalsByLevel[level].min < min { + min = fileIntervalsByLevel[level].min + } + if fileIntervalsByLevel[level].max > max { + max = fileIntervalsByLevel[level].max + } + index := sort.Search(len(s.Files[level]), func(i int) bool { + return s.Files[level][i].maxIntervalIndex >= min + }) + // start := index + for ; index < len(s.Files[level]); index++ { + f := s.Files[level][index] + if f.minIntervalIndex > max { + break + } + if c.isIntraL0 && f.LargestSeqNum >= c.earliestUnflushedSeqNum { + return errors.Errorf( + "sstable %s in compaction has sequence numbers higher than the earliest unflushed seqnum %d: %d-%d", + f.FileNum, c.earliestUnflushedSeqNum, f.SmallestSeqNum, + f.LargestSeqNum) + } + if !includedFiles[f.l0Index] { + var buf strings.Builder + fmt.Fprintf(&buf, "bug %t, seed interval: %d: level %d, sl index %d, f.index %d, min %d, max %d, pre-min %d, pre-max %d, f.min %d, f.max %d, filenum: %d, isCompacting: %t\n%s\n", + c.isIntraL0, c.seedInterval, level, index, f.l0Index, min, max, c.preExtensionMinInterval, c.preExtensionMaxInterval, + f.minIntervalIndex, f.maxIntervalIndex, + f.FileNum, f.Compacting, s) + fmt.Fprintf(&buf, "files included:\n") + for _, f := range c.Files { + fmt.Fprintf(&buf, "filenum: %d, sl: %d, index: %d, [%d, %d]\n", + f.FileNum, f.subLevel, f.l0Index, f.minIntervalIndex, f.maxIntervalIndex) + } + fmt.Fprintf(&buf, "files added:\n") + for _, f := range c.filesAdded { + fmt.Fprintf(&buf, "filenum: %d, sl: %d, index: %d, [%d, %d]\n", + f.FileNum, f.subLevel, f.l0Index, f.minIntervalIndex, f.maxIntervalIndex) + } + return errors.New(buf.String()) + } + } + } + return nil +} + +// UpdateStateForStartedCompaction updates internal L0SubLevels state for a +// recently started compaction. isBase specifies if this is a base compaction; +// if false, this is assumed to be an intra-L0 compaction. The specified +// compaction must be involving L0 SSTables. It's assumed that the Compacting +// and IsIntraL0Compacting fields are already set on all FileMetadatas passed +// in. +func (s *L0SubLevels) UpdateStateForStartedCompaction(inputs [][]*FileMetadata, isBase bool) error { + minIntervalIndex := -1 + maxIntervalIndex := 0 + for i := range inputs { + for _, f := range inputs[i] { + for i := f.minIntervalIndex; i <= f.maxIntervalIndex; i++ { + interval := &s.orderedIntervals[i] + interval.compactingFileCount++ + interval.isBaseCompacting = isBase + } + if f.minIntervalIndex < minIntervalIndex || minIntervalIndex == -1 { + minIntervalIndex = f.minIntervalIndex + } + if f.maxIntervalIndex > maxIntervalIndex { + maxIntervalIndex = f.maxIntervalIndex + } + } + } + if isBase { + for i := minIntervalIndex; i <= maxIntervalIndex; i++ { + interval := &s.orderedIntervals[i] + for j := interval.filesMinIntervalIndex; j <= interval.filesMaxIntervalIndex; j++ { + s.orderedIntervals[j].intervalRangeIsBaseCompacting = true + } + } + } + return nil +} + +// L0CompactionFiles represents a candidate set of L0 files for compaction. +// Also referred to as "lcf". Contains state information useful +// for generating the compaction (such as Files), as well as for picking +// between candidate compactions (eg. fileBytes and +// seedIntervalStackDepthReduction). +type L0CompactionFiles struct { + Files []*FileMetadata + FilesIncluded bitSet + // A "seed interval" is an interval with a high stack depth that was chosen + // to bootstrap this compaction candidate. seedIntervalStackDepthReduction + // is the number of sublevels that have a file in the seed interval that is + // a part of this compaction. + seedIntervalStackDepthReduction int + // For base compactions, seedIntervalMinLevel is 0, and for intra-L0 + // compactions, seedIntervalMaxLevel is len(s.Files)-1 i.e. the highest + // sublevel. + seedIntervalMinLevel int + seedIntervalMaxLevel int + // Index of the seed interval. + seedInterval int + // Sum of file sizes for all files in this compaction. + fileBytes uint64 + // Intervals with index [minIntervalIndex, maxIntervalIndex] are + // participating in this compaction; it's the union set of all intervals + // overlapped by participating files. + minIntervalIndex int + maxIntervalIndex int + + // Set for intra-L0 compactions. SSTables with sequence numbers greater + // than earliestUnflushedSeqNum cannot be a part of intra-L0 compactions. + isIntraL0 bool + earliestUnflushedSeqNum uint64 + + // For debugging purposes only. Used in checkCompaction(). + preExtensionMinInterval int + preExtensionMaxInterval int + filesAdded []*FileMetadata +} + +// Adds the specified file to the LCF. +func (l *L0CompactionFiles) addFile(f *FileMetadata) { + if l.FilesIncluded[f.l0Index] { + return + } + l.FilesIncluded.markBit(f.l0Index) + l.Files = append(l.Files, f) + l.filesAdded = append(l.filesAdded, f) + l.fileBytes += f.Size + if f.minIntervalIndex < l.minIntervalIndex { + l.minIntervalIndex = f.minIntervalIndex + } + if f.maxIntervalIndex > l.maxIntervalIndex { + l.maxIntervalIndex = f.maxIntervalIndex + } +} + +// Helper to order intervals being considered for compaction. +type intervalAndScore struct { + interval int + score int +} +type intervalSorterByDecreasingScore []intervalAndScore + +func (is intervalSorterByDecreasingScore) Len() int { return len(is) } +func (is intervalSorterByDecreasingScore) Less(i, j int) bool { + return is[i].score > is[j].score +} +func (is intervalSorterByDecreasingScore) Swap(i, j int) { + is[i], is[j] = is[j], is[i] +} + +// Compactions: +// +// The sub-levels and intervals can be visualized in 2 dimensions as the X +// axis containing intervals in increasing order and the Y axis containing +// sub-levels (older to younger). The intervals can be sparse wrt sub-levels. +// We observe that the system is typically under severe pressure in L0 during +// large numbers of ingestions where most files added to L0 are narrow and +// non-overlapping. +// +// L0.1 d---g +// L0.0 c--e g--j o--s u--x +// +// As opposed to a case with a lot of wide, overlapping L0 files: +// +// L0.3 d-----------r +// L0.2 c--------o +// L0.1 b-----------q +// L0.0 a----------------x +// +// In that case we expect the rectangle represented in the good visualization +// above (i.e. the first one) to be wide and short, and not too sparse (most +// intervals will have fileCount close to the sub-level count), which would make +// it amenable to concurrent L0 -> Lbase compactions. +// +// L0 -> Lbase: The high-level goal of a L0 -> Lbase compaction is to reduce +// stack depth, by compacting files in the intervals with the highest +// (fileCount - compactingCount). Additionally, we would like compactions to +// not involve a huge number of files, so that they finish quickly, and to +// allow for concurrent L0 -> Lbase compactions when needed. In order to +// achieve these goals we would like compactions to visualize as capturing +// thin and tall rectangles. The approach below is to consider intervals in +// some order and then try to construct a compaction using the interval. The +// first interval we can construct a compaction for is the compaction that is +// started. There can be multiple heuristics in choosing the ordering of the +// intervals -- the code uses one heuristic that worked well for a large +// ingestion stemming from a cockroachdb import, but additional experimentation +// is necessary to pick a general heuristic. Additionally, the compaction that +// gets picked may be not as desirable as one that could be constructed later +// in terms of reducing stack depth (since adding more files to the compaction +// can get blocked by needing to encompass files that are already being +// compacted). So an alternative would be to try to construct more than one +// compaction and pick the best one. +// +// Here's a visualization of an ideal L0->LBase compaction selection: +// +// L0.3 a--d g-j +// L0.2 f--j r-t +// L0.1 b-d e---j +// L0.0 a--d f--j l--o p-----x +// +// Lbase a--------i m---------w +// +// The [g,j] interval has the highest stack depth, so it would have the highest +// priority for selecting a base compaction candidate. Assuming none of the +// files are already compacting, this is the compaction that will be chosen: +// +// _______ +// L0.3 a--d | g-j| +// L0.2 | f--j| r-t +// L0.1 b-d |e---j| +// L0.0 a--d | f--j| l--o p-----x +// +// Lbase a--------i m---------w +// +// Note that running this compaction will mark the a--i file in Lbase as +// compacting, and when ExtendL0ForBaseCompactionTo is called with the bounds +// of that base file, it'll expand the compaction to also include all L0 files +// in the a-d interval. The resultant compaction would then be: +// +// _____________ +// L0.3 |a--d g-j| +// L0.2 | f--j| r-t +// L0.1 | b-d e---j| +// L0.0 |a--d f--j| l--o p-----x +// +// Lbase a--------i m---------w +// +// The next best interval for base compaction would therefore +// be the one including r--t in L0.2 and p--x in L0.0, and both this compaction +// and the one picked earlier can run in parallel. This is assuming +// minCompactionDepth >= 2, otherwise the second compaction has too little +// depth to pick. +// +// _____________ +// L0.3 |a--d g-j| _________ +// L0.2 | f--j| | r-t | +// L0.1 | b-d e---j| | | +// L0.0 |a--d f--j| l--o |p-----x| +// +// Lbase a--------i m---------w +// +// Intra-L0: If the L0 score is high, but PickBaseCompaction() is unable to +// pick a compaction, PickIntraL0Compaction will be used to pick an intra-L0 +// compaction. Similar to L0 -> Lbase compactions, we want to allow for +// multiple intra-L0 compactions and not generate wide output files that +// hinder later concurrency of L0 -> Lbase compactions. Also compactions +// that produce wide files don't reduce stack depth -- they represent wide +// rectangles in our visualization, which means many intervals have their +// depth reduced by a small amount. Typically, L0 files have non-overlapping +// sequence numbers, and sticking to that invariant would require us to +// consider intra-L0 compactions that proceed from youngest to oldest files, +// which could result in the aforementioned undesirable wide rectangle +// shape. But this non-overlapping sequence number is already relaxed in +// RocksDB -- sstables are primarily ordered by their largest sequence +// number. So we can arrange for intra-L0 compactions to capture thin and +// tall rectangles starting with the top of the stack (youngest files). +// Like the L0 -> Lbase case we order the intervals using a heuristic and +// consider each in turn. The same comment about better L0 -> Lbase heuristics +// and not being greedy applies here. +// +// Going back to a modified version of our example from earlier, let's say these +// are the base compactions in progress: +// _______ +// L0.3 a--d | g-j| _________ +// L0.2 | f--j| | r-t | +// L0.1 b-d |e---j| | | +// L0.0 a--d | f--j| l--o |p-----x| +// +// Lbase a---------i m---------w +// +// Since both LBase files are compacting, the only L0 compaction that can be +// picked is an intra-L0 compaction. For this, the b--d interval has the highest +// stack depth (3), and starting with a--d in L0.3 as the seed file, we can +// iterate downward and build this compaction, assuming all files in that +// interval are not compacting and have a highest sequence number less than +// earliestUnflushedSeqNum: +// +// _______ +// L0.3 |a--d| | g-j| _________ +// L0.2 | | | f--j| | r-t | +// L0.1 | b-d| |e---j| | | +// L0.0 |a--d| | f--j| l--o |p-----x| +// ------ +// Lbase a---------i m---------w +// + +// PickBaseCompaction picks a base compaction based on the above specified +// heuristics, for the specified Lbase files and a minimum depth of overlapping +// files that can be selected for compaction. Returns nil if no compaction is +// possible. +func (s *L0SubLevels) PickBaseCompaction( + minCompactionDepth int, baseFiles []*FileMetadata, +) (*L0CompactionFiles, error) { + // For LBase compactions, we consider intervals in a greedy manner in the + // following order: + // - Intervals that are unlikely to be blocked due + // to ongoing L0 -> Lbase compactions. These are the ones with + // !isBaseCompacting && !intervalRangeIsBaseCompacting. + // - Intervals that are !isBaseCompacting && intervalRangeIsBaseCompacting. + // + // The ordering heuristic exists just to avoid wasted work. Ideally, + // we would consider all intervals with isBaseCompacting = false and + // construct a compaction for it and compare the constructed compactions + // and pick the best one. If microbenchmarks show that we can afford + // this cost we can eliminate this heuristic. + scoredIntervals := make([]intervalAndScore, 0, len(s.orderedIntervals)) + sublevelCount := len(s.Files) + for i := range s.orderedIntervals { + interval := &s.orderedIntervals[i] + depth := interval.fileCount - interval.compactingFileCount + if interval.isBaseCompacting || minCompactionDepth > depth { + continue + } + if interval.intervalRangeIsBaseCompacting { + scoredIntervals = append(scoredIntervals, intervalAndScore{interval: i, score: depth}) + } else { + // Prioritize this interval by incrementing the score by the number + // of sublevels. + scoredIntervals = append(scoredIntervals, intervalAndScore{interval: i, score: depth+sublevelCount}) + } + } + sort.Sort(intervalSorterByDecreasingScore(scoredIntervals)) + + // Optimization to avoid considering different intervals that + // are likely to choose the same seed file. Again this is just + // to reduce wasted work. + consideredIntervals := newBitSet(len(s.orderedIntervals)) + for _, scoredInterval := range scoredIntervals { + interval := &s.orderedIntervals[scoredInterval.interval] + if consideredIntervals[interval.index] { + continue + } + + // Pick the seed file for the interval as the file + // in the lowest sub-level. + f := s.filesByAge[interval.files[0]] + // Don't bother considering the intervals that are + // covered by the seed file since they are likely + // nearby. Note that it is possible that those intervals + // have seed files at lower sub-levels so could be + // viable for compaction. + consideredIntervals.markBits(f.minIntervalIndex, f.maxIntervalIndex+1) + if f == nil { + return nil, errors.New("no seed file found in sublevel intervals") + } + if f.Compacting { + if f.IsIntraL0Compacting { + // If we're picking a base compaction and we came across a + // seed file candidate that's being intra-L0 compacted, skip + // the interval instead of erroring out. + continue + } + // We chose a compaction seed file that should not be + // compacting. Usually means the score is not accurately + // accounting for files already compacting, or internal state is + // inconsistent. + return nil, errors.Errorf("file %d chosen as seed file for compaction should not be compacting", f.FileNum) + } + + c := s.baseCompactionUsingSeed(f, interval.index, minCompactionDepth) + if c != nil { + // Check if the chosen compaction overlaps with any files + // in Lbase that have Compacting = true. If that's the case, + // this compaction cannot be chosen. + firstBaseIndex := sort.Search(len(baseFiles), func(i int) bool { + // An interval starting at ImmediateSuccessor(key) can never be the + // first interval of a compaction since no file can start at that + // interval. + return s.cmp(baseFiles[i].Largest.UserKey, s.orderedIntervals[c.minIntervalIndex].startKey.key) >= 0 + }) + // Exclusive + lastBaseIndex := sort.Search(len(baseFiles), func(i int) bool { + cmp := s.cmp(baseFiles[i].Smallest.UserKey, s.orderedIntervals[c.maxIntervalIndex+1].startKey.key) + // Compaction is ending at exclusive bound of c.maxIntervalIndex+1 + if cmp > 0 || (cmp == 0 && !s.orderedIntervals[c.maxIntervalIndex+1].startKey.isLargest) { + return true + } + return false + }) + baseCompacting := false + for j := firstBaseIndex; j < lastBaseIndex; j++ { + if baseFiles[j].Compacting { + baseCompacting = true + break + } + } + if baseCompacting { + continue + } + return c, nil + } + } + return nil, nil +} + +// Helper function for building an L0 -> Lbase compaction using a seed interval +// and seed file in that seed interval. +func (s *L0SubLevels) baseCompactionUsingSeed( + f *FileMetadata, intervalIndex int, minCompactionDepth int, +) *L0CompactionFiles { + c := &L0CompactionFiles{ + FilesIncluded: newBitSet(len(s.filesByAge)), + seedInterval: intervalIndex, + seedIntervalMinLevel: 0, + minIntervalIndex: f.minIntervalIndex, + maxIntervalIndex: f.maxIntervalIndex, + } + c.addFile(f) + + // The first iteration of this loop builds the compaction at the seed file's + // sublevel. Future iterations expand on this compaction by stacking + // more files from intervalIndex and repeating. This is an + // optional activity so when it fails we can fallback to the last + // successful candidate. + var lastCandidate *L0CompactionFiles + interval := &s.orderedIntervals[intervalIndex] + for i := 0; i < len(interval.files); i++ { + f2 := s.filesByAge[interval.files[i]] + sl := f2.subLevel + c.seedIntervalStackDepthReduction++ + c.seedIntervalMaxLevel = sl + c.addFile(f2) + // The seed file is in the lowest sublevel in the seed interval, but it may + // overlap with other files in even lower sublevels. For + // correctness we need to grow our interval to include those files, and + // capture all files in the next level that fall in this extended interval + // and so on. This can result in a triangular shape like the following + // where again the X axis is the key intervals and the Y axis + // is oldest to youngest. Note that it is not necessary for + // correctness to fill out the shape at the higher sub-levels + // to make it more rectangular since the invariant only requires + // that younger versions of a key not be moved to Lbase while + // leaving behind older versions. + // - + // --- + // ----- + // It may be better for performance to have a more rectangular + // shape since the files being left behind will overlap with the + // same Lbase key range as that of this compaction. But there is + // also the danger that in trying to construct a more rectangular + // shape we will be forced to pull in a file that is already + // compacting. We expect extendCandidateToRectangle to eventually be called + // on this compaction if it's chosen, at which point we would iterate + // backward and choose those files. This logic is similar to compaction.grow + // for non-L0 compactions. + done := false + for currLevel := sl - 1; currLevel >= 0; currLevel-- { + if !s.extendFiles(currLevel, math.MaxUint64, c) { + // Failed to extend due to ongoing compaction. + done = true + break + } + } + if done { + break + } + // Observed some compactions using > 1GB from L0 in an import + // experiment. Very long running compactions are not great as they + // reduce concurrency while they run, and take a while to produce + // results, though they're sometimes unavoidable. There is a tradeoff + // here in that adding more depth is more efficient in reducing stack + // depth, but long running compactions reduce flexibility in what can + // run concurrently in L0 and even Lbase -> Lbase+1. An increase more + // than 150% in bytes since the last candidate compaction (along with a + // total compaction size in excess of 100mb), or a total compaction + // size beyond a hard limit of 500mb, is criteria for rejecting this + // candidate. This lets us prefer slow growths as we add files, while + // still having a hard limit. Note that if this is the first compaction + // candidate to reach a stack depth reduction of minCompactionDepth or + // higher, this candidate will be chosen regardless. + if lastCandidate == nil { + lastCandidate = &L0CompactionFiles{} + } else if lastCandidate.seedIntervalStackDepthReduction >= minCompactionDepth && + c.fileBytes > 100<<20 && + (float64(c.fileBytes)/float64(lastCandidate.fileBytes) > 1.5 || c.fileBytes > 500<<20) { + break + } + *lastCandidate = *c + } + if lastCandidate != nil && lastCandidate.seedIntervalStackDepthReduction >= minCompactionDepth { + lastCandidate.FilesIncluded.clearAllBits() + for _, f := range lastCandidate.Files { + lastCandidate.FilesIncluded.markBit(f.l0Index) + } + return lastCandidate + } + return nil +} + +// Expands fields in the provided L0CompactionFiles instance (cFiles) to +// include overlapping files in the specified sublevel. Returns true if the +// compaction is possible (i.e. does not conflict with any base/intra-L0 +// compacting files). +func (s *L0SubLevels) extendFiles( + sl int, earliestUnflushedSeqNum uint64, cFiles *L0CompactionFiles, +) bool { + index := sort.Search(len(s.Files[sl]), func(i int) bool { + return s.Files[sl][i].maxIntervalIndex >= cFiles.minIntervalIndex + }) + for ; index < len(s.Files[sl]); index++ { + f := s.Files[sl][index] + if f.minIntervalIndex > cFiles.maxIntervalIndex { + break + } + if f.Compacting { + return false + } + // Skip over files that are newer than earliestUnflushedSeqNum. This is + // okay because this compaction can just pretend these files are not in + // L0 yet. These files must be in higher sublevels than any overlapping + // files with f.LargestSeqNum < earliestUnflushedSeqNum, and the output + // of the compaction will also go in a lower (older) sublevel than this + // file by definition. + if f.LargestSeqNum >= earliestUnflushedSeqNum { + continue + } + cFiles.addFile(f) + } + return true +} + +// PickIntraL0Compaction picks an intra-L0 compaction for files in this +// sublevel. This method is only called when a base compaction cannot be chosen. +// See comment above PickBaseCompaction for heuristics involved in this +// selection. +func (s *L0SubLevels) PickIntraL0Compaction( + earliestUnflushedSeqNum uint64, minCompactionDepth int, +) (*L0CompactionFiles, error) { + var scoredIntervals []intervalAndScore + for i := range s.orderedIntervals { + interval := &s.orderedIntervals[i] + depth := interval.fileCount - interval.compactingFileCount + if minCompactionDepth > depth { + continue + } + scoredIntervals = append(scoredIntervals, intervalAndScore{interval: i, score: depth}) + } + sort.Sort(intervalSorterByDecreasingScore(scoredIntervals)) + + // Optimization to avoid considering different intervals that + // are likely to choose the same seed file. Again this is just + // to reduce wasted work. + consideredIntervals := newBitSet(len(s.orderedIntervals)) + for _, scoredInterval := range scoredIntervals { + interval := &s.orderedIntervals[scoredInterval.interval] + if consideredIntervals[interval.index] { + continue + } + + var f *FileMetadata + // Pick the seed file for the interval as the file + // in the highest sub-level. + stackDepthReduction := scoredInterval.score + for i := len(interval.files)-1; i >= 0; i-- { + f = s.filesByAge[interval.files[i]] + if f.Compacting { + break + } + consideredIntervals.markBits(f.minIntervalIndex, f.maxIntervalIndex+1) + // Can this be the seed file? Files with newer sequence + // numbers than earliestUnflushedSeqNum cannot be in + // the compaction. + if f.LargestSeqNum >= earliestUnflushedSeqNum { + stackDepthReduction-- + if stackDepthReduction == 0 { + break + } + } else { + break + } + } + if stackDepthReduction < minCompactionDepth { + // Can't use this interval. + continue + } + + if f == nil { + return nil, errors.New("no seed file found in sublevel intervals") + } + if f.Compacting { + // We chose a compaction seed file that should not be + // compacting. Usually means the score is not accurately + // accounting for files already compacting, or internal state is + // inconsistent. + return nil, errors.Errorf("file %d chosen as seed file for compaction should not be compacting", f.FileNum) + } + + // We have a seed file. Build a compaction off of that seed. + c := s.intraL0CompactionUsingSeed( + f, interval.index, earliestUnflushedSeqNum, minCompactionDepth) + if c != nil { + return c, nil + } + } + return nil, nil +} + +func (s *L0SubLevels) intraL0CompactionUsingSeed( + f *FileMetadata, intervalIndex int, earliestUnflushedSeqNum uint64, minCompactionDepth int, +) *L0CompactionFiles { + // We know that all the files that overlap with intervalIndex have + // LargestSeqNum < earliestUnflushedSeqNum, but for other intervals + // we need to exclude files >= earliestUnflushedSeqNum + + c := &L0CompactionFiles{ + FilesIncluded: newBitSet(len(s.filesByAge)), + seedInterval: intervalIndex, + seedIntervalMaxLevel: len(s.Files) - 1, + minIntervalIndex: f.minIntervalIndex, + maxIntervalIndex: f.maxIntervalIndex, + isIntraL0: true, + earliestUnflushedSeqNum: earliestUnflushedSeqNum, + } + c.addFile(f) + + var lastCandidate *L0CompactionFiles + interval := &s.orderedIntervals[intervalIndex] + slIndex := len(interval.files) - 1 + for { + if interval.files[slIndex] == f.l0Index { + break + } + slIndex-- + } + // The first iteration of this loop produces an intra-L0 compaction at the + // seed level. Iterations after that optionally add to the compaction by + // stacking more files from intervalIndex and repeating. This is an + // optional activity so when it fails we can fallback to the last + // successful candidate. The code stops adding when it can't add more, or + // when fileBytes grows too large. + for ; slIndex >= 0; slIndex-- { + f2 := s.filesByAge[interval.files[slIndex]] + sl := f2.subLevel + if f2.Compacting { + break + } + c.seedIntervalStackDepthReduction++ + c.seedIntervalMinLevel = sl + c.addFile(f2) + // The seed file captures all files in the higher level that fall in the + // range of intervals. That may extend the range of intervals so for + // correctness we need to capture all files in the next higher level that + // fall in this extended interval and so on. This can result in an + // inverted triangular shape like the following where again the X axis is the + // key intervals and the Y axis is oldest to youngest. Note that it is not + // necessary for correctness to fill out the shape at lower sub-levels to + // make it more rectangular since the invariant only requires that if we + // move an older seqnum for key k into a file that has a higher seqnum, we + // also move all younger seqnums for that key k into that file. + // ----- + // --- + // - + // + // It may be better for performance to have a more rectangular shape since + // it will reduce the stack depth for more intervals. But there is also + // the danger that in explicitly trying to construct a more rectangular + // shape we will be forced to pull in a file that is already compacting. + // We assume that the performance concern is not a practical issue. + done := false + for currLevel := sl + 1; currLevel < len(s.Files); currLevel++ { + if !s.extendFiles(currLevel, earliestUnflushedSeqNum, c) { + // Failed to extend due to ongoing compaction. + done = true + break + } + } + if done { + break + } + if lastCandidate == nil { + lastCandidate = &L0CompactionFiles{} + } else if lastCandidate.seedIntervalStackDepthReduction >= minCompactionDepth && + c.fileBytes > 100<<20 && + (float64(c.fileBytes)/float64(lastCandidate.fileBytes) > 1.5 || c.fileBytes > 500<<20) { + break + } + *lastCandidate = *c + } + if lastCandidate != nil && lastCandidate.seedIntervalStackDepthReduction >= minCompactionDepth { + lastCandidate.FilesIncluded.clearAllBits() + for _, f := range lastCandidate.Files { + lastCandidate.FilesIncluded.markBit(f.l0Index) + } + s.extendCandidateToRectangle( + lastCandidate.minIntervalIndex, lastCandidate.maxIntervalIndex, lastCandidate, false) + return lastCandidate + } + return nil +} + +// ExtendL0ForBaseCompactionTo extends the specified base compaction candidate +// L0CompactionFiles to cover all L0 files in the specified key interval, +// by calling extendCandidateToRectangle. +func (s *L0SubLevels) ExtendL0ForBaseCompactionTo( + smallest []byte, largest []byte, candidate *L0CompactionFiles, +) bool { + firstIntervalIndex := sort.Search(len(s.orderedIntervals), func(i int) bool { + // Need to start at >= smallest since if we widen too much we may miss + // an Lbase file that overlaps with an L0 file that will get picked in + // this widening, which would be bad. This interval will not start with + // an immediate successor key. + return s.cmp(smallest, s.orderedIntervals[i].startKey.key) <= 0 + }) + // First interval that starts at or beyond the largest. This interval will not + // start with an immediate successor key. + lastIntervalIndex := sort.Search(len(s.orderedIntervals), func(i int) bool { + return s.cmp(largest, s.orderedIntervals[i].startKey.key) < 0 + }) + // Right now, lastIntervalIndex has a start that's higher than largest. + // The previous interval, by definition, has an end key higher than largest. + // Iterate back twice to get the last interval that's completely within + // [smallest, largest]. Except in the case where we went past the end of the + // list; in that case, the last interval to include is the very last + // interval in the list. + if lastIntervalIndex < len(s.orderedIntervals) { + lastIntervalIndex-- + } + lastIntervalIndex-- + if lastIntervalIndex < firstIntervalIndex { + return false + } + return s.extendCandidateToRectangle(firstIntervalIndex, lastIntervalIndex, candidate, true) +} + +// Best-effort attempt to make the compaction include more files in the +// rectangle defined by [minIntervalIndex, maxIntervalIndex] on the X axis and +// bounded on the Y axis by seedIntervalMinLevel and seedIntervalMaxLevel. +// +// This is strictly an optional extension; at any point where we can't feasibly +// add more files, the sublevel iteration can be halted early and candidate will +// still be a correct compaction candidate. +// +// Consider this scenario (original candidate is inside the rectangle), with +// isBase = true and interval bounds a-j (from the union of base file bounds and +// that of compaction candidate): +// _______ +// L0.3 a--d | g-j| +// L0.2 | f--j| r-t +// L0.1 b-d |e---j| +// L0.0 a--d | f--j| l--o p-----x +// +// Lbase a--------i m---------w +// +// This method will iterate from the bottom up. At L0.0, it will add a--d since +// it's in the bounds, then add b-d, then a--d, and so on, to produce this: +// +// _____________ +// L0.3 |a--d g-j| _________ +// L0.2 | f--j| | r-t | +// L0.1 | b-d e---j| | | +// L0.0 |a--d f--j| l--o |p-----x| +// +// Lbase a-------i m---------w +// +// Let's assume that, instead of a--d in the top sublevel, we had 3 files, a-b, +// bb-c, and cc-d, of which bb-c is compacting. Let's also add another sublevel +// L0.4 with some files, all of which aren't compacting: +// +// L0.4 a------c ca--d _______ +// L0.3 a-b bb-c cc-d | g-j| +// L0.2 | f--j| r-t +// L0.1 b----------d |e---j| +// L0.0 a------------d | f--j| l--o p-----x +// +// Lbase a------------------i m---------w +// +// This method then needs to choose between the left side of L0.3 bb-c +// (i.e. a-b), or the right side (i.e. cc-d and g-j) for inclusion in this +// compaction. Since the right side has more files as well as one file that has +// already been picked, it gets chosen at that sublevel, resulting in this +// intermediate compaction: +// +// L0.4 a------c ca--d +// ______________ +// L0.3 a-b bb-c| cc-d g-j| +// L0.2 _________| f--j| r-t +// L0.1 | b----------d e---j| +// L0.0 |a------------d f--j| l--o p-----x +// +// Lbase a------------------i m---------w +// +// Since bb-c had to be excluded at L0.3, the interval bounds for L0.4 are +// actually ca-j, since ca is the next interval start key after the end interval +// of bb-c. This would result in only ca-d being chosen at that sublevel, even +// though a--c is also not compacting. This is the final result: +// +// ______________ +// L0.4 a------c|ca--d | +// L0.3 a-b bb-c| cc-d g-j| +// L0.2 _________| f--j| r-t +// L0.1 | b----------d e---j| +// L0.0 |a------------d f--j| l--o p-----x +// +// Lbase a------------------i m---------w +// +// TODO(bilal): Add more targeted tests for this method, through +// ExtendL0ForBaseCompactionTo and intraL0CompactionUsingSeed. +func (s *L0SubLevels) extendCandidateToRectangle( + minIntervalIndex int, maxIntervalIndex int, candidate *L0CompactionFiles, isBase bool, +) bool { + candidate.preExtensionMinInterval = candidate.minIntervalIndex + candidate.preExtensionMaxInterval = candidate.maxIntervalIndex + // Extend {min,max}IntervalIndex to include all of the candidate's current + // bounds. + if minIntervalIndex > candidate.minIntervalIndex { + minIntervalIndex = candidate.minIntervalIndex + } + if maxIntervalIndex < candidate.maxIntervalIndex { + maxIntervalIndex = candidate.maxIntervalIndex + } + var startLevel, increment, endLevel int + if isBase { + startLevel = 0 + increment = +1 + // seedIntervalMaxLevel is inclusive, while endLevel is exclusive. + endLevel = candidate.seedIntervalMaxLevel + 1 + } else { + startLevel = len(s.Files) - 1 + increment = -1 + // seedIntervalMinLevel is inclusive, while endLevel is exclusive. + endLevel = candidate.seedIntervalMinLevel - 1 + } + // Stats for files. + addedCount := 0 + // Iterate from the oldest sub-level for L0 -> Lbase and youngest + // sub-level for intra-L0. The idea here is that anything that can't + // be included from that level constrains what can be included from + // the next level. This change in constraint is directly incorporated + // into minIntervalIndex, maxIntervalIndex. + for sl := startLevel; sl != endLevel; sl += increment { + files := s.Files[sl] + // Find the first file that overlaps with minIntervalIndex. + index := sort.Search(len(files), func(i int) bool { + return minIntervalIndex <= files[i].maxIntervalIndex + }) + // Track the files that are fully within the current constraint + // of [minIntervalIndex, maxIntervalIndex]. + firstIndex := -1 + lastIndex := -1 + for ; index < len(files); index++ { + f := files[index] + if f.minIntervalIndex > maxIntervalIndex { + break + } + include := true + // Extends out on the left so can't be included. This narrows + // what we can included in the next level. + if f.minIntervalIndex < minIntervalIndex { + include = false + minIntervalIndex = f.maxIntervalIndex + 1 + } + // Extends out on the right so can't be included. + if f.maxIntervalIndex > maxIntervalIndex { + include = false + maxIntervalIndex = f.minIntervalIndex - 1 + } + if !include { + continue + } + if firstIndex == -1 { + firstIndex = index + } + lastIndex = index + } + if minIntervalIndex > maxIntervalIndex { + // We excluded files that prevent continuation. + break + } + if firstIndex < 0 { + // No files to add in this sub-level. + continue + } + // We have the files in [firstIndex, lastIndex] as potential for + // inclusion. Some of these may already have been picked. Some + // of them may be already compacting. The latter is tricky since + // we have to decide whether to contract minIntervalIndex or + // maxIntervalIndex when we encounter an already compacting file. + // We pick the longest sequence between firstIndex + // and lastIndex of non-compacting files -- this is represented by + // [candidateNonCompactingFirst, candidateNonCompactingLast]. + nonCompactingFirst := -1 + currentRunHasAlreadyPickedFiles := false + candidateNonCompactingFirst := -1 + candidateNonCompactingLast := -1 + candidateHasAlreadyPickedFiles := false + for index = firstIndex; index <= lastIndex; index++ { + f := files[index] + if f.Compacting { + if nonCompactingFirst != -1 { + last := index - 1 + // Prioritize runs of consecutive non-compacting files that + // have files that have already been picked. That is to say, + // if candidateHasAlreadyPickedFiles == true, we stick with + // it, and if currentRunHasAlreadyPickedfiles == true, we + // pick that run even if it contains fewer files than the + // previous candidate. + if !candidateHasAlreadyPickedFiles && (candidateNonCompactingFirst == -1 || + currentRunHasAlreadyPickedFiles || + (last-nonCompactingFirst) > (candidateNonCompactingLast-candidateNonCompactingFirst)) { + candidateNonCompactingFirst = nonCompactingFirst + candidateNonCompactingLast = last + candidateHasAlreadyPickedFiles = currentRunHasAlreadyPickedFiles + } + } + nonCompactingFirst = -1 + currentRunHasAlreadyPickedFiles = false + continue + } + if nonCompactingFirst == -1 { + nonCompactingFirst = index + } + if candidate.FilesIncluded[f.l0Index] { + currentRunHasAlreadyPickedFiles = true + } + } + // Logic duplicated from inside the for loop above. + if nonCompactingFirst != -1 { + last := index - 1 + if !candidateHasAlreadyPickedFiles && (candidateNonCompactingFirst == -1 || + currentRunHasAlreadyPickedFiles || + (last-nonCompactingFirst) > (candidateNonCompactingLast-candidateNonCompactingFirst)) { + candidateNonCompactingFirst = nonCompactingFirst + candidateNonCompactingLast = last + } + } + if candidateNonCompactingFirst == -1 { + // All files are compacting. There will be gaps that we could exploit + // to continue, but don't bother. + break + } + // May need to shrink [minIntervalIndex, maxIntervalIndex] for the next level. + if candidateNonCompactingFirst > firstIndex { + minIntervalIndex = files[candidateNonCompactingFirst-1].maxIntervalIndex + 1 + } + if candidateNonCompactingLast < lastIndex { + maxIntervalIndex = files[candidateNonCompactingLast+1].minIntervalIndex - 1 + } + for index := candidateNonCompactingFirst; index <= candidateNonCompactingLast; index++ { + f := files[index] + if f.Compacting { + // TODO(bilal): Do a logger.Fatalf instead of a panic, for + // cleaner unwinding and error messages. + panic(fmt.Sprintf("expected %s to not be compacting", f.FileNum)) + } + if candidate.isIntraL0 && f.LargestSeqNum >= candidate.earliestUnflushedSeqNum { + continue + } + if !candidate.FilesIncluded[f.l0Index] { + addedCount++ + candidate.addFile(f) + } + } + } + return addedCount > 0 +} diff --git a/internal/manifest/l0_sublevels_test.go b/internal/manifest/l0_sublevels_test.go index 5723d12b7b..3c47a1f407 100644 --- a/internal/manifest/l0_sublevels_test.go +++ b/internal/manifest/l0_sublevels_test.go @@ -7,7 +7,9 @@ package manifest import ( "fmt" "io" + "math" "os" + "sort" "strconv" "strings" "testing" @@ -15,6 +17,7 @@ import ( "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/datadriven" "github.com/cockroachdb/pebble/internal/record" + "github.com/stretchr/testify/require" ) func readManifest(filename string) (*Version, error) { @@ -47,6 +50,112 @@ func readManifest(filename string) (*Version, error) { return v, nil } +func TestL0SubLevels_LargeImportL0(t *testing.T) { + // TODO(bilal): Fix this test. + t.Skip() + v, err := readManifest("testdata/MANIFEST_import") + require.NoError(t, err) + + subLevels, err := NewL0SubLevels(v.Files[0], base.DefaultComparer.Compare, base.DefaultFormatter, 5<<20) + require.NoError(t, err) + fmt.Printf("L0SubLevels:\n%s\n\n", subLevels) + + for i := 0; ; i++ { + c, err := subLevels.PickBaseCompaction(2, nil) + require.NoError(t, err) + if c == nil { + break + } + fmt.Printf("%d: base compaction: filecount: %d, bytes: %d, interval: [%d, %d], seed depth: %d\n", + i, len(c.Files), c.fileBytes, c.minIntervalIndex, c.maxIntervalIndex, c.seedIntervalStackDepthReduction) + var files []*FileMetadata + for i := range c.Files { + if c.FilesIncluded[i] { + c.Files[i].Compacting = true + files = append(files, c.Files[i]) + } + } + require.NoError(t, subLevels.UpdateStateForStartedCompaction([][]*FileMetadata{files}, true)) + } + + for i := 0; ; i++ { + c, err := subLevels.PickIntraL0Compaction(math.MaxUint64, 2) + require.NoError(t, err) + if c == nil { + break + } + fmt.Printf("%d: intra-L0 compaction: filecount: %d, bytes: %d, interval: [%d, %d], seed depth: %d\n", + i, len(c.Files), c.fileBytes, c.minIntervalIndex, c.maxIntervalIndex, c.seedIntervalStackDepthReduction) + var files []*FileMetadata + for i := range c.Files { + if c.FilesIncluded[i] { + c.Files[i].Compacting = true + c.Files[i].IsIntraL0Compacting = true + files = append(files, c.Files[i]) + } + } + require.NoError(t, subLevels.UpdateStateForStartedCompaction([][]*FileMetadata{files}, false)) + } +} + +func visualizeSublevels(s *L0SubLevels, compactionFiles bitSet) string { + var buf strings.Builder + if compactionFiles == nil { + compactionFiles = newBitSet(len(s.filesByAge)) + } + for i := len(s.Files) - 1; i >= 0; i-- { + lastChar := byte('a') + fmt.Fprintf(&buf, "L0.%d: ", i) + for j, f := range s.Files[i] { + for lastChar < f.Smallest.UserKey[0] { + buf.WriteString(" ") + lastChar++ + } + buf.WriteByte(f.Smallest.UserKey[0]) + middleChar := byte('-') + if compactionFiles[f.l0Index] { + middleChar = '+' + } + if f.Smallest.UserKey[0] == f.Largest.UserKey[0] { + buf.WriteByte(f.Largest.UserKey[0]) + if compactionFiles[f.l0Index] { + buf.WriteByte('+') + } else if j < len(s.Files[i]) - 1 { + buf.WriteByte(' ') + } + lastChar++ + continue + } + buf.WriteByte(middleChar) + buf.WriteByte(middleChar) + lastChar++ + for lastChar < f.Largest.UserKey[0] { + buf.WriteByte(middleChar) + buf.WriteByte(middleChar) + buf.WriteByte(middleChar) + lastChar++ + } + buf.WriteByte(middleChar) + buf.WriteByte(f.Largest.UserKey[0]) + if j < len(s.Files[i]) - 1 { + buf.WriteByte(' ') + } + lastChar++ + } + fmt.Fprintf(&buf, "\n") + } + buf.WriteString(" ") + for b := byte('a'); b <= 'z'; b++ { + buf.WriteByte(b) + buf.WriteByte(b) + if b < 'z' { + buf.WriteByte(' ') + } + } + buf.WriteByte('\n') + return buf.String() +} + func TestL0SubLevels(t *testing.T) { parseMeta := func(s string) (*FileMetadata, error) { parts := strings.Split(s, ":") @@ -101,6 +210,7 @@ func TestL0SubLevels(t *testing.T) { baseLevel := NumLevels - 1 datadriven.RunTest(t, "testdata/l0_sublevels", func(td *datadriven.TestData) string { + pickBaseCompaction := false switch td.Cmd { case "define": fileMetas = [NumLevels][]*FileMetadata{} @@ -185,6 +295,73 @@ func TestL0SubLevels(t *testing.T) { var builder strings.Builder builder.WriteString(sublevels.describe(true)) + builder.WriteString(visualizeSublevels(sublevels, nil)) + return builder.String() + case "pick-base-compaction": + pickBaseCompaction = true + fallthrough + case "pick-intra-l0-compaction": + minCompactionDepth := 3 + earliestUnflushedSeqNum := uint64(math.MaxUint64) + for _, arg := range td.CmdArgs { + switch arg.Key { + case "min_depth": + minCompactionDepth, err = strconv.Atoi(arg.Vals[0]) + if err != nil { + t.Fatal(err) + } + case "earliest_unflushed_seqnum": + eusnInt, err := strconv.Atoi(arg.Vals[0]) + if err != nil { + t.Fatal(err) + } + earliestUnflushedSeqNum = uint64(eusnInt) + } + } + + var lcf *L0CompactionFiles + if pickBaseCompaction { + lcf, err = sublevels.PickBaseCompaction(minCompactionDepth, fileMetas[baseLevel]) + if err == nil && lcf != nil { + // Try to extend the base compaction into a more rectangular + // shape, using the smallest/largest keys of overlapping + // base files. This mimics the logic the compactor is + // expected to implement. + baseFiles := fileMetas[baseLevel] + firstFile := sort.Search(len(baseFiles), func(i int) bool { + return sublevels.cmp(baseFiles[i].Largest.UserKey, sublevels.orderedIntervals[lcf.minIntervalIndex].startKey.key) >= 0 + }) + lastFile := sort.Search(len(baseFiles), func(i int) bool { + return sublevels.cmp(baseFiles[i].Smallest.UserKey, sublevels.orderedIntervals[lcf.maxIntervalIndex+1].startKey.key) >= 0 + }) + lastFile-- + sublevels.ExtendL0ForBaseCompactionTo( + baseFiles[firstFile].Smallest.UserKey, + baseFiles[lastFile].Largest.UserKey, + lcf) + } + } else { + lcf, err = sublevels.PickIntraL0Compaction(earliestUnflushedSeqNum, minCompactionDepth) + } + if err != nil { + return fmt.Sprintf("error: %s", err.Error()) + } + if lcf == nil { + return "no compaction picked" + } + var builder strings.Builder + builder.WriteString(fmt.Sprintf("compaction picked with stack depth reduction %d\n", lcf.seedIntervalStackDepthReduction)) + for i, file := range lcf.Files { + builder.WriteString(file.FileNum.String()) + if i < len(lcf.Files) - 1 { + builder.WriteByte(',') + } + } + startKey := sublevels.orderedIntervals[lcf.seedInterval].startKey + endKey := sublevels.orderedIntervals[lcf.seedInterval+1].startKey + builder.WriteString(fmt.Sprintf("\nseed interval: %s-%s\n", startKey.key, endKey.key)) + builder.WriteString(visualizeSublevels(sublevels, lcf.FilesIncluded)) + return builder.String() case "read-amp": return strconv.Itoa(sublevels.ReadAmplification()) @@ -208,6 +385,36 @@ func TestL0SubLevels(t *testing.T) { } } return "OK" + case "update-state-for-compaction": + var fileNums []base.FileNum + for _, arg := range td.CmdArgs { + switch arg.Key { + case "files": + for _, val := range arg.Vals { + fileNum, err := strconv.ParseUint(val, 10, 64) + if err != nil { + return err.Error() + } + fileNums = append(fileNums, base.FileNum(fileNum)) + } + } + } + files := make([]*FileMetadata, 0, len(fileNums)) + for _, num := range fileNums { + for _, f := range fileMetas[0] { + if f.FileNum == num { + f.Compacting = true + files = append(files, f) + break + } + } + } + if err := sublevels.UpdateStateForStartedCompaction([][]*FileMetadata{files}, true); err != nil { + return err.Error() + } + return "OK" + case "describe": + return sublevels.describe(true) } return fmt.Sprintf("unrecognized command: %s", td.Cmd) }) @@ -221,10 +428,29 @@ func BenchmarkL0SubLevelsInit(b *testing.B) { b.ResetTimer() for n := 0; n < b.N; n++ { sl, err := NewL0SubLevels(v.Files[0], base.DefaultComparer.Compare, base.DefaultFormatter, 5<<20) - if err != nil { - b.Fatal(err) - } else if sl == nil { + require.NoError(b, err) + if sl == nil { b.Fatal("expected non-nil L0SubLevels to be generated") } } } + +func BenchmarkL0SubLevelsInitAndPick(b *testing.B) { + v, err := readManifest("testdata/MANIFEST_import") + if err != nil { + b.Fatal(err) + } + b.ResetTimer() + for n := 0; n < b.N; n++ { + sl, err := NewL0SubLevels(v.Files[0], base.DefaultComparer.Compare, base.DefaultFormatter, 5<<20) + require.NoError(b, err) + if sl == nil { + b.Fatal("expected non-nil L0SubLevels to be generated") + } + c, err := sl.PickBaseCompaction(2, nil) + require.NoError(b, err) + if c == nil { + b.Fatal("expected non-nil compaction to be generated") + } + } +} diff --git a/internal/manifest/testdata/l0_sublevels b/internal/manifest/testdata/l0_sublevels index eb4b93d502..feacb086c5 100644 --- a/internal/manifest/testdata/l0_sublevels +++ b/internal/manifest/testdata/l0_sublevels @@ -12,7 +12,11 @@ flush split keys(3): [b, e, j] 000007:b#6,1-j#8,1 0.0: file count: 1, bytes: 256, width (mean, max): 1.0, 1, interval range: [3, 3] 000003:e#5,1-j#7,1 -compacting file count: 0, base compacting intervals: +compacting file count: 0, base compacting intervals: none +L0.2: a---b +L0.1: b------------------------j +L0.0: e---------------j + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz define no_initialize L0.2 @@ -30,7 +34,11 @@ flush split keys(0): [] 000003:e#5,1-j#7,1 0.0: file count: 1, bytes: 256, width (mean, max): 1.0, 1, interval range: [0, 0] 000007:b#6,1-j#8,1 -compacting file count: 0, base compacting intervals: +compacting file count: 0, base compacting intervals: none +L0.2: a---b +L0.1: e---------------j +L0.0: b------------------------j + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz l0-check-ordering ---- @@ -50,7 +58,10 @@ flush split keys(0): [] 0.0: file count: 2, bytes: 512, width (mean, max): 1.0, 1, interval range: [0, 0] 000007:b#6,1-j#8,1 000003:e#5,1-j#7,1 -compacting file count: 0, base compacting intervals: +compacting file count: 0, base compacting intervals: none +L0.1: a---b +L0.0: b------------------------j e---j + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz l0-check-ordering ---- @@ -83,14 +94,113 @@ flush split keys(5): [b, d, f, g, h] 000001:a#2,1-b#3,1 000002:c#3,1-d#5,1 000006:f#4,1-g#5,1 -compacting file count: 0, base compacting intervals: +compacting file count: 0, base compacting intervals: none +L0.4: f---g +L0.3: f---------i +L0.2: f------h +L0.1: e---f +L0.0: a---b c---d f---g + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz max-depth-after-ongoing-compactions ---- 5 +pick-base-compaction min_depth=3 +---- +compaction picked with stack depth reduction 5 +000006,000003,000005,000009,000010,000001,000002 +seed interval: f-f +L0.4: f+++g +L0.3: f+++++++++i +L0.2: f++++++h +L0.1: e+++f +L0.0: a+++b c+++d f+++g + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +# SSTables 0001 and 0002 are optional additions to the above compaction, as they +# overlap with base files that overlap with L0 files in the seed interval. +# Marking 0002 as compacting should be enough to exclude both from the +# chosen compaction. + +define +L0 + 0001:a.SET.2-b.SET.3 + 0002:c.SET.3-d.SET.5 intra_l0_compacting + 0003:e.SET.5-f.SET.7 + 0005:f.SET.6-h.SET.9 + 0006:f.SET.4-g.SET.5 + 0009:f.SET.10-i.SET.10 + 0010:f.SET.11-g.SET.11 +L6 + 0007:a.SET.0-f.SET.0 + 0008:g.SET.0-z.SET.0 +---- +file count: 7, sublevels: 5, intervals: 10 +flush split keys(5): [b, d, f, g, h] +0.4: file count: 1, bytes: 256, width (mean, max): 2.0, 2, interval range: [5, 6] + 000010:f#11,1-g#11,1 +0.3: file count: 1, bytes: 256, width (mean, max): 4.0, 4, interval range: [5, 8] + 000009:f#10,1-i#10,1 +0.2: file count: 1, bytes: 256, width (mean, max): 3.0, 3, interval range: [5, 7] + 000005:f#6,1-h#9,1 +0.1: file count: 1, bytes: 256, width (mean, max): 2.0, 2, interval range: [4, 5] + 000003:e#5,1-f#7,1 +0.0: file count: 3, bytes: 768, width (mean, max): 1.3, 2, interval range: [0, 6] + 000001:a#2,1-b#3,1 + 000002:c#3,1-d#5,1 + 000006:f#4,1-g#5,1 +compacting file count: 1, base compacting intervals: none +L0.4: f---g +L0.3: f---------i +L0.2: f------h +L0.1: e---f +L0.0: a---b c---d f---g + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + + +pick-base-compaction min_depth=3 +---- +compaction picked with stack depth reduction 5 +000006,000003,000005,000009,000010 +seed interval: f-f +L0.4: f+++g +L0.3: f+++++++++i +L0.2: f++++++h +L0.1: e+++f +L0.0: a---b c---d f+++g + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +# Mark the above compaction as started. + +update-state-for-compaction files=(000006,000003,000005,000009,000010) +---- +OK + +describe +---- +file count: 7, sublevels: 5, intervals: 10 +flush split keys(5): [b, d, f, g, h] +0.4: file count: 1, bytes: 256, width (mean, max): 2.0, 2, interval range: [5, 6] + 000010:f#11,1-g#11,1 +0.3: file count: 1, bytes: 256, width (mean, max): 4.0, 4, interval range: [5, 8] + 000009:f#10,1-i#10,1 +0.2: file count: 1, bytes: 256, width (mean, max): 3.0, 3, interval range: [5, 7] + 000005:f#6,1-h#9,1 +0.1: file count: 1, bytes: 256, width (mean, max): 2.0, 2, interval range: [4, 5] + 000003:e#5,1-f#7,1 +0.0: file count: 3, bytes: 768, width (mean, max): 1.3, 2, interval range: [0, 6] + 000001:a#2,1-b#3,1 + 000002:c#3,1-d#5,1 + 000006:f#4,1-g#5,1 +compacting file count: 6, base compacting intervals: [4, 9] + +pick-base-compaction min_depth=3 +---- +no compaction picked + # Extend one of the SSTables (0009) to the right, and place an SSTable "under" -# the extension (0011). +# the extension (0011). This adds it to the compaction. define L0 @@ -114,10 +224,131 @@ flush split keys(3): [g, h, p] 0.0: file count: 2, bytes: 512, width (mean, max): 1.0, 1, interval range: [0, 3] 000006:f#4,1-g#5,1 000011:n#8,1-p#10,1 -compacting file count: 0, base compacting intervals: +compacting file count: 0, base compacting intervals: none +L0.3: f---g +L0.2: f------------------------------p +L0.1: f------h +L0.0: f---g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-base-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000006,000005,000009,000011,000010 +seed interval: f-g +L0.3: f+++g +L0.2: f++++++++++++++++++++++++++++++p +L0.1: f++++++h +L0.0: f+++g n++++++p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +# Set SSTable 0011 which is "under" SSTable 0009 to IsBaseCompacting = true. +# This should prevent SSTable 0009 from participating in a base compaction. + +define +L0 + 0005:f.SET.6-h.SET.9 + 0006:f.SET.4-g.SET.5 + 0009:f.SET.10-p.SET.10 + 0010:f.SET.11-g.SET.11 + 0011:n.SET.8-p.SET.10 base_compacting +L6 + 0007:a.SET.0-f.SET.0 + 0008:g.SET.0-z.SET.0 +---- +file count: 5, sublevels: 4, intervals: 5 +flush split keys(3): [g, h, p] +0.3: file count: 1, bytes: 256, width (mean, max): 1.0, 1, interval range: [0, 0] + 000010:f#11,1-g#11,1 +0.2: file count: 1, bytes: 256, width (mean, max): 4.0, 4, interval range: [0, 3] + 000009:f#10,1-p#10,1 +0.1: file count: 1, bytes: 256, width (mean, max): 2.0, 2, interval range: [0, 1] + 000005:f#6,1-h#9,1 +0.0: file count: 2, bytes: 512, width (mean, max): 1.0, 1, interval range: [0, 3] + 000006:f#4,1-g#5,1 + 000011:n#8,1-p#10,1 +compacting file count: 1, base compacting intervals: [3, 4] +L0.3: f---g +L0.2: f------------------------------p +L0.1: f------h +L0.0: f---g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz -# Assume a base compaction from the above files is chosen. This should reduce -# max-depth-after-ongoing-compactions. +pick-base-compaction min_depth=3 +---- +no compaction picked + +pick-intra-l0-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000010,000009,000005,000006 +seed interval: f-g +L0.3: f+++g +L0.2: f++++++++++++++++++++++++++++++p +L0.1: f++++++h +L0.0: f+++g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +# Raise 00009 to a higher level, so that there's still a stack depth of 3 below +# it. This should make f-g a candidate for base compaction again. + +define +L0 + 0005:f.SET.6-h.SET.9 + 0006:f.SET.4-g.SET.5 + 0009:f.SET.12-p.SET.12 + 0010:f.SET.11-g.SET.11 + 0011:n.SET.8-p.SET.10 base_compacting +L6 + 0007:a.SET.0-f.SET.0 + 0008:g.SET.0-z.SET.0 +---- +file count: 5, sublevels: 4, intervals: 5 +flush split keys(3): [g, h, p] +0.3: file count: 1, bytes: 256, width (mean, max): 4.0, 4, interval range: [0, 3] + 000009:f#12,1-p#12,1 +0.2: file count: 1, bytes: 256, width (mean, max): 1.0, 1, interval range: [0, 0] + 000010:f#11,1-g#11,1 +0.1: file count: 1, bytes: 256, width (mean, max): 2.0, 2, interval range: [0, 1] + 000005:f#6,1-h#9,1 +0.0: file count: 2, bytes: 512, width (mean, max): 1.0, 1, interval range: [0, 3] + 000006:f#4,1-g#5,1 + 000011:n#8,1-p#10,1 +compacting file count: 1, base compacting intervals: [3, 4] +L0.3: f------------------------------p +L0.2: f---g +L0.1: f------h +L0.0: f---g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-base-compaction min_depth=3 +---- +compaction picked with stack depth reduction 3 +000006,000005,000010 +seed interval: f-g +L0.3: f------------------------------p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-intra-l0-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000009,000010,000005,000006 +seed interval: f-g +L0.3: f++++++++++++++++++++++++++++++p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +max-depth-after-ongoing-compactions +---- +4 + +# Assume the above base compaction is chosen. This should reduce max depth after +# ongoing compactions. define L0 @@ -141,16 +372,383 @@ flush split keys(3): [g, h, p] 0.0: file count: 2, bytes: 512, width (mean, max): 1.0, 1, interval range: [0, 3] 000006:f#4,1-g#5,1 000011:n#8,1-p#10,1 -compacting file count: 4, base compacting intervals: [0, 1], [3, 4], +compacting file count: 4, base compacting intervals: [0, 1], [3, 4] +L0.3: f------------------------------p +L0.2: f---g +L0.1: f------h +L0.0: f---g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-base-compaction min_depth=3 +---- +no compaction picked + +pick-intra-l0-compaction min_depth=3 +---- +no compaction picked max-depth-after-ongoing-compactions ---- 1 +# Ensure that when 0011 is not base compacting, it's chosen for compactions +# along with 0009. + +define +L0 + 0005:f.SET.6-h.SET.9 + 0006:f.SET.4-g.SET.5 + 0009:f.SET.12-p.SET.12 + 0010:f.SET.11-g.SET.11 + 0011:n.SET.8-p.SET.10 +L6 + 0007:a.SET.0-f.SET.0 + 0008:g.SET.0-z.SET.0 +---- +file count: 5, sublevels: 4, intervals: 5 +flush split keys(3): [g, h, p] +0.3: file count: 1, bytes: 256, width (mean, max): 4.0, 4, interval range: [0, 3] + 000009:f#12,1-p#12,1 +0.2: file count: 1, bytes: 256, width (mean, max): 1.0, 1, interval range: [0, 0] + 000010:f#11,1-g#11,1 +0.1: file count: 1, bytes: 256, width (mean, max): 2.0, 2, interval range: [0, 1] + 000005:f#6,1-h#9,1 +0.0: file count: 2, bytes: 512, width (mean, max): 1.0, 1, interval range: [0, 3] + 000006:f#4,1-g#5,1 + 000011:n#8,1-p#10,1 +compacting file count: 0, base compacting intervals: none +L0.3: f------------------------------p +L0.2: f---g +L0.1: f------h +L0.0: f---g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-base-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000006,000005,000010,000009,000011 +seed interval: f-g +L0.3: f++++++++++++++++++++++++++++++p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g n++++++p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-intra-l0-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000009,000010,000005,000006,000011 +seed interval: f-g +L0.3: f++++++++++++++++++++++++++++++p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g n++++++p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +# Don't pick a base compaction if the overlapping Lbase files are marked as +# compacting. + +define +L0 + 0005:f.SET.6-h.SET.9 + 0006:f.SET.4-g.SET.5 + 0009:f.SET.12-p.SET.12 + 0010:f.SET.11-g.SET.11 + 0011:n.SET.8-p.SET.10 +L6 + 0007:a.SET.0-f.SET.0 + 0008:g.SET.0-z.SET.0 compacting +---- +file count: 5, sublevels: 4, intervals: 5 +flush split keys(3): [g, h, p] +0.3: file count: 1, bytes: 256, width (mean, max): 4.0, 4, interval range: [0, 3] + 000009:f#12,1-p#12,1 +0.2: file count: 1, bytes: 256, width (mean, max): 1.0, 1, interval range: [0, 0] + 000010:f#11,1-g#11,1 +0.1: file count: 1, bytes: 256, width (mean, max): 2.0, 2, interval range: [0, 1] + 000005:f#6,1-h#9,1 +0.0: file count: 2, bytes: 512, width (mean, max): 1.0, 1, interval range: [0, 3] + 000006:f#4,1-g#5,1 + 000011:n#8,1-p#10,1 +compacting file count: 0, base compacting intervals: none +L0.3: f------------------------------p +L0.2: f---g +L0.1: f------h +L0.0: f---g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-base-compaction min_depth=3 +---- +no compaction picked + +# Greatly increase the size of SSTable 0009, past 100 << 20. This should make +# it no longer a candidate for base compaction. + +define +L0 + 0005:f.SET.6-h.SET.9 + 0006:f.SET.4-g.SET.5 + 0009:f.SET.12-p.SET.12 size=104859600 + 0010:f.SET.11-g.SET.11 + 0011:n.SET.8-p.SET.10 +L6 + 0007:a.SET.0-f.SET.0 + 0008:g.SET.0-z.SET.0 +---- +file count: 5, sublevels: 4, intervals: 5 +flush split keys(4): [g, h, n, p] +0.3: file count: 1, bytes: 104859600, width (mean, max): 4.0, 4, interval range: [0, 3] + 000009:f#12,1-p#12,1 +0.2: file count: 1, bytes: 256, width (mean, max): 1.0, 1, interval range: [0, 0] + 000010:f#11,1-g#11,1 +0.1: file count: 1, bytes: 256, width (mean, max): 2.0, 2, interval range: [0, 1] + 000005:f#6,1-h#9,1 +0.0: file count: 2, bytes: 512, width (mean, max): 1.0, 1, interval range: [0, 3] + 000006:f#4,1-g#5,1 + 000011:n#8,1-p#10,1 +compacting file count: 0, base compacting intervals: none +L0.3: f------------------------------p +L0.2: f---g +L0.1: f------h +L0.0: f---g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-base-compaction min_depth=3 +---- +compaction picked with stack depth reduction 3 +000006,000005,000010,000011 +seed interval: f-g +L0.3: f------------------------------p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g n++++++p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-intra-l0-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000009,000010,000005,000006,000011 +seed interval: f-g +L0.3: f++++++++++++++++++++++++++++++p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g n++++++p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +# However, when the size increase is applied to a lower sublevel that is +# necessary to include to meet the minimum stack depth reduction, we overlook +# the size difference and choose the file for compaction anyway. + +define +L0 + 0005:f.SET.6-h.SET.9 + 0006:f.SET.4-g.SET.5 + 0009:f.SET.12-p.SET.12 + 0010:f.SET.11-g.SET.11 size=104859600 + 0011:n.SET.8-p.SET.10 +L6 + 0007:a.SET.0-f.SET.0 + 0008:g.SET.0-z.SET.0 +---- +file count: 5, sublevels: 4, intervals: 5 +flush split keys(3): [g, h, p] +0.3: file count: 1, bytes: 256, width (mean, max): 4.0, 4, interval range: [0, 3] + 000009:f#12,1-p#12,1 +0.2: file count: 1, bytes: 104859600, width (mean, max): 1.0, 1, interval range: [0, 0] + 000010:f#11,1-g#11,1 +0.1: file count: 1, bytes: 256, width (mean, max): 2.0, 2, interval range: [0, 1] + 000005:f#6,1-h#9,1 +0.0: file count: 2, bytes: 512, width (mean, max): 1.0, 1, interval range: [0, 3] + 000006:f#4,1-g#5,1 + 000011:n#8,1-p#10,1 +compacting file count: 0, base compacting intervals: none +L0.3: f------------------------------p +L0.2: f---g +L0.1: f------h +L0.0: f---g n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-base-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000006,000005,000010,000009,000011 +seed interval: f-g +L0.3: f++++++++++++++++++++++++++++++p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g n++++++p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-intra-l0-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000009,000010,000005,000006,000011 +seed interval: f-g +L0.3: f++++++++++++++++++++++++++++++p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g n++++++p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + read-amp ---- 4 +# In L0.0, SST 0007 is marked as base compacting. There are two SSTs to the left +# of it in the sublevel, and one to its right. The ones to its left should be +# chosen by extendCandidateToRectangle. + +define +L0 + 0004:h.SET.2-j.SET.4 + 0005:f.SET.6-h.SET.9 + 0006:f.SET.4-g.SET.5 + 0007:k.SET.2-l.SET.4 base_compacting + 0009:f.SET.12-p.SET.12 + 0010:f.SET.11-g.SET.11 + 0011:n.SET.8-p.SET.10 +L6 + 0007:a.SET.0-f.SET.0 + 0008:g.SET.0-z.SET.0 +---- +file count: 7, sublevels: 4, intervals: 9 +flush split keys(5): [g, h, j, l, p] +0.3: file count: 1, bytes: 256, width (mean, max): 8.0, 8, interval range: [0, 7] + 000009:f#12,1-p#12,1 +0.2: file count: 1, bytes: 256, width (mean, max): 1.0, 1, interval range: [0, 0] + 000010:f#11,1-g#11,1 +0.1: file count: 1, bytes: 256, width (mean, max): 3.0, 3, interval range: [0, 2] + 000005:f#6,1-h#9,1 +0.0: file count: 4, bytes: 1024, width (mean, max): 1.2, 2, interval range: [0, 7] + 000006:f#4,1-g#5,1 + 000004:h#2,1-j#4,1 + 000007:k#2,1-l#4,1 + 000011:n#8,1-p#10,1 +compacting file count: 1, base compacting intervals: [5, 5] +L0.3: f------------------------------p +L0.2: f---g +L0.1: f------h +L0.0: f---g h------j k---l n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-intra-l0-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000009,000010,000005,000006,000004 +seed interval: f-g +L0.3: f++++++++++++++++++++++++++++++p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g h++++++j k---l n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-base-compaction min_depth=3 +---- +compaction picked with stack depth reduction 3 +000006,000005,000004,000010 +seed interval: f-g +L0.3: f------------------------------p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g h++++++j k---l n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + + +# Now shift the base_compacting marker one SST to the left. But since file 6 +# was already chosen as part of the seed compaction construction, we still +# prefer to choose it over files 7 and 11. + +define +L0 + 0004:h.SET.2-j.SET.4 base_compacting + 0005:f.SET.6-h.SET.9 + 0006:f.SET.4-g.SET.5 + 0007:k.SET.2-l.SET.4 + 0009:f.SET.12-p.SET.12 + 0010:f.SET.11-g.SET.11 + 0011:n.SET.8-p.SET.10 +L6 + 0007:a.SET.0-f.SET.0 + 0008:g.SET.0-z.SET.0 +---- +file count: 7, sublevels: 4, intervals: 9 +flush split keys(5): [g, h, j, l, p] +0.3: file count: 1, bytes: 256, width (mean, max): 8.0, 8, interval range: [0, 7] + 000009:f#12,1-p#12,1 +0.2: file count: 1, bytes: 256, width (mean, max): 1.0, 1, interval range: [0, 0] + 000010:f#11,1-g#11,1 +0.1: file count: 1, bytes: 256, width (mean, max): 3.0, 3, interval range: [0, 2] + 000005:f#6,1-h#9,1 +0.0: file count: 4, bytes: 1024, width (mean, max): 1.2, 2, interval range: [0, 7] + 000006:f#4,1-g#5,1 + 000004:h#2,1-j#4,1 + 000007:k#2,1-l#4,1 + 000011:n#8,1-p#10,1 +compacting file count: 1, base compacting intervals: [2, 3] +L0.3: f------------------------------p +L0.2: f---g +L0.1: f------h +L0.0: f---g h------j k---l n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-intra-l0-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000009,000010,000005,000006 +seed interval: f-g +L0.3: f++++++++++++++++++++++++++++++p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g h------j k---l n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +# Without any base_compacting markers, all SSTs in the bottom sublevel should +# be chosen for an intra-L0 compaction. + +define +L0 + 0004:h.SET.2-j.SET.4 + 0005:f.SET.6-h.SET.9 + 0006:f.SET.4-g.SET.5 + 0007:k.SET.2-l.SET.4 + 0009:f.SET.12-p.SET.12 + 0010:f.SET.11-g.SET.11 + 0011:n.SET.8-p.SET.10 +L6 + 0007:a.SET.0-f.SET.0 + 0008:g.SET.0-z.SET.0 +---- +file count: 7, sublevels: 4, intervals: 9 +flush split keys(5): [g, h, j, l, p] +0.3: file count: 1, bytes: 256, width (mean, max): 8.0, 8, interval range: [0, 7] + 000009:f#12,1-p#12,1 +0.2: file count: 1, bytes: 256, width (mean, max): 1.0, 1, interval range: [0, 0] + 000010:f#11,1-g#11,1 +0.1: file count: 1, bytes: 256, width (mean, max): 3.0, 3, interval range: [0, 2] + 000005:f#6,1-h#9,1 +0.0: file count: 4, bytes: 1024, width (mean, max): 1.2, 2, interval range: [0, 7] + 000006:f#4,1-g#5,1 + 000004:h#2,1-j#4,1 + 000007:k#2,1-l#4,1 + 000011:n#8,1-p#10,1 +compacting file count: 0, base compacting intervals: none +L0.3: f------------------------------p +L0.2: f---g +L0.1: f------h +L0.0: f---g h------j k---l n------p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + +pick-intra-l0-compaction min_depth=3 +---- +compaction picked with stack depth reduction 4 +000009,000010,000005,000006,000004,000007,000011 +seed interval: f-g +L0.3: f++++++++++++++++++++++++++++++p +L0.2: f+++g +L0.1: f++++++h +L0.0: f+++g h++++++j k+++l n++++++p + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz + define flush_split_max_bytes=32 L0 0001:a.SET.2-e.SET.5 size=64 @@ -168,7 +766,11 @@ flush split keys(1): [e] 000002:c#6,1-g#8,1 0.0: file count: 1, bytes: 64, width (mean, max): 2.0, 2, interval range: [0, 1] 000001:a#2,1-e#5,1 -compacting file count: 0, base compacting intervals: +compacting file count: 0, base compacting intervals: none +L0.2: f------------j +L0.1: c------------g +L0.0: a------------e + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz # Check that read amplification is the sublevel height of the tallest key # interval, not the overall count of sublevels. @@ -204,7 +806,11 @@ flush split keys(2): [c, e] 000002:c#6,1-g#8,1 0.0: file count: 1, bytes: 64, width (mean, max): 2.0, 2, interval range: [0, 1] 000001:a#2,1-e#5,1 -compacting file count: 0, base compacting intervals: +compacting file count: 0, base compacting intervals: none +L0.2: f------------j +L0.1: c------------g +L0.0: a------------e + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz flush-split-keys ---- @@ -229,7 +835,11 @@ flush split keys(2): [c, e] 000002:c#6,1-g#8,1 0.0: file count: 1, bytes: 64, width (mean, max): 2.0, 2, interval range: [0, 1] 000001:a#2,1-e#5,1 -compacting file count: 0, base compacting intervals: +compacting file count: 0, base compacting intervals: none +L0.2: f------------j +L0.1: c------------g +L0.0: a------------e + aa bb cc dd ee ff gg hh ii jj kk ll mm nn oo pp qq rr ss tt uu vv ww xx yy zz flush-split-keys ----