Skip to content

Commit

Permalink
db: refactor obsolete file deletion code
Browse files Browse the repository at this point in the history
The code around obsolete file deletion is very convoluted and there is
one known race (described in #2630).

This change reworks the deletion code as follows:
 - we add a cleanup manager which has a background goroutine.
 - all file and object deletions happen in the manager's background
   goroutine.
 - callers can enqueue cleanup jobs with the manager
 - callers can wait for outstanding cleanup jobs to finish

We also add a missing call to `deleteObsoleteObjects` from the ingest
path (which now can cause files to be removed through excising).

Finally, we fix an incorrect `Remove` call in the error path of
`Checkpoint()` (which was missing a `PathJoin`).

Fixes #2630.
  • Loading branch information
RaduBerinde committed Jun 21, 2023
1 parent 876b2e7 commit c0b26f2
Show file tree
Hide file tree
Showing 19 changed files with 386 additions and 550 deletions.
6 changes: 1 addition & 5 deletions checkpoint.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,11 +215,7 @@ func (d *DB) Checkpoint(
}
if ckErr != nil {
// Attempt to cleanup on error.
paths, _ := fs.List(destDir)
for _, path := range paths {
_ = fs.Remove(path)
}
_ = fs.Remove(destDir)
_ = fs.RemoveAll(destDir)
}
}()
dir, ckErr = mkdirAllAndSyncParents(fs, destDir)
Expand Down
3 changes: 2 additions & 1 deletion checkpoint_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,7 @@ func TestCheckpoint(t *testing.T) {
if err := d.Compact(nil, []byte("\xff"), false); err != nil {
return err.Error()
}
d.TestOnlyWaitForCleaning()
return memLog.String()

case "flush":
Expand Down Expand Up @@ -189,7 +190,7 @@ func TestCheckpointCompaction(t *testing.T) {
defer close(check)
defer wg.Done()
for i := 0; ctx.Err() == nil && i < 200; i++ {
dir := fmt.Sprintf("checkpoint%6d", i)
dir := fmt.Sprintf("checkpoint%06d", i)
if err := d.Checkpoint(dir); err != nil {
t.Error(err)
return
Expand Down
240 changes: 239 additions & 1 deletion cleaner.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,18 @@

package pebble

import "github.com/cockroachdb/pebble/internal/base"
import (
"context"
"runtime/pprof"
"sync"
"time"

"github.com/cockroachdb/errors/oserror"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/objstorage"
"github.com/cockroachdb/tokenbucket"
)

// Cleaner exports the base.Cleaner type.
type Cleaner = base.Cleaner
Expand All @@ -14,3 +25,230 @@ type DeleteCleaner = base.DeleteCleaner

// ArchiveCleaner exports the base.ArchiveCleaner type.
type ArchiveCleaner = base.ArchiveCleaner

type cleanupManager struct {
opts *Options
objProvider objstorage.Provider
onTableDeleteFn func(fileSize uint64)
deletePacer *deletionPacer

// jobsCh is used as the cleanup job queue.
jobsCh chan *cleanupJob
// waitGroup is used to wait for the background goroutine to exit.
waitGroup sync.WaitGroup

mu struct {
sync.Mutex
queuedJobs int
completedJobs int
completedJobsCond sync.Cond
}
}

// In practice, we should rarely have more than a couple of jobs (in most cases
// we Wait() after queueing a job).
const jobsChLen = 1000

// obsoleteFile holds information about a file that needs to be deleted soon.
type obsoleteFile struct {
dir string
fileNum base.DiskFileNum
fileType fileType
fileSize uint64
}

type cleanupJob struct {
jobID int
obsoleteFiles []obsoleteFile
}

// openCleanupManager creates a cleanupManager and starts its background goroutine.
// The cleanupManager must be Close()d.
func openCleanupManager(
opts *Options,
objProvider objstorage.Provider,
onTableDeleteFn func(fileSize uint64),
getDeletePacerInfo func() deletionPacerInfo,
) *cleanupManager {
cm := &cleanupManager{
opts: opts,
objProvider: objProvider,
onTableDeleteFn: onTableDeleteFn,
deletePacer: newDeletionPacer(getDeletePacerInfo),
jobsCh: make(chan *cleanupJob, jobsChLen),
}
cm.mu.completedJobsCond.L = &cm.mu.Mutex
cm.waitGroup.Add(1)

go func() {
pprof.Do(context.Background(), gcLabels, func(context.Context) {
cm.mainLoop()
})
}()

return cm
}

// Close stops the background goroutine, waiting until all queued jobs are completed.
// Delete pacing is disabled for the remaining jobs.
func (cm *cleanupManager) Close() {
close(cm.jobsCh)
cm.waitGroup.Wait()
}

// EnqueueJob adds a cleanup job to the manager's queue.
func (cm *cleanupManager) EnqueueJob(jobID int, obsoleteFiles []obsoleteFile) {
job := &cleanupJob{
jobID: jobID,
obsoleteFiles: obsoleteFiles,
}
cm.mu.Lock()
cm.mu.queuedJobs++
cm.mu.Unlock()

if invariants.Enabled && len(cm.jobsCh) >= cap(cm.jobsCh)-2 {
panic("cleanup jobs queue full")
}
cm.jobsCh <- job
}

// Wait until the completion of all jobs that were already queued.
//
// Does not wait for jobs that are enqueued during the call.
//
// Note that DB.mu should not be held while calling this method; the background
// goroutine needs to acquire DB.mu to update deleted table metrics.
func (cm *cleanupManager) Wait() {
cm.mu.Lock()
defer cm.mu.Unlock()
n := cm.mu.queuedJobs
for cm.mu.completedJobs < n {
cm.mu.completedJobsCond.Wait()
}
}

// mainLoop runs the manager's background goroutine.
func (cm *cleanupManager) mainLoop() {
defer cm.waitGroup.Done()
useLimiter := false
var limiter tokenbucket.TokenBucket

if r := cm.opts.TargetByteDeletionRate; r != 0 {
useLimiter = true
limiter.Init(tokenbucket.TokensPerSecond(r), tokenbucket.Tokens(r))
}

for job := range cm.jobsCh {
for _, of := range job.obsoleteFiles {
if of.fileType != fileTypeTable {
path := base.MakeFilepath(cm.opts.FS, of.dir, of.fileType, of.fileNum)
cm.deleteObsoleteFile(of.fileType, job.jobID, path, of.fileNum, of.fileSize)
} else {
if useLimiter {
cm.maybePace(&limiter, of.fileType, of.fileNum, of.fileSize)
}
cm.onTableDeleteFn(of.fileSize)
cm.deleteObsoleteObject(fileTypeTable, job.jobID, of.fileNum)
}
}
cm.mu.Lock()
cm.mu.completedJobs++
cm.mu.completedJobsCond.Broadcast()
cm.mu.Unlock()
}
}

// maybePace sleeps before deleting an object if appropriate. It is always
// called from the background goroutine.
func (cm *cleanupManager) maybePace(
limiter *tokenbucket.TokenBucket,
fileType base.FileType,
fileNum base.DiskFileNum,
fileSize uint64,
) {
meta, err := cm.objProvider.Lookup(fileType, fileNum)
if err != nil {
// The object was already removed from the provider; we won't actually
// delete anything, so we don't need to pace.
return
}
if meta.IsShared() {
// Don't throttle deletion of shared objects.
return
}
if !cm.deletePacer.shouldPace() {
// The deletion pacer decided that we shouldn't throttle; account
// for the operation but don't wait for tokens.
limiter.Adjust(-tokenbucket.Tokens(fileSize))
return
}
// Wait for tokens.
for {
ok, d := limiter.TryToFulfill(tokenbucket.Tokens(fileSize))
if ok {
break
}
time.Sleep(d)
}
}

// deleteObsoleteFile deletes a (non-object) file that is no longer needed.
func (cm *cleanupManager) deleteObsoleteFile(
fileType fileType, jobID int, path string, fileNum base.DiskFileNum, fileSize uint64,
) {
// TODO(peter): need to handle this error, probably by re-adding the
// file that couldn't be deleted to one of the obsolete slices map.
err := cm.opts.Cleaner.Clean(cm.opts.FS, fileType, path)
if oserror.IsNotExist(err) {
return
}

switch fileType {
case fileTypeLog:
cm.opts.EventListener.WALDeleted(WALDeleteInfo{
JobID: jobID,
Path: path,
FileNum: fileNum.FileNum(),
Err: err,
})
case fileTypeManifest:
cm.opts.EventListener.ManifestDeleted(ManifestDeleteInfo{
JobID: jobID,
Path: path,
FileNum: fileNum.FileNum(),
Err: err,
})
case fileTypeTable:
panic("invalid deletion of object file")
}
}

func (cm *cleanupManager) deleteObsoleteObject(
fileType fileType, jobID int, fileNum base.DiskFileNum,
) {
if fileType != fileTypeTable {
panic("not an object")
}

var path string
meta, err := cm.objProvider.Lookup(fileType, fileNum)
if err != nil {
path = "<nil>"
} else {
path = cm.objProvider.Path(meta)
err = cm.objProvider.Remove(fileType, fileNum)
}
if cm.objProvider.IsNotExistError(err) {
return
}

switch fileType {
case fileTypeTable:
cm.opts.EventListener.TableDeleted(TableDeleteInfo{
JobID: jobID,
Path: path,
FileNum: fileNum.FileNum(),
Err: err,
})
}
}
1 change: 1 addition & 0 deletions cleaner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ func TestCleaner(t *testing.T) {
if err := d.Compact(nil, []byte("\xff"), false); err != nil {
return err.Error()
}
d.TestOnlyWaitForCleaning()
return memLog.String()

case "flush":
Expand Down
Loading

0 comments on commit c0b26f2

Please sign in to comment.