github.com/cockroachdb/pebble@v1.1.2/compaction.go (about)

     1  // Copyright 2013 The LevelDB-Go and Pebble Authors. All rights reserved. Use
     2  // of this source code is governed by a BSD-style license that can be found in
     3  // the LICENSE file.
     4  
     5  package pebble
     6  
     7  import (
     8  	"bytes"
     9  	"context"
    10  	"fmt"
    11  	"io"
    12  	"math"
    13  	"runtime/pprof"
    14  	"sort"
    15  	"sync/atomic"
    16  	"time"
    17  
    18  	"github.com/cockroachdb/errors"
    19  	"github.com/cockroachdb/pebble/internal/base"
    20  	"github.com/cockroachdb/pebble/internal/invalidating"
    21  	"github.com/cockroachdb/pebble/internal/invariants"
    22  	"github.com/cockroachdb/pebble/internal/keyspan"
    23  	"github.com/cockroachdb/pebble/internal/manifest"
    24  	"github.com/cockroachdb/pebble/internal/private"
    25  	"github.com/cockroachdb/pebble/internal/rangedel"
    26  	"github.com/cockroachdb/pebble/internal/rangekey"
    27  	"github.com/cockroachdb/pebble/objstorage"
    28  	"github.com/cockroachdb/pebble/objstorage/objstorageprovider/objiotracing"
    29  	"github.com/cockroachdb/pebble/objstorage/remote"
    30  	"github.com/cockroachdb/pebble/sstable"
    31  	"github.com/cockroachdb/pebble/vfs"
    32  	"golang.org/x/exp/constraints"
    33  )
    34  
    35  var errEmptyTable = errors.New("pebble: empty table")
    36  
    37  // ErrCancelledCompaction is returned if a compaction is cancelled by a
    38  // concurrent excise or ingest-split operation.
    39  var ErrCancelledCompaction = errors.New("pebble: compaction cancelled by a concurrent operation, will retry compaction")
    40  
    41  var compactLabels = pprof.Labels("pebble", "compact")
    42  var flushLabels = pprof.Labels("pebble", "flush")
    43  var gcLabels = pprof.Labels("pebble", "gc")
    44  
    45  // getInternalWriterProperties accesses a private variable (in the
    46  // internal/private package) initialized by the sstable Writer. This indirection
    47  // is necessary to ensure non-Pebble users constructing sstables for ingestion
    48  // are unable to set internal-only properties.
    49  var getInternalWriterProperties = private.SSTableInternalProperties.(func(*sstable.Writer) *sstable.Properties)
    50  
    51  // expandedCompactionByteSizeLimit is the maximum number of bytes in all
    52  // compacted files. We avoid expanding the lower level file set of a compaction
    53  // if it would make the total compaction cover more than this many bytes.
    54  func expandedCompactionByteSizeLimit(opts *Options, level int, availBytes uint64) uint64 {
    55  	v := uint64(25 * opts.Level(level).TargetFileSize)
    56  
    57  	// Never expand a compaction beyond half the available capacity, divided
    58  	// by the maximum number of concurrent compactions. Each of the concurrent
    59  	// compactions may expand up to this limit, so this attempts to limit
    60  	// compactions to half of available disk space. Note that this will not
    61  	// prevent compaction picking from pursuing compactions that are larger
    62  	// than this threshold before expansion.
    63  	diskMax := (availBytes / 2) / uint64(opts.MaxConcurrentCompactions())
    64  	if v > diskMax {
    65  		v = diskMax
    66  	}
    67  	return v
    68  }
    69  
    70  // maxGrandparentOverlapBytes is the maximum bytes of overlap with level+1
    71  // before we stop building a single file in a level-1 to level compaction.
    72  func maxGrandparentOverlapBytes(opts *Options, level int) uint64 {
    73  	return uint64(10 * opts.Level(level).TargetFileSize)
    74  }
    75  
    76  // maxReadCompactionBytes is used to prevent read compactions which
    77  // are too wide.
    78  func maxReadCompactionBytes(opts *Options, level int) uint64 {
    79  	return uint64(10 * opts.Level(level).TargetFileSize)
    80  }
    81  
    82  // noCloseIter wraps around a FragmentIterator, intercepting and eliding
    83  // calls to Close. It is used during compaction to ensure that rangeDelIters
    84  // are not closed prematurely.
    85  type noCloseIter struct {
    86  	keyspan.FragmentIterator
    87  }
    88  
    89  func (i noCloseIter) Close() error {
    90  	return nil
    91  }
    92  
    93  type compactionLevel struct {
    94  	level int
    95  	files manifest.LevelSlice
    96  	// l0SublevelInfo contains information about L0 sublevels being compacted.
    97  	// It's only set for the start level of a compaction starting out of L0 and
    98  	// is nil for all other compactions.
    99  	l0SublevelInfo []sublevelInfo
   100  }
   101  
   102  func (cl compactionLevel) Clone() compactionLevel {
   103  	newCL := compactionLevel{
   104  		level: cl.level,
   105  		files: cl.files.Reslice(func(start, end *manifest.LevelIterator) {}),
   106  	}
   107  	return newCL
   108  }
   109  func (cl compactionLevel) String() string {
   110  	return fmt.Sprintf(`Level %d, Files %s`, cl.level, cl.files)
   111  }
   112  
   113  // Return output from compactionOutputSplitters. See comment on
   114  // compactionOutputSplitter.shouldSplitBefore() on how this value is used.
   115  type maybeSplit int
   116  
   117  const (
   118  	noSplit maybeSplit = iota
   119  	splitNow
   120  )
   121  
   122  // String implements the Stringer interface.
   123  func (c maybeSplit) String() string {
   124  	if c == noSplit {
   125  		return "no-split"
   126  	}
   127  	return "split-now"
   128  }
   129  
   130  // compactionOutputSplitter is an interface for encapsulating logic around
   131  // switching the output of a compaction to a new output file. Additional
   132  // constraints around switching compaction outputs that are specific to that
   133  // compaction type (eg. flush splits) are implemented in
   134  // compactionOutputSplitters that compose other child compactionOutputSplitters.
   135  type compactionOutputSplitter interface {
   136  	// shouldSplitBefore returns whether we should split outputs before the
   137  	// specified "current key". The return value is splitNow or noSplit.
   138  	// splitNow means a split is advised before the specified key, and noSplit
   139  	// means no split is advised. If shouldSplitBefore(a) advises a split then
   140  	// shouldSplitBefore(b) should also advise a split given b >= a, until
   141  	// onNewOutput is called.
   142  	shouldSplitBefore(key *InternalKey, tw *sstable.Writer) maybeSplit
   143  	// onNewOutput updates internal splitter state when the compaction switches
   144  	// to a new sstable, and returns the next limit for the new output which
   145  	// would get used to truncate range tombstones if the compaction iterator
   146  	// runs out of keys. The limit returned MUST be > key according to the
   147  	// compaction's comparator. The specified key is the first key in the new
   148  	// output, or nil if this sstable will only contain range tombstones already
   149  	// in the fragmenter.
   150  	onNewOutput(key []byte) []byte
   151  }
   152  
   153  // fileSizeSplitter is a compactionOutputSplitter that enforces target file
   154  // sizes. This splitter splits to a new output file when the estimated file size
   155  // is 0.5x-2x the target file size. If there are overlapping grandparent files,
   156  // this splitter will attempt to split at a grandparent boundary. For example,
   157  // consider the example where a compaction wrote 'd' to the current output file,
   158  // and the next key has a user key 'g':
   159  //
   160  //	                              previous key   next key
   161  //		                                 |           |
   162  //		                                 |           |
   163  //		                 +---------------|----+   +--|----------+
   164  //		  grandparents:  |       000006  |    |   |  | 000007   |
   165  //		                 +---------------|----+   +--|----------+
   166  //		                 a    b          d    e   f  g       i
   167  //
   168  // Splitting the output file F before 'g' will ensure that the current output
   169  // file F does not overlap the grandparent file 000007. Aligning sstable
   170  // boundaries like this can significantly reduce write amplification, since a
   171  // subsequent compaction of F into the grandparent level will avoid needlessly
   172  // rewriting any keys within 000007 that do not overlap F's bounds. Consider the
   173  // following compaction:
   174  //
   175  //	                       +----------------------+
   176  //		  input            |                      |
   177  //		  level            +----------------------+
   178  //		                              \/
   179  //		           +---------------+       +---------------+
   180  //		  output   |XXXXXXX|       |       |      |XXXXXXXX|
   181  //		  level    +---------------+       +---------------+
   182  //
   183  // The input-level file overlaps two files in the output level, but only
   184  // partially. The beginning of the first output-level file and the end of the
   185  // second output-level file will be rewritten verbatim. This write I/O is
   186  // "wasted" in the sense that no merging is being performed.
   187  //
   188  // To prevent the above waste, this splitter attempts to split output files
   189  // before the start key of grandparent files. It still strives to write output
   190  // files of approximately the target file size, by constraining this splitting
   191  // at grandparent points to apply only if the current output's file size is
   192  // about the right order of magnitude.
   193  //
   194  // Note that, unlike most other splitters, this splitter does not guarantee that
   195  // it will advise splits only at user key change boundaries.
   196  type fileSizeSplitter struct {
   197  	frontier              frontier
   198  	targetFileSize        uint64
   199  	atGrandparentBoundary bool
   200  	boundariesObserved    uint64
   201  	nextGrandparent       *fileMetadata
   202  	grandparents          manifest.LevelIterator
   203  }
   204  
   205  func newFileSizeSplitter(
   206  	f *frontiers, targetFileSize uint64, grandparents manifest.LevelIterator,
   207  ) *fileSizeSplitter {
   208  	s := &fileSizeSplitter{targetFileSize: targetFileSize}
   209  	s.nextGrandparent = grandparents.First()
   210  	s.grandparents = grandparents
   211  	if s.nextGrandparent != nil {
   212  		s.frontier.Init(f, s.nextGrandparent.Smallest.UserKey, s.reached)
   213  	}
   214  	return s
   215  }
   216  
   217  func (f *fileSizeSplitter) reached(nextKey []byte) []byte {
   218  	f.atGrandparentBoundary = true
   219  	f.boundariesObserved++
   220  	// NB: f.grandparents is a bounded iterator, constrained to the compaction
   221  	// key range.
   222  	f.nextGrandparent = f.grandparents.Next()
   223  	if f.nextGrandparent == nil {
   224  		return nil
   225  	}
   226  	// TODO(jackson): Should we also split before or immediately after
   227  	// grandparents' largest keys? Splitting before the start boundary prevents
   228  	// overlap with the grandparent. Also splitting after the end boundary may
   229  	// increase the probability of move compactions.
   230  	return f.nextGrandparent.Smallest.UserKey
   231  }
   232  
   233  func (f *fileSizeSplitter) shouldSplitBefore(key *InternalKey, tw *sstable.Writer) maybeSplit {
   234  	atGrandparentBoundary := f.atGrandparentBoundary
   235  
   236  	// Clear f.atGrandparentBoundary unconditionally.
   237  	//
   238  	// This is a bit subtle. Even if do decide to split, it's possible that a
   239  	// higher-level splitter will ignore our request (eg, because we're between
   240  	// two internal keys with the same user key). In this case, the next call to
   241  	// shouldSplitBefore will find atGrandparentBoundary=false. This is
   242  	// desirable, because in this case we would've already written the earlier
   243  	// key with the same user key to the output file. The current output file is
   244  	// already doomed to overlap the grandparent whose bound triggered
   245  	// atGrandparentBoundary=true. We should continue on, waiting for the next
   246  	// grandparent boundary.
   247  	f.atGrandparentBoundary = false
   248  
   249  	// If the key is a range tombstone, the EstimatedSize may not grow right
   250  	// away when a range tombstone is added to the fragmenter: It's dependent on
   251  	// whether or not the this new range deletion will start a new fragment.
   252  	// Range deletions are rare, so we choose to simply not split yet.
   253  	// TODO(jackson): Reconsider this, and consider range keys too as a part of
   254  	// #2321.
   255  	if key.Kind() == InternalKeyKindRangeDelete || tw == nil {
   256  		return noSplit
   257  	}
   258  
   259  	estSize := tw.EstimatedSize()
   260  	switch {
   261  	case estSize < f.targetFileSize/2:
   262  		// The estimated file size is less than half the target file size. Don't
   263  		// split it, even if currently aligned with a grandparent file because
   264  		// it's too small.
   265  		return noSplit
   266  	case estSize >= 2*f.targetFileSize:
   267  		// The estimated file size is double the target file size. Split it even
   268  		// if we were not aligned with a grandparent file boundary to avoid
   269  		// excessively exceeding the target file size.
   270  		return splitNow
   271  	case !atGrandparentBoundary:
   272  		// Don't split if we're not at a grandparent, except if we've exhausted
   273  		// all the grandparents overlapping this compaction's key range. Then we
   274  		// may want to split purely based on file size.
   275  		if f.nextGrandparent == nil {
   276  			// There are no more grandparents. Optimize for the target file size
   277  			// and split as soon as we hit the target file size.
   278  			if estSize >= f.targetFileSize {
   279  				return splitNow
   280  			}
   281  		}
   282  		return noSplit
   283  	default:
   284  		// INVARIANT: atGrandparentBoundary
   285  		// INVARIANT: targetSize/2 < estSize < 2*targetSize
   286  		//
   287  		// The estimated file size is close enough to the target file size that
   288  		// we should consider splitting.
   289  		//
   290  		// Determine whether to split now based on how many grandparent
   291  		// boundaries we have already observed while building this output file.
   292  		// The intuition here is that if the grandparent level is dense in this
   293  		// part of the keyspace, we're likely to continue to have more
   294  		// opportunities to split this file aligned with a grandparent. If this
   295  		// is the first grandparent boundary observed, we split immediately
   296  		// (we're already at ≥50% the target file size). Otherwise, each
   297  		// overlapping grandparent we've observed increases the minimum file
   298  		// size by 5% of the target file size, up to at most 90% of the target
   299  		// file size.
   300  		//
   301  		// TODO(jackson): The particular thresholds are somewhat unprincipled.
   302  		// This is the same heuristic as RocksDB implements. Is there are more
   303  		// principled formulation that can, further reduce w-amp, produce files
   304  		// closer to the target file size, or is more understandable?
   305  
   306  		// NB: Subtract 1 from `boundariesObserved` to account for the current
   307  		// boundary we're considering splitting at. `reached` will have
   308  		// incremented it at the same time it set `atGrandparentBoundary`.
   309  		minimumPctOfTargetSize := 50 + 5*minUint64(f.boundariesObserved-1, 8)
   310  		if estSize < (minimumPctOfTargetSize*f.targetFileSize)/100 {
   311  			return noSplit
   312  		}
   313  		return splitNow
   314  	}
   315  }
   316  
   317  func minUint64(a, b uint64) uint64 {
   318  	if b < a {
   319  		a = b
   320  	}
   321  	return a
   322  }
   323  
   324  func (f *fileSizeSplitter) onNewOutput(key []byte) []byte {
   325  	f.boundariesObserved = 0
   326  	return nil
   327  }
   328  
   329  func newLimitFuncSplitter(f *frontiers, limitFunc func(userKey []byte) []byte) *limitFuncSplitter {
   330  	s := &limitFuncSplitter{limitFunc: limitFunc}
   331  	s.frontier.Init(f, nil, s.reached)
   332  	return s
   333  }
   334  
   335  type limitFuncSplitter struct {
   336  	frontier  frontier
   337  	limitFunc func(userKey []byte) []byte
   338  	split     maybeSplit
   339  }
   340  
   341  func (lf *limitFuncSplitter) shouldSplitBefore(key *InternalKey, tw *sstable.Writer) maybeSplit {
   342  	return lf.split
   343  }
   344  
   345  func (lf *limitFuncSplitter) reached(nextKey []byte) []byte {
   346  	lf.split = splitNow
   347  	return nil
   348  }
   349  
   350  func (lf *limitFuncSplitter) onNewOutput(key []byte) []byte {
   351  	lf.split = noSplit
   352  	if key != nil {
   353  		// TODO(jackson): For some users, like L0 flush splits, there's no need
   354  		// to binary search over all the flush splits every time. The next split
   355  		// point must be ahead of the previous flush split point.
   356  		limit := lf.limitFunc(key)
   357  		lf.frontier.Update(limit)
   358  		return limit
   359  	}
   360  	lf.frontier.Update(nil)
   361  	return nil
   362  }
   363  
   364  // splitterGroup is a compactionOutputSplitter that splits whenever one of its
   365  // child splitters advises a compaction split.
   366  type splitterGroup struct {
   367  	cmp       Compare
   368  	splitters []compactionOutputSplitter
   369  }
   370  
   371  func (a *splitterGroup) shouldSplitBefore(
   372  	key *InternalKey, tw *sstable.Writer,
   373  ) (suggestion maybeSplit) {
   374  	for _, splitter := range a.splitters {
   375  		if splitter.shouldSplitBefore(key, tw) == splitNow {
   376  			return splitNow
   377  		}
   378  	}
   379  	return noSplit
   380  }
   381  
   382  func (a *splitterGroup) onNewOutput(key []byte) []byte {
   383  	var earliestLimit []byte
   384  	for _, splitter := range a.splitters {
   385  		limit := splitter.onNewOutput(key)
   386  		if limit == nil {
   387  			continue
   388  		}
   389  		if earliestLimit == nil || a.cmp(limit, earliestLimit) < 0 {
   390  			earliestLimit = limit
   391  		}
   392  	}
   393  	return earliestLimit
   394  }
   395  
   396  // userKeyChangeSplitter is a compactionOutputSplitter that takes in a child
   397  // splitter, and splits when 1) that child splitter has advised a split, and 2)
   398  // the compaction output is at the boundary between two user keys (also
   399  // the boundary between atomic compaction units). Use this splitter to wrap
   400  // any splitters that don't guarantee user key splits (i.e. splitters that make
   401  // their determination in ways other than comparing the current key against a
   402  // limit key.) If a wrapped splitter advises a split, it must continue
   403  // to advise a split until a new output.
   404  type userKeyChangeSplitter struct {
   405  	cmp               Compare
   406  	splitter          compactionOutputSplitter
   407  	unsafePrevUserKey func() []byte
   408  }
   409  
   410  func (u *userKeyChangeSplitter) shouldSplitBefore(key *InternalKey, tw *sstable.Writer) maybeSplit {
   411  	// NB: The userKeyChangeSplitter only needs to suffer a key comparison if
   412  	// the wrapped splitter requests a split.
   413  	//
   414  	// We could implement this splitter using frontiers: When the inner splitter
   415  	// requests a split before key `k`, we'd update a frontier to be
   416  	// ImmediateSuccessor(k). Then on the next key greater than >k, the
   417  	// frontier's `reached` func would be called and we'd return splitNow.
   418  	// This doesn't really save work since duplicate user keys are rare, and it
   419  	// requires us to materialize the ImmediateSuccessor key. It also prevents
   420  	// us from splitting on the same key that the inner splitter requested a
   421  	// split for—instead we need to wait until the next key. The current
   422  	// implementation uses `unsafePrevUserKey` to gain access to the previous
   423  	// key which allows it to immediately respect the inner splitter if
   424  	// possible.
   425  	if split := u.splitter.shouldSplitBefore(key, tw); split != splitNow {
   426  		return split
   427  	}
   428  	if u.cmp(key.UserKey, u.unsafePrevUserKey()) > 0 {
   429  		return splitNow
   430  	}
   431  	return noSplit
   432  }
   433  
   434  func (u *userKeyChangeSplitter) onNewOutput(key []byte) []byte {
   435  	return u.splitter.onNewOutput(key)
   436  }
   437  
   438  // compactionWritable is a objstorage.Writable wrapper that, on every write,
   439  // updates a metric in `versions` on bytes written by in-progress compactions so
   440  // far. It also increments a per-compaction `written` int.
   441  type compactionWritable struct {
   442  	objstorage.Writable
   443  
   444  	versions *versionSet
   445  	written  *int64
   446  }
   447  
   448  // Write is part of the objstorage.Writable interface.
   449  func (c *compactionWritable) Write(p []byte) error {
   450  	if err := c.Writable.Write(p); err != nil {
   451  		return err
   452  	}
   453  
   454  	*c.written += int64(len(p))
   455  	c.versions.incrementCompactionBytes(int64(len(p)))
   456  	return nil
   457  }
   458  
   459  type compactionKind int
   460  
   461  const (
   462  	compactionKindDefault compactionKind = iota
   463  	compactionKindFlush
   464  	// compactionKindMove denotes a move compaction where the input file is
   465  	// retained and linked in a new level without being obsoleted.
   466  	compactionKindMove
   467  	// compactionKindCopy denotes a copy compaction where the input file is
   468  	// copied byte-by-byte into a new file with a new FileNum in the output level.
   469  	compactionKindCopy
   470  	compactionKindDeleteOnly
   471  	compactionKindElisionOnly
   472  	compactionKindRead
   473  	compactionKindRewrite
   474  	compactionKindIngestedFlushable
   475  )
   476  
   477  func (k compactionKind) String() string {
   478  	switch k {
   479  	case compactionKindDefault:
   480  		return "default"
   481  	case compactionKindFlush:
   482  		return "flush"
   483  	case compactionKindMove:
   484  		return "move"
   485  	case compactionKindDeleteOnly:
   486  		return "delete-only"
   487  	case compactionKindElisionOnly:
   488  		return "elision-only"
   489  	case compactionKindRead:
   490  		return "read"
   491  	case compactionKindRewrite:
   492  		return "rewrite"
   493  	case compactionKindIngestedFlushable:
   494  		return "ingested-flushable"
   495  	case compactionKindCopy:
   496  		return "copy"
   497  	}
   498  	return "?"
   499  }
   500  
   501  // rangeKeyCompactionTransform is used to transform range key spans as part of the
   502  // keyspan.MergingIter. As part of this transformation step, we can elide range
   503  // keys in the last snapshot stripe, as well as coalesce range keys within
   504  // snapshot stripes.
   505  func rangeKeyCompactionTransform(
   506  	eq base.Equal, snapshots []uint64, elideRangeKey func(start, end []byte) bool,
   507  ) keyspan.Transformer {
   508  	return keyspan.TransformerFunc(func(cmp base.Compare, s keyspan.Span, dst *keyspan.Span) error {
   509  		elideInLastStripe := func(keys []keyspan.Key) []keyspan.Key {
   510  			// Unsets and deletes in the last snapshot stripe can be elided.
   511  			k := 0
   512  			for j := range keys {
   513  				if elideRangeKey(s.Start, s.End) &&
   514  					(keys[j].Kind() == InternalKeyKindRangeKeyUnset || keys[j].Kind() == InternalKeyKindRangeKeyDelete) {
   515  					continue
   516  				}
   517  				keys[k] = keys[j]
   518  				k++
   519  			}
   520  			keys = keys[:k]
   521  			return keys
   522  		}
   523  		// snapshots are in ascending order, while s.keys are in descending seqnum
   524  		// order. Partition s.keys by snapshot stripes, and call rangekey.Coalesce
   525  		// on each partition.
   526  		dst.Start = s.Start
   527  		dst.End = s.End
   528  		dst.Keys = dst.Keys[:0]
   529  		i, j := len(snapshots)-1, 0
   530  		usedLen := 0
   531  		for i >= 0 {
   532  			start := j
   533  			for j < len(s.Keys) && !base.Visible(s.Keys[j].SeqNum(), snapshots[i], base.InternalKeySeqNumMax) {
   534  				// Include j in current partition.
   535  				j++
   536  			}
   537  			if j > start {
   538  				keysDst := dst.Keys[usedLen:cap(dst.Keys)]
   539  				if err := rangekey.Coalesce(cmp, eq, s.Keys[start:j], &keysDst); err != nil {
   540  					return err
   541  				}
   542  				if j == len(s.Keys) {
   543  					// This is the last snapshot stripe. Unsets and deletes can be elided.
   544  					keysDst = elideInLastStripe(keysDst)
   545  				}
   546  				usedLen += len(keysDst)
   547  				dst.Keys = append(dst.Keys, keysDst...)
   548  			}
   549  			i--
   550  		}
   551  		if j < len(s.Keys) {
   552  			keysDst := dst.Keys[usedLen:cap(dst.Keys)]
   553  			if err := rangekey.Coalesce(cmp, eq, s.Keys[j:], &keysDst); err != nil {
   554  				return err
   555  			}
   556  			keysDst = elideInLastStripe(keysDst)
   557  			usedLen += len(keysDst)
   558  			dst.Keys = append(dst.Keys, keysDst...)
   559  		}
   560  		return nil
   561  	})
   562  }
   563  
   564  // compaction is a table compaction from one level to the next, starting from a
   565  // given version.
   566  type compaction struct {
   567  	// cancel is a bool that can be used by other goroutines to signal a compaction
   568  	// to cancel, such as if a conflicting excise operation raced it to manifest
   569  	// application. Only holders of the manifest lock will write to this atomic.
   570  	cancel atomic.Bool
   571  
   572  	kind      compactionKind
   573  	cmp       Compare
   574  	equal     Equal
   575  	comparer  *base.Comparer
   576  	formatKey base.FormatKey
   577  	logger    Logger
   578  	version   *version
   579  	stats     base.InternalIteratorStats
   580  	beganAt   time.Time
   581  	// versionEditApplied is set to true when a compaction has completed and the
   582  	// resulting version has been installed (if successful), but the compaction
   583  	// goroutine is still cleaning up (eg, deleting obsolete files).
   584  	versionEditApplied bool
   585  	bufferPool         sstable.BufferPool
   586  
   587  	// startLevel is the level that is being compacted. Inputs from startLevel
   588  	// and outputLevel will be merged to produce a set of outputLevel files.
   589  	startLevel *compactionLevel
   590  
   591  	// outputLevel is the level that files are being produced in. outputLevel is
   592  	// equal to startLevel+1 except when:
   593  	//    - if startLevel is 0, the output level equals compactionPicker.baseLevel().
   594  	//    - in multilevel compaction, the output level is the lowest level involved in
   595  	//      the compaction
   596  	// A compaction's outputLevel is nil for delete-only compactions.
   597  	outputLevel *compactionLevel
   598  
   599  	// extraLevels point to additional levels in between the input and output
   600  	// levels that get compacted in multilevel compactions
   601  	extraLevels []*compactionLevel
   602  
   603  	inputs []compactionLevel
   604  
   605  	// maxOutputFileSize is the maximum size of an individual table created
   606  	// during compaction.
   607  	maxOutputFileSize uint64
   608  	// maxOverlapBytes is the maximum number of bytes of overlap allowed for a
   609  	// single output table with the tables in the grandparent level.
   610  	maxOverlapBytes uint64
   611  	// disableSpanElision disables elision of range tombstones and range keys. Used
   612  	// by tests to allow range tombstones or range keys to be added to tables where
   613  	// they would otherwise be elided.
   614  	disableSpanElision bool
   615  
   616  	// flushing contains the flushables (aka memtables) that are being flushed.
   617  	flushing flushableList
   618  	// bytesIterated contains the number of bytes that have been flushed/compacted.
   619  	bytesIterated uint64
   620  	// bytesWritten contains the number of bytes that have been written to outputs.
   621  	bytesWritten int64
   622  
   623  	// The boundaries of the input data.
   624  	smallest InternalKey
   625  	largest  InternalKey
   626  
   627  	// The range deletion tombstone fragmenter. Adds range tombstones as they are
   628  	// returned from `compactionIter` and fragments them for output to files.
   629  	// Referenced by `compactionIter` which uses it to check whether keys are deleted.
   630  	rangeDelFrag keyspan.Fragmenter
   631  	// The range key fragmenter. Similar to rangeDelFrag in that it gets range
   632  	// keys from the compaction iter and fragments them for output to files.
   633  	rangeKeyFrag keyspan.Fragmenter
   634  	// The range deletion tombstone iterator, that merges and fragments
   635  	// tombstones across levels. This iterator is included within the compaction
   636  	// input iterator as a single level.
   637  	// TODO(jackson): Remove this when the refactor of FragmentIterator,
   638  	// InterleavingIterator, etc is complete.
   639  	rangeDelIter keyspan.InternalIteratorShim
   640  	// rangeKeyInterleaving is the interleaving iter for range keys.
   641  	rangeKeyInterleaving keyspan.InterleavingIter
   642  
   643  	// A list of objects to close when the compaction finishes. Used by input
   644  	// iteration to keep rangeDelIters open for the lifetime of the compaction,
   645  	// and only close them when the compaction finishes.
   646  	closers []io.Closer
   647  
   648  	// grandparents are the tables in level+2 that overlap with the files being
   649  	// compacted. Used to determine output table boundaries. Do not assume that the actual files
   650  	// in the grandparent when this compaction finishes will be the same.
   651  	grandparents manifest.LevelSlice
   652  
   653  	// Boundaries at which flushes to L0 should be split. Determined by
   654  	// L0Sublevels. If nil, flushes aren't split.
   655  	l0Limits [][]byte
   656  
   657  	// List of disjoint inuse key ranges the compaction overlaps with in
   658  	// grandparent and lower levels. See setupInuseKeyRanges() for the
   659  	// construction. Used by elideTombstone() and elideRangeTombstone() to
   660  	// determine if keys affected by a tombstone possibly exist at a lower level.
   661  	inuseKeyRanges []manifest.UserKeyRange
   662  	// inuseEntireRange is set if the above inuse key ranges wholly contain the
   663  	// compaction's key range. This allows compactions in higher levels to often
   664  	// elide key comparisons.
   665  	inuseEntireRange    bool
   666  	elideTombstoneIndex int
   667  
   668  	// allowedZeroSeqNum is true if seqnums can be zeroed if there are no
   669  	// snapshots requiring them to be kept. This determination is made by
   670  	// looking for an sstable which overlaps the bounds of the compaction at a
   671  	// lower level in the LSM during runCompaction.
   672  	allowedZeroSeqNum bool
   673  
   674  	metrics map[int]*LevelMetrics
   675  
   676  	pickerMetrics compactionPickerMetrics
   677  }
   678  
   679  func (c *compaction) makeInfo(jobID int) CompactionInfo {
   680  	info := CompactionInfo{
   681  		JobID:       jobID,
   682  		Reason:      c.kind.String(),
   683  		Input:       make([]LevelInfo, 0, len(c.inputs)),
   684  		Annotations: []string{},
   685  	}
   686  	for _, cl := range c.inputs {
   687  		inputInfo := LevelInfo{Level: cl.level, Tables: nil}
   688  		iter := cl.files.Iter()
   689  		for m := iter.First(); m != nil; m = iter.Next() {
   690  			inputInfo.Tables = append(inputInfo.Tables, m.TableInfo())
   691  		}
   692  		info.Input = append(info.Input, inputInfo)
   693  	}
   694  	if c.outputLevel != nil {
   695  		info.Output.Level = c.outputLevel.level
   696  
   697  		// If there are no inputs from the output level (eg, a move
   698  		// compaction), add an empty LevelInfo to info.Input.
   699  		if len(c.inputs) > 0 && c.inputs[len(c.inputs)-1].level != c.outputLevel.level {
   700  			info.Input = append(info.Input, LevelInfo{Level: c.outputLevel.level})
   701  		}
   702  	} else {
   703  		// For a delete-only compaction, set the output level to L6. The
   704  		// output level is not meaningful here, but complicating the
   705  		// info.Output interface with a pointer doesn't seem worth the
   706  		// semantic distinction.
   707  		info.Output.Level = numLevels - 1
   708  	}
   709  
   710  	for i, score := range c.pickerMetrics.scores {
   711  		info.Input[i].Score = score
   712  	}
   713  	info.SingleLevelOverlappingRatio = c.pickerMetrics.singleLevelOverlappingRatio
   714  	info.MultiLevelOverlappingRatio = c.pickerMetrics.multiLevelOverlappingRatio
   715  	if len(info.Input) > 2 {
   716  		info.Annotations = append(info.Annotations, "multilevel")
   717  	}
   718  	return info
   719  }
   720  
   721  func newCompaction(
   722  	pc *pickedCompaction, opts *Options, beganAt time.Time, provider objstorage.Provider,
   723  ) *compaction {
   724  	c := &compaction{
   725  		kind:              compactionKindDefault,
   726  		cmp:               pc.cmp,
   727  		equal:             opts.equal(),
   728  		comparer:          opts.Comparer,
   729  		formatKey:         opts.Comparer.FormatKey,
   730  		inputs:            pc.inputs,
   731  		smallest:          pc.smallest,
   732  		largest:           pc.largest,
   733  		logger:            opts.Logger,
   734  		version:           pc.version,
   735  		beganAt:           beganAt,
   736  		maxOutputFileSize: pc.maxOutputFileSize,
   737  		maxOverlapBytes:   pc.maxOverlapBytes,
   738  		pickerMetrics:     pc.pickerMetrics,
   739  	}
   740  	c.startLevel = &c.inputs[0]
   741  	if pc.startLevel.l0SublevelInfo != nil {
   742  		c.startLevel.l0SublevelInfo = pc.startLevel.l0SublevelInfo
   743  	}
   744  	c.outputLevel = &c.inputs[1]
   745  
   746  	if len(pc.extraLevels) > 0 {
   747  		c.extraLevels = pc.extraLevels
   748  		c.outputLevel = &c.inputs[len(c.inputs)-1]
   749  	}
   750  	// Compute the set of outputLevel+1 files that overlap this compaction (these
   751  	// are the grandparent sstables).
   752  	if c.outputLevel.level+1 < numLevels {
   753  		c.grandparents = c.version.Overlaps(c.outputLevel.level+1, c.cmp,
   754  			c.smallest.UserKey, c.largest.UserKey, c.largest.IsExclusiveSentinel())
   755  	}
   756  	c.setupInuseKeyRanges()
   757  	c.kind = pc.kind
   758  
   759  	if c.kind == compactionKindDefault && c.outputLevel.files.Empty() && !c.hasExtraLevelData() &&
   760  		c.startLevel.files.Len() == 1 && c.grandparents.SizeSum() <= c.maxOverlapBytes {
   761  		// This compaction can be converted into a move or copy from one level
   762  		// to the next. We avoid such a move if there is lots of overlapping
   763  		// grandparent data. Otherwise, the move could create a parent file
   764  		// that will require a very expensive merge later on.
   765  		iter := c.startLevel.files.Iter()
   766  		meta := iter.First()
   767  		isRemote := false
   768  		// We should always be passed a provider, except in some unit tests.
   769  		if provider != nil {
   770  			objMeta, err := provider.Lookup(fileTypeTable, meta.FileBacking.DiskFileNum)
   771  			if err != nil {
   772  				panic(errors.Wrapf(err, "cannot lookup table %s in provider", meta.FileBacking.DiskFileNum))
   773  			}
   774  			isRemote = objMeta.IsRemote()
   775  		}
   776  		// Avoid a trivial move or copy if all of these are true, as rewriting a
   777  		// new file is better:
   778  		//
   779  		// 1) The source file is a virtual sstable
   780  		// 2) The existing file `meta` is on non-remote storage
   781  		// 3) The output level prefers shared storage
   782  		mustCopy := !isRemote && remote.ShouldCreateShared(opts.Experimental.CreateOnShared, c.outputLevel.level)
   783  		if mustCopy {
   784  			// If the source is virtual, it's best to just rewrite the file as all
   785  			// conditions in the above comment are met.
   786  			if !meta.Virtual {
   787  				c.kind = compactionKindCopy
   788  			}
   789  		} else {
   790  			c.kind = compactionKindMove
   791  		}
   792  	}
   793  	return c
   794  }
   795  
   796  func newDeleteOnlyCompaction(
   797  	opts *Options, cur *version, inputs []compactionLevel, beganAt time.Time,
   798  ) *compaction {
   799  	c := &compaction{
   800  		kind:      compactionKindDeleteOnly,
   801  		cmp:       opts.Comparer.Compare,
   802  		equal:     opts.equal(),
   803  		comparer:  opts.Comparer,
   804  		formatKey: opts.Comparer.FormatKey,
   805  		logger:    opts.Logger,
   806  		version:   cur,
   807  		beganAt:   beganAt,
   808  		inputs:    inputs,
   809  	}
   810  
   811  	// Set c.smallest, c.largest.
   812  	files := make([]manifest.LevelIterator, 0, len(inputs))
   813  	for _, in := range inputs {
   814  		files = append(files, in.files.Iter())
   815  	}
   816  	c.smallest, c.largest = manifest.KeyRange(opts.Comparer.Compare, files...)
   817  	return c
   818  }
   819  
   820  func adjustGrandparentOverlapBytesForFlush(c *compaction, flushingBytes uint64) {
   821  	// Heuristic to place a lower bound on compaction output file size
   822  	// caused by Lbase. Prior to this heuristic we have observed an L0 in
   823  	// production with 310K files of which 290K files were < 10KB in size.
   824  	// Our hypothesis is that it was caused by L1 having 2600 files and
   825  	// ~10GB, such that each flush got split into many tiny files due to
   826  	// overlapping with most of the files in Lbase.
   827  	//
   828  	// The computation below is general in that it accounts
   829  	// for flushing different volumes of data (e.g. we may be flushing
   830  	// many memtables). For illustration, we consider the typical
   831  	// example of flushing a 64MB memtable. So 12.8MB output,
   832  	// based on the compression guess below. If the compressed bytes
   833  	// guess is an over-estimate we will end up with smaller files,
   834  	// and if an under-estimate we will end up with larger files.
   835  	// With a 2MB target file size, 7 files. We are willing to accept
   836  	// 4x the number of files, if it results in better write amplification
   837  	// when later compacting to Lbase, i.e., ~450KB files (target file
   838  	// size / 4).
   839  	//
   840  	// Note that this is a pessimistic heuristic in that
   841  	// fileCountUpperBoundDueToGrandparents could be far from the actual
   842  	// number of files produced due to the grandparent limits. For
   843  	// example, in the extreme, consider a flush that overlaps with 1000
   844  	// files in Lbase f0...f999, and the initially calculated value of
   845  	// maxOverlapBytes will cause splits at f10, f20,..., f990, which
   846  	// means an upper bound file count of 100 files. Say the input bytes
   847  	// in the flush are such that acceptableFileCount=10. We will fatten
   848  	// up maxOverlapBytes by 10x to ensure that the upper bound file count
   849  	// drops to 10. However, it is possible that in practice, even without
   850  	// this change, we would have produced no more than 10 files, and that
   851  	// this change makes the files unnecessarily wide. Say the input bytes
   852  	// are distributed such that 10% are in f0...f9, 10% in f10...f19, ...
   853  	// 10% in f80...f89 and 10% in f990...f999. The original value of
   854  	// maxOverlapBytes would have actually produced only 10 sstables. But
   855  	// by increasing maxOverlapBytes by 10x, we may produce 1 sstable that
   856  	// spans f0...f89, i.e., a much wider sstable than necessary.
   857  	//
   858  	// We could produce a tighter estimate of
   859  	// fileCountUpperBoundDueToGrandparents if we had knowledge of the key
   860  	// distribution of the flush. The 4x multiplier mentioned earlier is
   861  	// a way to try to compensate for this pessimism.
   862  	//
   863  	// TODO(sumeer): we don't have compression info for the data being
   864  	// flushed, but it is likely that existing files that overlap with
   865  	// this flush in Lbase are representative wrt compression ratio. We
   866  	// could store the uncompressed size in FileMetadata and estimate
   867  	// the compression ratio.
   868  	const approxCompressionRatio = 0.2
   869  	approxOutputBytes := approxCompressionRatio * float64(flushingBytes)
   870  	approxNumFilesBasedOnTargetSize :=
   871  		int(math.Ceil(approxOutputBytes / float64(c.maxOutputFileSize)))
   872  	acceptableFileCount := float64(4 * approxNumFilesBasedOnTargetSize)
   873  	// The byte calculation is linear in numGrandparentFiles, but we will
   874  	// incur this linear cost in findGrandparentLimit too, so we are also
   875  	// willing to pay it now. We could approximate this cheaply by using
   876  	// the mean file size of Lbase.
   877  	grandparentFileBytes := c.grandparents.SizeSum()
   878  	fileCountUpperBoundDueToGrandparents :=
   879  		float64(grandparentFileBytes) / float64(c.maxOverlapBytes)
   880  	if fileCountUpperBoundDueToGrandparents > acceptableFileCount {
   881  		c.maxOverlapBytes = uint64(
   882  			float64(c.maxOverlapBytes) *
   883  				(fileCountUpperBoundDueToGrandparents / acceptableFileCount))
   884  	}
   885  }
   886  
   887  func newFlush(
   888  	opts *Options, cur *version, baseLevel int, flushing flushableList, beganAt time.Time,
   889  ) *compaction {
   890  	c := &compaction{
   891  		kind:              compactionKindFlush,
   892  		cmp:               opts.Comparer.Compare,
   893  		equal:             opts.equal(),
   894  		comparer:          opts.Comparer,
   895  		formatKey:         opts.Comparer.FormatKey,
   896  		logger:            opts.Logger,
   897  		version:           cur,
   898  		beganAt:           beganAt,
   899  		inputs:            []compactionLevel{{level: -1}, {level: 0}},
   900  		maxOutputFileSize: math.MaxUint64,
   901  		maxOverlapBytes:   math.MaxUint64,
   902  		flushing:          flushing,
   903  	}
   904  	c.startLevel = &c.inputs[0]
   905  	c.outputLevel = &c.inputs[1]
   906  
   907  	if len(flushing) > 0 {
   908  		if _, ok := flushing[0].flushable.(*ingestedFlushable); ok {
   909  			if len(flushing) != 1 {
   910  				panic("pebble: ingestedFlushable must be flushed one at a time.")
   911  			}
   912  			c.kind = compactionKindIngestedFlushable
   913  			return c
   914  		}
   915  	}
   916  
   917  	// Make sure there's no ingestedFlushable after the first flushable in the
   918  	// list.
   919  	for _, f := range flushing {
   920  		if _, ok := f.flushable.(*ingestedFlushable); ok {
   921  			panic("pebble: flushing shouldn't contain ingestedFlushable flushable")
   922  		}
   923  	}
   924  
   925  	if cur.L0Sublevels != nil {
   926  		c.l0Limits = cur.L0Sublevels.FlushSplitKeys()
   927  	}
   928  
   929  	smallestSet, largestSet := false, false
   930  	updatePointBounds := func(iter internalIterator) {
   931  		if key, _ := iter.First(); key != nil {
   932  			if !smallestSet ||
   933  				base.InternalCompare(c.cmp, c.smallest, *key) > 0 {
   934  				smallestSet = true
   935  				c.smallest = key.Clone()
   936  			}
   937  		}
   938  		if key, _ := iter.Last(); key != nil {
   939  			if !largestSet ||
   940  				base.InternalCompare(c.cmp, c.largest, *key) < 0 {
   941  				largestSet = true
   942  				c.largest = key.Clone()
   943  			}
   944  		}
   945  	}
   946  
   947  	updateRangeBounds := func(iter keyspan.FragmentIterator) {
   948  		// File bounds require s != nil && !s.Empty(). We only need to check for
   949  		// s != nil here, as the memtable's FragmentIterator would never surface
   950  		// empty spans.
   951  		if s := iter.First(); s != nil {
   952  			if key := s.SmallestKey(); !smallestSet ||
   953  				base.InternalCompare(c.cmp, c.smallest, key) > 0 {
   954  				smallestSet = true
   955  				c.smallest = key.Clone()
   956  			}
   957  		}
   958  		if s := iter.Last(); s != nil {
   959  			if key := s.LargestKey(); !largestSet ||
   960  				base.InternalCompare(c.cmp, c.largest, key) < 0 {
   961  				largestSet = true
   962  				c.largest = key.Clone()
   963  			}
   964  		}
   965  	}
   966  
   967  	var flushingBytes uint64
   968  	for i := range flushing {
   969  		f := flushing[i]
   970  		updatePointBounds(f.newIter(nil))
   971  		if rangeDelIter := f.newRangeDelIter(nil); rangeDelIter != nil {
   972  			updateRangeBounds(rangeDelIter)
   973  		}
   974  		if rangeKeyIter := f.newRangeKeyIter(nil); rangeKeyIter != nil {
   975  			updateRangeBounds(rangeKeyIter)
   976  		}
   977  		flushingBytes += f.inuseBytes()
   978  	}
   979  
   980  	if opts.FlushSplitBytes > 0 {
   981  		c.maxOutputFileSize = uint64(opts.Level(0).TargetFileSize)
   982  		c.maxOverlapBytes = maxGrandparentOverlapBytes(opts, 0)
   983  		c.grandparents = c.version.Overlaps(baseLevel, c.cmp, c.smallest.UserKey,
   984  			c.largest.UserKey, c.largest.IsExclusiveSentinel())
   985  		adjustGrandparentOverlapBytesForFlush(c, flushingBytes)
   986  	}
   987  
   988  	c.setupInuseKeyRanges()
   989  	return c
   990  }
   991  
   992  func (c *compaction) hasExtraLevelData() bool {
   993  	if len(c.extraLevels) == 0 {
   994  		// not a multi level compaction
   995  		return false
   996  	} else if c.extraLevels[0].files.Empty() {
   997  		// a multi level compaction without data in the intermediate input level;
   998  		// e.g. for a multi level compaction with levels 4,5, and 6, this could
   999  		// occur if there is no files to compact in 5, or in 5 and 6 (i.e. a move).
  1000  		return false
  1001  	}
  1002  	return true
  1003  }
  1004  
  1005  func (c *compaction) setupInuseKeyRanges() {
  1006  	level := c.outputLevel.level + 1
  1007  	if c.outputLevel.level == 0 {
  1008  		level = 0
  1009  	}
  1010  	// calculateInuseKeyRanges will return a series of sorted spans. Overlapping
  1011  	// or abutting spans have already been merged.
  1012  	c.inuseKeyRanges = calculateInuseKeyRanges(
  1013  		c.version, c.cmp, level, numLevels-1, c.smallest.UserKey, c.largest.UserKey,
  1014  	)
  1015  	// Check if there's a single in-use span that encompasses the entire key
  1016  	// range of the compaction. This is an optimization to avoid key comparisons
  1017  	// against inuseKeyRanges during the compaction when every key within the
  1018  	// compaction overlaps with an in-use span.
  1019  	if len(c.inuseKeyRanges) > 0 {
  1020  		c.inuseEntireRange = c.cmp(c.inuseKeyRanges[0].Start, c.smallest.UserKey) <= 0 &&
  1021  			c.cmp(c.inuseKeyRanges[0].End, c.largest.UserKey) >= 0
  1022  	}
  1023  }
  1024  
  1025  func calculateInuseKeyRanges(
  1026  	v *version, cmp base.Compare, level, maxLevel int, smallest, largest []byte,
  1027  ) []manifest.UserKeyRange {
  1028  	// Use two slices, alternating which one is input and which one is output
  1029  	// as we descend the LSM.
  1030  	var input, output []manifest.UserKeyRange
  1031  
  1032  	// L0 requires special treatment, since sstables within L0 may overlap.
  1033  	// We use the L0 Sublevels structure to efficiently calculate the merged
  1034  	// in-use key ranges.
  1035  	if level == 0 {
  1036  		output = v.L0Sublevels.InUseKeyRanges(smallest, largest)
  1037  		level++
  1038  	}
  1039  
  1040  	for ; level <= maxLevel; level++ {
  1041  		// NB: We always treat `largest` as inclusive for simplicity, because
  1042  		// there's little consequence to calculating slightly broader in-use key
  1043  		// ranges.
  1044  		overlaps := v.Overlaps(level, cmp, smallest, largest, false /* exclusiveEnd */)
  1045  		iter := overlaps.Iter()
  1046  
  1047  		// We may already have in-use key ranges from higher levels. Iterate
  1048  		// through both our accumulated in-use key ranges and this level's
  1049  		// files, merging the two.
  1050  		//
  1051  		// Tables higher within the LSM have broader key spaces. We use this
  1052  		// when possible to seek past a level's files that are contained by
  1053  		// our current accumulated in-use key ranges. This helps avoid
  1054  		// per-sstable work during flushes or compactions in high levels which
  1055  		// overlap the majority of the LSM's sstables.
  1056  		input, output = output, input
  1057  		output = output[:0]
  1058  
  1059  		var currFile *fileMetadata
  1060  		var currAccum *manifest.UserKeyRange
  1061  		if len(input) > 0 {
  1062  			currAccum, input = &input[0], input[1:]
  1063  		}
  1064  
  1065  		// If we have an accumulated key range and its start is ≤ smallest,
  1066  		// we can seek to the accumulated range's end. Otherwise, we need to
  1067  		// start at the first overlapping file within the level.
  1068  		if currAccum != nil && cmp(currAccum.Start, smallest) <= 0 {
  1069  			currFile = seekGT(&iter, cmp, currAccum.End)
  1070  		} else {
  1071  			currFile = iter.First()
  1072  		}
  1073  
  1074  		for currFile != nil || currAccum != nil {
  1075  			// If we've exhausted either the files in the level or the
  1076  			// accumulated key ranges, we just need to append the one we have.
  1077  			// If we have both a currFile and a currAccum, they either overlap
  1078  			// or they're disjoint. If they're disjoint, we append whichever
  1079  			// one sorts first and move on to the next file or range. If they
  1080  			// overlap, we merge them into currAccum and proceed to the next
  1081  			// file.
  1082  			switch {
  1083  			case currAccum == nil || (currFile != nil && cmp(currFile.Largest.UserKey, currAccum.Start) < 0):
  1084  				// This file is strictly before the current accumulated range,
  1085  				// or there are no more accumulated ranges.
  1086  				output = append(output, manifest.UserKeyRange{
  1087  					Start: currFile.Smallest.UserKey,
  1088  					End:   currFile.Largest.UserKey,
  1089  				})
  1090  				currFile = iter.Next()
  1091  			case currFile == nil || (currAccum != nil && cmp(currAccum.End, currFile.Smallest.UserKey) < 0):
  1092  				// The current accumulated key range is strictly before the
  1093  				// current file, or there are no more files.
  1094  				output = append(output, *currAccum)
  1095  				currAccum = nil
  1096  				if len(input) > 0 {
  1097  					currAccum, input = &input[0], input[1:]
  1098  				}
  1099  			default:
  1100  				// The current accumulated range and the current file overlap.
  1101  				// Adjust the accumulated range to be the union.
  1102  				if cmp(currFile.Smallest.UserKey, currAccum.Start) < 0 {
  1103  					currAccum.Start = currFile.Smallest.UserKey
  1104  				}
  1105  				if cmp(currFile.Largest.UserKey, currAccum.End) > 0 {
  1106  					currAccum.End = currFile.Largest.UserKey
  1107  				}
  1108  
  1109  				// Extending `currAccum`'s end boundary may have caused it to
  1110  				// overlap with `input` key ranges that we haven't processed
  1111  				// yet. Merge any such key ranges.
  1112  				for len(input) > 0 && cmp(input[0].Start, currAccum.End) <= 0 {
  1113  					if cmp(input[0].End, currAccum.End) > 0 {
  1114  						currAccum.End = input[0].End
  1115  					}
  1116  					input = input[1:]
  1117  				}
  1118  				// Seek the level iterator past our current accumulated end.
  1119  				currFile = seekGT(&iter, cmp, currAccum.End)
  1120  			}
  1121  		}
  1122  	}
  1123  	return output
  1124  }
  1125  
  1126  func seekGT(iter *manifest.LevelIterator, cmp base.Compare, key []byte) *manifest.FileMetadata {
  1127  	f := iter.SeekGE(cmp, key)
  1128  	for f != nil && cmp(f.Largest.UserKey, key) == 0 {
  1129  		f = iter.Next()
  1130  	}
  1131  	return f
  1132  }
  1133  
  1134  // findGrandparentLimit takes the start user key for a table and returns the
  1135  // user key to which that table can extend without excessively overlapping
  1136  // the grandparent level. If no limit is needed considering the grandparent
  1137  // files, this function returns nil. This is done in order to prevent a table
  1138  // at level N from overlapping too much data at level N+1. We want to avoid
  1139  // such large overlaps because they translate into large compactions. The
  1140  // current heuristic stops output of a table if the addition of another key
  1141  // would cause the table to overlap more than 10x the target file size at
  1142  // level N. See maxGrandparentOverlapBytes.
  1143  func (c *compaction) findGrandparentLimit(start []byte) []byte {
  1144  	iter := c.grandparents.Iter()
  1145  	var overlappedBytes uint64
  1146  	var greater bool
  1147  	for f := iter.SeekGE(c.cmp, start); f != nil; f = iter.Next() {
  1148  		overlappedBytes += f.Size
  1149  		// To ensure forward progress we always return a larger user
  1150  		// key than where we started. See comments above clients of
  1151  		// this function for how this is used.
  1152  		greater = greater || c.cmp(f.Smallest.UserKey, start) > 0
  1153  		if !greater {
  1154  			continue
  1155  		}
  1156  
  1157  		// We return the smallest bound of a sstable rather than the
  1158  		// largest because the smallest is always inclusive, and limits
  1159  		// are used exlusively when truncating range tombstones. If we
  1160  		// truncated an output to the largest key while there's a
  1161  		// pending tombstone, the next output file would also overlap
  1162  		// the same grandparent f.
  1163  		if overlappedBytes > c.maxOverlapBytes {
  1164  			return f.Smallest.UserKey
  1165  		}
  1166  	}
  1167  	return nil
  1168  }
  1169  
  1170  // findL0Limit takes the start key for a table and returns the user key to which
  1171  // that table can be extended without hitting the next l0Limit. Having flushed
  1172  // sstables "bridging across" an l0Limit could lead to increased L0 -> LBase
  1173  // compaction sizes as well as elevated read amplification.
  1174  func (c *compaction) findL0Limit(start []byte) []byte {
  1175  	if c.startLevel.level > -1 || c.outputLevel.level != 0 || len(c.l0Limits) == 0 {
  1176  		return nil
  1177  	}
  1178  	index := sort.Search(len(c.l0Limits), func(i int) bool {
  1179  		return c.cmp(c.l0Limits[i], start) > 0
  1180  	})
  1181  	if index < len(c.l0Limits) {
  1182  		return c.l0Limits[index]
  1183  	}
  1184  	return nil
  1185  }
  1186  
  1187  // errorOnUserKeyOverlap returns an error if the last two written sstables in
  1188  // this compaction have revisions of the same user key present in both sstables,
  1189  // when it shouldn't (eg. when splitting flushes).
  1190  func (c *compaction) errorOnUserKeyOverlap(ve *versionEdit) error {
  1191  	if n := len(ve.NewFiles); n > 1 {
  1192  		meta := ve.NewFiles[n-1].Meta
  1193  		prevMeta := ve.NewFiles[n-2].Meta
  1194  		if !prevMeta.Largest.IsExclusiveSentinel() &&
  1195  			c.cmp(prevMeta.Largest.UserKey, meta.Smallest.UserKey) >= 0 {
  1196  			return errors.Errorf("pebble: compaction split user key across two sstables: %s in %s and %s",
  1197  				prevMeta.Largest.Pretty(c.formatKey),
  1198  				prevMeta.FileNum,
  1199  				meta.FileNum)
  1200  		}
  1201  	}
  1202  	return nil
  1203  }
  1204  
  1205  // allowZeroSeqNum returns true if seqnum's can be zeroed if there are no
  1206  // snapshots requiring them to be kept. It performs this determination by
  1207  // looking for an sstable which overlaps the bounds of the compaction at a
  1208  // lower level in the LSM.
  1209  func (c *compaction) allowZeroSeqNum() bool {
  1210  	return c.elideRangeTombstone(c.smallest.UserKey, c.largest.UserKey)
  1211  }
  1212  
  1213  // elideTombstone returns true if it is ok to elide a tombstone for the
  1214  // specified key. A return value of true guarantees that there are no key/value
  1215  // pairs at c.level+2 or higher that possibly contain the specified user
  1216  // key. The keys in multiple invocations to elideTombstone must be supplied in
  1217  // order.
  1218  func (c *compaction) elideTombstone(key []byte) bool {
  1219  	if c.inuseEntireRange || len(c.flushing) != 0 {
  1220  		return false
  1221  	}
  1222  
  1223  	for ; c.elideTombstoneIndex < len(c.inuseKeyRanges); c.elideTombstoneIndex++ {
  1224  		r := &c.inuseKeyRanges[c.elideTombstoneIndex]
  1225  		if c.cmp(key, r.End) <= 0 {
  1226  			if c.cmp(key, r.Start) >= 0 {
  1227  				return false
  1228  			}
  1229  			break
  1230  		}
  1231  	}
  1232  	return true
  1233  }
  1234  
  1235  // elideRangeTombstone returns true if it is ok to elide the specified range
  1236  // tombstone. A return value of true guarantees that there are no key/value
  1237  // pairs at c.outputLevel.level+1 or higher that possibly overlap the specified
  1238  // tombstone.
  1239  func (c *compaction) elideRangeTombstone(start, end []byte) bool {
  1240  	// Disable range tombstone elision if the testing knob for that is enabled,
  1241  	// or if we are flushing memtables. The latter requirement is due to
  1242  	// inuseKeyRanges not accounting for key ranges in other memtables that are
  1243  	// being flushed in the same compaction. It's possible for a range tombstone
  1244  	// in one memtable to overlap keys in a preceding memtable in c.flushing.
  1245  	//
  1246  	// This function is also used in setting allowZeroSeqNum, so disabling
  1247  	// elision of range tombstones also disables zeroing of SeqNums.
  1248  	//
  1249  	// TODO(peter): we disable zeroing of seqnums during flushing to match
  1250  	// RocksDB behavior and to avoid generating overlapping sstables during
  1251  	// DB.replayWAL. When replaying WAL files at startup, we flush after each
  1252  	// WAL is replayed building up a single version edit that is
  1253  	// applied. Because we don't apply the version edit after each flush, this
  1254  	// code doesn't know that L0 contains files and zeroing of seqnums should
  1255  	// be disabled. That is fixable, but it seems safer to just match the
  1256  	// RocksDB behavior for now.
  1257  	if c.disableSpanElision || len(c.flushing) != 0 {
  1258  		return false
  1259  	}
  1260  
  1261  	lower := sort.Search(len(c.inuseKeyRanges), func(i int) bool {
  1262  		return c.cmp(c.inuseKeyRanges[i].End, start) >= 0
  1263  	})
  1264  	upper := sort.Search(len(c.inuseKeyRanges), func(i int) bool {
  1265  		return c.cmp(c.inuseKeyRanges[i].Start, end) > 0
  1266  	})
  1267  	return lower >= upper
  1268  }
  1269  
  1270  // elideRangeKey returns true if it is ok to elide the specified range key. A
  1271  // return value of true guarantees that there are no key/value pairs at
  1272  // c.outputLevel.level+1 or higher that possibly overlap the specified range key.
  1273  func (c *compaction) elideRangeKey(start, end []byte) bool {
  1274  	// TODO(bilal): Track inuseKeyRanges separately for the range keyspace as
  1275  	// opposed to the point keyspace. Once that is done, elideRangeTombstone
  1276  	// can just check in the point keyspace, and this function can check for
  1277  	// inuseKeyRanges in the range keyspace.
  1278  	return c.elideRangeTombstone(start, end)
  1279  }
  1280  
  1281  // newInputIter returns an iterator over all the input tables in a compaction.
  1282  func (c *compaction) newInputIter(
  1283  	newIters tableNewIters, newRangeKeyIter keyspan.TableNewSpanIter, snapshots []uint64,
  1284  ) (_ internalIterator, retErr error) {
  1285  	// Validate the ordering of compaction input files for defense in depth.
  1286  	// TODO(jackson): Some of the CheckOrdering calls may be adapted to pass
  1287  	// ProhibitSplitUserKeys if we thread the active format major version in. Or
  1288  	// if we remove support for earlier FMVs, we can remove the parameter
  1289  	// altogether.
  1290  	if len(c.flushing) == 0 {
  1291  		if c.startLevel.level >= 0 {
  1292  			err := manifest.CheckOrdering(c.cmp, c.formatKey,
  1293  				manifest.Level(c.startLevel.level), c.startLevel.files.Iter(),
  1294  				manifest.AllowSplitUserKeys)
  1295  			if err != nil {
  1296  				return nil, err
  1297  			}
  1298  		}
  1299  		err := manifest.CheckOrdering(c.cmp, c.formatKey,
  1300  			manifest.Level(c.outputLevel.level), c.outputLevel.files.Iter(),
  1301  			manifest.AllowSplitUserKeys)
  1302  		if err != nil {
  1303  			return nil, err
  1304  		}
  1305  		if c.startLevel.level == 0 {
  1306  			if c.startLevel.l0SublevelInfo == nil {
  1307  				panic("l0SublevelInfo not created for compaction out of L0")
  1308  			}
  1309  			for _, info := range c.startLevel.l0SublevelInfo {
  1310  				err := manifest.CheckOrdering(c.cmp, c.formatKey,
  1311  					info.sublevel, info.Iter(),
  1312  					// NB: L0 sublevels have never allowed split user keys.
  1313  					manifest.ProhibitSplitUserKeys)
  1314  				if err != nil {
  1315  					return nil, err
  1316  				}
  1317  			}
  1318  		}
  1319  		if len(c.extraLevels) > 0 {
  1320  			if len(c.extraLevels) > 1 {
  1321  				panic("n>2 multi level compaction not implemented yet")
  1322  			}
  1323  			interLevel := c.extraLevels[0]
  1324  			err := manifest.CheckOrdering(c.cmp, c.formatKey,
  1325  				manifest.Level(interLevel.level), interLevel.files.Iter(),
  1326  				manifest.AllowSplitUserKeys)
  1327  			if err != nil {
  1328  				return nil, err
  1329  			}
  1330  		}
  1331  	}
  1332  
  1333  	// There are three classes of keys that a compaction needs to process: point
  1334  	// keys, range deletion tombstones and range keys. Collect all iterators for
  1335  	// all these classes of keys from all the levels. We'll aggregate them
  1336  	// together farther below.
  1337  	//
  1338  	// numInputLevels is an approximation of the number of iterator levels. Due
  1339  	// to idiosyncrasies in iterator construction, we may (rarely) exceed this
  1340  	// initial capacity.
  1341  	numInputLevels := max[int](len(c.flushing), len(c.inputs))
  1342  	iters := make([]internalIterator, 0, numInputLevels)
  1343  	rangeDelIters := make([]keyspan.FragmentIterator, 0, numInputLevels)
  1344  	rangeKeyIters := make([]keyspan.FragmentIterator, 0, numInputLevels)
  1345  
  1346  	// If construction of the iterator inputs fails, ensure that we close all
  1347  	// the consitutent iterators.
  1348  	defer func() {
  1349  		if retErr != nil {
  1350  			for _, iter := range iters {
  1351  				if iter != nil {
  1352  					iter.Close()
  1353  				}
  1354  			}
  1355  			for _, rangeDelIter := range rangeDelIters {
  1356  				rangeDelIter.Close()
  1357  			}
  1358  		}
  1359  	}()
  1360  	iterOpts := IterOptions{logger: c.logger}
  1361  
  1362  	// Populate iters, rangeDelIters and rangeKeyIters with the appropriate
  1363  	// constituent iterators. This depends on whether this is a flush or a
  1364  	// compaction.
  1365  	if len(c.flushing) != 0 {
  1366  		// If flushing, we need to build the input iterators over the memtables
  1367  		// stored in c.flushing.
  1368  		for i := range c.flushing {
  1369  			f := c.flushing[i]
  1370  			iters = append(iters, f.newFlushIter(nil, &c.bytesIterated))
  1371  			rangeDelIter := f.newRangeDelIter(nil)
  1372  			if rangeDelIter != nil {
  1373  				rangeDelIters = append(rangeDelIters, rangeDelIter)
  1374  			}
  1375  			if rangeKeyIter := f.newRangeKeyIter(nil); rangeKeyIter != nil {
  1376  				rangeKeyIters = append(rangeKeyIters, rangeKeyIter)
  1377  			}
  1378  		}
  1379  	} else {
  1380  		addItersForLevel := func(level *compactionLevel, l manifest.Level) error {
  1381  			// Add a *levelIter for point iterators. Because we don't call
  1382  			// initRangeDel, the levelIter will close and forget the range
  1383  			// deletion iterator when it steps on to a new file. Surfacing range
  1384  			// deletions to compactions are handled below.
  1385  			iters = append(iters, newLevelIter(iterOpts, c.comparer, newIters,
  1386  				level.files.Iter(), l, internalIterOpts{
  1387  					bytesIterated: &c.bytesIterated,
  1388  					bufferPool:    &c.bufferPool,
  1389  				}))
  1390  			// TODO(jackson): Use keyspan.LevelIter to avoid loading all the range
  1391  			// deletions into memory upfront. (See #2015, which reverted this.)
  1392  			// There will be no user keys that are split between sstables
  1393  			// within a level in Cockroach 23.1, which unblocks this optimization.
  1394  
  1395  			// Add the range deletion iterator for each file as an independent level
  1396  			// in mergingIter, as opposed to making a levelIter out of those. This
  1397  			// is safer as levelIter expects all keys coming from underlying
  1398  			// iterators to be in order. Due to compaction / tombstone writing
  1399  			// logic in finishOutput(), it is possible for range tombstones to not
  1400  			// be strictly ordered across all files in one level.
  1401  			//
  1402  			// Consider this example from the metamorphic tests (also repeated in
  1403  			// finishOutput()), consisting of three L3 files with their bounds
  1404  			// specified in square brackets next to the file name:
  1405  			//
  1406  			// ./000240.sst   [tmgc#391,MERGE-tmgc#391,MERGE]
  1407  			// tmgc#391,MERGE [786e627a]
  1408  			// tmgc-udkatvs#331,RANGEDEL
  1409  			//
  1410  			// ./000241.sst   [tmgc#384,MERGE-tmgc#384,MERGE]
  1411  			// tmgc#384,MERGE [666c7070]
  1412  			// tmgc-tvsalezade#383,RANGEDEL
  1413  			// tmgc-tvsalezade#331,RANGEDEL
  1414  			//
  1415  			// ./000242.sst   [tmgc#383,RANGEDEL-tvsalezade#72057594037927935,RANGEDEL]
  1416  			// tmgc-tvsalezade#383,RANGEDEL
  1417  			// tmgc#375,SET [72646c78766965616c72776865676e79]
  1418  			// tmgc-tvsalezade#356,RANGEDEL
  1419  			//
  1420  			// Here, the range tombstone in 000240.sst falls "after" one in
  1421  			// 000241.sst, despite 000240.sst being ordered "before" 000241.sst for
  1422  			// levelIter's purposes. While each file is still consistent before its
  1423  			// bounds, it's safer to have all rangedel iterators be visible to
  1424  			// mergingIter.
  1425  			iter := level.files.Iter()
  1426  			for f := iter.First(); f != nil; f = iter.Next() {
  1427  				rangeDelIter, closer, err := c.newRangeDelIter(newIters, iter.Take(), nil, l, &c.bytesIterated)
  1428  				if err != nil {
  1429  					// The error will already be annotated with the BackingFileNum, so
  1430  					// we annotate it with the FileNum.
  1431  					return errors.Wrapf(err, "pebble: could not open table %s", errors.Safe(f.FileNum))
  1432  				}
  1433  				if rangeDelIter == nil {
  1434  					continue
  1435  				}
  1436  				rangeDelIters = append(rangeDelIters, rangeDelIter)
  1437  				c.closers = append(c.closers, closer)
  1438  			}
  1439  
  1440  			// Check if this level has any range keys.
  1441  			hasRangeKeys := false
  1442  			for f := iter.First(); f != nil; f = iter.Next() {
  1443  				if f.HasRangeKeys {
  1444  					hasRangeKeys = true
  1445  					break
  1446  				}
  1447  			}
  1448  			if hasRangeKeys {
  1449  				li := &keyspan.LevelIter{}
  1450  				newRangeKeyIterWrapper := func(file *manifest.FileMetadata, iterOptions keyspan.SpanIterOptions) (keyspan.FragmentIterator, error) {
  1451  					iter, err := newRangeKeyIter(file, iterOptions)
  1452  					if err != nil {
  1453  						return nil, err
  1454  					} else if iter == nil {
  1455  						return emptyKeyspanIter, nil
  1456  					}
  1457  					// Ensure that the range key iter is not closed until the compaction is
  1458  					// finished. This is necessary because range key processing
  1459  					// requires the range keys to be held in memory for up to the
  1460  					// lifetime of the compaction.
  1461  					c.closers = append(c.closers, iter)
  1462  					iter = noCloseIter{iter}
  1463  
  1464  					// We do not need to truncate range keys to sstable boundaries, or
  1465  					// only read within the file's atomic compaction units, unlike with
  1466  					// range tombstones. This is because range keys were added after we
  1467  					// stopped splitting user keys across sstables, so all the range keys
  1468  					// in this sstable must wholly lie within the file's bounds.
  1469  					return iter, err
  1470  				}
  1471  				li.Init(keyspan.SpanIterOptions{}, c.cmp, newRangeKeyIterWrapper, level.files.Iter(), l, manifest.KeyTypeRange)
  1472  				rangeKeyIters = append(rangeKeyIters, li)
  1473  			}
  1474  			return nil
  1475  		}
  1476  
  1477  		for i := range c.inputs {
  1478  			// If the level is annotated with l0SublevelInfo, expand it into one
  1479  			// level per sublevel.
  1480  			// TODO(jackson): Perform this expansion even earlier when we pick the
  1481  			// compaction?
  1482  			if len(c.inputs[i].l0SublevelInfo) > 0 {
  1483  				for _, info := range c.startLevel.l0SublevelInfo {
  1484  					sublevelCompactionLevel := &compactionLevel{0, info.LevelSlice, nil}
  1485  					if err := addItersForLevel(sublevelCompactionLevel, info.sublevel); err != nil {
  1486  						return nil, err
  1487  					}
  1488  				}
  1489  				continue
  1490  			}
  1491  			if err := addItersForLevel(&c.inputs[i], manifest.Level(c.inputs[i].level)); err != nil {
  1492  				return nil, err
  1493  			}
  1494  		}
  1495  	}
  1496  
  1497  	// In normal operation, levelIter iterates over the point operations in a
  1498  	// level, and initializes a rangeDelIter pointer for the range deletions in
  1499  	// each table. During compaction, we want to iterate over the merged view of
  1500  	// point operations and range deletions. In order to do this we create one
  1501  	// levelIter per level to iterate over the point operations, and collect up
  1502  	// all the range deletion files.
  1503  	//
  1504  	// The range deletion levels are first combined with a keyspan.MergingIter
  1505  	// (currently wrapped by a keyspan.InternalIteratorShim to satisfy the
  1506  	// internal iterator interface). The resulting merged rangedel iterator is
  1507  	// then included with the point levels in a single mergingIter.
  1508  	//
  1509  	// Combine all the rangedel iterators using a keyspan.MergingIterator and a
  1510  	// InternalIteratorShim so that the range deletions may be interleaved in
  1511  	// the compaction input.
  1512  	// TODO(jackson): Replace the InternalIteratorShim with an interleaving
  1513  	// iterator.
  1514  	if len(rangeDelIters) > 0 {
  1515  		c.rangeDelIter.Init(c.cmp, rangeDelIters...)
  1516  		iters = append(iters, &c.rangeDelIter)
  1517  	}
  1518  
  1519  	// If there's only one constituent point iterator, we can avoid the overhead
  1520  	// of a *mergingIter. This is possible, for example, when performing a flush
  1521  	// of a single memtable. Otherwise, combine all the iterators into a merging
  1522  	// iter.
  1523  	iter := iters[0]
  1524  	if len(iters) > 0 {
  1525  		iter = newMergingIter(c.logger, &c.stats, c.cmp, nil, iters...)
  1526  	}
  1527  	// If there are range key iterators, we need to combine them using
  1528  	// keyspan.MergingIter, and then interleave them among the points.
  1529  	if len(rangeKeyIters) > 0 {
  1530  		mi := &keyspan.MergingIter{}
  1531  		mi.Init(c.cmp, rangeKeyCompactionTransform(c.equal, snapshots, c.elideRangeKey), new(keyspan.MergingBuffers), rangeKeyIters...)
  1532  		di := &keyspan.DefragmentingIter{}
  1533  		di.Init(c.comparer, mi, keyspan.DefragmentInternal, keyspan.StaticDefragmentReducer, new(keyspan.DefragmentingBuffers))
  1534  		c.rangeKeyInterleaving.Init(c.comparer, iter, di, keyspan.InterleavingIterOpts{})
  1535  		iter = &c.rangeKeyInterleaving
  1536  	}
  1537  	return iter, nil
  1538  }
  1539  
  1540  func (c *compaction) newRangeDelIter(
  1541  	newIters tableNewIters,
  1542  	f manifest.LevelFile,
  1543  	_ *IterOptions,
  1544  	l manifest.Level,
  1545  	bytesIterated *uint64,
  1546  ) (keyspan.FragmentIterator, io.Closer, error) {
  1547  	iter, rangeDelIter, err := newIters(context.Background(), f.FileMetadata,
  1548  		&IterOptions{level: l}, internalIterOpts{
  1549  			bytesIterated: &c.bytesIterated,
  1550  			bufferPool:    &c.bufferPool,
  1551  		})
  1552  	if err != nil {
  1553  		return nil, nil, err
  1554  	}
  1555  	// TODO(peter): It is mildly wasteful to open the point iterator only to
  1556  	// immediately close it. One way to solve this would be to add new
  1557  	// methods to tableCache for creating point and range-deletion iterators
  1558  	// independently. We'd only want to use those methods here,
  1559  	// though. Doesn't seem worth the hassle in the near term.
  1560  	if err = iter.Close(); err != nil {
  1561  		if rangeDelIter != nil {
  1562  			err = errors.CombineErrors(err, rangeDelIter.Close())
  1563  		}
  1564  		return nil, nil, err
  1565  	}
  1566  	if rangeDelIter == nil {
  1567  		// The file doesn't contain any range deletions.
  1568  		return nil, nil, nil
  1569  	}
  1570  
  1571  	// Ensure that rangeDelIter is not closed until the compaction is
  1572  	// finished. This is necessary because range tombstone processing
  1573  	// requires the range tombstones to be held in memory for up to the
  1574  	// lifetime of the compaction.
  1575  	closer := rangeDelIter
  1576  	rangeDelIter = noCloseIter{rangeDelIter}
  1577  
  1578  	// Truncate the range tombstones returned by the iterator to the
  1579  	// upper bound of the atomic compaction unit of the file. We want to
  1580  	// truncate the range tombstone to the bounds of the file, but files
  1581  	// with split user keys pose an obstacle: The file's largest bound
  1582  	// is inclusive whereas the range tombstone's end is exclusive.
  1583  	//
  1584  	// Consider the example:
  1585  	//
  1586  	//   000001:[b-f#200]         range del [c,k)
  1587  	//   000002:[f#190-g#inf]     range del [c,k)
  1588  	//   000003:[g#500-i#3]
  1589  	//
  1590  	// Files 000001 and 000002 contain the untruncated range tombstones
  1591  	// [c,k). While the keyspace covered by 000003 was at one point
  1592  	// deleted by the tombstone [c,k), the tombstone may have already
  1593  	// been compacted away and the file does not contain an untruncated
  1594  	// range tombstone. We want to bound 000001's tombstone to the file
  1595  	// bounds, but it's not possible to encode a range tombstone with an
  1596  	// end boundary within a user key (eg, between sequence numbers
  1597  	// f#200 and f#190). Instead, we expand 000001 to its atomic
  1598  	// compaction unit (000001 and 000002) and truncate the tombstone to
  1599  	// g#inf.
  1600  	//
  1601  	// NB: We must not use the atomic compaction unit of the entire
  1602  	// compaction, because the [c,k) tombstone contained in the file
  1603  	// 000001 ≥ g. If 000001, 000002 and 000003 are all included in the
  1604  	// same compaction, the compaction's atomic compaction unit includes
  1605  	// 000003. However 000003's keys must not be covered by 000001's
  1606  	// untruncated range tombstone.
  1607  	//
  1608  	// Note that we need do this truncation at read time in order to
  1609  	// handle sstables generated by RocksDB and earlier versions of
  1610  	// Pebble which do not truncate range tombstones to atomic
  1611  	// compaction unit boundaries at write time.
  1612  	//
  1613  	// The current Pebble compaction logic DOES truncate tombstones to
  1614  	// atomic unit boundaries at compaction time too.
  1615  	atomicUnit, _ := expandToAtomicUnit(c.cmp, f.Slice(), true /* disableIsCompacting */)
  1616  	lowerBound, upperBound := manifest.KeyRange(c.cmp, atomicUnit.Iter())
  1617  	// Range deletion tombstones are often written to sstables
  1618  	// untruncated on the end key side. However, they are still only
  1619  	// valid within a given file's bounds. The logic for writing range
  1620  	// tombstones to an output file sometimes has an incomplete view
  1621  	// of range tombstones outside the file's internal key bounds. Skip
  1622  	// any range tombstones completely outside file bounds.
  1623  	rangeDelIter = keyspan.Truncate(
  1624  		c.cmp, rangeDelIter, lowerBound.UserKey, upperBound.UserKey,
  1625  		&f.Smallest, &f.Largest, false, /* panicOnUpperTruncate */
  1626  	)
  1627  	return rangeDelIter, closer, nil
  1628  }
  1629  
  1630  func (c *compaction) String() string {
  1631  	if len(c.flushing) != 0 {
  1632  		return "flush\n"
  1633  	}
  1634  
  1635  	var buf bytes.Buffer
  1636  	for level := c.startLevel.level; level <= c.outputLevel.level; level++ {
  1637  		i := level - c.startLevel.level
  1638  		fmt.Fprintf(&buf, "%d:", level)
  1639  		iter := c.inputs[i].files.Iter()
  1640  		for f := iter.First(); f != nil; f = iter.Next() {
  1641  			fmt.Fprintf(&buf, " %s:%s-%s", f.FileNum, f.Smallest, f.Largest)
  1642  		}
  1643  		fmt.Fprintf(&buf, "\n")
  1644  	}
  1645  	return buf.String()
  1646  }
  1647  
  1648  type manualCompaction struct {
  1649  	// Count of the retries either due to too many concurrent compactions, or a
  1650  	// concurrent compaction to overlapping levels.
  1651  	retries     int
  1652  	level       int
  1653  	outputLevel int
  1654  	done        chan error
  1655  	start       []byte
  1656  	end         []byte
  1657  	split       bool
  1658  }
  1659  
  1660  type readCompaction struct {
  1661  	level int
  1662  	// [start, end] key ranges are used for de-duping.
  1663  	start []byte
  1664  	end   []byte
  1665  
  1666  	// The file associated with the compaction.
  1667  	// If the file no longer belongs in the same
  1668  	// level, then we skip the compaction.
  1669  	fileNum base.FileNum
  1670  }
  1671  
  1672  func (d *DB) addInProgressCompaction(c *compaction) {
  1673  	d.mu.compact.inProgress[c] = struct{}{}
  1674  	var isBase, isIntraL0 bool
  1675  	for _, cl := range c.inputs {
  1676  		iter := cl.files.Iter()
  1677  		for f := iter.First(); f != nil; f = iter.Next() {
  1678  			if f.IsCompacting() {
  1679  				d.opts.Logger.Fatalf("L%d->L%d: %s already being compacted", c.startLevel.level, c.outputLevel.level, f.FileNum)
  1680  			}
  1681  			f.SetCompactionState(manifest.CompactionStateCompacting)
  1682  			if c.startLevel != nil && c.outputLevel != nil && c.startLevel.level == 0 {
  1683  				if c.outputLevel.level == 0 {
  1684  					f.IsIntraL0Compacting = true
  1685  					isIntraL0 = true
  1686  				} else {
  1687  					isBase = true
  1688  				}
  1689  			}
  1690  		}
  1691  	}
  1692  
  1693  	if (isIntraL0 || isBase) && c.version.L0Sublevels != nil {
  1694  		l0Inputs := []manifest.LevelSlice{c.startLevel.files}
  1695  		if isIntraL0 {
  1696  			l0Inputs = append(l0Inputs, c.outputLevel.files)
  1697  		}
  1698  		if err := c.version.L0Sublevels.UpdateStateForStartedCompaction(l0Inputs, isBase); err != nil {
  1699  			d.opts.Logger.Fatalf("could not update state for compaction: %s", err)
  1700  		}
  1701  	}
  1702  }
  1703  
  1704  // Removes compaction markers from files in a compaction. The rollback parameter
  1705  // indicates whether the compaction state should be rolled back to its original
  1706  // state in the case of an unsuccessful compaction.
  1707  //
  1708  // DB.mu must be held when calling this method, however this method can drop and
  1709  // re-acquire that mutex. All writes to the manifest for this compaction should
  1710  // have completed by this point.
  1711  func (d *DB) clearCompactingState(c *compaction, rollback bool) {
  1712  	c.versionEditApplied = true
  1713  	for _, cl := range c.inputs {
  1714  		iter := cl.files.Iter()
  1715  		for f := iter.First(); f != nil; f = iter.Next() {
  1716  			if !f.IsCompacting() {
  1717  				d.opts.Logger.Fatalf("L%d->L%d: %s not being compacted", c.startLevel.level, c.outputLevel.level, f.FileNum)
  1718  			}
  1719  			if !rollback {
  1720  				// On success all compactions other than move-compactions transition the
  1721  				// file into the Compacted state. Move-compacted files become eligible
  1722  				// for compaction again and transition back to NotCompacting.
  1723  				if c.kind != compactionKindMove {
  1724  					f.SetCompactionState(manifest.CompactionStateCompacted)
  1725  				} else {
  1726  					f.SetCompactionState(manifest.CompactionStateNotCompacting)
  1727  				}
  1728  			} else {
  1729  				// Else, on rollback, all input files unconditionally transition back to
  1730  				// NotCompacting.
  1731  				f.SetCompactionState(manifest.CompactionStateNotCompacting)
  1732  			}
  1733  			f.IsIntraL0Compacting = false
  1734  		}
  1735  	}
  1736  	l0InProgress := inProgressL0Compactions(d.getInProgressCompactionInfoLocked(c))
  1737  	func() {
  1738  		// InitCompactingFileInfo requires that no other manifest writes be
  1739  		// happening in parallel with it, i.e. we're not in the midst of installing
  1740  		// another version. Otherwise, it's possible that we've created another
  1741  		// L0Sublevels instance, but not added it to the versions list, causing
  1742  		// all the indices in FileMetadata to be inaccurate. To ensure this,
  1743  		// grab the manifest lock.
  1744  		d.mu.versions.logLock()
  1745  		defer d.mu.versions.logUnlock()
  1746  		d.mu.versions.currentVersion().L0Sublevels.InitCompactingFileInfo(l0InProgress)
  1747  	}()
  1748  }
  1749  
  1750  func (d *DB) calculateDiskAvailableBytes() uint64 {
  1751  	if space, err := d.opts.FS.GetDiskUsage(d.dirname); err == nil {
  1752  		d.diskAvailBytes.Store(space.AvailBytes)
  1753  		return space.AvailBytes
  1754  	} else if !errors.Is(err, vfs.ErrUnsupported) {
  1755  		d.opts.EventListener.BackgroundError(err)
  1756  	}
  1757  	return d.diskAvailBytes.Load()
  1758  }
  1759  
  1760  func (d *DB) getDeletionPacerInfo() deletionPacerInfo {
  1761  	var pacerInfo deletionPacerInfo
  1762  	// Call GetDiskUsage after every file deletion. This may seem inefficient,
  1763  	// but in practice this was observed to take constant time, regardless of
  1764  	// volume size used, at least on linux with ext4 and zfs. All invocations
  1765  	// take 10 microseconds or less.
  1766  	pacerInfo.freeBytes = d.calculateDiskAvailableBytes()
  1767  	d.mu.Lock()
  1768  	pacerInfo.obsoleteBytes = d.mu.versions.metrics.Table.ObsoleteSize
  1769  	pacerInfo.liveBytes = uint64(d.mu.versions.metrics.Total().Size)
  1770  	d.mu.Unlock()
  1771  	return pacerInfo
  1772  }
  1773  
  1774  // onObsoleteTableDelete is called to update metrics when an sstable is deleted.
  1775  func (d *DB) onObsoleteTableDelete(fileSize uint64) {
  1776  	d.mu.Lock()
  1777  	d.mu.versions.metrics.Table.ObsoleteCount--
  1778  	d.mu.versions.metrics.Table.ObsoleteSize -= fileSize
  1779  	d.mu.Unlock()
  1780  }
  1781  
  1782  // maybeScheduleFlush schedules a flush if necessary.
  1783  //
  1784  // d.mu must be held when calling this.
  1785  func (d *DB) maybeScheduleFlush() {
  1786  	if d.mu.compact.flushing || d.closed.Load() != nil || d.opts.ReadOnly {
  1787  		return
  1788  	}
  1789  	if len(d.mu.mem.queue) <= 1 {
  1790  		return
  1791  	}
  1792  
  1793  	if !d.passedFlushThreshold() {
  1794  		return
  1795  	}
  1796  
  1797  	d.mu.compact.flushing = true
  1798  	go d.flush()
  1799  }
  1800  
  1801  func (d *DB) passedFlushThreshold() bool {
  1802  	var n int
  1803  	var size uint64
  1804  	for ; n < len(d.mu.mem.queue)-1; n++ {
  1805  		if !d.mu.mem.queue[n].readyForFlush() {
  1806  			break
  1807  		}
  1808  		if d.mu.mem.queue[n].flushForced {
  1809  			// A flush was forced. Pretend the memtable size is the configured
  1810  			// size. See minFlushSize below.
  1811  			size += d.opts.MemTableSize
  1812  		} else {
  1813  			size += d.mu.mem.queue[n].totalBytes()
  1814  		}
  1815  	}
  1816  	if n == 0 {
  1817  		// None of the immutable memtables are ready for flushing.
  1818  		return false
  1819  	}
  1820  
  1821  	// Only flush once the sum of the queued memtable sizes exceeds half the
  1822  	// configured memtable size. This prevents flushing of memtables at startup
  1823  	// while we're undergoing the ramp period on the memtable size. See
  1824  	// DB.newMemTable().
  1825  	minFlushSize := d.opts.MemTableSize / 2
  1826  	return size >= minFlushSize
  1827  }
  1828  
  1829  func (d *DB) maybeScheduleDelayedFlush(tbl *memTable, dur time.Duration) {
  1830  	var mem *flushableEntry
  1831  	for _, m := range d.mu.mem.queue {
  1832  		if m.flushable == tbl {
  1833  			mem = m
  1834  			break
  1835  		}
  1836  	}
  1837  	if mem == nil || mem.flushForced {
  1838  		return
  1839  	}
  1840  	deadline := d.timeNow().Add(dur)
  1841  	if !mem.delayedFlushForcedAt.IsZero() && deadline.After(mem.delayedFlushForcedAt) {
  1842  		// Already scheduled to flush sooner than within `dur`.
  1843  		return
  1844  	}
  1845  	mem.delayedFlushForcedAt = deadline
  1846  	go func() {
  1847  		timer := time.NewTimer(dur)
  1848  		defer timer.Stop()
  1849  
  1850  		select {
  1851  		case <-d.closedCh:
  1852  			return
  1853  		case <-mem.flushed:
  1854  			return
  1855  		case <-timer.C:
  1856  			d.commit.mu.Lock()
  1857  			defer d.commit.mu.Unlock()
  1858  			d.mu.Lock()
  1859  			defer d.mu.Unlock()
  1860  
  1861  			// NB: The timer may fire concurrently with a call to Close.  If a
  1862  			// Close call beat us to acquiring d.mu, d.closed holds ErrClosed,
  1863  			// and it's too late to flush anything. Otherwise, the Close call
  1864  			// will block on locking d.mu until we've finished scheduling the
  1865  			// flush and set `d.mu.compact.flushing` to true. Close will wait
  1866  			// for the current flush to complete.
  1867  			if d.closed.Load() != nil {
  1868  				return
  1869  			}
  1870  
  1871  			if d.mu.mem.mutable == tbl {
  1872  				d.makeRoomForWrite(nil)
  1873  			} else {
  1874  				mem.flushForced = true
  1875  			}
  1876  			d.maybeScheduleFlush()
  1877  		}
  1878  	}()
  1879  }
  1880  
  1881  func (d *DB) flush() {
  1882  	pprof.Do(context.Background(), flushLabels, func(context.Context) {
  1883  		flushingWorkStart := time.Now()
  1884  		d.mu.Lock()
  1885  		defer d.mu.Unlock()
  1886  		idleDuration := flushingWorkStart.Sub(d.mu.compact.noOngoingFlushStartTime)
  1887  		var bytesFlushed uint64
  1888  		var err error
  1889  		if bytesFlushed, err = d.flush1(); err != nil {
  1890  			// TODO(peter): count consecutive flush errors and backoff.
  1891  			d.opts.EventListener.BackgroundError(err)
  1892  		}
  1893  		d.mu.compact.flushing = false
  1894  		d.mu.compact.noOngoingFlushStartTime = time.Now()
  1895  		workDuration := d.mu.compact.noOngoingFlushStartTime.Sub(flushingWorkStart)
  1896  		d.mu.compact.flushWriteThroughput.Bytes += int64(bytesFlushed)
  1897  		d.mu.compact.flushWriteThroughput.WorkDuration += workDuration
  1898  		d.mu.compact.flushWriteThroughput.IdleDuration += idleDuration
  1899  		// More flush work may have arrived while we were flushing, so schedule
  1900  		// another flush if needed.
  1901  		d.maybeScheduleFlush()
  1902  		// The flush may have produced too many files in a level, so schedule a
  1903  		// compaction if needed.
  1904  		d.maybeScheduleCompaction()
  1905  		d.mu.compact.cond.Broadcast()
  1906  	})
  1907  }
  1908  
  1909  // runIngestFlush is used to generate a flush version edit for sstables which
  1910  // were ingested as flushables. Both DB.mu and the manifest lock must be held
  1911  // while runIngestFlush is called.
  1912  func (d *DB) runIngestFlush(c *compaction) (*manifest.VersionEdit, error) {
  1913  	if len(c.flushing) != 1 {
  1914  		panic("pebble: ingestedFlushable must be flushed one at a time.")
  1915  	}
  1916  
  1917  	// Construct the VersionEdit, levelMetrics etc.
  1918  	c.metrics = make(map[int]*LevelMetrics, numLevels)
  1919  	// Finding the target level for ingestion must use the latest version
  1920  	// after the logLock has been acquired.
  1921  	c.version = d.mu.versions.currentVersion()
  1922  
  1923  	baseLevel := d.mu.versions.picker.getBaseLevel()
  1924  	iterOpts := IterOptions{logger: d.opts.Logger}
  1925  	ve := &versionEdit{}
  1926  	var level int
  1927  	var err error
  1928  	var fileToSplit *fileMetadata
  1929  	var ingestSplitFiles []ingestSplitFile
  1930  	for _, file := range c.flushing[0].flushable.(*ingestedFlushable).files {
  1931  		suggestSplit := d.opts.Experimental.IngestSplit != nil && d.opts.Experimental.IngestSplit() &&
  1932  			d.FormatMajorVersion() >= FormatVirtualSSTables
  1933  		level, fileToSplit, err = ingestTargetLevel(
  1934  			d.newIters, d.tableNewRangeKeyIter, iterOpts, d.opts.Comparer,
  1935  			c.version, baseLevel, d.mu.compact.inProgress, file.FileMetadata,
  1936  			suggestSplit,
  1937  		)
  1938  		if err != nil {
  1939  			return nil, err
  1940  		}
  1941  		ve.NewFiles = append(ve.NewFiles, newFileEntry{Level: level, Meta: file.FileMetadata})
  1942  		if fileToSplit != nil {
  1943  			ingestSplitFiles = append(ingestSplitFiles, ingestSplitFile{
  1944  				ingestFile: file.FileMetadata,
  1945  				splitFile:  fileToSplit,
  1946  				level:      level,
  1947  			})
  1948  		}
  1949  		levelMetrics := c.metrics[level]
  1950  		if levelMetrics == nil {
  1951  			levelMetrics = &LevelMetrics{}
  1952  			c.metrics[level] = levelMetrics
  1953  		}
  1954  		levelMetrics.BytesIngested += file.Size
  1955  		levelMetrics.TablesIngested++
  1956  	}
  1957  
  1958  	updateLevelMetricsOnExcise := func(m *fileMetadata, level int, added []newFileEntry) {
  1959  		levelMetrics := c.metrics[level]
  1960  		if levelMetrics == nil {
  1961  			levelMetrics = &LevelMetrics{}
  1962  			c.metrics[level] = levelMetrics
  1963  		}
  1964  		levelMetrics.NumFiles--
  1965  		levelMetrics.Size -= int64(m.Size)
  1966  		for i := range added {
  1967  			levelMetrics.NumFiles++
  1968  			levelMetrics.Size += int64(added[i].Meta.Size)
  1969  		}
  1970  	}
  1971  
  1972  	if len(ingestSplitFiles) > 0 {
  1973  		ve.DeletedFiles = make(map[manifest.DeletedFileEntry]*manifest.FileMetadata)
  1974  		replacedFiles := make(map[base.FileNum][]newFileEntry)
  1975  		if err := d.ingestSplit(ve, updateLevelMetricsOnExcise, ingestSplitFiles, replacedFiles); err != nil {
  1976  			return nil, err
  1977  		}
  1978  	}
  1979  
  1980  	return ve, nil
  1981  }
  1982  
  1983  // flush runs a compaction that copies the immutable memtables from memory to
  1984  // disk.
  1985  //
  1986  // d.mu must be held when calling this, but the mutex may be dropped and
  1987  // re-acquired during the course of this method.
  1988  func (d *DB) flush1() (bytesFlushed uint64, err error) {
  1989  	// NB: The flushable queue can contain flushables of type ingestedFlushable.
  1990  	// The sstables in ingestedFlushable.files must be placed into the appropriate
  1991  	// level in the lsm. Let's say the flushable queue contains a prefix of
  1992  	// regular immutable memtables, then an ingestedFlushable, and then the
  1993  	// mutable memtable. When the flush of the ingestedFlushable is performed,
  1994  	// it needs an updated view of the lsm. That is, the prefix of immutable
  1995  	// memtables must have already been flushed. Similarly, if there are two
  1996  	// contiguous ingestedFlushables in the queue, then the first flushable must
  1997  	// be flushed, so that the second flushable can see an updated view of the
  1998  	// lsm.
  1999  	//
  2000  	// Given the above, we restrict flushes to either some prefix of regular
  2001  	// memtables, or a single flushable of type ingestedFlushable. The DB.flush
  2002  	// function will call DB.maybeScheduleFlush again, so a new flush to finish
  2003  	// the remaining flush work should be scheduled right away.
  2004  	//
  2005  	// NB: Large batches placed in the flushable queue share the WAL with the
  2006  	// previous memtable in the queue. We must ensure the property that both the
  2007  	// large batch and the memtable with which it shares a WAL are flushed
  2008  	// together. The property ensures that the minimum unflushed log number
  2009  	// isn't incremented incorrectly. Since a flushableBatch.readyToFlush always
  2010  	// returns true, and since the large batch will always be placed right after
  2011  	// the memtable with which it shares a WAL, the property is naturally
  2012  	// ensured. The large batch will always be placed after the memtable with
  2013  	// which it shares a WAL because we ensure it in DB.commitWrite by holding
  2014  	// the commitPipeline.mu and then holding DB.mu. As an extra defensive
  2015  	// measure, if we try to flush the memtable without also flushing the
  2016  	// flushable batch in the same flush, since the memtable and flushableBatch
  2017  	// have the same logNum, the logNum invariant check below will trigger.
  2018  	var n, inputs int
  2019  	var inputBytes uint64
  2020  	var ingest bool
  2021  	for ; n < len(d.mu.mem.queue)-1; n++ {
  2022  		if f, ok := d.mu.mem.queue[n].flushable.(*ingestedFlushable); ok {
  2023  			if n == 0 {
  2024  				// The first flushable is of type ingestedFlushable. Since these
  2025  				// must be flushed individually, we perform a flush for just
  2026  				// this.
  2027  				if !f.readyForFlush() {
  2028  					// This check is almost unnecessary, but we guard against it
  2029  					// just in case this invariant changes in the future.
  2030  					panic("pebble: ingestedFlushable should always be ready to flush.")
  2031  				}
  2032  				// By setting n = 1, we ensure that the first flushable(n == 0)
  2033  				// is scheduled for a flush. The number of tables added is equal to the
  2034  				// number of files in the ingest operation.
  2035  				n = 1
  2036  				inputs = len(f.files)
  2037  				ingest = true
  2038  				break
  2039  			} else {
  2040  				// There was some prefix of flushables which weren't of type
  2041  				// ingestedFlushable. So, perform a flush for those.
  2042  				break
  2043  			}
  2044  		}
  2045  		if !d.mu.mem.queue[n].readyForFlush() {
  2046  			break
  2047  		}
  2048  		inputBytes += d.mu.mem.queue[n].inuseBytes()
  2049  	}
  2050  	if n == 0 {
  2051  		// None of the immutable memtables are ready for flushing.
  2052  		return 0, nil
  2053  	}
  2054  	if !ingest {
  2055  		// Flushes of memtables add the prefix of n memtables from the flushable
  2056  		// queue.
  2057  		inputs = n
  2058  	}
  2059  
  2060  	// Require that every memtable being flushed has a log number less than the
  2061  	// new minimum unflushed log number.
  2062  	minUnflushedLogNum := d.mu.mem.queue[n].logNum
  2063  	if !d.opts.DisableWAL {
  2064  		for i := 0; i < n; i++ {
  2065  			if logNum := d.mu.mem.queue[i].logNum; logNum >= minUnflushedLogNum {
  2066  				panic(errors.AssertionFailedf("logNum invariant violated: flushing %d items; %d:type=%T,logNum=%d; %d:type=%T,logNum=%d",
  2067  					n,
  2068  					i, d.mu.mem.queue[i].flushable, logNum,
  2069  					n, d.mu.mem.queue[n].flushable, minUnflushedLogNum))
  2070  			}
  2071  		}
  2072  	}
  2073  
  2074  	c := newFlush(d.opts, d.mu.versions.currentVersion(),
  2075  		d.mu.versions.picker.getBaseLevel(), d.mu.mem.queue[:n], d.timeNow())
  2076  	d.addInProgressCompaction(c)
  2077  
  2078  	jobID := d.mu.nextJobID
  2079  	d.mu.nextJobID++
  2080  	d.opts.EventListener.FlushBegin(FlushInfo{
  2081  		JobID:      jobID,
  2082  		Input:      inputs,
  2083  		InputBytes: inputBytes,
  2084  		Ingest:     ingest,
  2085  	})
  2086  	startTime := d.timeNow()
  2087  
  2088  	var ve *manifest.VersionEdit
  2089  	var pendingOutputs []physicalMeta
  2090  	var stats compactStats
  2091  	// To determine the target level of the files in the ingestedFlushable, we
  2092  	// need to acquire the logLock, and not release it for that duration. Since,
  2093  	// we need to acquire the logLock below to perform the logAndApply step
  2094  	// anyway, we create the VersionEdit for ingestedFlushable outside of
  2095  	// runCompaction. For all other flush cases, we construct the VersionEdit
  2096  	// inside runCompaction.
  2097  	if c.kind != compactionKindIngestedFlushable {
  2098  		ve, pendingOutputs, stats, err = d.runCompaction(jobID, c)
  2099  	}
  2100  
  2101  	// Acquire logLock. This will be released either on an error, by way of
  2102  	// logUnlock, or through a call to logAndApply if there is no error.
  2103  	d.mu.versions.logLock()
  2104  
  2105  	if c.kind == compactionKindIngestedFlushable {
  2106  		ve, err = d.runIngestFlush(c)
  2107  	}
  2108  
  2109  	info := FlushInfo{
  2110  		JobID:      jobID,
  2111  		Input:      inputs,
  2112  		InputBytes: inputBytes,
  2113  		Duration:   d.timeNow().Sub(startTime),
  2114  		Done:       true,
  2115  		Ingest:     ingest,
  2116  		Err:        err,
  2117  	}
  2118  	if err == nil {
  2119  		for i := range ve.NewFiles {
  2120  			e := &ve.NewFiles[i]
  2121  			info.Output = append(info.Output, e.Meta.TableInfo())
  2122  			// Ingested tables are not necessarily flushed to L0. Record the level of
  2123  			// each ingested file explicitly.
  2124  			if ingest {
  2125  				info.IngestLevels = append(info.IngestLevels, e.Level)
  2126  			}
  2127  		}
  2128  		if len(ve.NewFiles) == 0 {
  2129  			info.Err = errEmptyTable
  2130  		}
  2131  
  2132  		// The flush succeeded or it produced an empty sstable. In either case we
  2133  		// want to bump the minimum unflushed log number to the log number of the
  2134  		// oldest unflushed memtable.
  2135  		ve.MinUnflushedLogNum = minUnflushedLogNum
  2136  		if c.kind != compactionKindIngestedFlushable {
  2137  			metrics := c.metrics[0]
  2138  			if d.opts.DisableWAL {
  2139  				// If the WAL is disabled, every flushable has a zero [logSize],
  2140  				// resulting in zero bytes in. Instead, use the number of bytes we
  2141  				// flushed as the BytesIn. This ensures we get a reasonable w-amp
  2142  				// calculation even when the WAL is disabled.
  2143  				metrics.BytesIn = metrics.BytesFlushed
  2144  			} else {
  2145  				for i := 0; i < n; i++ {
  2146  					metrics.BytesIn += d.mu.mem.queue[i].logSize
  2147  				}
  2148  			}
  2149  		} else if len(ve.DeletedFiles) > 0 {
  2150  			// c.kind == compactionKindIngestedFlushable && we have deleted files due
  2151  			// to ingest-time splits.
  2152  			//
  2153  			// Iterate through all other compactions, and check if their inputs have
  2154  			// been replaced due to an ingest-time split. In that case, cancel the
  2155  			// compaction.
  2156  			for c2 := range d.mu.compact.inProgress {
  2157  				for i := range c2.inputs {
  2158  					iter := c2.inputs[i].files.Iter()
  2159  					for f := iter.First(); f != nil; f = iter.Next() {
  2160  						if _, ok := ve.DeletedFiles[deletedFileEntry{FileNum: f.FileNum, Level: c2.inputs[i].level}]; ok {
  2161  							c2.cancel.Store(true)
  2162  							break
  2163  						}
  2164  					}
  2165  				}
  2166  			}
  2167  		}
  2168  		err = d.mu.versions.logAndApply(jobID, ve, c.metrics, false, /* forceRotation */
  2169  			func() []compactionInfo { return d.getInProgressCompactionInfoLocked(c) })
  2170  		if err != nil {
  2171  			info.Err = err
  2172  			// TODO(peter): untested.
  2173  			for _, f := range pendingOutputs {
  2174  				// Note that the FileBacking for the file metadata might not have
  2175  				// been set yet. So, we directly use the FileNum. Since these
  2176  				// files were generated as compaction outputs, these must be
  2177  				// physical files on disk. This property might not hold once
  2178  				// https://github.com/cockroachdb/pebble/issues/389 is
  2179  				// implemented if #389 creates virtual sstables as output files.
  2180  				d.mu.versions.obsoleteTables = append(
  2181  					d.mu.versions.obsoleteTables,
  2182  					fileInfo{f.FileNum.DiskFileNum(), f.Size},
  2183  				)
  2184  			}
  2185  			d.mu.versions.updateObsoleteTableMetricsLocked()
  2186  		}
  2187  	} else {
  2188  		// We won't be performing the logAndApply step because of the error,
  2189  		// so logUnlock.
  2190  		d.mu.versions.logUnlock()
  2191  	}
  2192  
  2193  	bytesFlushed = c.bytesIterated
  2194  
  2195  	// If err != nil, then the flush will be retried, and we will recalculate
  2196  	// these metrics.
  2197  	if err == nil {
  2198  		d.mu.snapshots.cumulativePinnedCount += stats.cumulativePinnedKeys
  2199  		d.mu.snapshots.cumulativePinnedSize += stats.cumulativePinnedSize
  2200  		d.mu.versions.metrics.Keys.MissizedTombstonesCount += stats.countMissizedDels
  2201  		d.maybeUpdateDeleteCompactionHints(c)
  2202  	}
  2203  
  2204  	d.clearCompactingState(c, err != nil)
  2205  	delete(d.mu.compact.inProgress, c)
  2206  	d.mu.versions.incrementCompactions(c.kind, c.extraLevels, c.pickerMetrics)
  2207  
  2208  	var flushed flushableList
  2209  	if err == nil {
  2210  		flushed = d.mu.mem.queue[:n]
  2211  		d.mu.mem.queue = d.mu.mem.queue[n:]
  2212  		d.updateReadStateLocked(d.opts.DebugCheck)
  2213  		d.updateTableStatsLocked(ve.NewFiles)
  2214  		if ingest {
  2215  			d.mu.versions.metrics.Flush.AsIngestCount++
  2216  			for _, l := range c.metrics {
  2217  				d.mu.versions.metrics.Flush.AsIngestBytes += l.BytesIngested
  2218  				d.mu.versions.metrics.Flush.AsIngestTableCount += l.TablesIngested
  2219  			}
  2220  		}
  2221  
  2222  		// Update if any eventually file-only snapshots have now transitioned to
  2223  		// being file-only.
  2224  		earliestUnflushedSeqNum := d.getEarliestUnflushedSeqNumLocked()
  2225  		currentVersion := d.mu.versions.currentVersion()
  2226  		for s := d.mu.snapshots.root.next; s != &d.mu.snapshots.root; {
  2227  			if s.efos == nil {
  2228  				s = s.next
  2229  				continue
  2230  			}
  2231  			if base.Visible(earliestUnflushedSeqNum, s.efos.seqNum, InternalKeySeqNumMax) {
  2232  				s = s.next
  2233  				continue
  2234  			}
  2235  			if s.efos.excised.Load() {
  2236  				// If a concurrent excise has happened that overlaps with one of the key
  2237  				// ranges this snapshot is interested in, this EFOS cannot transition to
  2238  				// a file-only snapshot as keys in that range could now be deleted. Move
  2239  				// onto the next snapshot.
  2240  				s = s.next
  2241  				continue
  2242  			}
  2243  			currentVersion.Ref()
  2244  
  2245  			// NB: s.efos.transitionToFileOnlySnapshot could close s, in which
  2246  			// case s.next would be nil. Save it before calling it.
  2247  			next := s.next
  2248  			_ = s.efos.transitionToFileOnlySnapshot(currentVersion)
  2249  			s = next
  2250  		}
  2251  	}
  2252  	// Signal FlushEnd after installing the new readState. This helps for unit
  2253  	// tests that use the callback to trigger a read using an iterator with
  2254  	// IterOptions.OnlyReadGuaranteedDurable.
  2255  	info.TotalDuration = d.timeNow().Sub(startTime)
  2256  	d.opts.EventListener.FlushEnd(info)
  2257  
  2258  	// The order of these operations matters here for ease of testing.
  2259  	// Removing the reader reference first allows tests to be guaranteed that
  2260  	// the memtable reservation has been released by the time a synchronous
  2261  	// flush returns. readerUnrefLocked may also produce obsolete files so the
  2262  	// call to deleteObsoleteFiles must happen after it.
  2263  	for i := range flushed {
  2264  		flushed[i].readerUnrefLocked(true)
  2265  	}
  2266  
  2267  	d.deleteObsoleteFiles(jobID)
  2268  
  2269  	// Mark all the memtables we flushed as flushed.
  2270  	for i := range flushed {
  2271  		close(flushed[i].flushed)
  2272  	}
  2273  
  2274  	return bytesFlushed, err
  2275  }
  2276  
  2277  // maybeScheduleCompactionAsync should be used when
  2278  // we want to possibly schedule a compaction, but don't
  2279  // want to eat the cost of running maybeScheduleCompaction.
  2280  // This method should be launched in a separate goroutine.
  2281  // d.mu must not be held when this is called.
  2282  func (d *DB) maybeScheduleCompactionAsync() {
  2283  	defer d.compactionSchedulers.Done()
  2284  
  2285  	d.mu.Lock()
  2286  	d.maybeScheduleCompaction()
  2287  	d.mu.Unlock()
  2288  }
  2289  
  2290  // maybeScheduleCompaction schedules a compaction if necessary.
  2291  //
  2292  // d.mu must be held when calling this.
  2293  func (d *DB) maybeScheduleCompaction() {
  2294  	d.maybeScheduleCompactionPicker(pickAuto)
  2295  }
  2296  
  2297  func pickAuto(picker compactionPicker, env compactionEnv) *pickedCompaction {
  2298  	return picker.pickAuto(env)
  2299  }
  2300  
  2301  func pickElisionOnly(picker compactionPicker, env compactionEnv) *pickedCompaction {
  2302  	return picker.pickElisionOnlyCompaction(env)
  2303  }
  2304  
  2305  // maybeScheduleCompactionPicker schedules a compaction if necessary,
  2306  // calling `pickFunc` to pick automatic compactions.
  2307  //
  2308  // d.mu must be held when calling this.
  2309  func (d *DB) maybeScheduleCompactionPicker(
  2310  	pickFunc func(compactionPicker, compactionEnv) *pickedCompaction,
  2311  ) {
  2312  	if d.closed.Load() != nil || d.opts.ReadOnly {
  2313  		return
  2314  	}
  2315  	maxConcurrentCompactions := d.opts.MaxConcurrentCompactions()
  2316  	if d.mu.compact.compactingCount >= maxConcurrentCompactions {
  2317  		if len(d.mu.compact.manual) > 0 {
  2318  			// Inability to run head blocks later manual compactions.
  2319  			d.mu.compact.manual[0].retries++
  2320  		}
  2321  		return
  2322  	}
  2323  
  2324  	// Compaction picking needs a coherent view of a Version. In particular, we
  2325  	// need to exlude concurrent ingestions from making a decision on which level
  2326  	// to ingest into that conflicts with our compaction
  2327  	// decision. versionSet.logLock provides the necessary mutual exclusion.
  2328  	d.mu.versions.logLock()
  2329  	defer d.mu.versions.logUnlock()
  2330  
  2331  	// Check for the closed flag again, in case the DB was closed while we were
  2332  	// waiting for logLock().
  2333  	if d.closed.Load() != nil {
  2334  		return
  2335  	}
  2336  
  2337  	env := compactionEnv{
  2338  		diskAvailBytes:          d.diskAvailBytes.Load(),
  2339  		earliestSnapshotSeqNum:  d.mu.snapshots.earliest(),
  2340  		earliestUnflushedSeqNum: d.getEarliestUnflushedSeqNumLocked(),
  2341  	}
  2342  
  2343  	// Check for delete-only compactions first, because they're expected to be
  2344  	// cheap and reduce future compaction work.
  2345  	if !d.opts.private.disableDeleteOnlyCompactions &&
  2346  		len(d.mu.compact.deletionHints) > 0 &&
  2347  		d.mu.compact.compactingCount < maxConcurrentCompactions &&
  2348  		!d.opts.DisableAutomaticCompactions {
  2349  		v := d.mu.versions.currentVersion()
  2350  		snapshots := d.mu.snapshots.toSlice()
  2351  		inputs, unresolvedHints := checkDeleteCompactionHints(d.cmp, v, d.mu.compact.deletionHints, snapshots)
  2352  		d.mu.compact.deletionHints = unresolvedHints
  2353  
  2354  		if len(inputs) > 0 {
  2355  			c := newDeleteOnlyCompaction(d.opts, v, inputs, d.timeNow())
  2356  			d.mu.compact.compactingCount++
  2357  			d.addInProgressCompaction(c)
  2358  			go d.compact(c, nil)
  2359  		}
  2360  	}
  2361  
  2362  	for len(d.mu.compact.manual) > 0 && d.mu.compact.compactingCount < maxConcurrentCompactions {
  2363  		v := d.mu.versions.currentVersion()
  2364  		manual := d.mu.compact.manual[0]
  2365  		env.inProgressCompactions = d.getInProgressCompactionInfoLocked(nil)
  2366  		pc, retryLater := pickManualCompaction(v, d.opts, env, d.mu.versions.picker.getBaseLevel(), manual)
  2367  		if pc != nil {
  2368  			c := newCompaction(pc, d.opts, d.timeNow(), d.ObjProvider())
  2369  			d.mu.compact.manual = d.mu.compact.manual[1:]
  2370  			d.mu.compact.compactingCount++
  2371  			d.addInProgressCompaction(c)
  2372  			go d.compact(c, manual.done)
  2373  		} else if !retryLater {
  2374  			// Noop
  2375  			d.mu.compact.manual = d.mu.compact.manual[1:]
  2376  			manual.done <- nil
  2377  		} else {
  2378  			// Inability to run head blocks later manual compactions.
  2379  			manual.retries++
  2380  			break
  2381  		}
  2382  	}
  2383  
  2384  	for !d.opts.DisableAutomaticCompactions && d.mu.compact.compactingCount < maxConcurrentCompactions {
  2385  		env.inProgressCompactions = d.getInProgressCompactionInfoLocked(nil)
  2386  		env.readCompactionEnv = readCompactionEnv{
  2387  			readCompactions:          &d.mu.compact.readCompactions,
  2388  			flushing:                 d.mu.compact.flushing || d.passedFlushThreshold(),
  2389  			rescheduleReadCompaction: &d.mu.compact.rescheduleReadCompaction,
  2390  		}
  2391  		pc := pickFunc(d.mu.versions.picker, env)
  2392  		if pc == nil {
  2393  			break
  2394  		}
  2395  		c := newCompaction(pc, d.opts, d.timeNow(), d.ObjProvider())
  2396  		d.mu.compact.compactingCount++
  2397  		d.addInProgressCompaction(c)
  2398  		go d.compact(c, nil)
  2399  	}
  2400  }
  2401  
  2402  // deleteCompactionHintType indicates whether the deleteCompactionHint was
  2403  // generated from a span containing a range del (point key only), a range key
  2404  // delete (range key only), or both a point and range key.
  2405  type deleteCompactionHintType uint8
  2406  
  2407  const (
  2408  	// NOTE: While these are primarily used as enumeration types, they are also
  2409  	// used for some bitwise operations. Care should be taken when updating.
  2410  	deleteCompactionHintTypeUnknown deleteCompactionHintType = iota
  2411  	deleteCompactionHintTypePointKeyOnly
  2412  	deleteCompactionHintTypeRangeKeyOnly
  2413  	deleteCompactionHintTypePointAndRangeKey
  2414  )
  2415  
  2416  // String implements fmt.Stringer.
  2417  func (h deleteCompactionHintType) String() string {
  2418  	switch h {
  2419  	case deleteCompactionHintTypeUnknown:
  2420  		return "unknown"
  2421  	case deleteCompactionHintTypePointKeyOnly:
  2422  		return "point-key-only"
  2423  	case deleteCompactionHintTypeRangeKeyOnly:
  2424  		return "range-key-only"
  2425  	case deleteCompactionHintTypePointAndRangeKey:
  2426  		return "point-and-range-key"
  2427  	default:
  2428  		panic(fmt.Sprintf("unknown hint type: %d", h))
  2429  	}
  2430  }
  2431  
  2432  // compactionHintFromKeys returns a deleteCompactionHintType given a slice of
  2433  // keyspan.Keys.
  2434  func compactionHintFromKeys(keys []keyspan.Key) deleteCompactionHintType {
  2435  	var hintType deleteCompactionHintType
  2436  	for _, k := range keys {
  2437  		switch k.Kind() {
  2438  		case base.InternalKeyKindRangeDelete:
  2439  			hintType |= deleteCompactionHintTypePointKeyOnly
  2440  		case base.InternalKeyKindRangeKeyDelete:
  2441  			hintType |= deleteCompactionHintTypeRangeKeyOnly
  2442  		default:
  2443  			panic(fmt.Sprintf("unsupported key kind: %s", k.Kind()))
  2444  		}
  2445  	}
  2446  	return hintType
  2447  }
  2448  
  2449  // A deleteCompactionHint records a user key and sequence number span that has been
  2450  // deleted by a range tombstone. A hint is recorded if at least one sstable
  2451  // falls completely within both the user key and sequence number spans.
  2452  // Once the tombstones and the observed completely-contained sstables fall
  2453  // into the same snapshot stripe, a delete-only compaction may delete any
  2454  // sstables within the range.
  2455  type deleteCompactionHint struct {
  2456  	// The type of key span that generated this hint (point key, range key, or
  2457  	// both).
  2458  	hintType deleteCompactionHintType
  2459  	// start and end are user keys specifying a key range [start, end) of
  2460  	// deleted keys.
  2461  	start []byte
  2462  	end   []byte
  2463  	// The level of the file containing the range tombstone(s) when the hint
  2464  	// was created. Only lower levels need to be searched for files that may
  2465  	// be deleted.
  2466  	tombstoneLevel int
  2467  	// The file containing the range tombstone(s) that created the hint.
  2468  	tombstoneFile *fileMetadata
  2469  	// The smallest and largest sequence numbers of the abutting tombstones
  2470  	// merged to form this hint. All of a tables' keys must be less than the
  2471  	// tombstone smallest sequence number to be deleted. All of a tables'
  2472  	// sequence numbers must fall into the same snapshot stripe as the
  2473  	// tombstone largest sequence number to be deleted.
  2474  	tombstoneLargestSeqNum  uint64
  2475  	tombstoneSmallestSeqNum uint64
  2476  	// The smallest sequence number of a sstable that was found to be covered
  2477  	// by this hint. The hint cannot be resolved until this sequence number is
  2478  	// in the same snapshot stripe as the largest tombstone sequence number.
  2479  	// This is set when a hint is created, so the LSM may look different and
  2480  	// notably no longer contain the sstable that contained the key at this
  2481  	// sequence number.
  2482  	fileSmallestSeqNum uint64
  2483  }
  2484  
  2485  func (h deleteCompactionHint) String() string {
  2486  	return fmt.Sprintf(
  2487  		"L%d.%s %s-%s seqnums(tombstone=%d-%d, file-smallest=%d, type=%s)",
  2488  		h.tombstoneLevel, h.tombstoneFile.FileNum, h.start, h.end,
  2489  		h.tombstoneSmallestSeqNum, h.tombstoneLargestSeqNum, h.fileSmallestSeqNum,
  2490  		h.hintType,
  2491  	)
  2492  }
  2493  
  2494  func (h *deleteCompactionHint) canDelete(cmp Compare, m *fileMetadata, snapshots []uint64) bool {
  2495  	// The file can only be deleted if all of its keys are older than the
  2496  	// earliest tombstone aggregated into the hint.
  2497  	if m.LargestSeqNum >= h.tombstoneSmallestSeqNum || m.SmallestSeqNum < h.fileSmallestSeqNum {
  2498  		return false
  2499  	}
  2500  
  2501  	// The file's oldest key must  be in the same snapshot stripe as the
  2502  	// newest tombstone. NB: We already checked the hint's sequence numbers,
  2503  	// but this file's oldest sequence number might be lower than the hint's
  2504  	// smallest sequence number despite the file falling within the key range
  2505  	// if this file was constructed after the hint by a compaction.
  2506  	ti, _ := snapshotIndex(h.tombstoneLargestSeqNum, snapshots)
  2507  	fi, _ := snapshotIndex(m.SmallestSeqNum, snapshots)
  2508  	if ti != fi {
  2509  		return false
  2510  	}
  2511  
  2512  	switch h.hintType {
  2513  	case deleteCompactionHintTypePointKeyOnly:
  2514  		// A hint generated by a range del span cannot delete tables that contain
  2515  		// range keys.
  2516  		if m.HasRangeKeys {
  2517  			return false
  2518  		}
  2519  	case deleteCompactionHintTypeRangeKeyOnly:
  2520  		// A hint generated by a range key del span cannot delete tables that
  2521  		// contain point keys.
  2522  		if m.HasPointKeys {
  2523  			return false
  2524  		}
  2525  	case deleteCompactionHintTypePointAndRangeKey:
  2526  		// A hint from a span that contains both range dels *and* range keys can
  2527  		// only be deleted if both bounds fall within the hint. The next check takes
  2528  		// care of this.
  2529  	default:
  2530  		panic(fmt.Sprintf("pebble: unknown delete compaction hint type: %d", h.hintType))
  2531  	}
  2532  
  2533  	// The file's keys must be completely contained within the hint range.
  2534  	return cmp(h.start, m.Smallest.UserKey) <= 0 && cmp(m.Largest.UserKey, h.end) < 0
  2535  }
  2536  
  2537  func (d *DB) maybeUpdateDeleteCompactionHints(c *compaction) {
  2538  	// Compactions that zero sequence numbers can interfere with compaction
  2539  	// deletion hints. Deletion hints apply to tables containing keys older
  2540  	// than a threshold. If a key more recent than the threshold is zeroed in
  2541  	// a compaction, a delete-only compaction may mistake it as meeting the
  2542  	// threshold and drop a table containing live data.
  2543  	//
  2544  	// To avoid this scenario, compactions that zero sequence numbers remove
  2545  	// any conflicting deletion hints. A deletion hint is conflicting if both
  2546  	// of the following conditions apply:
  2547  	// * its key space overlaps with the compaction
  2548  	// * at least one of its inputs contains a key as recent as one of the
  2549  	//   hint's tombstones.
  2550  	//
  2551  	if !c.allowedZeroSeqNum {
  2552  		return
  2553  	}
  2554  
  2555  	updatedHints := d.mu.compact.deletionHints[:0]
  2556  	for _, h := range d.mu.compact.deletionHints {
  2557  		// If the compaction's key space is disjoint from the hint's key
  2558  		// space, the zeroing of sequence numbers won't affect the hint. Keep
  2559  		// the hint.
  2560  		keysDisjoint := d.cmp(h.end, c.smallest.UserKey) < 0 || d.cmp(h.start, c.largest.UserKey) > 0
  2561  		if keysDisjoint {
  2562  			updatedHints = append(updatedHints, h)
  2563  			continue
  2564  		}
  2565  
  2566  		// All of the compaction's inputs must be older than the hint's
  2567  		// tombstones.
  2568  		inputsOlder := true
  2569  		for _, in := range c.inputs {
  2570  			iter := in.files.Iter()
  2571  			for f := iter.First(); f != nil; f = iter.Next() {
  2572  				inputsOlder = inputsOlder && f.LargestSeqNum < h.tombstoneSmallestSeqNum
  2573  			}
  2574  		}
  2575  		if inputsOlder {
  2576  			updatedHints = append(updatedHints, h)
  2577  			continue
  2578  		}
  2579  
  2580  		// Drop h, because the compaction c may have zeroed sequence numbers
  2581  		// of keys more recent than some of h's tombstones.
  2582  	}
  2583  	d.mu.compact.deletionHints = updatedHints
  2584  }
  2585  
  2586  func checkDeleteCompactionHints(
  2587  	cmp Compare, v *version, hints []deleteCompactionHint, snapshots []uint64,
  2588  ) ([]compactionLevel, []deleteCompactionHint) {
  2589  	var files map[*fileMetadata]bool
  2590  	var byLevel [numLevels][]*fileMetadata
  2591  
  2592  	unresolvedHints := hints[:0]
  2593  	for _, h := range hints {
  2594  		// Check each compaction hint to see if it's resolvable. Resolvable
  2595  		// hints are removed and trigger a delete-only compaction if any files
  2596  		// in the current LSM still meet their criteria. Unresolvable hints
  2597  		// are saved and don't trigger a delete-only compaction.
  2598  		//
  2599  		// When a compaction hint is created, the sequence numbers of the
  2600  		// range tombstones and the covered file with the oldest key are
  2601  		// recorded. The largest tombstone sequence number and the smallest
  2602  		// file sequence number must be in the same snapshot stripe for the
  2603  		// hint to be resolved. The below graphic models a compaction hint
  2604  		// covering the keyspace [b, r). The hint completely contains two
  2605  		// files, 000002 and 000003. The file 000003 contains the lowest
  2606  		// covered sequence number at #90. The tombstone b.RANGEDEL.230:h has
  2607  		// the highest tombstone sequence number incorporated into the hint.
  2608  		// The hint may be resolved only once the snapshots at #100, #180 and
  2609  		// #210 are all closed. File 000001 is not included within the hint
  2610  		// because it extends beyond the range tombstones in user key space.
  2611  		//
  2612  		// 250
  2613  		//
  2614  		//       |-b...230:h-|
  2615  		// _____________________________________________________ snapshot #210
  2616  		// 200               |--h.RANGEDEL.200:r--|
  2617  		//
  2618  		// _____________________________________________________ snapshot #180
  2619  		//
  2620  		// 150                     +--------+
  2621  		//           +---------+   | 000003 |
  2622  		//           | 000002  |   |        |
  2623  		//           +_________+   |        |
  2624  		// 100_____________________|________|___________________ snapshot #100
  2625  		//                         +--------+
  2626  		// _____________________________________________________ snapshot #70
  2627  		//                             +---------------+
  2628  		//  50                         | 000001        |
  2629  		//                             |               |
  2630  		//                             +---------------+
  2631  		// ______________________________________________________________
  2632  		//     a b c d e f g h i j k l m n o p q r s t u v w x y z
  2633  
  2634  		ti, _ := snapshotIndex(h.tombstoneLargestSeqNum, snapshots)
  2635  		fi, _ := snapshotIndex(h.fileSmallestSeqNum, snapshots)
  2636  		if ti != fi {
  2637  			// Cannot resolve yet.
  2638  			unresolvedHints = append(unresolvedHints, h)
  2639  			continue
  2640  		}
  2641  
  2642  		// The hint h will be resolved and dropped, regardless of whether
  2643  		// there are any tables that can be deleted.
  2644  		for l := h.tombstoneLevel + 1; l < numLevels; l++ {
  2645  			overlaps := v.Overlaps(l, cmp, h.start, h.end, true /* exclusiveEnd */)
  2646  			iter := overlaps.Iter()
  2647  			for m := iter.First(); m != nil; m = iter.Next() {
  2648  				if m.IsCompacting() || !h.canDelete(cmp, m, snapshots) || files[m] {
  2649  					continue
  2650  				}
  2651  				if files == nil {
  2652  					// Construct files lazily, assuming most calls will not
  2653  					// produce delete-only compactions.
  2654  					files = make(map[*fileMetadata]bool)
  2655  				}
  2656  				files[m] = true
  2657  				byLevel[l] = append(byLevel[l], m)
  2658  			}
  2659  		}
  2660  	}
  2661  
  2662  	var compactLevels []compactionLevel
  2663  	for l, files := range byLevel {
  2664  		if len(files) == 0 {
  2665  			continue
  2666  		}
  2667  		compactLevels = append(compactLevels, compactionLevel{
  2668  			level: l,
  2669  			files: manifest.NewLevelSliceKeySorted(cmp, files),
  2670  		})
  2671  	}
  2672  	return compactLevels, unresolvedHints
  2673  }
  2674  
  2675  // compact runs one compaction and maybe schedules another call to compact.
  2676  func (d *DB) compact(c *compaction, errChannel chan error) {
  2677  	pprof.Do(context.Background(), compactLabels, func(context.Context) {
  2678  		d.mu.Lock()
  2679  		defer d.mu.Unlock()
  2680  		if err := d.compact1(c, errChannel); err != nil {
  2681  			// TODO(peter): count consecutive compaction errors and backoff.
  2682  			d.opts.EventListener.BackgroundError(err)
  2683  		}
  2684  		d.mu.compact.compactingCount--
  2685  		delete(d.mu.compact.inProgress, c)
  2686  		// Add this compaction's duration to the cumulative duration. NB: This
  2687  		// must be atomic with the above removal of c from
  2688  		// d.mu.compact.InProgress to ensure Metrics.Compact.Duration does not
  2689  		// miss or double count a completing compaction's duration.
  2690  		d.mu.compact.duration += d.timeNow().Sub(c.beganAt)
  2691  
  2692  		// The previous compaction may have produced too many files in a
  2693  		// level, so reschedule another compaction if needed.
  2694  		d.maybeScheduleCompaction()
  2695  		d.mu.compact.cond.Broadcast()
  2696  	})
  2697  }
  2698  
  2699  // compact1 runs one compaction.
  2700  //
  2701  // d.mu must be held when calling this, but the mutex may be dropped and
  2702  // re-acquired during the course of this method.
  2703  func (d *DB) compact1(c *compaction, errChannel chan error) (err error) {
  2704  	if errChannel != nil {
  2705  		defer func() {
  2706  			errChannel <- err
  2707  		}()
  2708  	}
  2709  
  2710  	jobID := d.mu.nextJobID
  2711  	d.mu.nextJobID++
  2712  	info := c.makeInfo(jobID)
  2713  	d.opts.EventListener.CompactionBegin(info)
  2714  	startTime := d.timeNow()
  2715  
  2716  	ve, pendingOutputs, stats, err := d.runCompaction(jobID, c)
  2717  
  2718  	info.Duration = d.timeNow().Sub(startTime)
  2719  	if err == nil {
  2720  		err = func() error {
  2721  			var err error
  2722  			d.mu.versions.logLock()
  2723  			// Check if this compaction had a conflicting operation (eg. a d.excise())
  2724  			// that necessitates it restarting from scratch. Note that since we hold
  2725  			// the manifest lock, we don't expect this bool to change its value
  2726  			// as only the holder of the manifest lock will ever write to it.
  2727  			if c.cancel.Load() {
  2728  				err = firstError(err, ErrCancelledCompaction)
  2729  			}
  2730  			if err != nil {
  2731  				// logAndApply calls logUnlock. If we didn't call it, we need to call
  2732  				// logUnlock ourselves.
  2733  				d.mu.versions.logUnlock()
  2734  				return err
  2735  			}
  2736  			return d.mu.versions.logAndApply(jobID, ve, c.metrics, false /* forceRotation */, func() []compactionInfo {
  2737  				return d.getInProgressCompactionInfoLocked(c)
  2738  			})
  2739  		}()
  2740  		if err != nil {
  2741  			// TODO(peter): untested.
  2742  			for _, f := range pendingOutputs {
  2743  				// Note that the FileBacking for the file metadata might not have
  2744  				// been set yet. So, we directly use the FileNum. Since these
  2745  				// files were generated as compaction outputs, these must be
  2746  				// physical files on disk. This property might not hold once
  2747  				// https://github.com/cockroachdb/pebble/issues/389 is
  2748  				// implemented if #389 creates virtual sstables as output files.
  2749  				d.mu.versions.obsoleteTables = append(
  2750  					d.mu.versions.obsoleteTables,
  2751  					fileInfo{f.FileNum.DiskFileNum(), f.Size},
  2752  				)
  2753  			}
  2754  			d.mu.versions.updateObsoleteTableMetricsLocked()
  2755  		}
  2756  	}
  2757  
  2758  	info.Done = true
  2759  	info.Err = err
  2760  	if err == nil {
  2761  		for i := range ve.NewFiles {
  2762  			e := &ve.NewFiles[i]
  2763  			info.Output.Tables = append(info.Output.Tables, e.Meta.TableInfo())
  2764  		}
  2765  		d.mu.snapshots.cumulativePinnedCount += stats.cumulativePinnedKeys
  2766  		d.mu.snapshots.cumulativePinnedSize += stats.cumulativePinnedSize
  2767  		d.mu.versions.metrics.Keys.MissizedTombstonesCount += stats.countMissizedDels
  2768  		d.maybeUpdateDeleteCompactionHints(c)
  2769  	}
  2770  
  2771  	// NB: clearing compacting state must occur before updating the read state;
  2772  	// L0Sublevels initialization depends on it.
  2773  	d.clearCompactingState(c, err != nil)
  2774  	d.mu.versions.incrementCompactions(c.kind, c.extraLevels, c.pickerMetrics)
  2775  	d.mu.versions.incrementCompactionBytes(-c.bytesWritten)
  2776  
  2777  	info.TotalDuration = d.timeNow().Sub(c.beganAt)
  2778  	d.opts.EventListener.CompactionEnd(info)
  2779  
  2780  	// Update the read state before deleting obsolete files because the
  2781  	// read-state update will cause the previous version to be unref'd and if
  2782  	// there are no references obsolete tables will be added to the obsolete
  2783  	// table list.
  2784  	if err == nil {
  2785  		d.updateReadStateLocked(d.opts.DebugCheck)
  2786  		d.updateTableStatsLocked(ve.NewFiles)
  2787  	}
  2788  	d.deleteObsoleteFiles(jobID)
  2789  
  2790  	return err
  2791  }
  2792  
  2793  type compactStats struct {
  2794  	cumulativePinnedKeys uint64
  2795  	cumulativePinnedSize uint64
  2796  	countMissizedDels    uint64
  2797  }
  2798  
  2799  // runCopyCompaction runs a copy compaction where a new FileNum is created that
  2800  // is a byte-for-byte copy of the input file. This is used in lieu of a move
  2801  // compaction when a file is being moved across the local/remote storage
  2802  // boundary.
  2803  //
  2804  // d.mu must be held when calling this method.
  2805  func (d *DB) runCopyCompaction(
  2806  	jobID int,
  2807  	c *compaction,
  2808  	meta *fileMetadata,
  2809  	objMeta objstorage.ObjectMetadata,
  2810  	versionEdit *versionEdit,
  2811  ) (ve *versionEdit, pendingOutputs []physicalMeta, retErr error) {
  2812  	ve = versionEdit
  2813  	if objMeta.IsRemote() || !remote.ShouldCreateShared(d.opts.Experimental.CreateOnShared, c.outputLevel.level) {
  2814  		panic("pebble: scheduled a copy compaction that is not actually moving files to shared storage")
  2815  	}
  2816  	// Note that based on logic in the compaction picker, we're guaranteed
  2817  	// meta.Virtual is false.
  2818  	if meta.Virtual {
  2819  		panic(errors.AssertionFailedf("cannot do a copy compaction of a virtual sstable across local/remote storage"))
  2820  	}
  2821  	// We are in the relatively more complex case where we need to copy this
  2822  	// file to remote/shared storage. Drop the db mutex while we do the
  2823  	// copy.
  2824  	//
  2825  	// To ease up cleanup of the local file and tracking of refs, we create
  2826  	// a new FileNum. This has the potential of making the block cache less
  2827  	// effective, however.
  2828  	metaCopy := new(fileMetadata)
  2829  	*metaCopy = fileMetadata{
  2830  		Size:           meta.Size,
  2831  		CreationTime:   meta.CreationTime,
  2832  		SmallestSeqNum: meta.SmallestSeqNum,
  2833  		LargestSeqNum:  meta.LargestSeqNum,
  2834  		Stats:          meta.Stats,
  2835  		Virtual:        meta.Virtual,
  2836  	}
  2837  	if meta.HasPointKeys {
  2838  		metaCopy.ExtendPointKeyBounds(c.cmp, meta.SmallestPointKey, meta.LargestPointKey)
  2839  	}
  2840  	if meta.HasRangeKeys {
  2841  		metaCopy.ExtendRangeKeyBounds(c.cmp, meta.SmallestRangeKey, meta.LargestRangeKey)
  2842  	}
  2843  	metaCopy.FileNum = d.mu.versions.getNextFileNum()
  2844  	metaCopy.InitPhysicalBacking()
  2845  	c.metrics = map[int]*LevelMetrics{
  2846  		c.outputLevel.level: {
  2847  			BytesIn:         meta.Size,
  2848  			BytesCompacted:  meta.Size,
  2849  			TablesCompacted: 1,
  2850  		},
  2851  	}
  2852  	pendingOutputs = append(pendingOutputs, metaCopy.PhysicalMeta())
  2853  
  2854  	d.mu.Unlock()
  2855  	defer d.mu.Lock()
  2856  	_, err := d.objProvider.LinkOrCopyFromLocal(context.TODO(), d.opts.FS,
  2857  		d.objProvider.Path(objMeta), fileTypeTable, metaCopy.FileBacking.DiskFileNum,
  2858  		objstorage.CreateOptions{PreferSharedStorage: true})
  2859  	if err != nil {
  2860  		return ve, pendingOutputs, err
  2861  	}
  2862  	ve.NewFiles[0].Meta = metaCopy
  2863  
  2864  	if err := d.objProvider.Sync(); err != nil {
  2865  		return nil, pendingOutputs, err
  2866  	}
  2867  	return ve, pendingOutputs, nil
  2868  }
  2869  
  2870  // runCompactions runs a compaction that produces new on-disk tables from
  2871  // memtables or old on-disk tables.
  2872  //
  2873  // d.mu must be held when calling this, but the mutex may be dropped and
  2874  // re-acquired during the course of this method.
  2875  func (d *DB) runCompaction(
  2876  	jobID int, c *compaction,
  2877  ) (ve *versionEdit, pendingOutputs []physicalMeta, stats compactStats, retErr error) {
  2878  	// As a sanity check, confirm that the smallest / largest keys for new and
  2879  	// deleted files in the new versionEdit pass a validation function before
  2880  	// returning the edit.
  2881  	defer func() {
  2882  		// If we're handling a panic, don't expect the version edit to validate.
  2883  		if r := recover(); r != nil {
  2884  			panic(r)
  2885  		} else if ve != nil {
  2886  			err := validateVersionEdit(ve, d.opts.Experimental.KeyValidationFunc, d.opts.Comparer.FormatKey)
  2887  			if err != nil {
  2888  				d.opts.Logger.Fatalf("pebble: version edit validation failed: %s", err)
  2889  			}
  2890  		}
  2891  	}()
  2892  
  2893  	// Check for a delete-only compaction. This can occur when wide range
  2894  	// tombstones completely contain sstables.
  2895  	if c.kind == compactionKindDeleteOnly {
  2896  		c.metrics = make(map[int]*LevelMetrics, len(c.inputs))
  2897  		ve := &versionEdit{
  2898  			DeletedFiles: map[deletedFileEntry]*fileMetadata{},
  2899  		}
  2900  		for _, cl := range c.inputs {
  2901  			levelMetrics := &LevelMetrics{}
  2902  			iter := cl.files.Iter()
  2903  			for f := iter.First(); f != nil; f = iter.Next() {
  2904  				ve.DeletedFiles[deletedFileEntry{
  2905  					Level:   cl.level,
  2906  					FileNum: f.FileNum,
  2907  				}] = f
  2908  			}
  2909  			c.metrics[cl.level] = levelMetrics
  2910  		}
  2911  		return ve, nil, stats, nil
  2912  	}
  2913  
  2914  	if c.kind == compactionKindIngestedFlushable {
  2915  		panic("pebble: runCompaction cannot handle compactionKindIngestedFlushable.")
  2916  	}
  2917  
  2918  	// Check for a move or copy of one table from one level to the next. We avoid
  2919  	// such a move if there is lots of overlapping grandparent data. Otherwise,
  2920  	// the move could create a parent file that will require a very expensive
  2921  	// merge later on.
  2922  	if c.kind == compactionKindMove || c.kind == compactionKindCopy {
  2923  		iter := c.startLevel.files.Iter()
  2924  		meta := iter.First()
  2925  		if invariants.Enabled {
  2926  			if iter.Next() != nil {
  2927  				panic("got more than one file for a move or copy compaction")
  2928  			}
  2929  		}
  2930  		objMeta, err := d.objProvider.Lookup(fileTypeTable, meta.FileBacking.DiskFileNum)
  2931  		if err != nil {
  2932  			return ve, pendingOutputs, stats, err
  2933  		}
  2934  		c.metrics = map[int]*LevelMetrics{
  2935  			c.outputLevel.level: {
  2936  				BytesMoved:  meta.Size,
  2937  				TablesMoved: 1,
  2938  			},
  2939  		}
  2940  		ve := &versionEdit{
  2941  			DeletedFiles: map[deletedFileEntry]*fileMetadata{
  2942  				{Level: c.startLevel.level, FileNum: meta.FileNum}: meta,
  2943  			},
  2944  			NewFiles: []newFileEntry{
  2945  				{Level: c.outputLevel.level, Meta: meta},
  2946  			},
  2947  		}
  2948  		if c.kind == compactionKindCopy {
  2949  			ve, pendingOutputs, retErr = d.runCopyCompaction(jobID, c, meta, objMeta, ve)
  2950  			if retErr != nil {
  2951  				return ve, pendingOutputs, stats, retErr
  2952  			}
  2953  		}
  2954  		return ve, nil, stats, nil
  2955  	}
  2956  
  2957  	defer func() {
  2958  		if retErr != nil {
  2959  			pendingOutputs = nil
  2960  		}
  2961  	}()
  2962  
  2963  	snapshots := d.mu.snapshots.toSlice()
  2964  	formatVers := d.FormatMajorVersion()
  2965  
  2966  	if c.flushing == nil {
  2967  		// Before dropping the db mutex, grab a ref to the current version. This
  2968  		// prevents any concurrent excises from deleting files that this compaction
  2969  		// needs to read/maintain a reference to.
  2970  		//
  2971  		// Note that unlike user iterators, compactionIter does not maintain a ref
  2972  		// of the version or read state.
  2973  		vers := d.mu.versions.currentVersion()
  2974  		vers.Ref()
  2975  		defer vers.UnrefLocked()
  2976  	}
  2977  
  2978  	if c.cancel.Load() {
  2979  		return ve, nil, stats, ErrCancelledCompaction
  2980  	}
  2981  
  2982  	// Release the d.mu lock while doing I/O.
  2983  	// Note the unusual order: Unlock and then Lock.
  2984  	d.mu.Unlock()
  2985  	defer d.mu.Lock()
  2986  
  2987  	// Compactions use a pool of buffers to read blocks, avoiding polluting the
  2988  	// block cache with blocks that will not be read again. We initialize the
  2989  	// buffer pool with a size 12. This initial size does not need to be
  2990  	// accurate, because the pool will grow to accommodate the maximum number of
  2991  	// blocks allocated at a given time over the course of the compaction. But
  2992  	// choosing a size larger than that working set avoids any additional
  2993  	// allocations to grow the size of the pool over the course of iteration.
  2994  	//
  2995  	// Justification for initial size 12: In a two-level compaction, at any
  2996  	// given moment we'll have 2 index blocks in-use and 2 data blocks in-use.
  2997  	// Additionally, when decoding a compressed block, we'll temporarily
  2998  	// allocate 1 additional block to hold the compressed buffer. In the worst
  2999  	// case that all input sstables have two-level index blocks (+2), value
  3000  	// blocks (+2), range deletion blocks (+n) and range key blocks (+n), we'll
  3001  	// additionally require 2n+4 blocks where n is the number of input sstables.
  3002  	// Range deletion and range key blocks are relatively rare, and the cost of
  3003  	// an additional allocation or two over the course of the compaction is
  3004  	// considered to be okay. A larger initial size would cause the pool to hold
  3005  	// on to more memory, even when it's not in-use because the pool will
  3006  	// recycle buffers up to the current capacity of the pool. The memory use of
  3007  	// a 12-buffer pool is expected to be within reason, even if all the buffers
  3008  	// grow to the typical size of an index block (256 KiB) which would
  3009  	// translate to 3 MiB per compaction.
  3010  	c.bufferPool.Init(12)
  3011  	defer c.bufferPool.Release()
  3012  
  3013  	iiter, err := c.newInputIter(d.newIters, d.tableNewRangeKeyIter, snapshots)
  3014  	if err != nil {
  3015  		return nil, pendingOutputs, stats, err
  3016  	}
  3017  	c.allowedZeroSeqNum = c.allowZeroSeqNum()
  3018  	iiter = invalidating.MaybeWrapIfInvariants(iiter)
  3019  	iter := newCompactionIter(c.cmp, c.equal, c.formatKey, d.merge, iiter, snapshots,
  3020  		&c.rangeDelFrag, &c.rangeKeyFrag, c.allowedZeroSeqNum, c.elideTombstone,
  3021  		c.elideRangeTombstone, d.opts.Experimental.IneffectualSingleDeleteCallback,
  3022  		d.opts.Experimental.SingleDeleteInvariantViolationCallback,
  3023  		d.FormatMajorVersion())
  3024  
  3025  	var (
  3026  		createdFiles    []base.DiskFileNum
  3027  		tw              *sstable.Writer
  3028  		pinnedKeySize   uint64
  3029  		pinnedValueSize uint64
  3030  		pinnedCount     uint64
  3031  	)
  3032  	defer func() {
  3033  		if iter != nil {
  3034  			retErr = firstError(retErr, iter.Close())
  3035  		}
  3036  		if tw != nil {
  3037  			retErr = firstError(retErr, tw.Close())
  3038  		}
  3039  		if retErr != nil {
  3040  			for _, fileNum := range createdFiles {
  3041  				_ = d.objProvider.Remove(fileTypeTable, fileNum)
  3042  			}
  3043  		}
  3044  		for _, closer := range c.closers {
  3045  			retErr = firstError(retErr, closer.Close())
  3046  		}
  3047  	}()
  3048  
  3049  	ve = &versionEdit{
  3050  		DeletedFiles: map[deletedFileEntry]*fileMetadata{},
  3051  	}
  3052  
  3053  	startLevelBytes := c.startLevel.files.SizeSum()
  3054  	outputMetrics := &LevelMetrics{
  3055  		BytesIn:   startLevelBytes,
  3056  		BytesRead: c.outputLevel.files.SizeSum(),
  3057  	}
  3058  	if len(c.extraLevels) > 0 {
  3059  		outputMetrics.BytesIn += c.extraLevels[0].files.SizeSum()
  3060  	}
  3061  	outputMetrics.BytesRead += outputMetrics.BytesIn
  3062  
  3063  	c.metrics = map[int]*LevelMetrics{
  3064  		c.outputLevel.level: outputMetrics,
  3065  	}
  3066  	if len(c.flushing) == 0 && c.metrics[c.startLevel.level] == nil {
  3067  		c.metrics[c.startLevel.level] = &LevelMetrics{}
  3068  	}
  3069  	if len(c.extraLevels) > 0 {
  3070  		c.metrics[c.extraLevels[0].level] = &LevelMetrics{}
  3071  		outputMetrics.MultiLevel.BytesInTop = startLevelBytes
  3072  		outputMetrics.MultiLevel.BytesIn = outputMetrics.BytesIn
  3073  		outputMetrics.MultiLevel.BytesRead = outputMetrics.BytesRead
  3074  	}
  3075  
  3076  	// The table is typically written at the maximum allowable format implied by
  3077  	// the current format major version of the DB.
  3078  	tableFormat := formatVers.MaxTableFormat()
  3079  
  3080  	// In format major versions with maximum table formats of Pebblev3, value
  3081  	// blocks were conditional on an experimental setting. In format major
  3082  	// versions with maximum table formats of Pebblev4 and higher, value blocks
  3083  	// are always enabled.
  3084  	if tableFormat == sstable.TableFormatPebblev3 &&
  3085  		(d.opts.Experimental.EnableValueBlocks == nil || !d.opts.Experimental.EnableValueBlocks()) {
  3086  		tableFormat = sstable.TableFormatPebblev2
  3087  	}
  3088  
  3089  	writerOpts := d.opts.MakeWriterOptions(c.outputLevel.level, tableFormat)
  3090  	if formatVers < FormatBlockPropertyCollector {
  3091  		// Cannot yet write block properties.
  3092  		writerOpts.BlockPropertyCollectors = nil
  3093  	}
  3094  
  3095  	// prevPointKey is a sstable.WriterOption that provides access to
  3096  	// the last point key written to a writer's sstable. When a new
  3097  	// output begins in newOutput, prevPointKey is updated to point to
  3098  	// the new output's sstable.Writer. This allows the compaction loop
  3099  	// to access the last written point key without requiring the
  3100  	// compaction loop to make a copy of each key ahead of time. Users
  3101  	// must be careful, because the byte slice returned by UnsafeKey
  3102  	// points directly into the Writer's block buffer.
  3103  	var prevPointKey sstable.PreviousPointKeyOpt
  3104  	var cpuWorkHandle CPUWorkHandle
  3105  	defer func() {
  3106  		if cpuWorkHandle != nil {
  3107  			d.opts.Experimental.CPUWorkPermissionGranter.CPUWorkDone(cpuWorkHandle)
  3108  		}
  3109  	}()
  3110  
  3111  	newOutput := func() error {
  3112  		// Check if we've been cancelled by a concurrent operation.
  3113  		if c.cancel.Load() {
  3114  			return ErrCancelledCompaction
  3115  		}
  3116  		fileMeta := &fileMetadata{}
  3117  		d.mu.Lock()
  3118  		fileNum := d.mu.versions.getNextFileNum()
  3119  		fileMeta.FileNum = fileNum
  3120  		pendingOutputs = append(pendingOutputs, fileMeta.PhysicalMeta())
  3121  		d.mu.Unlock()
  3122  
  3123  		ctx := context.TODO()
  3124  		if objiotracing.Enabled {
  3125  			ctx = objiotracing.WithLevel(ctx, c.outputLevel.level)
  3126  			switch c.kind {
  3127  			case compactionKindFlush:
  3128  				ctx = objiotracing.WithReason(ctx, objiotracing.ForFlush)
  3129  			case compactionKindIngestedFlushable:
  3130  				ctx = objiotracing.WithReason(ctx, objiotracing.ForIngestion)
  3131  			default:
  3132  				ctx = objiotracing.WithReason(ctx, objiotracing.ForCompaction)
  3133  			}
  3134  		}
  3135  		// Prefer shared storage if present.
  3136  		createOpts := objstorage.CreateOptions{
  3137  			PreferSharedStorage: remote.ShouldCreateShared(d.opts.Experimental.CreateOnShared, c.outputLevel.level),
  3138  		}
  3139  		writable, objMeta, err := d.objProvider.Create(ctx, fileTypeTable, fileNum.DiskFileNum(), createOpts)
  3140  		if err != nil {
  3141  			return err
  3142  		}
  3143  
  3144  		reason := "flushing"
  3145  		if c.flushing == nil {
  3146  			reason = "compacting"
  3147  		}
  3148  		d.opts.EventListener.TableCreated(TableCreateInfo{
  3149  			JobID:   jobID,
  3150  			Reason:  reason,
  3151  			Path:    d.objProvider.Path(objMeta),
  3152  			FileNum: fileNum,
  3153  		})
  3154  		if c.kind != compactionKindFlush {
  3155  			writable = &compactionWritable{
  3156  				Writable: writable,
  3157  				versions: d.mu.versions,
  3158  				written:  &c.bytesWritten,
  3159  			}
  3160  		}
  3161  		createdFiles = append(createdFiles, fileNum.DiskFileNum())
  3162  		cacheOpts := private.SSTableCacheOpts(d.cacheID, fileNum.DiskFileNum()).(sstable.WriterOption)
  3163  
  3164  		const MaxFileWriteAdditionalCPUTime = time.Millisecond * 100
  3165  		cpuWorkHandle = d.opts.Experimental.CPUWorkPermissionGranter.GetPermission(
  3166  			MaxFileWriteAdditionalCPUTime,
  3167  		)
  3168  		writerOpts.Parallelism =
  3169  			d.opts.Experimental.MaxWriterConcurrency > 0 &&
  3170  				(cpuWorkHandle.Permitted() || d.opts.Experimental.ForceWriterParallelism)
  3171  
  3172  		tw = sstable.NewWriter(writable, writerOpts, cacheOpts, &prevPointKey)
  3173  
  3174  		fileMeta.CreationTime = time.Now().Unix()
  3175  		ve.NewFiles = append(ve.NewFiles, newFileEntry{
  3176  			Level: c.outputLevel.level,
  3177  			Meta:  fileMeta,
  3178  		})
  3179  		return nil
  3180  	}
  3181  
  3182  	// splitL0Outputs is true during flushes and intra-L0 compactions with flush
  3183  	// splits enabled.
  3184  	splitL0Outputs := c.outputLevel.level == 0 && d.opts.FlushSplitBytes > 0
  3185  
  3186  	// finishOutput is called with the a user key up to which all tombstones
  3187  	// should be flushed. Typically, this is the first key of the next
  3188  	// sstable or an empty key if this output is the final sstable.
  3189  	finishOutput := func(splitKey []byte) error {
  3190  		// If we haven't output any point records to the sstable (tw == nil) then the
  3191  		// sstable will only contain range tombstones and/or range keys. The smallest
  3192  		// key in the sstable will be the start key of the first range tombstone or
  3193  		// range key added. We need to ensure that this start key is distinct from
  3194  		// the splitKey passed to finishOutput (if set), otherwise we would generate
  3195  		// an sstable where the largest key is smaller than the smallest key due to
  3196  		// how the largest key boundary is set below. NB: It is permissible for the
  3197  		// range tombstone / range key start key to be the empty string.
  3198  		//
  3199  		// TODO: It is unfortunate that we have to do this check here rather than
  3200  		// when we decide to finish the sstable in the runCompaction loop. A better
  3201  		// structure currently eludes us.
  3202  		if tw == nil {
  3203  			startKey := c.rangeDelFrag.Start()
  3204  			if len(iter.tombstones) > 0 {
  3205  				startKey = iter.tombstones[0].Start
  3206  			}
  3207  			if startKey == nil {
  3208  				startKey = c.rangeKeyFrag.Start()
  3209  				if len(iter.rangeKeys) > 0 {
  3210  					startKey = iter.rangeKeys[0].Start
  3211  				}
  3212  			}
  3213  			if splitKey != nil && d.cmp(startKey, splitKey) == 0 {
  3214  				return nil
  3215  			}
  3216  		}
  3217  
  3218  		// NB: clone the key because the data can be held on to by the call to
  3219  		// compactionIter.Tombstones via keyspan.Fragmenter.FlushTo, and by the
  3220  		// WriterMetadata.LargestRangeDel.UserKey.
  3221  		splitKey = append([]byte(nil), splitKey...)
  3222  		for _, v := range iter.Tombstones(splitKey) {
  3223  			if tw == nil {
  3224  				if err := newOutput(); err != nil {
  3225  					return err
  3226  				}
  3227  			}
  3228  			// The tombstone being added could be completely outside the
  3229  			// eventual bounds of the sstable. Consider this example (bounds
  3230  			// in square brackets next to table filename):
  3231  			//
  3232  			// ./000240.sst   [tmgc#391,MERGE-tmgc#391,MERGE]
  3233  			// tmgc#391,MERGE [786e627a]
  3234  			// tmgc-udkatvs#331,RANGEDEL
  3235  			//
  3236  			// ./000241.sst   [tmgc#384,MERGE-tmgc#384,MERGE]
  3237  			// tmgc#384,MERGE [666c7070]
  3238  			// tmgc-tvsalezade#383,RANGEDEL
  3239  			// tmgc-tvsalezade#331,RANGEDEL
  3240  			//
  3241  			// ./000242.sst   [tmgc#383,RANGEDEL-tvsalezade#72057594037927935,RANGEDEL]
  3242  			// tmgc-tvsalezade#383,RANGEDEL
  3243  			// tmgc#375,SET [72646c78766965616c72776865676e79]
  3244  			// tmgc-tvsalezade#356,RANGEDEL
  3245  			//
  3246  			// Note that both of the top two SSTables have range tombstones
  3247  			// that start after the file's end keys. Since the file bound
  3248  			// computation happens well after all range tombstones have been
  3249  			// added to the writer, eliding out-of-file range tombstones based
  3250  			// on sequence number at this stage is difficult, and necessitates
  3251  			// read-time logic to ignore range tombstones outside file bounds.
  3252  			if err := rangedel.Encode(&v, tw.Add); err != nil {
  3253  				return err
  3254  			}
  3255  		}
  3256  		for _, v := range iter.RangeKeys(splitKey) {
  3257  			// Same logic as for range tombstones, except added using tw.AddRangeKey.
  3258  			if tw == nil {
  3259  				if err := newOutput(); err != nil {
  3260  					return err
  3261  				}
  3262  			}
  3263  			if err := rangekey.Encode(&v, tw.AddRangeKey); err != nil {
  3264  				return err
  3265  			}
  3266  		}
  3267  
  3268  		if tw == nil {
  3269  			return nil
  3270  		}
  3271  		{
  3272  			// Set internal sstable properties.
  3273  			p := getInternalWriterProperties(tw)
  3274  			// Set the external sst version to 0. This is what RocksDB expects for
  3275  			// db-internal sstables; otherwise, it could apply a global sequence number.
  3276  			p.ExternalFormatVersion = 0
  3277  			// Set the snapshot pinned totals.
  3278  			p.SnapshotPinnedKeys = pinnedCount
  3279  			p.SnapshotPinnedKeySize = pinnedKeySize
  3280  			p.SnapshotPinnedValueSize = pinnedValueSize
  3281  			stats.cumulativePinnedKeys += pinnedCount
  3282  			stats.cumulativePinnedSize += pinnedKeySize + pinnedValueSize
  3283  			pinnedCount = 0
  3284  			pinnedKeySize = 0
  3285  			pinnedValueSize = 0
  3286  		}
  3287  		if err := tw.Close(); err != nil {
  3288  			tw = nil
  3289  			return err
  3290  		}
  3291  		d.opts.Experimental.CPUWorkPermissionGranter.CPUWorkDone(cpuWorkHandle)
  3292  		cpuWorkHandle = nil
  3293  		writerMeta, err := tw.Metadata()
  3294  		if err != nil {
  3295  			tw = nil
  3296  			return err
  3297  		}
  3298  		tw = nil
  3299  		meta := ve.NewFiles[len(ve.NewFiles)-1].Meta
  3300  		meta.Size = writerMeta.Size
  3301  		meta.SmallestSeqNum = writerMeta.SmallestSeqNum
  3302  		meta.LargestSeqNum = writerMeta.LargestSeqNum
  3303  		meta.InitPhysicalBacking()
  3304  
  3305  		// If the file didn't contain any range deletions, we can fill its
  3306  		// table stats now, avoiding unnecessarily loading the table later.
  3307  		maybeSetStatsFromProperties(
  3308  			meta.PhysicalMeta(), &writerMeta.Properties,
  3309  		)
  3310  
  3311  		if c.flushing == nil {
  3312  			outputMetrics.TablesCompacted++
  3313  			outputMetrics.BytesCompacted += meta.Size
  3314  		} else {
  3315  			outputMetrics.TablesFlushed++
  3316  			outputMetrics.BytesFlushed += meta.Size
  3317  		}
  3318  		outputMetrics.Size += int64(meta.Size)
  3319  		outputMetrics.NumFiles++
  3320  		outputMetrics.Additional.BytesWrittenDataBlocks += writerMeta.Properties.DataSize
  3321  		outputMetrics.Additional.BytesWrittenValueBlocks += writerMeta.Properties.ValueBlocksSize
  3322  
  3323  		if n := len(ve.NewFiles); n > 1 {
  3324  			// This is not the first output file. Ensure the sstable boundaries
  3325  			// are nonoverlapping.
  3326  			prevMeta := ve.NewFiles[n-2].Meta
  3327  			if writerMeta.SmallestRangeDel.UserKey != nil {
  3328  				c := d.cmp(writerMeta.SmallestRangeDel.UserKey, prevMeta.Largest.UserKey)
  3329  				if c < 0 {
  3330  					return errors.Errorf(
  3331  						"pebble: smallest range tombstone start key is less than previous sstable largest key: %s < %s",
  3332  						writerMeta.SmallestRangeDel.Pretty(d.opts.Comparer.FormatKey),
  3333  						prevMeta.Largest.Pretty(d.opts.Comparer.FormatKey))
  3334  				} else if c == 0 && !prevMeta.Largest.IsExclusiveSentinel() {
  3335  					// The user key portion of the range boundary start key is
  3336  					// equal to the previous table's largest key user key, and
  3337  					// the previous table's largest key is not exclusive. This
  3338  					// violates the invariant that tables are key-space
  3339  					// partitioned.
  3340  					return errors.Errorf(
  3341  						"pebble: invariant violation: previous sstable largest key %s, current sstable smallest rangedel: %s",
  3342  						prevMeta.Largest.Pretty(d.opts.Comparer.FormatKey),
  3343  						writerMeta.SmallestRangeDel.Pretty(d.opts.Comparer.FormatKey),
  3344  					)
  3345  				}
  3346  			}
  3347  		}
  3348  
  3349  		// Verify that all range deletions outputted to the sstable are
  3350  		// truncated to split key.
  3351  		if splitKey != nil && writerMeta.LargestRangeDel.UserKey != nil &&
  3352  			d.cmp(writerMeta.LargestRangeDel.UserKey, splitKey) > 0 {
  3353  			return errors.Errorf(
  3354  				"pebble: invariant violation: rangedel largest key %q extends beyond split key %q",
  3355  				writerMeta.LargestRangeDel.Pretty(d.opts.Comparer.FormatKey),
  3356  				d.opts.Comparer.FormatKey(splitKey),
  3357  			)
  3358  		}
  3359  
  3360  		if writerMeta.HasPointKeys {
  3361  			meta.ExtendPointKeyBounds(d.cmp, writerMeta.SmallestPoint, writerMeta.LargestPoint)
  3362  		}
  3363  		if writerMeta.HasRangeDelKeys {
  3364  			meta.ExtendPointKeyBounds(d.cmp, writerMeta.SmallestRangeDel, writerMeta.LargestRangeDel)
  3365  		}
  3366  		if writerMeta.HasRangeKeys {
  3367  			meta.ExtendRangeKeyBounds(d.cmp, writerMeta.SmallestRangeKey, writerMeta.LargestRangeKey)
  3368  		}
  3369  
  3370  		// Verify that the sstable bounds fall within the compaction input
  3371  		// bounds. This is a sanity check that we don't have a logic error
  3372  		// elsewhere that causes the sstable bounds to accidentally expand past the
  3373  		// compaction input bounds as doing so could lead to various badness such
  3374  		// as keys being deleted by a range tombstone incorrectly.
  3375  		if c.smallest.UserKey != nil {
  3376  			switch v := d.cmp(meta.Smallest.UserKey, c.smallest.UserKey); {
  3377  			case v >= 0:
  3378  				// Nothing to do.
  3379  			case v < 0:
  3380  				return errors.Errorf("pebble: compaction output grew beyond bounds of input: %s < %s",
  3381  					meta.Smallest.Pretty(d.opts.Comparer.FormatKey),
  3382  					c.smallest.Pretty(d.opts.Comparer.FormatKey))
  3383  			}
  3384  		}
  3385  		if c.largest.UserKey != nil {
  3386  			switch v := d.cmp(meta.Largest.UserKey, c.largest.UserKey); {
  3387  			case v <= 0:
  3388  				// Nothing to do.
  3389  			case v > 0:
  3390  				return errors.Errorf("pebble: compaction output grew beyond bounds of input: %s > %s",
  3391  					meta.Largest.Pretty(d.opts.Comparer.FormatKey),
  3392  					c.largest.Pretty(d.opts.Comparer.FormatKey))
  3393  			}
  3394  		}
  3395  		// Verify that we never split different revisions of the same user key
  3396  		// across two different sstables.
  3397  		if err := c.errorOnUserKeyOverlap(ve); err != nil {
  3398  			return err
  3399  		}
  3400  		if err := meta.Validate(d.cmp, d.opts.Comparer.FormatKey); err != nil {
  3401  			return err
  3402  		}
  3403  		return nil
  3404  	}
  3405  
  3406  	// Build a compactionOutputSplitter that contains all logic to determine
  3407  	// whether the compaction loop should stop writing to one output sstable and
  3408  	// switch to a new one. Some splitters can wrap other splitters, and the
  3409  	// splitterGroup can be composed of multiple splitters. In this case, we
  3410  	// start off with splitters for file sizes, grandparent limits, and (for L0
  3411  	// splits) L0 limits, before wrapping them in an splitterGroup.
  3412  	sizeSplitter := newFileSizeSplitter(&iter.frontiers, c.maxOutputFileSize, c.grandparents.Iter())
  3413  	unsafePrevUserKey := func() []byte {
  3414  		// Return the largest point key written to tw or the start of
  3415  		// the current range deletion in the fragmenter, whichever is
  3416  		// greater.
  3417  		prevPoint := prevPointKey.UnsafeKey()
  3418  		if c.cmp(prevPoint.UserKey, c.rangeDelFrag.Start()) > 0 {
  3419  			return prevPoint.UserKey
  3420  		}
  3421  		return c.rangeDelFrag.Start()
  3422  	}
  3423  	outputSplitters := []compactionOutputSplitter{
  3424  		// We do not split the same user key across different sstables within
  3425  		// one flush or compaction. The fileSizeSplitter may request a split in
  3426  		// the middle of a user key, so the userKeyChangeSplitter ensures we are
  3427  		// at a user key change boundary when doing a split.
  3428  		&userKeyChangeSplitter{
  3429  			cmp:               c.cmp,
  3430  			splitter:          sizeSplitter,
  3431  			unsafePrevUserKey: unsafePrevUserKey,
  3432  		},
  3433  		newLimitFuncSplitter(&iter.frontiers, c.findGrandparentLimit),
  3434  	}
  3435  	if splitL0Outputs {
  3436  		outputSplitters = append(outputSplitters, newLimitFuncSplitter(&iter.frontiers, c.findL0Limit))
  3437  	}
  3438  	splitter := &splitterGroup{cmp: c.cmp, splitters: outputSplitters}
  3439  
  3440  	// Each outer loop iteration produces one output file. An iteration that
  3441  	// produces a file containing point keys (and optionally range tombstones)
  3442  	// guarantees that the input iterator advanced. An iteration that produces
  3443  	// a file containing only range tombstones guarantees the limit passed to
  3444  	// `finishOutput()` advanced to a strictly greater user key corresponding
  3445  	// to a grandparent file largest key, or nil. Taken together, these
  3446  	// progress guarantees ensure that eventually the input iterator will be
  3447  	// exhausted and the range tombstone fragments will all be flushed.
  3448  	for key, val := iter.First(); key != nil || !c.rangeDelFrag.Empty() || !c.rangeKeyFrag.Empty(); {
  3449  		var firstKey []byte
  3450  		if key != nil {
  3451  			firstKey = key.UserKey
  3452  		} else if startKey := c.rangeDelFrag.Start(); startKey != nil {
  3453  			// Pass the start key of the first pending tombstone to find the
  3454  			// next limit. All pending tombstones have the same start key. We
  3455  			// use this as opposed to the end key of the last written sstable to
  3456  			// effectively handle cases like these:
  3457  			//
  3458  			// a.SET.3
  3459  			// (lf.limit at b)
  3460  			// d.RANGEDEL.4:f
  3461  			//
  3462  			// In this case, the partition after b has only range deletions, so
  3463  			// if we were to find the limit after the last written key at the
  3464  			// split point (key a), we'd get the limit b again, and
  3465  			// finishOutput() would not advance any further because the next
  3466  			// range tombstone to write does not start until after the L0 split
  3467  			// point.
  3468  			firstKey = startKey
  3469  		}
  3470  		splitterSuggestion := splitter.onNewOutput(firstKey)
  3471  
  3472  		// Each inner loop iteration processes one key from the input iterator.
  3473  		for ; key != nil; key, val = iter.Next() {
  3474  			if split := splitter.shouldSplitBefore(key, tw); split == splitNow {
  3475  				break
  3476  			}
  3477  
  3478  			switch key.Kind() {
  3479  			case InternalKeyKindRangeDelete:
  3480  				// Range tombstones are handled specially. They are fragmented,
  3481  				// and they're not written until later during `finishOutput()`.
  3482  				// We add them to the `Fragmenter` now to make them visible to
  3483  				// `compactionIter` so covered keys in the same snapshot stripe
  3484  				// can be elided.
  3485  
  3486  				// The interleaved range deletion might only be one of many with
  3487  				// these bounds. Some fragmenting is performed ahead of time by
  3488  				// keyspan.MergingIter.
  3489  				if s := c.rangeDelIter.Span(); !s.Empty() {
  3490  					// The memory management here is subtle. Range deletions
  3491  					// blocks do NOT use prefix compression, which ensures that
  3492  					// range deletion spans' memory is available as long we keep
  3493  					// the iterator open. However, the keyspan.MergingIter that
  3494  					// merges spans across levels only guarantees the lifetime
  3495  					// of the [start, end) bounds until the next positioning
  3496  					// method is called.
  3497  					//
  3498  					// Additionally, the Span.Keys slice is owned by the the
  3499  					// range deletion iterator stack, and it may be overwritten
  3500  					// when we advance.
  3501  					//
  3502  					// Clone the Keys slice and the start and end keys.
  3503  					//
  3504  					// TODO(jackson): Avoid the clone by removing c.rangeDelFrag
  3505  					// and performing explicit truncation of the pending
  3506  					// rangedel span as necessary.
  3507  					clone := keyspan.Span{
  3508  						Start: iter.cloneKey(s.Start),
  3509  						End:   iter.cloneKey(s.End),
  3510  						Keys:  make([]keyspan.Key, len(s.Keys)),
  3511  					}
  3512  					copy(clone.Keys, s.Keys)
  3513  					c.rangeDelFrag.Add(clone)
  3514  				}
  3515  				continue
  3516  			case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete:
  3517  				// Range keys are handled in the same way as range tombstones, except
  3518  				// with a dedicated fragmenter.
  3519  				if s := c.rangeKeyInterleaving.Span(); !s.Empty() {
  3520  					clone := keyspan.Span{
  3521  						Start: iter.cloneKey(s.Start),
  3522  						End:   iter.cloneKey(s.End),
  3523  						Keys:  make([]keyspan.Key, len(s.Keys)),
  3524  					}
  3525  					// Since the keys' Suffix and Value fields are not deep cloned, the
  3526  					// underlying blockIter must be kept open for the lifetime of the
  3527  					// compaction.
  3528  					copy(clone.Keys, s.Keys)
  3529  					c.rangeKeyFrag.Add(clone)
  3530  				}
  3531  				continue
  3532  			}
  3533  			if tw == nil {
  3534  				if err := newOutput(); err != nil {
  3535  					return nil, pendingOutputs, stats, err
  3536  				}
  3537  			}
  3538  			if err := tw.AddWithForceObsolete(*key, val, iter.forceObsoleteDueToRangeDel); err != nil {
  3539  				return nil, pendingOutputs, stats, err
  3540  			}
  3541  			if iter.snapshotPinned {
  3542  				// The kv pair we just added to the sstable was only surfaced by
  3543  				// the compaction iterator because an open snapshot prevented
  3544  				// its elision. Increment the stats.
  3545  				pinnedCount++
  3546  				pinnedKeySize += uint64(len(key.UserKey)) + base.InternalTrailerLen
  3547  				pinnedValueSize += uint64(len(val))
  3548  			}
  3549  		}
  3550  
  3551  		// A splitter requested a split, and we're ready to finish the output.
  3552  		// We need to choose the key at which to split any pending range
  3553  		// tombstones. There are two options:
  3554  		// 1. splitterSuggestion — The key suggested by the splitter. This key
  3555  		//    is guaranteed to be greater than the last key written to the
  3556  		//    current output.
  3557  		// 2. key.UserKey — the first key of the next sstable output. This user
  3558  		//     key is also guaranteed to be greater than the last user key
  3559  		//     written to the current output (see userKeyChangeSplitter).
  3560  		//
  3561  		// Use whichever is smaller. Using the smaller of the two limits
  3562  		// overlap with grandparents. Consider the case where the
  3563  		// grandparent limit is calculated to be 'b', key is 'x', and
  3564  		// there exist many sstables between 'b' and 'x'. If the range
  3565  		// deletion fragmenter has a pending tombstone [a,x), splitting
  3566  		// at 'x' would cause the output table to overlap many
  3567  		// grandparents well beyond the calculated grandparent limit
  3568  		// 'b'. Splitting at the smaller `splitterSuggestion` avoids
  3569  		// this unbounded overlap with grandparent tables.
  3570  		splitKey := splitterSuggestion
  3571  		if key != nil && (splitKey == nil || c.cmp(splitKey, key.UserKey) > 0) {
  3572  			splitKey = key.UserKey
  3573  		}
  3574  		if err := finishOutput(splitKey); err != nil {
  3575  			return nil, pendingOutputs, stats, err
  3576  		}
  3577  	}
  3578  
  3579  	for _, cl := range c.inputs {
  3580  		iter := cl.files.Iter()
  3581  		for f := iter.First(); f != nil; f = iter.Next() {
  3582  			ve.DeletedFiles[deletedFileEntry{
  3583  				Level:   cl.level,
  3584  				FileNum: f.FileNum,
  3585  			}] = f
  3586  		}
  3587  	}
  3588  
  3589  	// The compaction iterator keeps track of a count of the number of DELSIZED
  3590  	// keys that encoded an incorrect size. Propagate it up as a part of
  3591  	// compactStats.
  3592  	stats.countMissizedDels = iter.stats.countMissizedDels
  3593  
  3594  	if err := d.objProvider.Sync(); err != nil {
  3595  		return nil, pendingOutputs, stats, err
  3596  	}
  3597  
  3598  	// Refresh the disk available statistic whenever a compaction/flush
  3599  	// completes, before re-acquiring the mutex.
  3600  	_ = d.calculateDiskAvailableBytes()
  3601  
  3602  	return ve, pendingOutputs, stats, nil
  3603  }
  3604  
  3605  // validateVersionEdit validates that start and end keys across new and deleted
  3606  // files in a versionEdit pass the given validation function.
  3607  func validateVersionEdit(
  3608  	ve *versionEdit, validateFn func([]byte) error, format base.FormatKey,
  3609  ) error {
  3610  	validateMetaFn := func(f *manifest.FileMetadata) error {
  3611  		for _, key := range []InternalKey{f.Smallest, f.Largest} {
  3612  			if err := validateFn(key.UserKey); err != nil {
  3613  				return errors.Wrapf(err, "key=%q; file=%s", format(key.UserKey), f)
  3614  			}
  3615  		}
  3616  		return nil
  3617  	}
  3618  
  3619  	// Validate both new and deleted files.
  3620  	for _, f := range ve.NewFiles {
  3621  		if err := validateMetaFn(f.Meta); err != nil {
  3622  			return err
  3623  		}
  3624  	}
  3625  	for _, m := range ve.DeletedFiles {
  3626  		if err := validateMetaFn(m); err != nil {
  3627  			return err
  3628  		}
  3629  	}
  3630  
  3631  	return nil
  3632  }
  3633  
  3634  // scanObsoleteFiles scans the filesystem for files that are no longer needed
  3635  // and adds those to the internal lists of obsolete files. Note that the files
  3636  // are not actually deleted by this method. A subsequent call to
  3637  // deleteObsoleteFiles must be performed. Must be not be called concurrently
  3638  // with compactions and flushes. db.mu must be held when calling this function.
  3639  func (d *DB) scanObsoleteFiles(list []string) {
  3640  	// Disable automatic compactions temporarily to avoid concurrent compactions /
  3641  	// flushes from interfering. The original value is restored on completion.
  3642  	disabledPrev := d.opts.DisableAutomaticCompactions
  3643  	defer func() {
  3644  		d.opts.DisableAutomaticCompactions = disabledPrev
  3645  	}()
  3646  	d.opts.DisableAutomaticCompactions = true
  3647  
  3648  	// Wait for any ongoing compaction to complete before continuing.
  3649  	for d.mu.compact.compactingCount > 0 || d.mu.compact.flushing {
  3650  		d.mu.compact.cond.Wait()
  3651  	}
  3652  
  3653  	liveFileNums := make(map[base.DiskFileNum]struct{})
  3654  	d.mu.versions.addLiveFileNums(liveFileNums)
  3655  	// Protect against files which are only referred to by the ingestedFlushable
  3656  	// from being deleted. These are added to the flushable queue on WAL replay
  3657  	// during read only mode and aren't part of the Version. Note that if
  3658  	// !d.opts.ReadOnly, then all flushables of type ingestedFlushable have
  3659  	// already been flushed.
  3660  	for _, fEntry := range d.mu.mem.queue {
  3661  		if f, ok := fEntry.flushable.(*ingestedFlushable); ok {
  3662  			for _, file := range f.files {
  3663  				liveFileNums[file.FileBacking.DiskFileNum] = struct{}{}
  3664  			}
  3665  		}
  3666  	}
  3667  
  3668  	minUnflushedLogNum := d.mu.versions.minUnflushedLogNum
  3669  	manifestFileNum := d.mu.versions.manifestFileNum
  3670  
  3671  	var obsoleteLogs []fileInfo
  3672  	var obsoleteTables []fileInfo
  3673  	var obsoleteManifests []fileInfo
  3674  	var obsoleteOptions []fileInfo
  3675  
  3676  	for _, filename := range list {
  3677  		fileType, diskFileNum, ok := base.ParseFilename(d.opts.FS, filename)
  3678  		if !ok {
  3679  			continue
  3680  		}
  3681  		switch fileType {
  3682  		case fileTypeLog:
  3683  			if diskFileNum.FileNum() >= minUnflushedLogNum {
  3684  				continue
  3685  			}
  3686  			fi := fileInfo{fileNum: diskFileNum}
  3687  			if stat, err := d.opts.FS.Stat(filename); err == nil {
  3688  				fi.fileSize = uint64(stat.Size())
  3689  			}
  3690  			obsoleteLogs = append(obsoleteLogs, fi)
  3691  		case fileTypeManifest:
  3692  			if diskFileNum.FileNum() >= manifestFileNum {
  3693  				continue
  3694  			}
  3695  			fi := fileInfo{fileNum: diskFileNum}
  3696  			if stat, err := d.opts.FS.Stat(filename); err == nil {
  3697  				fi.fileSize = uint64(stat.Size())
  3698  			}
  3699  			obsoleteManifests = append(obsoleteManifests, fi)
  3700  		case fileTypeOptions:
  3701  			if diskFileNum.FileNum() >= d.optionsFileNum.FileNum() {
  3702  				continue
  3703  			}
  3704  			fi := fileInfo{fileNum: diskFileNum}
  3705  			if stat, err := d.opts.FS.Stat(filename); err == nil {
  3706  				fi.fileSize = uint64(stat.Size())
  3707  			}
  3708  			obsoleteOptions = append(obsoleteOptions, fi)
  3709  		case fileTypeTable:
  3710  			// Objects are handled through the objstorage provider below.
  3711  		default:
  3712  			// Don't delete files we don't know about.
  3713  		}
  3714  	}
  3715  
  3716  	objects := d.objProvider.List()
  3717  	for _, obj := range objects {
  3718  		switch obj.FileType {
  3719  		case fileTypeTable:
  3720  			if _, ok := liveFileNums[obj.DiskFileNum]; ok {
  3721  				continue
  3722  			}
  3723  			fileInfo := fileInfo{
  3724  				fileNum: obj.DiskFileNum,
  3725  			}
  3726  			if size, err := d.objProvider.Size(obj); err == nil {
  3727  				fileInfo.fileSize = uint64(size)
  3728  			}
  3729  			obsoleteTables = append(obsoleteTables, fileInfo)
  3730  
  3731  		default:
  3732  			// Ignore object types we don't know about.
  3733  		}
  3734  	}
  3735  
  3736  	d.mu.log.queue = merge(d.mu.log.queue, obsoleteLogs)
  3737  	d.mu.versions.metrics.WAL.Files = int64(len(d.mu.log.queue))
  3738  	d.mu.versions.obsoleteTables = mergeFileInfo(d.mu.versions.obsoleteTables, obsoleteTables)
  3739  	d.mu.versions.updateObsoleteTableMetricsLocked()
  3740  	d.mu.versions.obsoleteManifests = merge(d.mu.versions.obsoleteManifests, obsoleteManifests)
  3741  	d.mu.versions.obsoleteOptions = merge(d.mu.versions.obsoleteOptions, obsoleteOptions)
  3742  }
  3743  
  3744  // disableFileDeletions disables file deletions and then waits for any
  3745  // in-progress deletion to finish. The caller is required to call
  3746  // enableFileDeletions in order to enable file deletions again. It is ok for
  3747  // multiple callers to disable file deletions simultaneously, though they must
  3748  // all invoke enableFileDeletions in order for file deletions to be re-enabled
  3749  // (there is an internal reference count on file deletion disablement).
  3750  //
  3751  // d.mu must be held when calling this method.
  3752  func (d *DB) disableFileDeletions() {
  3753  	d.mu.disableFileDeletions++
  3754  	d.mu.Unlock()
  3755  	defer d.mu.Lock()
  3756  	d.cleanupManager.Wait()
  3757  }
  3758  
  3759  // enableFileDeletions enables previously disabled file deletions. A cleanup job
  3760  // is queued if necessary.
  3761  //
  3762  // d.mu must be held when calling this method.
  3763  func (d *DB) enableFileDeletions() {
  3764  	if d.mu.disableFileDeletions <= 0 {
  3765  		panic("pebble: file deletion disablement invariant violated")
  3766  	}
  3767  	d.mu.disableFileDeletions--
  3768  	if d.mu.disableFileDeletions > 0 {
  3769  		return
  3770  	}
  3771  	jobID := d.mu.nextJobID
  3772  	d.mu.nextJobID++
  3773  	d.deleteObsoleteFiles(jobID)
  3774  }
  3775  
  3776  type fileInfo struct {
  3777  	fileNum  base.DiskFileNum
  3778  	fileSize uint64
  3779  }
  3780  
  3781  // deleteObsoleteFiles enqueues a cleanup job to the cleanup manager, if necessary.
  3782  //
  3783  // d.mu must be held when calling this. The function will release and re-aquire the mutex.
  3784  //
  3785  // Does nothing if file deletions are disabled (see disableFileDeletions). A
  3786  // cleanup job will be scheduled when file deletions are re-enabled.
  3787  func (d *DB) deleteObsoleteFiles(jobID int) {
  3788  	if d.mu.disableFileDeletions > 0 {
  3789  		return
  3790  	}
  3791  
  3792  	var obsoleteLogs []fileInfo
  3793  	for i := range d.mu.log.queue {
  3794  		// NB: d.mu.versions.minUnflushedLogNum is the log number of the earliest
  3795  		// log that has not had its contents flushed to an sstable. We can recycle
  3796  		// the prefix of d.mu.log.queue with log numbers less than
  3797  		// minUnflushedLogNum.
  3798  		if d.mu.log.queue[i].fileNum.FileNum() >= d.mu.versions.minUnflushedLogNum {
  3799  			obsoleteLogs = d.mu.log.queue[:i]
  3800  			d.mu.log.queue = d.mu.log.queue[i:]
  3801  			d.mu.versions.metrics.WAL.Files -= int64(len(obsoleteLogs))
  3802  			break
  3803  		}
  3804  	}
  3805  
  3806  	obsoleteTables := append([]fileInfo(nil), d.mu.versions.obsoleteTables...)
  3807  	d.mu.versions.obsoleteTables = nil
  3808  
  3809  	for _, tbl := range obsoleteTables {
  3810  		delete(d.mu.versions.zombieTables, tbl.fileNum)
  3811  	}
  3812  
  3813  	// Sort the manifests cause we want to delete some contiguous prefix
  3814  	// of the older manifests.
  3815  	sort.Slice(d.mu.versions.obsoleteManifests, func(i, j int) bool {
  3816  		return d.mu.versions.obsoleteManifests[i].fileNum.FileNum() <
  3817  			d.mu.versions.obsoleteManifests[j].fileNum.FileNum()
  3818  	})
  3819  
  3820  	var obsoleteManifests []fileInfo
  3821  	manifestsToDelete := len(d.mu.versions.obsoleteManifests) - d.opts.NumPrevManifest
  3822  	if manifestsToDelete > 0 {
  3823  		obsoleteManifests = d.mu.versions.obsoleteManifests[:manifestsToDelete]
  3824  		d.mu.versions.obsoleteManifests = d.mu.versions.obsoleteManifests[manifestsToDelete:]
  3825  		if len(d.mu.versions.obsoleteManifests) == 0 {
  3826  			d.mu.versions.obsoleteManifests = nil
  3827  		}
  3828  	}
  3829  
  3830  	obsoleteOptions := d.mu.versions.obsoleteOptions
  3831  	d.mu.versions.obsoleteOptions = nil
  3832  
  3833  	// Release d.mu while preparing the cleanup job and possibly waiting.
  3834  	// Note the unusual order: Unlock and then Lock.
  3835  	d.mu.Unlock()
  3836  	defer d.mu.Lock()
  3837  
  3838  	files := [4]struct {
  3839  		fileType fileType
  3840  		obsolete []fileInfo
  3841  	}{
  3842  		{fileTypeLog, obsoleteLogs},
  3843  		{fileTypeTable, obsoleteTables},
  3844  		{fileTypeManifest, obsoleteManifests},
  3845  		{fileTypeOptions, obsoleteOptions},
  3846  	}
  3847  	_, noRecycle := d.opts.Cleaner.(base.NeedsFileContents)
  3848  	filesToDelete := make([]obsoleteFile, 0, len(obsoleteLogs)+len(obsoleteTables)+len(obsoleteManifests)+len(obsoleteOptions))
  3849  	for _, f := range files {
  3850  		// We sort to make the order of deletions deterministic, which is nice for
  3851  		// tests.
  3852  		sort.Slice(f.obsolete, func(i, j int) bool {
  3853  			return f.obsolete[i].fileNum.FileNum() < f.obsolete[j].fileNum.FileNum()
  3854  		})
  3855  		for _, fi := range f.obsolete {
  3856  			dir := d.dirname
  3857  			switch f.fileType {
  3858  			case fileTypeLog:
  3859  				if !noRecycle && d.logRecycler.add(fi) {
  3860  					continue
  3861  				}
  3862  				dir = d.walDirname
  3863  			case fileTypeTable:
  3864  				d.tableCache.evict(fi.fileNum)
  3865  			}
  3866  
  3867  			filesToDelete = append(filesToDelete, obsoleteFile{
  3868  				dir:      dir,
  3869  				fileNum:  fi.fileNum,
  3870  				fileType: f.fileType,
  3871  				fileSize: fi.fileSize,
  3872  			})
  3873  		}
  3874  	}
  3875  	if len(filesToDelete) > 0 {
  3876  		d.cleanupManager.EnqueueJob(jobID, filesToDelete)
  3877  	}
  3878  	if d.opts.private.testingAlwaysWaitForCleanup {
  3879  		d.cleanupManager.Wait()
  3880  	}
  3881  }
  3882  
  3883  func (d *DB) maybeScheduleObsoleteTableDeletion() {
  3884  	d.mu.Lock()
  3885  	defer d.mu.Unlock()
  3886  	d.maybeScheduleObsoleteTableDeletionLocked()
  3887  }
  3888  
  3889  func (d *DB) maybeScheduleObsoleteTableDeletionLocked() {
  3890  	if len(d.mu.versions.obsoleteTables) > 0 {
  3891  		jobID := d.mu.nextJobID
  3892  		d.mu.nextJobID++
  3893  		d.deleteObsoleteFiles(jobID)
  3894  	}
  3895  }
  3896  
  3897  func merge(a, b []fileInfo) []fileInfo {
  3898  	if len(b) == 0 {
  3899  		return a
  3900  	}
  3901  
  3902  	a = append(a, b...)
  3903  	sort.Slice(a, func(i, j int) bool {
  3904  		return a[i].fileNum.FileNum() < a[j].fileNum.FileNum()
  3905  	})
  3906  
  3907  	n := 0
  3908  	for i := 0; i < len(a); i++ {
  3909  		if n == 0 || a[i].fileNum != a[n-1].fileNum {
  3910  			a[n] = a[i]
  3911  			n++
  3912  		}
  3913  	}
  3914  	return a[:n]
  3915  }
  3916  
  3917  func mergeFileInfo(a, b []fileInfo) []fileInfo {
  3918  	if len(b) == 0 {
  3919  		return a
  3920  	}
  3921  
  3922  	a = append(a, b...)
  3923  	sort.Slice(a, func(i, j int) bool {
  3924  		return a[i].fileNum.FileNum() < a[j].fileNum.FileNum()
  3925  	})
  3926  
  3927  	n := 0
  3928  	for i := 0; i < len(a); i++ {
  3929  		if n == 0 || a[i].fileNum != a[n-1].fileNum {
  3930  			a[n] = a[i]
  3931  			n++
  3932  		}
  3933  	}
  3934  	return a[:n]
  3935  }
  3936  
  3937  func max[I constraints.Ordered](a, b I) I {
  3938  	if b > a {
  3939  		return b
  3940  	}
  3941  	return a
  3942  }