diff --git a/pkg/base/config.go b/pkg/base/config.go index dd7a759d0c1b..86ab34550621 100644 --- a/pkg/base/config.go +++ b/pkg/base/config.go @@ -522,6 +522,9 @@ type StorageConfig struct { // MaxSize is used for calculating free space and making rebalancing // decisions. Zero indicates that there is no maximum size. MaxSize int64 + // BallastSize is the amount reserved by a ballast file for manual + // out-of-disk recovery. + BallastSize int64 // Settings instance for cluster-wide knobs. Settings *cluster.Settings // UseFileRegistry is true if the file registry is needed (eg: encryption-at-rest). diff --git a/pkg/base/store_spec.go b/pkg/base/store_spec.go index 052026ded334..527cd1167d08 100644 --- a/pkg/base/store_spec.go +++ b/pkg/base/store_spec.go @@ -162,10 +162,11 @@ func (ss *SizeSpec) Set(value string) error { // StoreSpec contains the details that can be specified in the cli pertaining // to the --store flag. type StoreSpec struct { - Path string - Size SizeSpec - InMemory bool - Attributes roachpb.Attributes + Path string + Size SizeSpec + BallastSize *SizeSpec + InMemory bool + Attributes roachpb.Attributes // StickyInMemoryEngineID is a unique identifier associated with a given // store which will remain in memory even after the default Engine close // until it has been explicitly cleaned up by CleanupStickyInMemEngine[s] @@ -203,6 +204,14 @@ func (ss StoreSpec) String() string { if ss.Size.Percent > 0 { fmt.Fprintf(&buffer, "size=%s%%,", humanize.Ftoa(ss.Size.Percent)) } + if ss.BallastSize != nil { + if ss.BallastSize.InBytes > 0 { + fmt.Fprintf(&buffer, "ballast-size=%s,", humanizeutil.IBytes(ss.BallastSize.InBytes)) + } + if ss.BallastSize.Percent > 0 { + fmt.Fprintf(&buffer, "ballast-size=%s%%,", humanize.Ftoa(ss.BallastSize.Percent)) + } + } if len(ss.Attributes.Attrs) > 0 { fmt.Fprint(&buffer, "attrs=") for i, attr := range ss.Attributes.Attrs { @@ -315,6 +324,19 @@ func NewStoreSpec(value string) (StoreSpec, error) { if err != nil { return StoreSpec{}, err } + case "ballast-size": + var minBytesAllowed int64 + var minPercent float64 = 1 + var maxPercent float64 = 50 + ballastSize, err := NewSizeSpec( + value, + &intInterval{min: &minBytesAllowed}, + &floatInterval{min: &minPercent, max: &maxPercent}, + ) + if err != nil { + return StoreSpec{}, err + } + ss.BallastSize = &ballastSize case "attrs": // Check to make sure there are no duplicate attributes. attrMap := make(map[string]struct{}) @@ -384,6 +406,9 @@ func NewStoreSpec(value string) (StoreSpec, error) { if ss.Size.Percent == 0 && ss.Size.InBytes == 0 { return StoreSpec{}, fmt.Errorf("size must be specified for an in memory store") } + if ss.BallastSize != nil { + return StoreSpec{}, fmt.Errorf("ballast-size specified for in memory store") + } } else if ss.Path == "" { return StoreSpec{}, fmt.Errorf("no path specified") } @@ -417,6 +442,12 @@ func (ssl StoreSpecList) String() string { // root directory. It must not be changed without a proper migration. const AuxiliaryDir = "auxiliary" +// EmergencyBallastFile returns the path (relative to a data directory) used +// for an emergency ballast file. +func EmergencyBallastFile(pathJoin func(...string) string, dataDir string) string { + return pathJoin(dataDir, AuxiliaryDir, "EMERGENCY_BALLAST") +} + // PreventedStartupFile is the filename (relative to 'dir') used for files that // can block server startup. func PreventedStartupFile(dir string) string { diff --git a/pkg/cli/exit/codes.go b/pkg/cli/exit/codes.go index ac41c99f8626..bc96e85e46b6 100644 --- a/pkg/cli/exit/codes.go +++ b/pkg/cli/exit/codes.go @@ -58,6 +58,10 @@ func TimeoutAfterFatalError() Code { return Code{8} } // during a logging operation to a network collector. func LoggingNetCollectorUnavailable() Code { return Code{9} } +// DiskFull (10) indicates an emergency shutdown in response to a +// store's full disk. +func DiskFull() Code { return Code{10} } + // Codes that are specific to client commands follow. It's possible // for codes to be reused across separate client or server commands. // Command-specific exit codes should be allocated down from 125. diff --git a/pkg/cli/start.go b/pkg/cli/start.go index ac09db92d849..abb6ddbb0cb0 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -54,6 +54,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" + "github.com/cockroachdb/pebble/vfs" "github.com/cockroachdb/redact" "github.com/spf13/cobra" "google.golang.org/grpc" @@ -331,6 +332,17 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr }() } + // Check for stores with full disks and exit with an informative exit + // code. + for _, spec := range serverCfg.Stores.Specs { + if isDiskFull, err := storage.IsDiskFull(vfs.Default, spec); err != nil { + return err + } else if isDiskFull { + err := errors.Errorf("store %s: out of disk space", spec.Path) + return &cliError{exitCode: exit.DiskFull(), cause: err} + } + } + // Set up a cancellable context for the entire start command. // The context will be canceled at the end. ctx, cancel := context.WithCancel(context.Background()) diff --git a/pkg/server/config.go b/pkg/server/config.go index de59099fb958..5c8a4a222901 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -514,11 +514,11 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { engines = append(engines, storage.NewInMem(ctx, spec.Attributes, cfg.CacheSize, sizeInBytes, cfg.Settings)) } } else { + du, err := vfs.Default.GetDiskUsage(spec.Path) + if err != nil { + return Engines{}, err + } if spec.Size.Percent > 0 { - du, err := vfs.Default.GetDiskUsage(spec.Path) - if err != nil { - return Engines{}, err - } sizeInBytes = int64(float64(du.TotalBytes) * spec.Size.Percent / 100) } if sizeInBytes != 0 && !skipSizeCheck && sizeInBytes < base.MinimumStoreSize { @@ -532,6 +532,7 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { storageConfig := base.StorageConfig{ Attrs: spec.Attributes, Dir: spec.Path, + BallastSize: storage.BallastSizeBytes(spec, du), MaxSize: sizeInBytes, Settings: cfg.Settings, UseFileRegistry: spec.UseFileRegistry, diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 08123d6c0cb7..d5bbe9710bb3 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "array_32bit.go", "array_64bit.go", + "ballast.go", "batch.go", "disk_map.go", "doc.go", @@ -73,6 +74,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/cli/exit", "//pkg/clusterversion", "//pkg/keys", "//pkg/kv/kvserver/concurrency/lock", @@ -94,6 +96,7 @@ go_library( "//pkg/util/protoutil", "//pkg/util/stop", "//pkg/util/syncutil", + "//pkg/util/sysutil", "//pkg/util/timeutil", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", @@ -113,6 +116,7 @@ go_test( name = "storage_test", size = "medium", srcs = [ + "ballast_test.go", "batch_test.go", "bench_pebble_test.go", "bench_test.go", @@ -164,6 +168,7 @@ go_test( "//pkg/util/randutil", "//pkg/util/shuffle", "//pkg/util/stop", + "//pkg/util/sysutil", "//pkg/util/timeutil", "//pkg/util/uint128", "//pkg/util/uuid", diff --git a/pkg/storage/ballast.go b/pkg/storage/ballast.go new file mode 100644 index 000000000000..c657b9198dfd --- /dev/null +++ b/pkg/storage/ballast.go @@ -0,0 +1,133 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +import ( + "os" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/util/sysutil" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/errors/oserror" + "github.com/cockroachdb/pebble/vfs" +) + +// IsDiskFull examines the store indicated by spec, determining whether the +// store's underlying disk is out of disk space. A disk is considered to be +// full if available capacity is less than half of the store's ballast size. +// +// If the current on-disk ballast is larger than the ballast size configured +// through spec, IsDiskFull will truncate the ballast to the configured size. +func IsDiskFull(fs vfs.FS, spec base.StoreSpec) (bool, error) { + diskUsage, err := fs.GetDiskUsage(spec.Path) + if err != nil { + return false, errors.Wrapf(err, "retrieving disk usage: %s", spec.Path) + } + desiredSizeBytes := BallastSizeBytes(spec, diskUsage) + + ballastPath := base.EmergencyBallastFile(fs.PathJoin, spec.Path) + var currentSizeBytes int64 + if fi, err := fs.Stat(ballastPath); err != nil && !oserror.IsNotExist(err) { + return false, err + } else if err == nil { + currentSizeBytes = fi.Size() + } + + // If the ballast is larger than desired, truncate it now in case the + // freed disk space will allow us to start. Generally, re-sizing the + // ballast is the responsibility of the Engine. + if currentSizeBytes > desiredSizeBytes { + if err := fs.MkdirAll(fs.PathDir(ballastPath), 0755); err != nil { + return false, err + } + // TODO(jackson): Expose Truncate on vfs.FS. + if err := os.Truncate(ballastPath, desiredSizeBytes); err != nil { + return false, errors.Wrap(err, "truncating ballast") + } + diskUsage, err = fs.GetDiskUsage(spec.Path) + if err != nil { + return false, errors.Wrapf(err, "retrieving disk usage: %s", spec.Path) + } + } + + // If the filesystem reports less than half the disk space available, + // consider the disk full. If the ballast hasn't been removed yet, + // removing it will free enough disk space to start. We don't use exactly + // the ballast size in case some of the headroom gets consumed elsewhere: + // eg, the operator's shell history, system logs, copy-on-write filesystem + // metadata, etc. + return diskUsage.AvailBytes < uint64(desiredSizeBytes/2), nil +} + +// BallastSizeBytes returns the desired size of the emergency ballast, +// calculated from the provided store spec and disk usage. If the store spec +// contains an explicit ballast size (either in bytes or as a perecentage of +// the disk's total capacity), the store spec's size is used. Otherwise, +// BallastSizeBytes returns 1GiB or 1% of total capacity, whichever is +// smaller. +func BallastSizeBytes(spec base.StoreSpec, diskUsage vfs.DiskUsage) int64 { + if spec.BallastSize != nil { + v := spec.BallastSize.InBytes + if spec.BallastSize.Percent != 0 { + v = int64(float64(diskUsage.TotalBytes) * spec.BallastSize.Percent / 100) + } + return v + } + + // Default to a 1% or 1GiB ballast, whichever is smaller. + var v int64 = 1 << 30 // 1 GiB + if p := int64(float64(diskUsage.TotalBytes) * 0.01); p < v { + v = p + } + return v +} + +func maybeEstablishBallast( + fs vfs.FS, ballastPath string, ballastSizeBytes int64, diskUsage vfs.DiskUsage, +) (resized bool, err error) { + var currentSizeBytes int64 + fi, err := fs.Stat(ballastPath) + if err != nil && !oserror.IsNotExist(err) { + return false, err + } else if err == nil { + currentSizeBytes = fi.Size() + } + + switch { + case currentSizeBytes > ballastSizeBytes: + // If the current ballast is too big, shrink it regardless of current + // disk space availability. + return true, sysutil.ResizeLargeFile(ballastPath, ballastSizeBytes) + case currentSizeBytes < ballastSizeBytes: + // We need to either create the ballast or extend the current ballast + // to make it larger. The ballast may have been intentionally removed + // to enable recovery. Only create/extend the ballast if there's + // sufficient disk space. + extendBytes := ballastSizeBytes - currentSizeBytes + + // If available disk space is >= 4x the required amount, create the + // ballast. + if extendBytes <= int64(diskUsage.AvailBytes)/4 { + return true, sysutil.ResizeLargeFile(ballastPath, ballastSizeBytes) + } + + // If the user configured a really large ballast, we might not ever + // have >= 4x the required amount available. Also allow extending the + // ballast if we will have 10 GiB available after the extension. + if int64(diskUsage.AvailBytes)-extendBytes > (10 << 30 /* 10 GiB */) { + return true, sysutil.ResizeLargeFile(ballastPath, ballastSizeBytes) + } + + return false, nil + default: + return false, nil + } +} diff --git a/pkg/storage/ballast_test.go b/pkg/storage/ballast_test.go new file mode 100644 index 000000000000..ffb2ffedf9a4 --- /dev/null +++ b/pkg/storage/ballast_test.go @@ -0,0 +1,225 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. +package storage + +import ( + "fmt" + "path/filepath" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/sysutil" + "github.com/cockroachdb/errors/oserror" + "github.com/cockroachdb/pebble/vfs" + "github.com/stretchr/testify/require" +) + +func TestBallastSizeBytes(t *testing.T) { + testCases := []struct { + base.StoreSpec + totalBytes uint64 + want int64 + }{ + { + StoreSpec: base.StoreSpec{}, + totalBytes: 500 << 30, // 500 GiB + want: 1 << 30, // 1 GiB + }, + { + StoreSpec: base.StoreSpec{}, + totalBytes: 25 << 30, // 25 GiB + want: 256 << 20, // 256 MiB + }, + { + StoreSpec: base.StoreSpec{BallastSize: &base.SizeSpec{InBytes: 1 << 30 /* 1 GiB */}}, + totalBytes: 25 << 30, // 25 GiB + want: 1 << 30, // 1 GiB + }, + { + StoreSpec: base.StoreSpec{BallastSize: &base.SizeSpec{Percent: 20}}, + totalBytes: 25 << 30, // 25 GiB + want: 5 << 30, // 5 GiB + }, + { + StoreSpec: base.StoreSpec{BallastSize: &base.SizeSpec{Percent: 20}}, + totalBytes: 500 << 30, // 500 GiB + want: 100 << 30, // 100 GiB + }, + } + + for _, tc := range testCases { + du := vfs.DiskUsage{TotalBytes: tc.totalBytes} + got := BallastSizeBytes(tc.StoreSpec, du) + require.Equal(t, tc.want, got) + } + +} + +func TestIsDiskFull(t *testing.T) { + // TODO(jackson): This test could be adapted to use a MemFS if we add + // Truncate to vfs.FS. + + setup := func(t *testing.T, spec *base.StoreSpec, ballastSize int64, du ...vfs.DiskUsage) (vfs.FS, func()) { + dir, dirCleanupFn := testutils.TempDir(t) + fs := mockDiskUsageFS{ + FS: vfs.Default, + diskUsages: du, + } + spec.Path = dir + + if ballastSize > 0 { + path := base.EmergencyBallastFile(fs.PathJoin, spec.Path) + require.NoError(t, fs.MkdirAll(fs.PathDir(path), 0755)) + err := sysutil.ResizeLargeFile(path, ballastSize) + fmt.Printf("Created ballast at %s\n", path) + require.NoError(t, err) + } + return &fs, dirCleanupFn + } + + t.Run("default ballast, full disk", func(t *testing.T) { + spec := base.StoreSpec{ + // NB: A missing ballast size defaults to Min(1GiB, 0.5% of + // total) = 1GiB > available bytes. + BallastSize: nil, + } + fs, cleanup := setup(t, &spec, 0 /* ballastSize */, vfs.DiskUsage{ + AvailBytes: (1 << 28), // 256 MiB + TotalBytes: 500 << 30, // 500 GiB + }) + defer cleanup() + got, err := IsDiskFull(fs, spec) + require.NoError(t, err) + require.True(t, got) + }) + t.Run("default ballast, plenty of space", func(t *testing.T) { + spec := base.StoreSpec{ + // NB: A missing ballast size defaults to Min(1GiB, 0.5% of + // total) = 1GiB > available bytes. + BallastSize: nil, + } + fs, cleanup := setup(t, &spec, 0 /* ballastSize */, vfs.DiskUsage{ + AvailBytes: 25 << 30, // 25 GiB + TotalBytes: 500 << 30, // 500 GiB + }) + defer cleanup() + got, err := IsDiskFull(fs, spec) + require.NoError(t, err) + require.False(t, got) + }) + t.Run("truncating ballast frees enough space", func(t *testing.T) { + spec := base.StoreSpec{ + BallastSize: &base.SizeSpec{InBytes: 1024}, + } + // Provide two disk usages. The second one will be returned + // post-truncation. + fs, cleanup := setup(t, &spec, 2048, /* ballastSize */ + vfs.DiskUsage{AvailBytes: 256, TotalBytes: 500 << 30 /* 500 GiB */}, + vfs.DiskUsage{AvailBytes: 1280, TotalBytes: 500 << 30 /* 500 GiB */}) + defer cleanup() + + got, err := IsDiskFull(fs, spec) + require.NoError(t, err) + require.False(t, got) + // The ballast should've been truncated. + fi, err := fs.Stat(base.EmergencyBallastFile(fs.PathJoin, spec.Path)) + require.NoError(t, err) + require.Equal(t, int64(1024), fi.Size()) + }) + t.Run("configured ballast, plenty of space", func(t *testing.T) { + spec := base.StoreSpec{ + BallastSize: &base.SizeSpec{InBytes: 5 << 30 /* 5 GiB */}, + } + fs, cleanup := setup(t, &spec, 0 /* ballastSize */, vfs.DiskUsage{ + AvailBytes: 25 << 30, // 25 GiB + TotalBytes: 500 << 30, // 500 GiB + }) + defer cleanup() + got, err := IsDiskFull(fs, spec) + require.NoError(t, err) + require.False(t, got) + }) +} + +type mockDiskUsageFS struct { + vfs.FS + diskUsagesIdx int + diskUsages []vfs.DiskUsage +} + +func (fs *mockDiskUsageFS) GetDiskUsage(string) (vfs.DiskUsage, error) { + ret := fs.diskUsages[fs.diskUsagesIdx] + if fs.diskUsagesIdx+1 < len(fs.diskUsages) { + fs.diskUsagesIdx++ + } + return ret, nil +} + +func TestMaybeEstablishBallast(t *testing.T) { + setup := func(t *testing.T, ballastSize int64) (string, func()) { + dir, dirCleanupFn := testutils.TempDir(t) + path := filepath.Join(dir, "ballast") + if ballastSize > 0 { + err := sysutil.ResizeLargeFile(path, ballastSize) + require.NoError(t, err) + } + return path, dirCleanupFn + } + getSize := func(t *testing.T, path string) int { + fi, err := vfs.Default.Stat(path) + if oserror.IsNotExist(err) { + return 0 + } + require.NoError(t, err) + return int(fi.Size()) + } + + t.Run("insufficient disk space, no ballast", func(t *testing.T) { + path, cleanup := setup(t, 0) + defer cleanup() + resized, err := maybeEstablishBallast(vfs.Default, path, 1<<30 /* 1 GiB */, vfs.DiskUsage{ + AvailBytes: 3 << 30, /* 3 GiB */ + }) + require.NoError(t, err) + require.False(t, resized) + require.Equal(t, 0, getSize(t, path)) + }) + t.Run("sufficient disk space, no ballast", func(t *testing.T) { + path, cleanup := setup(t, 0) + defer cleanup() + resized, err := maybeEstablishBallast(vfs.Default, path, 1024, vfs.DiskUsage{ + AvailBytes: 500 << 20, /* 500 MiB */ + }) + require.NoError(t, err) + require.True(t, resized) + require.Equal(t, 1024, getSize(t, path)) + }) + t.Run("truncates ballast if necessary", func(t *testing.T) { + path, cleanup := setup(t, 2048) + defer cleanup() + resized, err := maybeEstablishBallast(vfs.Default, path, 1024, vfs.DiskUsage{ + AvailBytes: 500 << 20, /* 500 MiB */ + }) + require.NoError(t, err) + require.True(t, resized) + require.Equal(t, 1024, getSize(t, path)) + }) + t.Run("does nothing if ballast is correct size", func(t *testing.T) { + path, cleanup := setup(t, 4096) + defer cleanup() + resized, err := maybeEstablishBallast(vfs.Default, path, 4096, vfs.DiskUsage{ + AvailBytes: 500 << 20, /* 500 MiB */ + }) + require.NoError(t, err) + require.False(t, resized) + require.Equal(t, 4096, getSize(t, path)) + }) +} diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 9c0d307c038b..6a5ffebee04c 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -27,6 +27,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/cli/exit" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" @@ -389,6 +390,10 @@ func DefaultPebbleOptions() *pebble.Options { if diskHealthCheckInterval.Seconds() > maxSyncDurationDefault.Seconds() { diskHealthCheckInterval = maxSyncDurationDefault } + // If we encounter ENOSPC, exit with an informative exit code. + opts.FS = vfs.OnDiskFull(opts.FS, func() { + exit.WithCode(exit.DiskFull()) + }) // Instantiate a file system with disk health checking enabled. This FS wraps // vfs.Default, and can be wrapped for encryption-at-rest. opts.FS = vfs.WithDiskHealthChecks(vfs.Default, diskHealthCheckInterval, @@ -443,11 +448,13 @@ type EncryptionStatsHandler interface { type Pebble struct { db *pebble.DB - closed bool - path string - auxDir string - maxSize int64 - attrs roachpb.Attributes + closed bool + path string + auxDir string + ballastPath string + ballastSize int64 + maxSize int64 + attrs roachpb.Attributes // settings must be non-nil if this Pebble instance will be used to write // intents. settings *cluster.Settings @@ -553,6 +560,7 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { if err := cfg.Opts.FS.MkdirAll(auxDir, 0755); err != nil { return nil, err } + ballastPath := base.EmergencyBallastFile(cfg.Opts.FS.PathJoin, cfg.Dir) fileRegistry, statsHandler, err := ResolveEncryptedEnvOptions(&cfg) if err != nil { @@ -576,9 +584,32 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { ctx: logCtx, depth: 2, // skip over the EventListener stack frame }) + + // Establish the emergency ballast if we can. If there's not sufficient + // disk space, the ballast will be reestablished from Capacity when the + // store's capacity is queried periodically. Only try if the store is not + // an in-memory store, because vfs.MemFS will error if you retrieve disk + // usage. + if cfg.Dir != "" /* if not in-memory */ { + du, err := cfg.Opts.FS.GetDiskUsage(cfg.Dir) + if err != nil { + return nil, err + } + resized, err := maybeEstablishBallast(cfg.Opts.FS, ballastPath, cfg.BallastSize, du) + if err != nil { + return nil, errors.Wrap(err, "resizing ballast") + } + if resized { + cfg.Opts.Logger.Infof("resized ballast %s to size %s", + ballastPath, humanizeutil.IBytes(cfg.BallastSize)) + } + } + p := &Pebble{ path: cfg.Dir, auxDir: auxDir, + ballastPath: ballastPath, + ballastSize: cfg.BallastSize, maxSize: cfg.MaxSize, attrs: cfg.Attrs, settings: cfg.Settings, @@ -995,6 +1026,22 @@ func (p *Pebble) Capacity() (roachpb.StoreCapacity, error) { fsuTotal := int64(du.TotalBytes) fsuAvail := int64(du.AvailBytes) + // If the emergency ballast isn't appropriately sized, try to resize it. + // This is a no-op if the ballast is already sized or if there's not + // enough available capacity to resize it. + resized, err := maybeEstablishBallast(p.fs, p.ballastPath, p.ballastSize, du) + if err != nil { + return roachpb.StoreCapacity{}, errors.Wrap(err, "resizing ballast") + } + if resized { + p.logger.Infof("resized ballast %s to size %s", + p.ballastPath, humanizeutil.IBytes(p.ballastSize)) + du, err = p.fs.GetDiskUsage(dir) + if err != nil { + return roachpb.StoreCapacity{}, err + } + } + // Pebble has detailed accounting of its own disk space usage, and it's // incrementally updated which helps avoid O(# files) work here. m := p.db.Metrics() @@ -1018,6 +1065,12 @@ func (p *Pebble) Capacity() (roachpb.StoreCapacity, error) { } 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() } diff --git a/pkg/storage/temp_engine.go b/pkg/storage/temp_engine.go index e69d3e268c1b..519d580ee817 100644 --- a/pkg/storage/temp_engine.go +++ b/pkg/storage/temp_engine.go @@ -38,6 +38,7 @@ func storageConfigFromTempStorageConfigAndStoreSpec( return base.StorageConfig{ Attrs: roachpb.Attributes{}, Dir: config.Path, + BallastSize: 0, // redundant with primary store ballast MaxSize: 0, // doesn't matter for temp storage - it's not enforced in any way. Settings: config.Settings, UseFileRegistry: spec.UseFileRegistry,