github.com/cockroachdb/pebble@v0.0.0-20231214172447-ab4952c5f87b/checkpoint.go (about)

     1  // Copyright 2019 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  	"io"
     9  	"os"
    10  
    11  	"github.com/cockroachdb/errors/oserror"
    12  	"github.com/cockroachdb/pebble/internal/base"
    13  	"github.com/cockroachdb/pebble/record"
    14  	"github.com/cockroachdb/pebble/vfs"
    15  	"github.com/cockroachdb/pebble/vfs/atomicfs"
    16  )
    17  
    18  // checkpointOptions hold the optional parameters to construct checkpoint
    19  // snapshots.
    20  type checkpointOptions struct {
    21  	// flushWAL set to true will force a flush and sync of the WAL prior to
    22  	// checkpointing.
    23  	flushWAL bool
    24  
    25  	// If set, any SSTs that don't overlap with these spans are excluded from a checkpoint.
    26  	restrictToSpans []CheckpointSpan
    27  }
    28  
    29  // CheckpointOption set optional parameters used by `DB.Checkpoint`.
    30  type CheckpointOption func(*checkpointOptions)
    31  
    32  // WithFlushedWAL enables flushing and syncing the WAL prior to constructing a
    33  // checkpoint. This guarantees that any writes committed before calling
    34  // DB.Checkpoint will be part of that checkpoint.
    35  //
    36  // Note that this setting can only be useful in cases when some writes are
    37  // performed with Sync = false. Otherwise, the guarantee will already be met.
    38  //
    39  // Passing this option is functionally equivalent to calling
    40  // DB.LogData(nil, Sync) right before DB.Checkpoint.
    41  func WithFlushedWAL() CheckpointOption {
    42  	return func(opt *checkpointOptions) {
    43  		opt.flushWAL = true
    44  	}
    45  }
    46  
    47  // WithRestrictToSpans specifies spans of interest for the checkpoint. Any SSTs
    48  // that don't overlap with any of these spans are excluded from the checkpoint.
    49  //
    50  // Note that the checkpoint can still surface keys outside of these spans (from
    51  // the WAL and from SSTs that partially overlap with these spans). Moreover,
    52  // these surface keys aren't necessarily "valid" in that they could have been
    53  // modified but the SST containing the modification is excluded.
    54  func WithRestrictToSpans(spans []CheckpointSpan) CheckpointOption {
    55  	return func(opt *checkpointOptions) {
    56  		opt.restrictToSpans = spans
    57  	}
    58  }
    59  
    60  // CheckpointSpan is a key range [Start, End) (inclusive on Start, exclusive on
    61  // End) of interest for a checkpoint.
    62  type CheckpointSpan struct {
    63  	Start []byte
    64  	End   []byte
    65  }
    66  
    67  // excludeFromCheckpoint returns true if an SST file should be excluded from the
    68  // checkpoint because it does not overlap with the spans of interest
    69  // (opt.restrictToSpans).
    70  func excludeFromCheckpoint(f *fileMetadata, opt *checkpointOptions, cmp Compare) bool {
    71  	if len(opt.restrictToSpans) == 0 {
    72  		// Option not set; don't exclude anything.
    73  		return false
    74  	}
    75  	for _, s := range opt.restrictToSpans {
    76  		if f.Overlaps(cmp, s.Start, s.End, true /* exclusiveEnd */) {
    77  			return false
    78  		}
    79  	}
    80  	// None of the restrictToSpans overlapped; we can exclude this file.
    81  	return true
    82  }
    83  
    84  // mkdirAllAndSyncParents creates destDir and any of its missing parents.
    85  // Those missing parents, as well as the closest existing ancestor, are synced.
    86  // Returns a handle to the directory created at destDir.
    87  func mkdirAllAndSyncParents(fs vfs.FS, destDir string) (vfs.File, error) {
    88  	// Collect paths for all directories between destDir (excluded) and its
    89  	// closest existing ancestor (included).
    90  	var parentPaths []string
    91  	foundExistingAncestor := false
    92  	for parentPath := fs.PathDir(destDir); parentPath != "."; parentPath = fs.PathDir(parentPath) {
    93  		parentPaths = append(parentPaths, parentPath)
    94  		_, err := fs.Stat(parentPath)
    95  		if err == nil {
    96  			// Exit loop at the closest existing ancestor.
    97  			foundExistingAncestor = true
    98  			break
    99  		}
   100  		if !oserror.IsNotExist(err) {
   101  			return nil, err
   102  		}
   103  	}
   104  	// Handle empty filesystem edge case.
   105  	if !foundExistingAncestor {
   106  		parentPaths = append(parentPaths, "")
   107  	}
   108  	// Create destDir and any of its missing parents.
   109  	if err := fs.MkdirAll(destDir, 0755); err != nil {
   110  		return nil, err
   111  	}
   112  	// Sync all the parent directories up to the closest existing ancestor,
   113  	// included.
   114  	for _, parentPath := range parentPaths {
   115  		parentDir, err := fs.OpenDir(parentPath)
   116  		if err != nil {
   117  			return nil, err
   118  		}
   119  		err = parentDir.Sync()
   120  		if err != nil {
   121  			_ = parentDir.Close()
   122  			return nil, err
   123  		}
   124  		err = parentDir.Close()
   125  		if err != nil {
   126  			return nil, err
   127  		}
   128  	}
   129  	return fs.OpenDir(destDir)
   130  }
   131  
   132  // Checkpoint constructs a snapshot of the DB instance in the specified
   133  // directory. The WAL, MANIFEST, OPTIONS, and sstables will be copied into the
   134  // snapshot. Hard links will be used when possible. Beware of the significant
   135  // space overhead for a checkpoint if hard links are disabled. Also beware that
   136  // even if hard links are used, the space overhead for the checkpoint will
   137  // increase over time as the DB performs compactions.
   138  func (d *DB) Checkpoint(
   139  	destDir string, opts ...CheckpointOption,
   140  ) (
   141  	ckErr error, /* used in deferred cleanup */
   142  ) {
   143  	opt := &checkpointOptions{}
   144  	for _, fn := range opts {
   145  		fn(opt)
   146  	}
   147  
   148  	if _, err := d.opts.FS.Stat(destDir); !oserror.IsNotExist(err) {
   149  		if err == nil {
   150  			return &os.PathError{
   151  				Op:   "checkpoint",
   152  				Path: destDir,
   153  				Err:  oserror.ErrExist,
   154  			}
   155  		}
   156  		return err
   157  	}
   158  
   159  	if opt.flushWAL && !d.opts.DisableWAL {
   160  		// Write an empty log-data record to flush and sync the WAL.
   161  		if err := d.LogData(nil /* data */, Sync); err != nil {
   162  			return err
   163  		}
   164  	}
   165  
   166  	// Disable file deletions.
   167  	d.mu.Lock()
   168  	d.disableFileDeletions()
   169  	defer func() {
   170  		d.mu.Lock()
   171  		defer d.mu.Unlock()
   172  		d.enableFileDeletions()
   173  	}()
   174  
   175  	// TODO(peter): RocksDB provides the option to roll the manifest if the
   176  	// MANIFEST size is too large. Should we do this too?
   177  
   178  	// Lock the manifest before getting the current version. We need the
   179  	// length of the manifest that we read to match the current version that
   180  	// we read, otherwise we might copy a versionEdit not reflected in the
   181  	// sstables we copy/link.
   182  	d.mu.versions.logLock()
   183  	// Get the unflushed log files, the current version, and the current manifest
   184  	// file number.
   185  	memQueue := d.mu.mem.queue
   186  	current := d.mu.versions.currentVersion()
   187  	formatVers := d.FormatMajorVersion()
   188  	manifestFileNum := d.mu.versions.manifestFileNum
   189  	manifestSize := d.mu.versions.manifest.Size()
   190  	optionsFileNum := d.optionsFileNum
   191  	virtualBackingFiles := make(map[base.DiskFileNum]struct{})
   192  	for diskFileNum := range d.mu.versions.backingState.fileBackingMap {
   193  		virtualBackingFiles[diskFileNum] = struct{}{}
   194  	}
   195  	// Release the manifest and DB.mu so we don't block other operations on
   196  	// the database.
   197  	d.mu.versions.logUnlock()
   198  	d.mu.Unlock()
   199  
   200  	// Wrap the normal filesystem with one which wraps newly created files with
   201  	// vfs.NewSyncingFile.
   202  	fs := vfs.NewSyncingFS(d.opts.FS, vfs.SyncingFileOptions{
   203  		NoSyncOnClose: d.opts.NoSyncOnClose,
   204  		BytesPerSync:  d.opts.BytesPerSync,
   205  	})
   206  
   207  	// Create the dir and its parents (if necessary), and sync them.
   208  	var dir vfs.File
   209  	defer func() {
   210  		if dir != nil {
   211  			_ = dir.Close()
   212  		}
   213  		if ckErr != nil {
   214  			// Attempt to cleanup on error.
   215  			_ = fs.RemoveAll(destDir)
   216  		}
   217  	}()
   218  	dir, ckErr = mkdirAllAndSyncParents(fs, destDir)
   219  	if ckErr != nil {
   220  		return ckErr
   221  	}
   222  
   223  	{
   224  		// Link or copy the OPTIONS.
   225  		srcPath := base.MakeFilepath(fs, d.dirname, fileTypeOptions, optionsFileNum)
   226  		destPath := fs.PathJoin(destDir, fs.PathBase(srcPath))
   227  		ckErr = vfs.LinkOrCopy(fs, srcPath, destPath)
   228  		if ckErr != nil {
   229  			return ckErr
   230  		}
   231  	}
   232  
   233  	{
   234  		// Set the format major version in the destination directory.
   235  		var versionMarker *atomicfs.Marker
   236  		versionMarker, _, ckErr = atomicfs.LocateMarker(fs, destDir, formatVersionMarkerName)
   237  		if ckErr != nil {
   238  			return ckErr
   239  		}
   240  
   241  		// We use the marker to encode the active format version in the
   242  		// marker filename. Unlike other uses of the atomic marker,
   243  		// there is no file with the filename `formatVers.String()` on
   244  		// the filesystem.
   245  		ckErr = versionMarker.Move(formatVers.String())
   246  		if ckErr != nil {
   247  			return ckErr
   248  		}
   249  		ckErr = versionMarker.Close()
   250  		if ckErr != nil {
   251  			return ckErr
   252  		}
   253  	}
   254  
   255  	var excludedFiles map[deletedFileEntry]*fileMetadata
   256  	// Set of FileBacking.DiskFileNum which will be required by virtual sstables
   257  	// in the checkpoint.
   258  	requiredVirtualBackingFiles := make(map[base.DiskFileNum]struct{})
   259  	// Link or copy the sstables.
   260  	for l := range current.Levels {
   261  		iter := current.Levels[l].Iter()
   262  		for f := iter.First(); f != nil; f = iter.Next() {
   263  			if excludeFromCheckpoint(f, opt, d.cmp) {
   264  				if excludedFiles == nil {
   265  					excludedFiles = make(map[deletedFileEntry]*fileMetadata)
   266  				}
   267  				excludedFiles[deletedFileEntry{
   268  					Level:   l,
   269  					FileNum: f.FileNum,
   270  				}] = f
   271  				continue
   272  			}
   273  
   274  			fileBacking := f.FileBacking
   275  			if f.Virtual {
   276  				if _, ok := requiredVirtualBackingFiles[fileBacking.DiskFileNum]; ok {
   277  					continue
   278  				}
   279  				requiredVirtualBackingFiles[fileBacking.DiskFileNum] = struct{}{}
   280  			}
   281  
   282  			srcPath := base.MakeFilepath(fs, d.dirname, fileTypeTable, fileBacking.DiskFileNum)
   283  			destPath := fs.PathJoin(destDir, fs.PathBase(srcPath))
   284  			ckErr = vfs.LinkOrCopy(fs, srcPath, destPath)
   285  			if ckErr != nil {
   286  				return ckErr
   287  			}
   288  		}
   289  	}
   290  
   291  	var removeBackingTables []base.DiskFileNum
   292  	for diskFileNum := range virtualBackingFiles {
   293  		if _, ok := requiredVirtualBackingFiles[diskFileNum]; !ok {
   294  			// The backing sstable associated with fileNum is no longer
   295  			// required.
   296  			removeBackingTables = append(removeBackingTables, diskFileNum)
   297  		}
   298  	}
   299  
   300  	ckErr = d.writeCheckpointManifest(
   301  		fs, formatVers, destDir, dir, manifestFileNum, manifestSize,
   302  		excludedFiles, removeBackingTables,
   303  	)
   304  	if ckErr != nil {
   305  		return ckErr
   306  	}
   307  
   308  	// Copy the WAL files. We copy rather than link because WAL file recycling
   309  	// will cause the WAL files to be reused which would invalidate the
   310  	// checkpoint.
   311  	for i := range memQueue {
   312  		logNum := memQueue[i].logNum
   313  		if logNum == 0 {
   314  			continue
   315  		}
   316  		srcPath := base.MakeFilepath(fs, d.walDirname, fileTypeLog, logNum)
   317  		destPath := fs.PathJoin(destDir, fs.PathBase(srcPath))
   318  		ckErr = vfs.Copy(fs, srcPath, destPath)
   319  		if ckErr != nil {
   320  			return ckErr
   321  		}
   322  	}
   323  
   324  	// Sync and close the checkpoint directory.
   325  	ckErr = dir.Sync()
   326  	if ckErr != nil {
   327  		return ckErr
   328  	}
   329  	ckErr = dir.Close()
   330  	dir = nil
   331  	return ckErr
   332  }
   333  
   334  func (d *DB) writeCheckpointManifest(
   335  	fs vfs.FS,
   336  	formatVers FormatMajorVersion,
   337  	destDirPath string,
   338  	destDir vfs.File,
   339  	manifestFileNum base.DiskFileNum,
   340  	manifestSize int64,
   341  	excludedFiles map[deletedFileEntry]*fileMetadata,
   342  	removeBackingTables []base.DiskFileNum,
   343  ) error {
   344  	// Copy the MANIFEST, and create a pointer to it. We copy rather
   345  	// than link because additional version edits added to the
   346  	// MANIFEST after we took our snapshot of the sstables will
   347  	// reference sstables that aren't in our checkpoint. For a
   348  	// similar reason, we need to limit how much of the MANIFEST we
   349  	// copy.
   350  	// If some files are excluded from the checkpoint, also append a block that
   351  	// records those files as deleted.
   352  	if err := func() error {
   353  		srcPath := base.MakeFilepath(fs, d.dirname, fileTypeManifest, manifestFileNum)
   354  		destPath := fs.PathJoin(destDirPath, fs.PathBase(srcPath))
   355  		src, err := fs.Open(srcPath, vfs.SequentialReadsOption)
   356  		if err != nil {
   357  			return err
   358  		}
   359  		defer src.Close()
   360  
   361  		dst, err := fs.Create(destPath)
   362  		if err != nil {
   363  			return err
   364  		}
   365  		defer dst.Close()
   366  
   367  		// Copy all existing records. We need to copy at the record level in case we
   368  		// need to append another record with the excluded files (we cannot simply
   369  		// append a record after a raw data copy; see
   370  		// https://github.com/cockroachdb/cockroach/issues/100935).
   371  		r := record.NewReader(&io.LimitedReader{R: src, N: manifestSize}, manifestFileNum)
   372  		w := record.NewWriter(dst)
   373  		for {
   374  			rr, err := r.Next()
   375  			if err != nil {
   376  				if err == io.EOF {
   377  					break
   378  				}
   379  				return err
   380  			}
   381  
   382  			rw, err := w.Next()
   383  			if err != nil {
   384  				return err
   385  			}
   386  			if _, err := io.Copy(rw, rr); err != nil {
   387  				return err
   388  			}
   389  		}
   390  
   391  		if len(excludedFiles) > 0 {
   392  			// Write out an additional VersionEdit that deletes the excluded SST files.
   393  			ve := versionEdit{
   394  				DeletedFiles:         excludedFiles,
   395  				RemovedBackingTables: removeBackingTables,
   396  			}
   397  
   398  			rw, err := w.Next()
   399  			if err != nil {
   400  				return err
   401  			}
   402  			if err := ve.Encode(rw); err != nil {
   403  				return err
   404  			}
   405  		}
   406  		if err := w.Close(); err != nil {
   407  			return err
   408  		}
   409  		return dst.Sync()
   410  	}(); err != nil {
   411  		return err
   412  	}
   413  
   414  	// Recent format versions use an atomic marker for setting the
   415  	// active manifest. Older versions use the CURRENT file. The
   416  	// setCurrentFunc function will return a closure that will
   417  	// take the appropriate action for the database's format
   418  	// version.
   419  	var manifestMarker *atomicfs.Marker
   420  	manifestMarker, _, err := atomicfs.LocateMarker(fs, destDirPath, manifestMarkerName)
   421  	if err != nil {
   422  		return err
   423  	}
   424  	if err := setCurrentFunc(formatVers, manifestMarker, fs, destDirPath, destDir)(manifestFileNum); err != nil {
   425  		return err
   426  	}
   427  	return manifestMarker.Close()
   428  }