github.com/cockroachdb/pebble@v1.1.2/open.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
     6  
     7  import (
     8  	"bytes"
     9  	"context"
    10  	"encoding/binary"
    11  	"fmt"
    12  	"io"
    13  	"math"
    14  	"os"
    15  	"sort"
    16  	"sync/atomic"
    17  	"time"
    18  
    19  	"github.com/cockroachdb/errors"
    20  	"github.com/cockroachdb/errors/oserror"
    21  	"github.com/cockroachdb/pebble/internal/arenaskl"
    22  	"github.com/cockroachdb/pebble/internal/base"
    23  	"github.com/cockroachdb/pebble/internal/cache"
    24  	"github.com/cockroachdb/pebble/internal/constants"
    25  	"github.com/cockroachdb/pebble/internal/invariants"
    26  	"github.com/cockroachdb/pebble/internal/manifest"
    27  	"github.com/cockroachdb/pebble/internal/manual"
    28  	"github.com/cockroachdb/pebble/objstorage"
    29  	"github.com/cockroachdb/pebble/objstorage/objstorageprovider"
    30  	"github.com/cockroachdb/pebble/record"
    31  	"github.com/cockroachdb/pebble/sstable"
    32  	"github.com/cockroachdb/pebble/vfs"
    33  	"github.com/prometheus/client_golang/prometheus"
    34  )
    35  
    36  const (
    37  	initialMemTableSize = 256 << 10 // 256 KB
    38  
    39  	// The max batch size is limited by the uint32 offsets stored in
    40  	// internal/batchskl.node, DeferredBatchOp, and flushableBatchEntry.
    41  	//
    42  	// We limit the size to MaxUint32 (just short of 4GB) so that the exclusive
    43  	// end of an allocation fits in uint32.
    44  	//
    45  	// On 32-bit systems, slices are naturally limited to MaxInt (just short of
    46  	// 2GB).
    47  	maxBatchSize = constants.MaxUint32OrInt
    48  
    49  	// The max memtable size is limited by the uint32 offsets stored in
    50  	// internal/arenaskl.node, DeferredBatchOp, and flushableBatchEntry.
    51  	//
    52  	// We limit the size to MaxUint32 (just short of 4GB) so that the exclusive
    53  	// end of an allocation fits in uint32.
    54  	//
    55  	// On 32-bit systems, slices are naturally limited to MaxInt (just short of
    56  	// 2GB).
    57  	maxMemTableSize = constants.MaxUint32OrInt
    58  )
    59  
    60  // TableCacheSize can be used to determine the table
    61  // cache size for a single db, given the maximum open
    62  // files which can be used by a table cache which is
    63  // only used by a single db.
    64  func TableCacheSize(maxOpenFiles int) int {
    65  	tableCacheSize := maxOpenFiles - numNonTableCacheFiles
    66  	if tableCacheSize < minTableCacheSize {
    67  		tableCacheSize = minTableCacheSize
    68  	}
    69  	return tableCacheSize
    70  }
    71  
    72  // Open opens a DB whose files live in the given directory.
    73  func Open(dirname string, opts *Options) (db *DB, _ error) {
    74  	// Make a copy of the options so that we don't mutate the passed in options.
    75  	opts = opts.Clone()
    76  	opts = opts.EnsureDefaults()
    77  	if err := opts.Validate(); err != nil {
    78  		return nil, err
    79  	}
    80  	if opts.LoggerAndTracer == nil {
    81  		opts.LoggerAndTracer = &base.LoggerWithNoopTracer{Logger: opts.Logger}
    82  	} else {
    83  		opts.Logger = opts.LoggerAndTracer
    84  	}
    85  
    86  	// In all error cases, we return db = nil; this is used by various
    87  	// deferred cleanups.
    88  
    89  	// Open the database and WAL directories first.
    90  	walDirname, dataDir, walDir, err := prepareAndOpenDirs(dirname, opts)
    91  	if err != nil {
    92  		return nil, errors.Wrapf(err, "error opening database at %q", dirname)
    93  	}
    94  	defer func() {
    95  		if db == nil {
    96  			if walDir != dataDir {
    97  				walDir.Close()
    98  			}
    99  			dataDir.Close()
   100  		}
   101  	}()
   102  
   103  	// Lock the database directory.
   104  	var fileLock *Lock
   105  	if opts.Lock != nil {
   106  		// The caller already acquired the database lock. Ensure that the
   107  		// directory matches.
   108  		if dirname != opts.Lock.dirname {
   109  			return nil, errors.Newf("pebble: opts.Lock acquired in %q not %q", opts.Lock.dirname, dirname)
   110  		}
   111  		if err := opts.Lock.refForOpen(); err != nil {
   112  			return nil, err
   113  		}
   114  		fileLock = opts.Lock
   115  	} else {
   116  		fileLock, err = LockDirectory(dirname, opts.FS)
   117  		if err != nil {
   118  			return nil, err
   119  		}
   120  	}
   121  	defer func() {
   122  		if db == nil {
   123  			fileLock.Close()
   124  		}
   125  	}()
   126  
   127  	// Establish the format major version.
   128  	formatVersion, formatVersionMarker, err := lookupFormatMajorVersion(opts.FS, dirname)
   129  	if err != nil {
   130  		return nil, err
   131  	}
   132  	defer func() {
   133  		if db == nil {
   134  			formatVersionMarker.Close()
   135  		}
   136  	}()
   137  
   138  	// Find the currently active manifest, if there is one.
   139  	manifestMarker, manifestFileNum, manifestExists, err := findCurrentManifest(formatVersion, opts.FS, dirname)
   140  	if err != nil {
   141  		return nil, errors.Wrapf(err, "pebble: database %q", dirname)
   142  	}
   143  	defer func() {
   144  		if db == nil {
   145  			manifestMarker.Close()
   146  		}
   147  	}()
   148  
   149  	// Atomic markers may leave behind obsolete files if there's a crash
   150  	// mid-update. Clean these up if we're not in read-only mode.
   151  	if !opts.ReadOnly {
   152  		if err := formatVersionMarker.RemoveObsolete(); err != nil {
   153  			return nil, err
   154  		}
   155  		if err := manifestMarker.RemoveObsolete(); err != nil {
   156  			return nil, err
   157  		}
   158  	}
   159  
   160  	if opts.Cache == nil {
   161  		opts.Cache = cache.New(cacheDefaultSize)
   162  	} else {
   163  		opts.Cache.Ref()
   164  	}
   165  
   166  	d := &DB{
   167  		cacheID:             opts.Cache.NewID(),
   168  		dirname:             dirname,
   169  		walDirname:          walDirname,
   170  		opts:                opts,
   171  		cmp:                 opts.Comparer.Compare,
   172  		equal:               opts.equal(),
   173  		merge:               opts.Merger.Merge,
   174  		split:               opts.Comparer.Split,
   175  		abbreviatedKey:      opts.Comparer.AbbreviatedKey,
   176  		largeBatchThreshold: (opts.MemTableSize - uint64(memTableEmptySize)) / 2,
   177  		fileLock:            fileLock,
   178  		dataDir:             dataDir,
   179  		walDir:              walDir,
   180  		logRecycler:         logRecycler{limit: opts.MemTableStopWritesThreshold + 1},
   181  		closed:              new(atomic.Value),
   182  		closedCh:            make(chan struct{}),
   183  	}
   184  	d.mu.versions = &versionSet{}
   185  	d.diskAvailBytes.Store(math.MaxUint64)
   186  
   187  	defer func() {
   188  		// If an error or panic occurs during open, attempt to release the manually
   189  		// allocated memory resources. Note that rather than look for an error, we
   190  		// look for the return of a nil DB pointer.
   191  		if r := recover(); db == nil {
   192  			// Release our references to the Cache. Note that both the DB, and
   193  			// tableCache have a reference. When we release the reference to
   194  			// the tableCache, and if there are no other references to
   195  			// the tableCache, then the tableCache will also release its
   196  			// reference to the cache.
   197  			opts.Cache.Unref()
   198  
   199  			if d.tableCache != nil {
   200  				_ = d.tableCache.close()
   201  			}
   202  
   203  			for _, mem := range d.mu.mem.queue {
   204  				switch t := mem.flushable.(type) {
   205  				case *memTable:
   206  					manual.Free(t.arenaBuf)
   207  					t.arenaBuf = nil
   208  				}
   209  			}
   210  			if d.cleanupManager != nil {
   211  				d.cleanupManager.Close()
   212  			}
   213  			if d.objProvider != nil {
   214  				d.objProvider.Close()
   215  			}
   216  			if r != nil {
   217  				panic(r)
   218  			}
   219  		}
   220  	}()
   221  
   222  	d.commit = newCommitPipeline(commitEnv{
   223  		logSeqNum:     &d.mu.versions.logSeqNum,
   224  		visibleSeqNum: &d.mu.versions.visibleSeqNum,
   225  		apply:         d.commitApply,
   226  		write:         d.commitWrite,
   227  	})
   228  	d.mu.nextJobID = 1
   229  	d.mu.mem.nextSize = opts.MemTableSize
   230  	if d.mu.mem.nextSize > initialMemTableSize {
   231  		d.mu.mem.nextSize = initialMemTableSize
   232  	}
   233  	d.mu.compact.cond.L = &d.mu.Mutex
   234  	d.mu.compact.inProgress = make(map[*compaction]struct{})
   235  	d.mu.compact.noOngoingFlushStartTime = time.Now()
   236  	d.mu.snapshots.init()
   237  	// logSeqNum is the next sequence number that will be assigned.
   238  	// Start assigning sequence numbers from base.SeqNumStart to leave
   239  	// room for reserved sequence numbers (see comments around
   240  	// SeqNumStart).
   241  	d.mu.versions.logSeqNum.Store(base.SeqNumStart)
   242  	d.mu.formatVers.vers.Store(uint64(formatVersion))
   243  	d.mu.formatVers.marker = formatVersionMarker
   244  
   245  	d.timeNow = time.Now
   246  	d.openedAt = d.timeNow()
   247  
   248  	d.mu.Lock()
   249  	defer d.mu.Unlock()
   250  
   251  	jobID := d.mu.nextJobID
   252  	d.mu.nextJobID++
   253  
   254  	setCurrent := setCurrentFunc(d.FormatMajorVersion(), manifestMarker, opts.FS, dirname, d.dataDir)
   255  
   256  	if !manifestExists {
   257  		// DB does not exist.
   258  		if d.opts.ErrorIfNotExists || d.opts.ReadOnly {
   259  			return nil, errors.Wrapf(ErrDBDoesNotExist, "dirname=%q", dirname)
   260  		}
   261  
   262  		// Create the DB.
   263  		if err := d.mu.versions.create(jobID, dirname, opts, manifestMarker, setCurrent, d.FormatMajorVersion, &d.mu.Mutex); err != nil {
   264  			return nil, err
   265  		}
   266  	} else {
   267  		if opts.ErrorIfExists {
   268  			return nil, errors.Wrapf(ErrDBAlreadyExists, "dirname=%q", dirname)
   269  		}
   270  		// Load the version set.
   271  		if err := d.mu.versions.load(dirname, opts, manifestFileNum.FileNum(), manifestMarker, setCurrent, d.FormatMajorVersion, &d.mu.Mutex); err != nil {
   272  			return nil, err
   273  		}
   274  		if opts.ErrorIfNotPristine {
   275  			liveFileNums := make(map[base.DiskFileNum]struct{})
   276  			d.mu.versions.addLiveFileNums(liveFileNums)
   277  			if len(liveFileNums) != 0 {
   278  				return nil, errors.Wrapf(ErrDBNotPristine, "dirname=%q", dirname)
   279  			}
   280  		}
   281  	}
   282  
   283  	// In read-only mode, we replay directly into the mutable memtable but never
   284  	// flush it. We need to delay creation of the memtable until we know the
   285  	// sequence number of the first batch that will be inserted.
   286  	if !d.opts.ReadOnly {
   287  		var entry *flushableEntry
   288  		d.mu.mem.mutable, entry = d.newMemTable(0 /* logNum */, d.mu.versions.logSeqNum.Load())
   289  		d.mu.mem.queue = append(d.mu.mem.queue, entry)
   290  	}
   291  
   292  	// List the objects
   293  	ls, err := opts.FS.List(d.walDirname)
   294  	if err != nil {
   295  		return nil, err
   296  	}
   297  	if d.dirname != d.walDirname {
   298  		ls2, err := opts.FS.List(d.dirname)
   299  		if err != nil {
   300  			return nil, err
   301  		}
   302  		ls = append(ls, ls2...)
   303  	}
   304  	providerSettings := objstorageprovider.Settings{
   305  		Logger:              opts.Logger,
   306  		FS:                  opts.FS,
   307  		FSDirName:           dirname,
   308  		FSDirInitialListing: ls,
   309  		FSCleaner:           opts.Cleaner,
   310  		NoSyncOnClose:       opts.NoSyncOnClose,
   311  		BytesPerSync:        opts.BytesPerSync,
   312  	}
   313  	providerSettings.Local.ReadaheadConfigFn = opts.Local.ReadaheadConfigFn
   314  	providerSettings.Remote.StorageFactory = opts.Experimental.RemoteStorage
   315  	providerSettings.Remote.CreateOnShared = opts.Experimental.CreateOnShared
   316  	providerSettings.Remote.CreateOnSharedLocator = opts.Experimental.CreateOnSharedLocator
   317  	providerSettings.Remote.CacheSizeBytes = opts.Experimental.SecondaryCacheSizeBytes
   318  
   319  	d.objProvider, err = objstorageprovider.Open(providerSettings)
   320  	if err != nil {
   321  		return nil, err
   322  	}
   323  
   324  	d.cleanupManager = openCleanupManager(opts, d.objProvider, d.onObsoleteTableDelete, d.getDeletionPacerInfo)
   325  
   326  	if manifestExists {
   327  		curVersion := d.mu.versions.currentVersion()
   328  		if err := checkConsistency(curVersion, dirname, d.objProvider); err != nil {
   329  			return nil, err
   330  		}
   331  	}
   332  
   333  	tableCacheSize := TableCacheSize(opts.MaxOpenFiles)
   334  	d.tableCache = newTableCacheContainer(opts.TableCache, d.cacheID, d.objProvider, d.opts, tableCacheSize)
   335  	d.newIters = d.tableCache.newIters
   336  	d.tableNewRangeKeyIter = d.tableCache.newRangeKeyIter
   337  
   338  	// Replay any newer log files than the ones named in the manifest.
   339  	type fileNumAndName struct {
   340  		num  FileNum
   341  		name string
   342  	}
   343  	var logFiles []fileNumAndName
   344  	var previousOptionsFileNum FileNum
   345  	var previousOptionsFilename string
   346  	for _, filename := range ls {
   347  		ft, fn, ok := base.ParseFilename(opts.FS, filename)
   348  		if !ok {
   349  			continue
   350  		}
   351  
   352  		// Don't reuse any obsolete file numbers to avoid modifying an
   353  		// ingested sstable's original external file.
   354  		if d.mu.versions.nextFileNum <= fn.FileNum() {
   355  			d.mu.versions.nextFileNum = fn.FileNum() + 1
   356  		}
   357  
   358  		switch ft {
   359  		case fileTypeLog:
   360  			if fn.FileNum() >= d.mu.versions.minUnflushedLogNum {
   361  				logFiles = append(logFiles, fileNumAndName{fn.FileNum(), filename})
   362  			}
   363  			if d.logRecycler.minRecycleLogNum <= fn.FileNum() {
   364  				d.logRecycler.minRecycleLogNum = fn.FileNum() + 1
   365  			}
   366  		case fileTypeOptions:
   367  			if previousOptionsFileNum < fn.FileNum() {
   368  				previousOptionsFileNum = fn.FileNum()
   369  				previousOptionsFilename = filename
   370  			}
   371  		case fileTypeTemp, fileTypeOldTemp:
   372  			if !d.opts.ReadOnly {
   373  				// Some codepaths write to a temporary file and then
   374  				// rename it to its final location when complete.  A
   375  				// temp file is leftover if a process exits before the
   376  				// rename.  Remove it.
   377  				err := opts.FS.Remove(opts.FS.PathJoin(dirname, filename))
   378  				if err != nil {
   379  					return nil, err
   380  				}
   381  			}
   382  		}
   383  	}
   384  
   385  	// Ratchet d.mu.versions.nextFileNum ahead of all known objects in the
   386  	// objProvider. This avoids FileNum collisions with obsolete sstables.
   387  	objects := d.objProvider.List()
   388  	for _, obj := range objects {
   389  		if d.mu.versions.nextFileNum <= obj.DiskFileNum.FileNum() {
   390  			d.mu.versions.nextFileNum = obj.DiskFileNum.FileNum() + 1
   391  		}
   392  	}
   393  
   394  	// Validate the most-recent OPTIONS file, if there is one.
   395  	var strictWALTail bool
   396  	if previousOptionsFilename != "" {
   397  		path := opts.FS.PathJoin(dirname, previousOptionsFilename)
   398  		strictWALTail, err = checkOptions(opts, path)
   399  		if err != nil {
   400  			return nil, err
   401  		}
   402  	}
   403  
   404  	sort.Slice(logFiles, func(i, j int) bool {
   405  		return logFiles[i].num < logFiles[j].num
   406  	})
   407  
   408  	var ve versionEdit
   409  	var toFlush flushableList
   410  	for i, lf := range logFiles {
   411  		lastWAL := i == len(logFiles)-1
   412  		flush, maxSeqNum, err := d.replayWAL(jobID, &ve, opts.FS,
   413  			opts.FS.PathJoin(d.walDirname, lf.name), lf.num, strictWALTail && !lastWAL)
   414  		if err != nil {
   415  			return nil, err
   416  		}
   417  		toFlush = append(toFlush, flush...)
   418  		d.mu.versions.markFileNumUsed(lf.num)
   419  		if d.mu.versions.logSeqNum.Load() < maxSeqNum {
   420  			d.mu.versions.logSeqNum.Store(maxSeqNum)
   421  		}
   422  	}
   423  	d.mu.versions.visibleSeqNum.Store(d.mu.versions.logSeqNum.Load())
   424  
   425  	if !d.opts.ReadOnly {
   426  		// Create an empty .log file.
   427  		newLogNum := d.mu.versions.getNextFileNum()
   428  
   429  		// This logic is slightly different than RocksDB's. Specifically, RocksDB
   430  		// sets MinUnflushedLogNum to max-recovered-log-num + 1. We set it to the
   431  		// newLogNum. There should be no difference in using either value.
   432  		ve.MinUnflushedLogNum = newLogNum
   433  
   434  		// Create the manifest with the updated MinUnflushedLogNum before
   435  		// creating the new log file. If we created the log file first, a
   436  		// crash before the manifest is synced could leave two WALs with
   437  		// unclean tails.
   438  		d.mu.versions.logLock()
   439  		if err := d.mu.versions.logAndApply(jobID, &ve, newFileMetrics(ve.NewFiles), false /* forceRotation */, func() []compactionInfo {
   440  			return nil
   441  		}); err != nil {
   442  			return nil, err
   443  		}
   444  
   445  		for _, entry := range toFlush {
   446  			entry.readerUnrefLocked(true)
   447  		}
   448  
   449  		newLogName := base.MakeFilepath(opts.FS, d.walDirname, fileTypeLog, newLogNum.DiskFileNum())
   450  		d.mu.log.queue = append(d.mu.log.queue, fileInfo{fileNum: newLogNum.DiskFileNum(), fileSize: 0})
   451  		logFile, err := opts.FS.Create(newLogName)
   452  		if err != nil {
   453  			return nil, err
   454  		}
   455  		if err := d.walDir.Sync(); err != nil {
   456  			return nil, err
   457  		}
   458  		d.opts.EventListener.WALCreated(WALCreateInfo{
   459  			JobID:   jobID,
   460  			Path:    newLogName,
   461  			FileNum: newLogNum,
   462  		})
   463  		// This isn't strictly necessary as we don't use the log number for
   464  		// memtables being flushed, only for the next unflushed memtable.
   465  		d.mu.mem.queue[len(d.mu.mem.queue)-1].logNum = newLogNum
   466  
   467  		logFile = vfs.NewSyncingFile(logFile, vfs.SyncingFileOptions{
   468  			NoSyncOnClose:   d.opts.NoSyncOnClose,
   469  			BytesPerSync:    d.opts.WALBytesPerSync,
   470  			PreallocateSize: d.walPreallocateSize(),
   471  		})
   472  		d.mu.log.metrics.fsyncLatency = prometheus.NewHistogram(prometheus.HistogramOpts{
   473  			Buckets: FsyncLatencyBuckets,
   474  		})
   475  
   476  		logWriterConfig := record.LogWriterConfig{
   477  			WALMinSyncInterval: d.opts.WALMinSyncInterval,
   478  			WALFsyncLatency:    d.mu.log.metrics.fsyncLatency,
   479  			QueueSemChan:       d.commit.logSyncQSem,
   480  		}
   481  		d.mu.log.LogWriter = record.NewLogWriter(logFile, newLogNum, logWriterConfig)
   482  		d.mu.versions.metrics.WAL.Files++
   483  	}
   484  	d.updateReadStateLocked(d.opts.DebugCheck)
   485  
   486  	// If the Options specify a format major version higher than the
   487  	// loaded database's, upgrade it. If this is a new database, this
   488  	// code path also performs an initial upgrade from the starting
   489  	// implicit MostCompatible version.
   490  	//
   491  	// We ratchet the version this far into Open so that migrations have a read
   492  	// state available.
   493  	if !d.opts.ReadOnly && opts.FormatMajorVersion > d.FormatMajorVersion() {
   494  		if err := d.ratchetFormatMajorVersionLocked(opts.FormatMajorVersion); err != nil {
   495  			return nil, err
   496  		}
   497  	}
   498  
   499  	if !d.opts.ReadOnly {
   500  		// Write the current options to disk.
   501  		d.optionsFileNum = d.mu.versions.getNextFileNum().DiskFileNum()
   502  		tmpPath := base.MakeFilepath(opts.FS, dirname, fileTypeTemp, d.optionsFileNum)
   503  		optionsPath := base.MakeFilepath(opts.FS, dirname, fileTypeOptions, d.optionsFileNum)
   504  
   505  		// Write them to a temporary file first, in case we crash before
   506  		// we're done. A corrupt options file prevents opening the
   507  		// database.
   508  		optionsFile, err := opts.FS.Create(tmpPath)
   509  		if err != nil {
   510  			return nil, err
   511  		}
   512  		serializedOpts := []byte(opts.String())
   513  		if _, err := optionsFile.Write(serializedOpts); err != nil {
   514  			return nil, errors.CombineErrors(err, optionsFile.Close())
   515  		}
   516  		d.optionsFileSize = uint64(len(serializedOpts))
   517  		if err := optionsFile.Sync(); err != nil {
   518  			return nil, errors.CombineErrors(err, optionsFile.Close())
   519  		}
   520  		if err := optionsFile.Close(); err != nil {
   521  			return nil, err
   522  		}
   523  		// Atomically rename to the OPTIONS-XXXXXX path. This rename is
   524  		// guaranteed to be atomic because the destination path does not
   525  		// exist.
   526  		if err := opts.FS.Rename(tmpPath, optionsPath); err != nil {
   527  			return nil, err
   528  		}
   529  		if err := d.dataDir.Sync(); err != nil {
   530  			return nil, err
   531  		}
   532  	}
   533  
   534  	if !d.opts.ReadOnly {
   535  		d.scanObsoleteFiles(ls)
   536  		d.deleteObsoleteFiles(jobID)
   537  	} else {
   538  		// All the log files are obsolete.
   539  		d.mu.versions.metrics.WAL.Files = int64(len(logFiles))
   540  	}
   541  	d.mu.tableStats.cond.L = &d.mu.Mutex
   542  	d.mu.tableValidation.cond.L = &d.mu.Mutex
   543  	if !d.opts.ReadOnly {
   544  		d.maybeCollectTableStatsLocked()
   545  	}
   546  	d.calculateDiskAvailableBytes()
   547  
   548  	d.maybeScheduleFlush()
   549  	d.maybeScheduleCompaction()
   550  
   551  	// Note: this is a no-op if invariants are disabled or race is enabled.
   552  	//
   553  	// Setting a finalizer on *DB causes *DB to never be reclaimed and the
   554  	// finalizer to never be run. The problem is due to this limitation of
   555  	// finalizers mention in the SetFinalizer docs:
   556  	//
   557  	//   If a cyclic structure includes a block with a finalizer, that cycle is
   558  	//   not guaranteed to be garbage collected and the finalizer is not
   559  	//   guaranteed to run, because there is no ordering that respects the
   560  	//   dependencies.
   561  	//
   562  	// DB has cycles with several of its internal structures: readState,
   563  	// newIters, tableCache, versions, etc. Each of this individually cause a
   564  	// cycle and prevent the finalizer from being run. But we can workaround this
   565  	// finializer limitation by setting a finalizer on another object that is
   566  	// tied to the lifetime of DB: the DB.closed atomic.Value.
   567  	dPtr := fmt.Sprintf("%p", d)
   568  	invariants.SetFinalizer(d.closed, func(obj interface{}) {
   569  		v := obj.(*atomic.Value)
   570  		if err := v.Load(); err == nil {
   571  			fmt.Fprintf(os.Stderr, "%s: unreferenced DB not closed\n", dPtr)
   572  			os.Exit(1)
   573  		}
   574  	})
   575  
   576  	return d, nil
   577  }
   578  
   579  // prepareAndOpenDirs opens the directories for the store (and creates them if
   580  // necessary).
   581  //
   582  // Returns an error if ReadOnly is set and the directories don't exist.
   583  func prepareAndOpenDirs(
   584  	dirname string, opts *Options,
   585  ) (walDirname string, dataDir vfs.File, walDir vfs.File, err error) {
   586  	walDirname = opts.WALDir
   587  	if opts.WALDir == "" {
   588  		walDirname = dirname
   589  	}
   590  
   591  	// Create directories if needed.
   592  	if !opts.ReadOnly {
   593  		if err := opts.FS.MkdirAll(dirname, 0755); err != nil {
   594  			return "", nil, nil, err
   595  		}
   596  		if walDirname != dirname {
   597  			if err := opts.FS.MkdirAll(walDirname, 0755); err != nil {
   598  				return "", nil, nil, err
   599  			}
   600  		}
   601  	}
   602  
   603  	dataDir, err = opts.FS.OpenDir(dirname)
   604  	if err != nil {
   605  		if opts.ReadOnly && oserror.IsNotExist(err) {
   606  			return "", nil, nil, errors.Errorf("pebble: database %q does not exist", dirname)
   607  		}
   608  		return "", nil, nil, err
   609  	}
   610  
   611  	if walDirname == dirname {
   612  		walDir = dataDir
   613  	} else {
   614  		walDir, err = opts.FS.OpenDir(walDirname)
   615  		if err != nil {
   616  			dataDir.Close()
   617  			return "", nil, nil, err
   618  		}
   619  	}
   620  	return walDirname, dataDir, walDir, nil
   621  }
   622  
   623  // GetVersion returns the engine version string from the latest options
   624  // file present in dir. Used to check what Pebble or RocksDB version was last
   625  // used to write to the database stored in this directory. An empty string is
   626  // returned if no valid OPTIONS file with a version key was found.
   627  func GetVersion(dir string, fs vfs.FS) (string, error) {
   628  	ls, err := fs.List(dir)
   629  	if err != nil {
   630  		return "", err
   631  	}
   632  	var version string
   633  	lastOptionsSeen := FileNum(0)
   634  	for _, filename := range ls {
   635  		ft, fn, ok := base.ParseFilename(fs, filename)
   636  		if !ok {
   637  			continue
   638  		}
   639  		switch ft {
   640  		case fileTypeOptions:
   641  			// If this file has a higher number than the last options file
   642  			// processed, reset version. This is because rocksdb often
   643  			// writes multiple options files without deleting previous ones.
   644  			// Otherwise, skip parsing this options file.
   645  			if fn.FileNum() > lastOptionsSeen {
   646  				version = ""
   647  				lastOptionsSeen = fn.FileNum()
   648  			} else {
   649  				continue
   650  			}
   651  			f, err := fs.Open(fs.PathJoin(dir, filename))
   652  			if err != nil {
   653  				return "", err
   654  			}
   655  			data, err := io.ReadAll(f)
   656  			f.Close()
   657  
   658  			if err != nil {
   659  				return "", err
   660  			}
   661  			err = parseOptions(string(data), func(section, key, value string) error {
   662  				switch {
   663  				case section == "Version":
   664  					switch key {
   665  					case "pebble_version":
   666  						version = value
   667  					case "rocksdb_version":
   668  						version = fmt.Sprintf("rocksdb v%s", value)
   669  					}
   670  				}
   671  				return nil
   672  			})
   673  			if err != nil {
   674  				return "", err
   675  			}
   676  		}
   677  	}
   678  	return version, nil
   679  }
   680  
   681  // replayWAL replays the edits in the specified log file. If the DB is in
   682  // read only mode, then the WALs are replayed into memtables and not flushed. If
   683  // the DB is not in read only mode, then the contents of the WAL are guaranteed
   684  // to be flushed.
   685  //
   686  // The toFlush return value is a list of flushables associated with the WAL
   687  // being replayed which will be flushed. Once the version edit has been applied
   688  // to the manifest, it is up to the caller of replayWAL to unreference the
   689  // toFlush flushables returned by replayWAL.
   690  //
   691  // d.mu must be held when calling this, but the mutex may be dropped and
   692  // re-acquired during the course of this method.
   693  func (d *DB) replayWAL(
   694  	jobID int, ve *versionEdit, fs vfs.FS, filename string, logNum FileNum, strictWALTail bool,
   695  ) (toFlush flushableList, maxSeqNum uint64, err error) {
   696  	file, err := fs.Open(filename)
   697  	if err != nil {
   698  		return nil, 0, err
   699  	}
   700  	defer file.Close()
   701  	var (
   702  		b               Batch
   703  		buf             bytes.Buffer
   704  		mem             *memTable
   705  		entry           *flushableEntry
   706  		rr              = record.NewReader(file, logNum)
   707  		offset          int64 // byte offset in rr
   708  		lastFlushOffset int64
   709  		keysReplayed    int64 // number of keys replayed
   710  		batchesReplayed int64 // number of batches replayed
   711  	)
   712  
   713  	// TODO(jackson): This function is interspersed with panics, in addition to
   714  	// corruption error propagation. Audit them to ensure we're truly only
   715  	// panicking where the error points to Pebble bug and not user or
   716  	// hardware-induced corruption.
   717  
   718  	if d.opts.ReadOnly {
   719  		// In read-only mode, we replay directly into the mutable memtable which will
   720  		// never be flushed.
   721  		mem = d.mu.mem.mutable
   722  		if mem != nil {
   723  			entry = d.mu.mem.queue[len(d.mu.mem.queue)-1]
   724  		}
   725  	}
   726  
   727  	// Flushes the current memtable, if not nil.
   728  	flushMem := func() {
   729  		if mem == nil {
   730  			return
   731  		}
   732  		var logSize uint64
   733  		if offset >= lastFlushOffset {
   734  			logSize = uint64(offset - lastFlushOffset)
   735  		}
   736  		// Else, this was the initial memtable in the read-only case which must have
   737  		// been empty, but we need to flush it since we don't want to add to it later.
   738  		lastFlushOffset = offset
   739  		entry.logSize = logSize
   740  		if !d.opts.ReadOnly {
   741  			toFlush = append(toFlush, entry)
   742  		}
   743  		mem, entry = nil, nil
   744  	}
   745  	// Creates a new memtable if there is no current memtable.
   746  	ensureMem := func(seqNum uint64) {
   747  		if mem != nil {
   748  			return
   749  		}
   750  		mem, entry = d.newMemTable(logNum, seqNum)
   751  		if d.opts.ReadOnly {
   752  			d.mu.mem.mutable = mem
   753  			d.mu.mem.queue = append(d.mu.mem.queue, entry)
   754  		}
   755  	}
   756  
   757  	// updateVE is used to update ve with information about new files created
   758  	// during the flush of any flushable not of type ingestedFlushable. For the
   759  	// flushable of type ingestedFlushable we use custom handling below.
   760  	updateVE := func() error {
   761  		// TODO(bananabrick): See if we can use the actual base level here,
   762  		// instead of using 1.
   763  		c := newFlush(d.opts, d.mu.versions.currentVersion(),
   764  			1 /* base level */, toFlush, d.timeNow())
   765  		newVE, _, _, err := d.runCompaction(jobID, c)
   766  		if err != nil {
   767  			return errors.Wrapf(err, "running compaction during WAL replay")
   768  		}
   769  		ve.NewFiles = append(ve.NewFiles, newVE.NewFiles...)
   770  		return nil
   771  	}
   772  	defer func() {
   773  		if err != nil {
   774  			err = errors.WithDetailf(err, "replaying log %s, offset %d", logNum, offset)
   775  		}
   776  	}()
   777  
   778  	for {
   779  		offset = rr.Offset()
   780  		r, err := rr.Next()
   781  		if err == nil {
   782  			_, err = io.Copy(&buf, r)
   783  		}
   784  		if err != nil {
   785  			// It is common to encounter a zeroed or invalid chunk due to WAL
   786  			// preallocation and WAL recycling. We need to distinguish these
   787  			// errors from EOF in order to recognize that the record was
   788  			// truncated and to avoid replaying subsequent WALs, but want
   789  			// to otherwise treat them like EOF.
   790  			if err == io.EOF {
   791  				break
   792  			} else if record.IsInvalidRecord(err) && !strictWALTail {
   793  				break
   794  			}
   795  			return nil, 0, errors.Wrap(err, "pebble: error when replaying WAL")
   796  		}
   797  
   798  		if buf.Len() < batchHeaderLen {
   799  			return nil, 0, base.CorruptionErrorf("pebble: corrupt log file %q (num %s)",
   800  				filename, errors.Safe(logNum))
   801  		}
   802  
   803  		if d.opts.ErrorIfNotPristine {
   804  			return nil, 0, errors.WithDetailf(ErrDBNotPristine, "location: %q", d.dirname)
   805  		}
   806  
   807  		// Specify Batch.db so that Batch.SetRepr will compute Batch.memTableSize
   808  		// which is used below.
   809  		b = Batch{}
   810  		b.db = d
   811  		b.SetRepr(buf.Bytes())
   812  		seqNum := b.SeqNum()
   813  		maxSeqNum = seqNum + uint64(b.Count())
   814  		keysReplayed += int64(b.Count())
   815  		batchesReplayed++
   816  		{
   817  			br := b.Reader()
   818  			if kind, encodedFileNum, _, ok, err := br.Next(); err != nil {
   819  				return nil, 0, err
   820  			} else if ok && kind == InternalKeyKindIngestSST {
   821  				fileNums := make([]base.DiskFileNum, 0, b.Count())
   822  				addFileNum := func(encodedFileNum []byte) {
   823  					fileNum, n := binary.Uvarint(encodedFileNum)
   824  					if n <= 0 {
   825  						panic("pebble: ingest sstable file num is invalid.")
   826  					}
   827  					fileNums = append(fileNums, base.FileNum(fileNum).DiskFileNum())
   828  				}
   829  				addFileNum(encodedFileNum)
   830  
   831  				for i := 1; i < int(b.Count()); i++ {
   832  					kind, encodedFileNum, _, ok, err := br.Next()
   833  					if err != nil {
   834  						return nil, 0, err
   835  					}
   836  					if kind != InternalKeyKindIngestSST {
   837  						panic("pebble: invalid batch key kind.")
   838  					}
   839  					if !ok {
   840  						panic("pebble: invalid batch count.")
   841  					}
   842  					addFileNum(encodedFileNum)
   843  				}
   844  
   845  				if _, _, _, ok, err := br.Next(); err != nil {
   846  					return nil, 0, err
   847  				} else if ok {
   848  					panic("pebble: invalid number of entries in batch.")
   849  				}
   850  
   851  				meta := make([]*fileMetadata, len(fileNums))
   852  				for i, n := range fileNums {
   853  					var readable objstorage.Readable
   854  					objMeta, err := d.objProvider.Lookup(fileTypeTable, n)
   855  					if err != nil {
   856  						return nil, 0, errors.Wrap(err, "pebble: error when looking up ingested SSTs")
   857  					}
   858  					if objMeta.IsRemote() {
   859  						readable, err = d.objProvider.OpenForReading(context.TODO(), fileTypeTable, n, objstorage.OpenOptions{MustExist: true})
   860  						if err != nil {
   861  							return nil, 0, errors.Wrap(err, "pebble: error when opening flushable ingest files")
   862  						}
   863  					} else {
   864  						path := base.MakeFilepath(d.opts.FS, d.dirname, fileTypeTable, n)
   865  						f, err := d.opts.FS.Open(path)
   866  						if err != nil {
   867  							return nil, 0, err
   868  						}
   869  
   870  						readable, err = sstable.NewSimpleReadable(f)
   871  						if err != nil {
   872  							return nil, 0, err
   873  						}
   874  					}
   875  					// NB: ingestLoad1 will close readable.
   876  					meta[i], err = ingestLoad1(d.opts, d.FormatMajorVersion(), readable, d.cacheID, n)
   877  					if err != nil {
   878  						return nil, 0, errors.Wrap(err, "pebble: error when loading flushable ingest files")
   879  					}
   880  				}
   881  
   882  				if uint32(len(meta)) != b.Count() {
   883  					panic("pebble: couldn't load all files in WAL entry.")
   884  				}
   885  
   886  				entry, err = d.newIngestedFlushableEntry(
   887  					meta, seqNum, logNum,
   888  				)
   889  				if err != nil {
   890  					return nil, 0, err
   891  				}
   892  
   893  				if d.opts.ReadOnly {
   894  					d.mu.mem.queue = append(d.mu.mem.queue, entry)
   895  					// We added the IngestSST flushable to the queue. But there
   896  					// must be at least one WAL entry waiting to be replayed. We
   897  					// have to ensure this newer WAL entry isn't replayed into
   898  					// the current value of d.mu.mem.mutable because the current
   899  					// mutable memtable exists before this flushable entry in
   900  					// the memtable queue. To ensure this, we just need to unset
   901  					// d.mu.mem.mutable. When a newer WAL is replayed, we will
   902  					// set d.mu.mem.mutable to a newer value.
   903  					d.mu.mem.mutable = nil
   904  				} else {
   905  					toFlush = append(toFlush, entry)
   906  					// During WAL replay, the lsm only has L0, hence, the
   907  					// baseLevel is 1. For the sake of simplicity, we place the
   908  					// ingested files in L0 here, instead of finding their
   909  					// target levels. This is a simplification for the sake of
   910  					// simpler code. It is expected that WAL replay should be
   911  					// rare, and that flushables of type ingestedFlushable
   912  					// should also be rare. So, placing the ingested files in L0
   913  					// is alright.
   914  					//
   915  					// TODO(bananabrick): Maybe refactor this function to allow
   916  					// us to easily place ingested files in levels as low as
   917  					// possible during WAL replay. It would require breaking up
   918  					// the application of ve to the manifest into chunks and is
   919  					// not pretty w/o a refactor to this function and how it's
   920  					// used.
   921  					c := newFlush(
   922  						d.opts, d.mu.versions.currentVersion(),
   923  						1, /* base level */
   924  						[]*flushableEntry{entry},
   925  						d.timeNow(),
   926  					)
   927  					for _, file := range c.flushing[0].flushable.(*ingestedFlushable).files {
   928  						ve.NewFiles = append(ve.NewFiles, newFileEntry{Level: 0, Meta: file.FileMetadata})
   929  					}
   930  				}
   931  				return toFlush, maxSeqNum, nil
   932  			}
   933  		}
   934  
   935  		if b.memTableSize >= uint64(d.largeBatchThreshold) {
   936  			flushMem()
   937  			// Make a copy of the data slice since it is currently owned by buf and will
   938  			// be reused in the next iteration.
   939  			b.data = append([]byte(nil), b.data...)
   940  			b.flushable, err = newFlushableBatch(&b, d.opts.Comparer)
   941  			if err != nil {
   942  				return nil, 0, err
   943  			}
   944  			entry := d.newFlushableEntry(b.flushable, logNum, b.SeqNum())
   945  			// Disable memory accounting by adding a reader ref that will never be
   946  			// removed.
   947  			entry.readerRefs.Add(1)
   948  			if d.opts.ReadOnly {
   949  				d.mu.mem.queue = append(d.mu.mem.queue, entry)
   950  				// We added the flushable batch to the flushable to the queue.
   951  				// But there must be at least one WAL entry waiting to be
   952  				// replayed. We have to ensure this newer WAL entry isn't
   953  				// replayed into the current value of d.mu.mem.mutable because
   954  				// the current mutable memtable exists before this flushable
   955  				// entry in the memtable queue. To ensure this, we just need to
   956  				// unset d.mu.mem.mutable. When a newer WAL is replayed, we will
   957  				// set d.mu.mem.mutable to a newer value.
   958  				d.mu.mem.mutable = nil
   959  			} else {
   960  				toFlush = append(toFlush, entry)
   961  			}
   962  		} else {
   963  			ensureMem(seqNum)
   964  			if err = mem.prepare(&b); err != nil && err != arenaskl.ErrArenaFull {
   965  				return nil, 0, err
   966  			}
   967  			// We loop since DB.newMemTable() slowly grows the size of allocated memtables, so the
   968  			// batch may not initially fit, but will eventually fit (since it is smaller than
   969  			// largeBatchThreshold).
   970  			for err == arenaskl.ErrArenaFull {
   971  				flushMem()
   972  				ensureMem(seqNum)
   973  				err = mem.prepare(&b)
   974  				if err != nil && err != arenaskl.ErrArenaFull {
   975  					return nil, 0, err
   976  				}
   977  			}
   978  			if err = mem.apply(&b, seqNum); err != nil {
   979  				return nil, 0, err
   980  			}
   981  			mem.writerUnref()
   982  		}
   983  		buf.Reset()
   984  	}
   985  
   986  	d.opts.Logger.Infof("[JOB %d] WAL file %s with log number %s stopped reading at offset: %d; replayed %d keys in %d batches", jobID, filename, logNum.String(), offset, keysReplayed, batchesReplayed)
   987  	flushMem()
   988  
   989  	// mem is nil here.
   990  	if !d.opts.ReadOnly {
   991  		err = updateVE()
   992  		if err != nil {
   993  			return nil, 0, err
   994  		}
   995  	}
   996  	return toFlush, maxSeqNum, err
   997  }
   998  
   999  func checkOptions(opts *Options, path string) (strictWALTail bool, err error) {
  1000  	f, err := opts.FS.Open(path)
  1001  	if err != nil {
  1002  		return false, err
  1003  	}
  1004  	defer f.Close()
  1005  
  1006  	data, err := io.ReadAll(f)
  1007  	if err != nil {
  1008  		return false, err
  1009  	}
  1010  	return opts.checkOptions(string(data))
  1011  }
  1012  
  1013  // DBDesc briefly describes high-level state about a database.
  1014  type DBDesc struct {
  1015  	// Exists is true if an existing database was found.
  1016  	Exists bool
  1017  	// FormatMajorVersion indicates the database's current format
  1018  	// version.
  1019  	FormatMajorVersion FormatMajorVersion
  1020  	// ManifestFilename is the filename of the current active manifest,
  1021  	// if the database exists.
  1022  	ManifestFilename string
  1023  }
  1024  
  1025  // Peek looks for an existing database in dirname on the provided FS. It
  1026  // returns a brief description of the database. Peek is read-only and
  1027  // does not open the database
  1028  func Peek(dirname string, fs vfs.FS) (*DBDesc, error) {
  1029  	vers, versMarker, err := lookupFormatMajorVersion(fs, dirname)
  1030  	if err != nil {
  1031  		return nil, err
  1032  	}
  1033  	// TODO(jackson): Immediately closing the marker is clunky. Add a
  1034  	// PeekMarker variant that avoids opening the directory.
  1035  	if err := versMarker.Close(); err != nil {
  1036  		return nil, err
  1037  	}
  1038  
  1039  	// Find the currently active manifest, if there is one.
  1040  	manifestMarker, manifestFileNum, exists, err := findCurrentManifest(vers, fs, dirname)
  1041  	if err != nil {
  1042  		return nil, err
  1043  	}
  1044  	// TODO(jackson): Immediately closing the marker is clunky. Add a
  1045  	// PeekMarker variant that avoids opening the directory.
  1046  	if err := manifestMarker.Close(); err != nil {
  1047  		return nil, err
  1048  	}
  1049  
  1050  	desc := &DBDesc{
  1051  		Exists:             exists,
  1052  		FormatMajorVersion: vers,
  1053  	}
  1054  	if exists {
  1055  		desc.ManifestFilename = base.MakeFilepath(fs, dirname, fileTypeManifest, manifestFileNum)
  1056  	}
  1057  	return desc, nil
  1058  }
  1059  
  1060  // LockDirectory acquires the database directory lock in the named directory,
  1061  // preventing another process from opening the database. LockDirectory returns a
  1062  // handle to the held lock that may be passed to Open through Options.Lock to
  1063  // subsequently open the database, skipping lock acquistion during Open.
  1064  //
  1065  // LockDirectory may be used to expand the critical section protected by the
  1066  // database lock to include setup before the call to Open.
  1067  func LockDirectory(dirname string, fs vfs.FS) (*Lock, error) {
  1068  	fileLock, err := fs.Lock(base.MakeFilepath(fs, dirname, fileTypeLock, base.FileNum(0).DiskFileNum()))
  1069  	if err != nil {
  1070  		return nil, err
  1071  	}
  1072  	l := &Lock{dirname: dirname, fileLock: fileLock}
  1073  	l.refs.Store(1)
  1074  	invariants.SetFinalizer(l, func(obj interface{}) {
  1075  		if refs := obj.(*Lock).refs.Load(); refs > 0 {
  1076  			panic(errors.AssertionFailedf("lock for %q finalized with %d refs", dirname, refs))
  1077  		}
  1078  	})
  1079  	return l, nil
  1080  }
  1081  
  1082  // Lock represents a file lock on a directory. It may be passed to Open through
  1083  // Options.Lock to elide lock aquisition during Open.
  1084  type Lock struct {
  1085  	dirname  string
  1086  	fileLock io.Closer
  1087  	// refs is a count of the number of handles on the lock. refs must be 0, 1
  1088  	// or 2.
  1089  	//
  1090  	// When acquired by the client and passed to Open, refs = 1 and the Open
  1091  	// call increments it to 2. When the database is closed, it's decremented to
  1092  	// 1. Finally when the original caller, calls Close on the Lock, it's
  1093  	// drecemented to zero and the underlying file lock is released.
  1094  	//
  1095  	// When Open acquires the file lock, refs remains at 1 until the database is
  1096  	// closed.
  1097  	refs atomic.Int32
  1098  }
  1099  
  1100  func (l *Lock) refForOpen() error {
  1101  	// During Open, when a user passed in a lock, the reference count must be
  1102  	// exactly 1. If it's zero, the lock is no longer held and is invalid. If
  1103  	// it's 2, the lock is already in use by another database within the
  1104  	// process.
  1105  	if !l.refs.CompareAndSwap(1, 2) {
  1106  		return errors.Errorf("pebble: unexpected Lock reference count; is the lock already in use?")
  1107  	}
  1108  	return nil
  1109  }
  1110  
  1111  // Close releases the lock, permitting another process to lock and open the
  1112  // database. Close must not be called until after a database using the Lock has
  1113  // been closed.
  1114  func (l *Lock) Close() error {
  1115  	if l.refs.Add(-1) > 0 {
  1116  		return nil
  1117  	}
  1118  	defer func() { l.fileLock = nil }()
  1119  	return l.fileLock.Close()
  1120  }
  1121  
  1122  // ErrDBDoesNotExist is generated when ErrorIfNotExists is set and the database
  1123  // does not exist.
  1124  //
  1125  // Note that errors can be wrapped with more details; use errors.Is().
  1126  var ErrDBDoesNotExist = errors.New("pebble: database does not exist")
  1127  
  1128  // ErrDBAlreadyExists is generated when ErrorIfExists is set and the database
  1129  // already exists.
  1130  //
  1131  // Note that errors can be wrapped with more details; use errors.Is().
  1132  var ErrDBAlreadyExists = errors.New("pebble: database already exists")
  1133  
  1134  // ErrDBNotPristine is generated when ErrorIfNotPristine is set and the database
  1135  // already exists and is not pristine.
  1136  //
  1137  // Note that errors can be wrapped with more details; use errors.Is().
  1138  var ErrDBNotPristine = errors.New("pebble: database already exists and is not pristine")
  1139  
  1140  // IsCorruptionError returns true if the given error indicates database
  1141  // corruption.
  1142  func IsCorruptionError(err error) bool {
  1143  	return errors.Is(err, base.ErrCorruption)
  1144  }
  1145  
  1146  func checkConsistency(v *manifest.Version, dirname string, objProvider objstorage.Provider) error {
  1147  	var buf bytes.Buffer
  1148  	var args []interface{}
  1149  
  1150  	dedup := make(map[base.DiskFileNum]struct{})
  1151  	for level, files := range v.Levels {
  1152  		iter := files.Iter()
  1153  		for f := iter.First(); f != nil; f = iter.Next() {
  1154  			backingState := f.FileBacking
  1155  			if _, ok := dedup[backingState.DiskFileNum]; ok {
  1156  				continue
  1157  			}
  1158  			dedup[backingState.DiskFileNum] = struct{}{}
  1159  			fileNum := backingState.DiskFileNum
  1160  			fileSize := backingState.Size
  1161  			// We allow foreign objects to have a mismatch between sizes. This is
  1162  			// because we might skew the backing size stored by our objprovider
  1163  			// to prevent us from over-prioritizing this file for compaction.
  1164  			meta, err := objProvider.Lookup(base.FileTypeTable, fileNum)
  1165  			var size int64
  1166  			if err == nil {
  1167  				if objProvider.IsSharedForeign(meta) {
  1168  					continue
  1169  				}
  1170  				size, err = objProvider.Size(meta)
  1171  			}
  1172  			if err != nil {
  1173  				buf.WriteString("L%d: %s: %v\n")
  1174  				args = append(args, errors.Safe(level), errors.Safe(fileNum), err)
  1175  				continue
  1176  			}
  1177  
  1178  			if size != int64(fileSize) {
  1179  				buf.WriteString("L%d: %s: object size mismatch (%s): %d (disk) != %d (MANIFEST)\n")
  1180  				args = append(args, errors.Safe(level), errors.Safe(fileNum), objProvider.Path(meta),
  1181  					errors.Safe(size), errors.Safe(fileSize))
  1182  				continue
  1183  			}
  1184  		}
  1185  	}
  1186  
  1187  	if buf.Len() == 0 {
  1188  		return nil
  1189  	}
  1190  	return errors.Errorf(buf.String(), args...)
  1191  }