github.com/petermattis/pebble@v0.0.0-20190905164901-ab51a2166067/db.go (about)

     1  // Copyright 2012 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 provides an ordered key/value store.
     6  package pebble // import "github.com/petermattis/pebble"
     7  
     8  import (
     9  	"errors"
    10  	"fmt"
    11  	"io"
    12  	"sync"
    13  	"sync/atomic"
    14  
    15  	"github.com/petermattis/pebble/internal/arenaskl"
    16  	"github.com/petermattis/pebble/internal/base"
    17  	"github.com/petermattis/pebble/internal/record"
    18  	"github.com/petermattis/pebble/vfs"
    19  )
    20  
    21  const (
    22  	// minTableCacheSize is the minimum size of the table cache.
    23  	minTableCacheSize = 64
    24  
    25  	// numNonTableCacheFiles is an approximation for the number of MaxOpenFiles
    26  	// that we don't use for table caches.
    27  	numNonTableCacheFiles = 10
    28  )
    29  
    30  var (
    31  	// ErrNotFound is returned when a get operation does not find the requested
    32  	// key.
    33  	ErrNotFound = base.ErrNotFound
    34  	// ErrClosed is returned when an operation is performed on a closed snapshot
    35  	// or DB.
    36  	ErrClosed = errors.New("pebble: closed")
    37  	// ErrReadOnly is returned when a write operation is performed on a read-only
    38  	// database.
    39  	ErrReadOnly = errors.New("pebble: read-only")
    40  )
    41  
    42  type flushable interface {
    43  	newIter(o *IterOptions) internalIterator
    44  	newFlushIter(o *IterOptions, bytesFlushed *uint64) internalIterator
    45  	newRangeDelIter(o *IterOptions) internalIterator
    46  	totalBytes() uint64
    47  	flushed() chan struct{}
    48  	readyForFlush() bool
    49  	logInfo() (num, size uint64)
    50  }
    51  
    52  // Reader is a readable key/value store.
    53  //
    54  // It is safe to call Get and NewIter from concurrent goroutines.
    55  type Reader interface {
    56  	// Get gets the value for the given key. It returns ErrNotFound if the DB
    57  	// does not contain the key.
    58  	//
    59  	// The caller should not modify the contents of the returned slice, but
    60  	// it is safe to modify the contents of the argument after Get returns.
    61  	Get(key []byte) (value []byte, err error)
    62  
    63  	// NewIter returns an iterator that is unpositioned (Iterator.Valid() will
    64  	// return false). The iterator can be positioned via a call to SeekGE,
    65  	// SeekLT, First or Last.
    66  	NewIter(o *IterOptions) *Iterator
    67  
    68  	// Close closes the Reader. It may or may not close any underlying io.Reader
    69  	// or io.Writer, depending on how the DB was created.
    70  	//
    71  	// It is not safe to close a DB until all outstanding iterators are closed.
    72  	// It is valid to call Close multiple times. Other methods should not be
    73  	// called after the DB has been closed.
    74  	Close() error
    75  }
    76  
    77  // Writer is a writable key/value store.
    78  //
    79  // Goroutine safety is dependent on the specific implementation.
    80  type Writer interface {
    81  	// Apply the operations contained in the batch to the DB.
    82  	//
    83  	// It is safe to modify the contents of the arguments after Apply returns.
    84  	Apply(batch *Batch, o *WriteOptions) error
    85  
    86  	// Delete deletes the value for the given key. Deletes are blind all will
    87  	// succeed even if the given key does not exist.
    88  	//
    89  	// It is safe to modify the contents of the arguments after Delete returns.
    90  	Delete(key []byte, o *WriteOptions) error
    91  
    92  	// DeleteRange deletes all of the keys (and values) in the range [start,end)
    93  	// (inclusive on start, exclusive on end).
    94  	//
    95  	// It is safe to modify the contents of the arguments after Delete returns.
    96  	DeleteRange(start, end []byte, o *WriteOptions) error
    97  
    98  	// LogData adds the specified to the batch. The data will be written to the
    99  	// WAL, but not added to memtables or sstables. Log data is never indexed,
   100  	// which makes it useful for testing WAL performance.
   101  	//
   102  	// It is safe to modify the contents of the argument after LogData returns.
   103  	LogData(data []byte, opts *WriteOptions) error
   104  
   105  	// Merge merges the value for the given key. The details of the merge are
   106  	// dependent upon the configured merge operation.
   107  	//
   108  	// It is safe to modify the contents of the arguments after Merge returns.
   109  	Merge(key, value []byte, o *WriteOptions) error
   110  
   111  	// Set sets the value for the given key. It overwrites any previous value
   112  	// for that key; a DB is not a multi-map.
   113  	//
   114  	// It is safe to modify the contents of the arguments after Set returns.
   115  	Set(key, value []byte, o *WriteOptions) error
   116  }
   117  
   118  // DB provides a concurrent, persistent ordered key/value store.
   119  //
   120  // A DB's basic operations (Get, Set, Delete) should be self-explanatory. Get
   121  // and Delete will return ErrNotFound if the requested key is not in the store.
   122  // Callers are free to ignore this error.
   123  //
   124  // A DB also allows for iterating over the key/value pairs in key order. If d
   125  // is a DB, the code below prints all key/value pairs whose keys are 'greater
   126  // than or equal to' k:
   127  //
   128  //	iter := d.NewIter(readOptions)
   129  //	for iter.SeekGE(k); iter.Valid(); iter.Next() {
   130  //		fmt.Printf("key=%q value=%q\n", iter.Key(), iter.Value())
   131  //	}
   132  //	return iter.Close()
   133  //
   134  // The Options struct holds the optional parameters for the DB, including a
   135  // Comparer to define a 'less than' relationship over keys. It is always valid
   136  // to pass a nil *Options, which means to use the default parameter values. Any
   137  // zero field of a non-nil *Options also means to use the default value for
   138  // that parameter. Thus, the code below uses a custom Comparer, but the default
   139  // values for every other parameter:
   140  //
   141  //	db := pebble.Open(&Options{
   142  //		Comparer: myComparer,
   143  //	})
   144  type DB struct {
   145  	dbNum          uint64
   146  	dirname        string
   147  	walDirname     string
   148  	opts           *Options
   149  	cmp            Compare
   150  	equal          Equal
   151  	merge          Merge
   152  	split          Split
   153  	abbreviatedKey AbbreviatedKey
   154  
   155  	dataDir vfs.File
   156  	walDir  vfs.File
   157  
   158  	tableCache tableCache
   159  	newIters   tableNewIters
   160  
   161  	commit   *commitPipeline
   162  	fileLock io.Closer
   163  
   164  	largeBatchThreshold int
   165  	optionsFileNum      uint64
   166  
   167  	// readState provides access to the state needed for reading without needing
   168  	// to acquire DB.mu.
   169  	readState struct {
   170  		sync.RWMutex
   171  		val *readState
   172  	}
   173  
   174  	logRecycler logRecycler
   175  
   176  	closed int32 // updated atomically
   177  
   178  	compactionLimiter limiter
   179  
   180  	// bytesFlushed is the number of bytes flushed in the current flush. This
   181  	// must be read/written atomically since it is accessed by both the flush
   182  	// and compaction routines.
   183  	bytesFlushed uint64
   184  	// bytesCompacted is the number of bytes compacted in the current compaction.
   185  	// This is used as a dummy variable to increment during compaction, and the
   186  	// value is not used anywhere.
   187  	bytesCompacted uint64
   188  
   189  	flushLimiter limiter
   190  
   191  	// TODO(peter): describe exactly what this mutex protects. So far: every
   192  	// field in the struct.
   193  	mu struct {
   194  		sync.Mutex
   195  
   196  		nextJobID int
   197  
   198  		versions versionSet
   199  
   200  		log struct {
   201  			queue   []uint64
   202  			size    uint64
   203  			bytesIn uint64
   204  			// The LogWriter is protected by commitPipeline.mu. This allows log
   205  			// writes to be performed without holding DB.mu, but requires both
   206  			// commitPipeline.mu and DB.mu to be held when rotating the WAL/memtable
   207  			// (i.e. makeRoomForWrite).
   208  			*record.LogWriter
   209  		}
   210  
   211  		mem struct {
   212  			cond sync.Cond
   213  			// The current mutable memTable.
   214  			mutable *memTable
   215  			// Queue of flushables (the mutable memtable is at end). Elements are
   216  			// added to the end of the slice and removed from the beginning. Once an
   217  			// index is set it is never modified making a fixed slice immutable and
   218  			// safe for concurrent reads.
   219  			queue []flushable
   220  			// True when the memtable is actively been switched. Both mem.mutable and
   221  			// log.LogWriter are invalid while switching is true.
   222  			switching bool
   223  		}
   224  
   225  		compact struct {
   226  			cond           sync.Cond
   227  			flushing       bool
   228  			compacting     bool
   229  			pendingOutputs map[uint64]struct{}
   230  			manual         []*manualCompaction
   231  		}
   232  
   233  		cleaner struct {
   234  			cond     sync.Cond
   235  			cleaning bool
   236  		}
   237  
   238  		// The list of active snapshots.
   239  		snapshots snapshotList
   240  	}
   241  }
   242  
   243  var _ Reader = (*DB)(nil)
   244  var _ Writer = (*DB)(nil)
   245  
   246  // Get gets the value for the given key. It returns ErrNotFound if the DB does
   247  // not contain the key.
   248  //
   249  // The caller should not modify the contents of the returned slice, but it is
   250  // safe to modify the contents of the argument after Get returns.
   251  func (d *DB) Get(key []byte) ([]byte, error) {
   252  	return d.getInternal(key, nil /* batch */, nil /* snapshot */)
   253  }
   254  
   255  func (d *DB) getInternal(key []byte, b *Batch, s *Snapshot) ([]byte, error) {
   256  	if atomic.LoadInt32(&d.closed) != 0 {
   257  		panic(ErrClosed)
   258  	}
   259  
   260  	// Grab and reference the current readState. This prevents the underlying
   261  	// files in the associated version from being deleted if there is a current
   262  	// compaction. The readState is unref'd by Iterator.Close().
   263  	readState := d.loadReadState()
   264  
   265  	// Determine the seqnum to read at after grabbing the read state (current and
   266  	// memtables) above.
   267  	var seqNum uint64
   268  	if s != nil {
   269  		seqNum = s.seqNum
   270  	} else {
   271  		seqNum = atomic.LoadUint64(&d.mu.versions.visibleSeqNum)
   272  	}
   273  
   274  	var buf struct {
   275  		dbi Iterator
   276  		get getIter
   277  	}
   278  
   279  	get := &buf.get
   280  	get.cmp = d.cmp
   281  	get.equal = d.equal
   282  	get.newIters = d.newIters
   283  	get.snapshot = seqNum
   284  	get.key = key
   285  	get.batch = b
   286  	get.mem = readState.memtables
   287  	get.l0 = readState.current.Files[0]
   288  	get.version = readState.current
   289  
   290  	i := &buf.dbi
   291  	i.cmp = d.cmp
   292  	i.equal = d.equal
   293  	i.merge = d.merge
   294  	i.split = d.split
   295  	i.iter = get
   296  	i.readState = readState
   297  
   298  	defer i.Close()
   299  	if !i.First() {
   300  		err := i.Error()
   301  		if err != nil {
   302  			return nil, err
   303  		}
   304  		return nil, ErrNotFound
   305  	}
   306  	return i.Value(), nil
   307  }
   308  
   309  // Set sets the value for the given key. It overwrites any previous value
   310  // for that key; a DB is not a multi-map.
   311  //
   312  // It is safe to modify the contents of the arguments after Set returns.
   313  func (d *DB) Set(key, value []byte, opts *WriteOptions) error {
   314  	b := newBatch(d)
   315  	defer b.release()
   316  	_ = b.Set(key, value, opts)
   317  	return d.Apply(b, opts)
   318  }
   319  
   320  // Delete deletes the value for the given key. Deletes are blind all will
   321  // succeed even if the given key does not exist.
   322  //
   323  // It is safe to modify the contents of the arguments after Delete returns.
   324  func (d *DB) Delete(key []byte, opts *WriteOptions) error {
   325  	b := newBatch(d)
   326  	defer b.release()
   327  	_ = b.Delete(key, opts)
   328  	return d.Apply(b, opts)
   329  }
   330  
   331  // DeleteRange deletes all of the keys (and values) in the range [start,end)
   332  // (inclusive on start, exclusive on end).
   333  //
   334  // It is safe to modify the contents of the arguments after DeleteRange
   335  // returns.
   336  func (d *DB) DeleteRange(start, end []byte, opts *WriteOptions) error {
   337  	b := newBatch(d)
   338  	defer b.release()
   339  	_ = b.DeleteRange(start, end, opts)
   340  	return d.Apply(b, opts)
   341  }
   342  
   343  // Merge adds an action to the DB that merges the value at key with the new
   344  // value. The details of the merge are dependent upon the configured merge
   345  // operator.
   346  //
   347  // It is safe to modify the contents of the arguments after Merge returns.
   348  func (d *DB) Merge(key, value []byte, opts *WriteOptions) error {
   349  	b := newBatch(d)
   350  	defer b.release()
   351  	_ = b.Merge(key, value, opts)
   352  	return d.Apply(b, opts)
   353  }
   354  
   355  // LogData adds the specified to the batch. The data will be written to the
   356  // WAL, but not added to memtables or sstables. Log data is never indexed,
   357  // which makes it useful for testing WAL performance.
   358  //
   359  // It is safe to modify the contents of the argument after LogData returns.
   360  //
   361  // TODO(peter): untested.
   362  func (d *DB) LogData(data []byte, opts *WriteOptions) error {
   363  	b := newBatch(d)
   364  	defer b.release()
   365  	_ = b.LogData(data, opts)
   366  	return d.Apply(b, opts)
   367  }
   368  
   369  // Apply the operations contained in the batch to the DB. If the batch is large
   370  // the contents of the batch may be retained by the database. If that occurs
   371  // the batch contents will be cleared preventing the caller from attempting to
   372  // reuse them.
   373  //
   374  // It is safe to modify the contents of the arguments after Apply returns.
   375  func (d *DB) Apply(batch *Batch, opts *WriteOptions) error {
   376  	if atomic.LoadInt32(&d.closed) != 0 {
   377  		panic(ErrClosed)
   378  	}
   379  	if d.opts.ReadOnly {
   380  		return ErrReadOnly
   381  	}
   382  
   383  	sync := opts.GetSync()
   384  	if sync && d.opts.DisableWAL {
   385  		return errors.New("pebble: WAL disabled")
   386  	}
   387  
   388  	if int(batch.memTableSize) >= d.largeBatchThreshold {
   389  		batch.flushable = newFlushableBatch(batch, d.opts.Comparer)
   390  	}
   391  	err := d.commit.Commit(batch, sync)
   392  	if err == nil {
   393  		// If this is a large batch, we need to clear the batch contents as the
   394  		// flushable batch may still be present in the flushables queue.
   395  		if batch.flushable != nil {
   396  			batch.storage.data = nil
   397  		}
   398  	}
   399  	return err
   400  }
   401  
   402  func (d *DB) commitApply(b *Batch, mem *memTable) error {
   403  	if b.flushable != nil {
   404  		// This is a large batch which was already added to the immutable queue.
   405  		return nil
   406  	}
   407  	err := mem.apply(b, b.SeqNum())
   408  	if err != nil {
   409  		return err
   410  	}
   411  	if mem.unref() {
   412  		d.mu.Lock()
   413  		d.maybeScheduleFlush()
   414  		d.mu.Unlock()
   415  	}
   416  	return nil
   417  }
   418  
   419  func (d *DB) commitWrite(b *Batch, wg *sync.WaitGroup) (*memTable, error) {
   420  	repr := b.Repr()
   421  
   422  	d.mu.Lock()
   423  
   424  	if b.flushable != nil {
   425  		b.flushable.seqNum = b.SeqNum()
   426  	}
   427  
   428  	// Switch out the memtable if there was not enough room to store the batch.
   429  	err := d.makeRoomForWrite(b)
   430  
   431  	if err == nil {
   432  		d.mu.log.bytesIn += uint64(len(repr))
   433  	}
   434  
   435  	// Grab a reference to the memtable while holding DB.mu. Note that
   436  	// makeRoomForWrite() add a reference to the memtable which will prevent it
   437  	// from being flushed until we unreference it.
   438  	mem := d.mu.mem.mutable
   439  
   440  	d.mu.Unlock()
   441  	if err != nil {
   442  		return nil, err
   443  	}
   444  
   445  	if d.opts.DisableWAL {
   446  		return mem, nil
   447  	}
   448  
   449  	size, err := d.mu.log.SyncRecord(repr, wg)
   450  	if err != nil {
   451  		panic(err)
   452  	}
   453  
   454  	atomic.StoreUint64(&d.mu.log.size, uint64(size))
   455  	return mem, err
   456  }
   457  
   458  type iterAlloc struct {
   459  	dbi             Iterator
   460  	merging         mergingIter
   461  	iters           [3 + numLevels]internalIterator
   462  	rangeDelIters   [3 + numLevels]internalIterator
   463  	largestUserKeys [3 + numLevels][]byte
   464  	levels          [numLevels]levelIter
   465  }
   466  
   467  var iterAllocPool = sync.Pool{
   468  	New: func() interface{} {
   469  		return &iterAlloc{}
   470  	},
   471  }
   472  
   473  // newIterInternal constructs a new iterator, merging in batchIter as an extra
   474  // level.
   475  func (d *DB) newIterInternal(
   476  	batchIter internalIterator,
   477  	batchRangeDelIter internalIterator,
   478  	s *Snapshot,
   479  	o *IterOptions,
   480  ) *Iterator {
   481  	if atomic.LoadInt32(&d.closed) != 0 {
   482  		panic(ErrClosed)
   483  	}
   484  
   485  	// Grab and reference the current readState. This prevents the underlying
   486  	// files in the associated version from being deleted if there is a current
   487  	// compaction. The readState is unref'd by Iterator.Close().
   488  	readState := d.loadReadState()
   489  
   490  	// Determine the seqnum to read at after grabbing the read state (current and
   491  	// memtables) above.
   492  	var seqNum uint64
   493  	if s != nil {
   494  		seqNum = s.seqNum
   495  	} else {
   496  		seqNum = atomic.LoadUint64(&d.mu.versions.visibleSeqNum)
   497  	}
   498  
   499  	// Bundle various structures under a single umbrella in order to allocate
   500  	// them together.
   501  	buf := iterAllocPool.Get().(*iterAlloc)
   502  	dbi := &buf.dbi
   503  	dbi.alloc = buf
   504  	dbi.cmp = d.cmp
   505  	dbi.equal = d.equal
   506  	dbi.merge = d.merge
   507  	dbi.split = d.split
   508  	dbi.readState = readState
   509  	if o != nil {
   510  		dbi.opts = *o
   511  	}
   512  
   513  	iters := buf.iters[:0]
   514  	rangeDelIters := buf.rangeDelIters[:0]
   515  	largestUserKeys := buf.largestUserKeys[:0]
   516  	if batchIter != nil {
   517  		iters = append(iters, batchIter)
   518  		rangeDelIters = append(rangeDelIters, batchRangeDelIter)
   519  		largestUserKeys = append(largestUserKeys, nil)
   520  	}
   521  
   522  	// TODO(peter): We only need to add memtables which contain sequence numbers
   523  	// older than seqNum. Unfortunately, memtables don't track their oldest
   524  	// sequence number currently.
   525  	memtables := readState.memtables
   526  	for i := len(memtables) - 1; i >= 0; i-- {
   527  		mem := memtables[i]
   528  		iters = append(iters, mem.newIter(&dbi.opts))
   529  		rangeDelIters = append(rangeDelIters, mem.newRangeDelIter(&dbi.opts))
   530  		largestUserKeys = append(largestUserKeys, nil)
   531  	}
   532  
   533  	// The level 0 files need to be added from newest to oldest.
   534  	current := readState.current
   535  	for i := len(current.Files[0]) - 1; i >= 0; i-- {
   536  		f := &current.Files[0][i]
   537  		iter, rangeDelIter, err := d.newIters(f, &dbi.opts, nil)
   538  		if err != nil {
   539  			dbi.err = err
   540  			return dbi
   541  		}
   542  		iters = append(iters, iter)
   543  		rangeDelIters = append(rangeDelIters, rangeDelIter)
   544  		largestUserKeys = append(largestUserKeys, nil)
   545  	}
   546  
   547  	start := len(rangeDelIters)
   548  	for level := 1; level < len(current.Files); level++ {
   549  		if len(current.Files[level]) == 0 {
   550  			continue
   551  		}
   552  		rangeDelIters = append(rangeDelIters, nil)
   553  		largestUserKeys = append(largestUserKeys, nil)
   554  	}
   555  	buf.merging.rangeDelIters = rangeDelIters
   556  	buf.merging.largestUserKeys = largestUserKeys
   557  	rangeDelIters = rangeDelIters[start:]
   558  	largestUserKeys = largestUserKeys[start:]
   559  
   560  	// Add level iterators for the remaining files.
   561  	levels := buf.levels[:]
   562  	for level := 1; level < len(current.Files); level++ {
   563  		if len(current.Files[level]) == 0 {
   564  			continue
   565  		}
   566  
   567  		var li *levelIter
   568  		if len(levels) > 0 {
   569  			li = &levels[0]
   570  			levels = levels[1:]
   571  		} else {
   572  			li = &levelIter{}
   573  		}
   574  
   575  		li.init(&dbi.opts, d.cmp, d.newIters, current.Files[level], nil)
   576  		li.initRangeDel(&rangeDelIters[0])
   577  		li.initLargestUserKey(&largestUserKeys[0])
   578  		iters = append(iters, li)
   579  		rangeDelIters = rangeDelIters[1:]
   580  		largestUserKeys = largestUserKeys[1:]
   581  	}
   582  
   583  	buf.merging.init(d.cmp, iters...)
   584  	buf.merging.snapshot = seqNum
   585  	dbi.iter = &buf.merging
   586  	return dbi
   587  }
   588  
   589  // NewBatch returns a new empty write-only batch. Any reads on the batch will
   590  // return an error. If the batch is committed it will be applied to the DB.
   591  func (d *DB) NewBatch() *Batch {
   592  	return newBatch(d)
   593  }
   594  
   595  // NewIndexedBatch returns a new empty read-write batch. Any reads on the batch
   596  // will read from both the batch and the DB. If the batch is committed it will
   597  // be applied to the DB. An indexed batch is slower that a non-indexed batch
   598  // for insert operations. If you do not need to perform reads on the batch, use
   599  // NewBatch instead.
   600  func (d *DB) NewIndexedBatch() *Batch {
   601  	return newIndexedBatch(d, d.opts.Comparer)
   602  }
   603  
   604  // NewIter returns an iterator that is unpositioned (Iterator.Valid() will
   605  // return false). The iterator can be positioned via a call to SeekGE, SeekLT,
   606  // First or Last. The iterator provides a point-in-time view of the current DB
   607  // state. This view is maintained by preventing file deletions and preventing
   608  // memtables referenced by the iterator from being deleted. Using an iterator
   609  // to maintain a long-lived point-in-time view of the DB state can lead to an
   610  // apparent memory and disk usage leak. Use snapshots (see NewSnapshot) for
   611  // point-in-time snapshots which avoids these problems.
   612  func (d *DB) NewIter(o *IterOptions) *Iterator {
   613  	return d.newIterInternal(nil, /* batchIter */
   614  		nil /* batchRangeDelIter */, nil /* snapshot */, o)
   615  }
   616  
   617  // NewSnapshot returns a point-in-time view of the current DB state. Iterators
   618  // created with this handle will all observe a stable snapshot of the current
   619  // DB state. The caller must call Snapshot.Close() when the snapshot is no
   620  // longer needed. Snapshots are not persisted across DB restarts (close ->
   621  // open). Unlike the implicit snapshot maintained by an iterator, a snapshot
   622  // will not prevent memtables from being released or sstables from being
   623  // deleted. Instead, a snapshot prevents deletion of sequence numbers
   624  // referenced by the snapshot.
   625  func (d *DB) NewSnapshot() *Snapshot {
   626  	if atomic.LoadInt32(&d.closed) != 0 {
   627  		panic(ErrClosed)
   628  	}
   629  
   630  	s := &Snapshot{
   631  		db:     d,
   632  		seqNum: atomic.LoadUint64(&d.mu.versions.visibleSeqNum),
   633  	}
   634  	d.mu.Lock()
   635  	d.mu.snapshots.pushBack(s)
   636  	d.mu.Unlock()
   637  	return s
   638  }
   639  
   640  // Close closes the DB.
   641  //
   642  // It is not safe to close a DB until all outstanding iterators are closed.
   643  // It is valid to call Close multiple times. Other methods should not be
   644  // called after the DB has been closed.
   645  func (d *DB) Close() error {
   646  	d.mu.Lock()
   647  	defer d.mu.Unlock()
   648  	if atomic.LoadInt32(&d.closed) != 0 {
   649  		panic(ErrClosed)
   650  	}
   651  	atomic.StoreInt32(&d.closed, 1)
   652  	for d.mu.compact.compacting || d.mu.compact.flushing {
   653  		d.mu.compact.cond.Wait()
   654  	}
   655  	err := d.tableCache.Close()
   656  	if !d.opts.ReadOnly {
   657  		err = firstError(err, d.mu.log.Close())
   658  	} else if d.mu.log.LogWriter != nil {
   659  		panic("pebble: log-writer should be nil in read-only mode")
   660  	}
   661  	err = firstError(err, d.fileLock.Close())
   662  	d.commit.Close()
   663  
   664  	err = firstError(err, d.dataDir.Close())
   665  
   666  	if err == nil {
   667  		d.readState.val.unrefLocked()
   668  
   669  		current := d.mu.versions.currentVersion()
   670  		for v := d.mu.versions.versions.Front(); true; v = v.Next() {
   671  			refs := v.Refs()
   672  			if v == current {
   673  				if refs != 1 {
   674  					return fmt.Errorf("leaked iterators: current\n%s", v)
   675  				}
   676  				break
   677  			}
   678  			if refs != 0 {
   679  				return fmt.Errorf("leaked iterators:\n%s", v)
   680  			}
   681  		}
   682  	}
   683  	return err
   684  }
   685  
   686  // Compact the specified range of keys in the database.
   687  func (d *DB) Compact(start, end []byte /* CompactionOptions */) error {
   688  	if atomic.LoadInt32(&d.closed) != 0 {
   689  		panic(ErrClosed)
   690  	}
   691  	if d.opts.ReadOnly {
   692  		return ErrReadOnly
   693  	}
   694  
   695  	iStart := base.MakeInternalKey(start, InternalKeySeqNumMax, InternalKeyKindMax)
   696  	iEnd := base.MakeInternalKey(end, 0, 0)
   697  	meta := []*fileMetadata{&fileMetadata{Smallest: iStart, Largest: iEnd}}
   698  
   699  	d.mu.Lock()
   700  	maxLevelWithFiles := 1
   701  	cur := d.mu.versions.currentVersion()
   702  	for level := 0; level < numLevels; level++ {
   703  		if len(cur.Overlaps(level, d.cmp, start, end)) > 0 {
   704  			maxLevelWithFiles = level + 1
   705  		}
   706  	}
   707  
   708  	// Determine if any memtable overlaps with the compaction range. We wait for
   709  	// any such overlap to flush (initiating a flush if necessary).
   710  	mem, err := func() (flushable, error) {
   711  		if ingestMemtableOverlaps(d.cmp, d.mu.mem.mutable, meta) {
   712  			mem := d.mu.mem.mutable
   713  
   714  			// We have to hold both commitPipeline.mu and DB.mu when calling
   715  			// makeRoomForWrite(). Lock order requirements elsewhere force us to
   716  			// unlock DB.mu in order to grab commitPipeline.mu first.
   717  			d.mu.Unlock()
   718  			d.commit.mu.Lock()
   719  			d.mu.Lock()
   720  			defer d.commit.mu.Unlock()
   721  			if mem == d.mu.mem.mutable {
   722  				// Only flush if the active memtable is unchanged.
   723  				return mem, d.makeRoomForWrite(nil)
   724  			}
   725  			return mem, nil
   726  		}
   727  		// Check to see if any files overlap with any of the immutable
   728  		// memtables. The queue is ordered from oldest to newest. We want to wait
   729  		// for the newest table that overlaps.
   730  		for i := len(d.mu.mem.queue) - 1; i >= 0; i-- {
   731  			mem := d.mu.mem.queue[i]
   732  			if ingestMemtableOverlaps(d.cmp, mem, meta) {
   733  				return mem, nil
   734  			}
   735  		}
   736  		return nil, nil
   737  	}()
   738  
   739  	d.mu.Unlock()
   740  
   741  	if err != nil {
   742  		return err
   743  	}
   744  	if mem != nil {
   745  		<-mem.flushed()
   746  	}
   747  
   748  	for level := 0; level < maxLevelWithFiles; {
   749  		manual := &manualCompaction{
   750  			done:  make(chan error, 1),
   751  			level: level,
   752  			start: iStart,
   753  			end:   iEnd,
   754  		}
   755  		if err := d.manualCompact(manual); err != nil {
   756  			return err
   757  		}
   758  		level = manual.outputLevel
   759  		if level == numLevels-1 {
   760  			// A manual compaction of the bottommost level occured. There is no next
   761  			// level to try and compact.
   762  			break
   763  		}
   764  	}
   765  	return nil
   766  }
   767  
   768  func (d *DB) manualCompact(manual *manualCompaction) error {
   769  	d.mu.Lock()
   770  	d.mu.compact.manual = append(d.mu.compact.manual, manual)
   771  	d.maybeScheduleCompaction()
   772  	d.mu.Unlock()
   773  	return <-manual.done
   774  }
   775  
   776  // Flush the memtable to stable storage.
   777  func (d *DB) Flush() error {
   778  	flushDone, err := d.AsyncFlush()
   779  	if err != nil {
   780  		return err
   781  	}
   782  	<-flushDone
   783  	return nil
   784  }
   785  
   786  // AsyncFlush asynchronously flushes the memtable to stable storage.
   787  //
   788  // If no error is returned, the caller can receive from the returned channel in
   789  // order to wait for the flush to complete.
   790  func (d *DB) AsyncFlush() (<-chan struct{}, error) {
   791  	if atomic.LoadInt32(&d.closed) != 0 {
   792  		panic(ErrClosed)
   793  	}
   794  	if d.opts.ReadOnly {
   795  		return nil, ErrReadOnly
   796  	}
   797  
   798  	d.commit.mu.Lock()
   799  	d.mu.Lock()
   800  	mem := d.mu.mem.mutable
   801  	err := d.makeRoomForWrite(nil)
   802  	d.mu.Unlock()
   803  	d.commit.mu.Unlock()
   804  	if err != nil {
   805  		return nil, err
   806  	}
   807  	return mem.flushed(), nil
   808  }
   809  
   810  // Metrics returns metrics about the database.
   811  func (d *DB) Metrics() *VersionMetrics {
   812  	metrics := &VersionMetrics{}
   813  	recycledLogs := d.logRecycler.count()
   814  	d.mu.Lock()
   815  	*metrics = d.mu.versions.metrics
   816  	metrics.WAL.ObsoleteFiles = int64(recycledLogs)
   817  	metrics.WAL.Size = atomic.LoadUint64(&d.mu.log.size)
   818  	metrics.WAL.BytesIn = d.mu.log.bytesIn // protected by d.mu
   819  	for i, n := 0, len(d.mu.mem.queue)-1; i < n; i++ {
   820  		_, size := d.mu.mem.queue[i].logInfo()
   821  		metrics.WAL.Size += size
   822  	}
   823  	metrics.WAL.BytesWritten = metrics.Levels[0].BytesIn + metrics.WAL.Size
   824  	metrics.Levels[0].Score = float64(metrics.Levels[0].NumFiles) / float64(d.opts.L0CompactionThreshold)
   825  	if p := d.mu.versions.picker; p != nil {
   826  		for level := 1; level < numLevels; level++ {
   827  			metrics.Levels[level].Score = float64(metrics.Levels[level].Size) / float64(p.levelMaxBytes[level])
   828  		}
   829  	}
   830  	d.mu.Unlock()
   831  	return metrics
   832  }
   833  
   834  func (d *DB) walPreallocateSize() int {
   835  	// Set the WAL preallocate size to 110% of the memtable size. Note that there
   836  	// is a bit of apples and oranges in units here as the memtabls size
   837  	// corresponds to the memory usage of the memtable while the WAL size is the
   838  	// size of the batches (plus overhead) stored in the WAL.
   839  	//
   840  	// TODO(peter): 110% of the memtable size is quite hefty for a block
   841  	// size. This logic is taken from GetWalPreallocateBlockSize in
   842  	// RocksDB. Could a smaller preallocation block size be used?
   843  	size := d.opts.MemTableSize
   844  	size = (size / 10) + size
   845  	return size
   846  }
   847  
   848  // makeRoomForWrite ensures that the memtable has room to hold the contents of
   849  // Batch. It reserves the space in the memtable and adds a reference to the
   850  // memtable. The caller must later ensure that the memtable is unreferenced. If
   851  // the memtable is full, or a nil Batch is provided, the current memtable is
   852  // rotated (marked as immutable) and a new mutable memtable is allocated. This
   853  // memtable rotation also causes a log rotation.
   854  //
   855  // Both DB.mu and commitPipeline.mu must be held by the caller. Note that DB.mu
   856  // may be released and reacquired.
   857  func (d *DB) makeRoomForWrite(b *Batch) error {
   858  	force := b == nil || b.flushable != nil
   859  	stalled := false
   860  	for {
   861  		if d.mu.mem.switching {
   862  			d.mu.mem.cond.Wait()
   863  			continue
   864  		}
   865  		if b != nil && b.flushable == nil {
   866  			err := d.mu.mem.mutable.prepare(b)
   867  			if err != arenaskl.ErrArenaFull {
   868  				if stalled {
   869  					stalled = false
   870  					if d.opts.EventListener.WriteStallEnd != nil {
   871  						d.opts.EventListener.WriteStallEnd()
   872  					}
   873  				}
   874  				return err
   875  			}
   876  		} else if !force {
   877  			if stalled {
   878  				stalled = false
   879  				if d.opts.EventListener.WriteStallEnd != nil {
   880  					d.opts.EventListener.WriteStallEnd()
   881  				}
   882  			}
   883  			return nil
   884  		}
   885  		if len(d.mu.mem.queue) >= d.opts.MemTableStopWritesThreshold {
   886  			// We have filled up the current memtable, but the previous one is still
   887  			// being compacted, so we wait.
   888  			if !stalled {
   889  				stalled = true
   890  				if d.opts.EventListener.WriteStallBegin != nil {
   891  					d.opts.EventListener.WriteStallBegin(WriteStallBeginInfo{
   892  						Reason: "memtable count limit reached",
   893  					})
   894  				}
   895  			}
   896  			d.mu.compact.cond.Wait()
   897  			continue
   898  		}
   899  		if len(d.mu.versions.currentVersion().Files[0]) > d.opts.L0StopWritesThreshold {
   900  			// There are too many level-0 files, so we wait.
   901  			if !stalled {
   902  				stalled = true
   903  				if d.opts.EventListener.WriteStallBegin != nil {
   904  					d.opts.EventListener.WriteStallBegin(WriteStallBeginInfo{
   905  						Reason: "L0 file count limit exceeded",
   906  					})
   907  				}
   908  			}
   909  			d.mu.compact.cond.Wait()
   910  			continue
   911  		}
   912  
   913  		var newLogNumber uint64
   914  		var newLogFile vfs.File
   915  		var prevLogSize uint64
   916  		var err error
   917  
   918  		if !d.opts.DisableWAL {
   919  			jobID := d.mu.nextJobID
   920  			d.mu.nextJobID++
   921  			newLogNumber = d.mu.versions.getNextFileNum()
   922  			d.mu.mem.switching = true
   923  			d.mu.Unlock()
   924  
   925  			newLogName := base.MakeFilename(d.walDirname, fileTypeLog, newLogNumber)
   926  
   927  			// Try to use a recycled log file. Recycling log files is an important
   928  			// performance optimization as it is faster to sync a file that has
   929  			// already been written, than one which is being written for the first
   930  			// time. This is due to the need to sync file metadata when a file is
   931  			// being written for the first time. Note this is true even if file
   932  			// preallocation is performed (e.g. fallocate).
   933  			recycleLogNumber := d.logRecycler.peek()
   934  			if recycleLogNumber > 0 {
   935  				recycleLogName := base.MakeFilename(d.walDirname, fileTypeLog, recycleLogNumber)
   936  				err = d.opts.FS.Rename(recycleLogName, newLogName)
   937  			}
   938  
   939  			if err == nil {
   940  				newLogFile, err = d.opts.FS.Create(newLogName)
   941  			}
   942  
   943  			if err == nil {
   944  				// TODO(peter): RocksDB delays sync of the parent directory until the
   945  				// first time the log is synced. Is that worthwhile?
   946  				err = d.walDir.Sync()
   947  			}
   948  
   949  			if err == nil {
   950  				prevLogSize = uint64(d.mu.log.Size())
   951  				err = d.mu.log.Close()
   952  				if err != nil {
   953  					newLogFile.Close()
   954  				} else {
   955  					newLogFile = vfs.NewSyncingFile(newLogFile, vfs.SyncingFileOptions{
   956  						BytesPerSync:    d.opts.BytesPerSync,
   957  						PreallocateSize: d.walPreallocateSize(),
   958  					})
   959  				}
   960  			}
   961  
   962  			if recycleLogNumber > 0 {
   963  				err = d.logRecycler.pop(recycleLogNumber)
   964  			}
   965  
   966  			if d.opts.EventListener.WALCreated != nil {
   967  				d.opts.EventListener.WALCreated(WALCreateInfo{
   968  					JobID:           jobID,
   969  					Path:            newLogName,
   970  					FileNum:         newLogNumber,
   971  					RecycledFileNum: recycleLogNumber,
   972  					Err:             err,
   973  				})
   974  			}
   975  
   976  			d.mu.Lock()
   977  			d.mu.mem.switching = false
   978  			d.mu.mem.cond.Broadcast()
   979  
   980  			d.mu.versions.metrics.WAL.Files++
   981  		}
   982  
   983  		if err != nil {
   984  			// TODO(peter): avoid chewing through file numbers in a tight loop if there
   985  			// is an error here.
   986  			//
   987  			// What to do here? Stumbling on doesn't seem worthwhile. If we failed to
   988  			// close the previous log it is possible we lost a write.
   989  			panic(err)
   990  		}
   991  
   992  		if !d.opts.DisableWAL {
   993  			d.mu.log.queue = append(d.mu.log.queue, newLogNumber)
   994  			d.mu.log.LogWriter = record.NewLogWriter(newLogFile, newLogNumber)
   995  		}
   996  
   997  		imm := d.mu.mem.mutable
   998  		imm.logSize = prevLogSize
   999  		prevLogNumber := imm.logNum
  1000  
  1001  		var scheduleFlush bool
  1002  		if b != nil && b.flushable != nil {
  1003  			// The batch is too large to fit in the memtable so add it directly to
  1004  			// the immutable queue.
  1005  			b.flushable.logNum = prevLogNumber
  1006  			d.mu.mem.queue = append(d.mu.mem.queue, b.flushable)
  1007  			scheduleFlush = true
  1008  		}
  1009  
  1010  		// Create a new memtable, scheduling the previous one for flushing. We do
  1011  		// this even if the previous memtable was empty because the DB.Flush
  1012  		// mechanism is dependent on being able to wait for the empty memtable to
  1013  		// flush. We can't just mark the empty memtable as flushed here because we
  1014  		// also have to wait for all previous immutable tables to
  1015  		// flush. Additionally, the memtable is tied to particular WAL file and we
  1016  		// want to go through the flush path in order to recycle that WAL file.
  1017  		d.mu.mem.mutable = newMemTable(d.opts)
  1018  		// NB: When the immutable memtable is flushed to disk it will apply a
  1019  		// versionEdit to the manifest telling it that log files < newLogNumber
  1020  		// have been applied. newLogNumber corresponds to the WAL that contains
  1021  		// mutations that are present in the new memtable.
  1022  		d.mu.mem.mutable.logNum = newLogNumber
  1023  		d.mu.mem.queue = append(d.mu.mem.queue, d.mu.mem.mutable)
  1024  		d.updateReadStateLocked()
  1025  		if (imm != nil && imm.unref()) || scheduleFlush {
  1026  			d.maybeScheduleFlush()
  1027  		}
  1028  		force = false
  1029  	}
  1030  }
  1031  
  1032  // firstError returns the first non-nil error of err0 and err1, or nil if both
  1033  // are nil.
  1034  func firstError(err0, err1 error) error {
  1035  	if err0 != nil {
  1036  		return err0
  1037  	}
  1038  	return err1
  1039  }