From 364d86df6ef6f96e03de36ce701883f40d7b437b Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Thu, 20 Nov 2025 16:26:14 -0500 Subject: [PATCH] storage: avoid excessive walking of the auxiliary directory Pebble incrementally accounts its disk usage and this accounting is used to cheaply service Capacity calls. The auxiliary directory is a subdirectory of the data directory, and various Cockroach subsystems write to it directly without any incremental accounting of their disk usage. As a result, Capacity calculations walk the auxiliary directory to calculate the directory's disk usage anew each time. Previously, this calculation occurred during every call to Capacity. This commit adapts Pebble.Capacity to cache the computed capacity and recalculate it at most every minute. Ideally, we would incrementally account disk usage within the auxiliary directory (#96344). This change is a stopgap, avoiding the bulk of the CPU usage incurred by Capacity recalculations during IMPORTs. Epic: none Release note: none --- pkg/storage/BUILD.bazel | 1 + pkg/storage/pebble.go | 99 +++++++++++++++++++++++++++-------------- 2 files changed, 66 insertions(+), 34 deletions(-) diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 2375de0ff4aa..4ab0209d7850 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -91,6 +91,7 @@ go_library( "//pkg/util/timeutil", "//pkg/util/tracing", "//pkg/util/uuid", + "@com_github_cockroachdb_crlib//crtime", "@com_github_cockroachdb_crlib//fifo", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_errors//oserror", diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 2fe7e8f70811..1b5a74768bbb 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -43,6 +43,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/crlib/crtime" "github.com/cockroachdb/crlib/fifo" "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/oserror" @@ -803,10 +804,15 @@ type engineConfig struct { // Pebble is a wrapper around a Pebble database instance. type Pebble struct { - cfg engineConfig - db *pebble.DB - closed atomic.Bool - auxDir string + cfg engineConfig + db *pebble.DB + closed atomic.Bool + auxDir string + auxiliarySize struct { + mu syncutil.Mutex + computedAt crtime.Mono + size int64 + } ballastPath string properties roachpb.StoreProperties @@ -1919,37 +1925,11 @@ func (p *Pebble) Capacity() (roachpb.StoreCapacity, error) { m := p.db.Metrics() totalUsedBytes := int64(m.DiskSpaceUsage()) - // We don't have incremental accounting of the disk space usage of files - // in the auxiliary directory. Walk the auxiliary directory and all its - // subdirectories, adding to the total used bytes. - if errOuter := filepath.Walk(p.auxDir, func(path string, info os.FileInfo, err error) error { - if err != nil { - // This can happen if CockroachDB removes files out from under us - - // just keep going to get the best estimate we can. - if oserror.IsNotExist(err) { - return nil - } - // Special-case: if the store-dir is configured using the root of some fs, - // e.g. "/mnt/db", we might have special fs-created files like lost+found - // that we can't read, so just ignore them rather than crashing. - if oserror.IsPermission(err) && filepath.Base(path) == "lost+found" { - return nil - } - return err - } - if path == p.ballastPath { - // Skip the ballast. Counting it as used is likely to confuse - // users, and it's more akin to space that is just unavailable - // like disk space often restricted to a root user. - return nil - } - if info.Mode().IsRegular() { - totalUsedBytes += info.Size() - } - return nil - }); errOuter != nil { - return roachpb.StoreCapacity{}, errOuter + auxiliarySize, err := p.auxiliaryDirSize() + if err != nil { + return roachpb.StoreCapacity{}, err } + totalUsedBytes += auxiliarySize // If no size limitation have been placed on the store size or if the // limitation is greater than what's available, just return the actual @@ -1982,6 +1962,57 @@ func (p *Pebble) Capacity() (roachpb.StoreCapacity, error) { }, nil } +// auxiliaryDirSize computes the size of the auxiliary directory. There are +// multiple Cockroach subsystems that write into the auxiliary directory, and +// they don't incrementally account for their disk space usage. This function +// walks the auxiliary directory and all its subdirectories, summing the file +// sizes. This walk can be expensive, so we cache the result and only recompute +// if it's over 1 minute stale. +// +// TODO(jackson): Eventually we should update the various subsystems writing +// into the auxiliary directory to incrementally account for their disk space +// usage. See #96344. +func (p *Pebble) auxiliaryDirSize() (int64, error) { + p.auxiliarySize.mu.Lock() + defer p.auxiliarySize.mu.Unlock() + if crtime.NowMono().Sub(p.auxiliarySize.computedAt) < time.Minute { + return p.auxiliarySize.size, nil + } + + p.auxiliarySize.size = 0 + err := filepath.Walk(p.auxDir, func(path string, info os.FileInfo, err error) error { + if err != nil { + // This can happen if CockroachDB removes files out from under us - + // just keep going to get the best estimate we can. + if oserror.IsNotExist(err) { + return nil + } + // Special-case: if the store-dir is configured using the root of some fs, + // e.g. "/mnt/db", we might have special fs-created files like lost+found + // that we can't read, so just ignore them rather than erroring out. + if oserror.IsPermission(err) && filepath.Base(path) == "lost+found" { + return nil + } + return err + } + if path == p.ballastPath { + // Skip the ballast. Counting it as used is likely to confuse + // users, and it's more akin to space that is just unavailable + // like disk space often restricted to a root user. + return nil + } + if info.Mode().IsRegular() { + p.auxiliarySize.size += info.Size() + } + return nil + }) + if err != nil { + return 0, err + } + p.auxiliarySize.computedAt = crtime.NowMono() + return p.auxiliarySize.size, err +} + // Flush implements the Engine interface. func (p *Pebble) Flush() error { return p.db.Flush()