Skip to content

Commit

Permalink
cli,storage: add emergency ballast
Browse files Browse the repository at this point in the history
Add an automatically created, on-by-default emergency ballast file. This
new ballast defaults to the minimum of 1% total disk capacity or 1GiB.
The size of the ballast may be configured via the `--store` flag with a
`ballast-size` field, accepting the same value formats as the `size`
field.

The ballast is automatically created when either available disk space is
at least four times the ballast size, or when available disk space after
creating the ballast is at least 10 GiB. Creation of the ballast happens
either when the engine is opened or during the periodic Capacity
calculations driven by the `kvserver.Store`.

During node start, if available disk space is less than or equal to half
the ballast size, exit immediately with a new Disk Full (10) exit code.

See cockroachdb#66493.

Release note (ops change): Add an automatically created, on by default
emergency ballast file. This new ballast defaults to the minimum of 1%
total disk capacity or 1GiB.  The size of the ballast may be configured
via the `--store` flag with a `ballast-size` field, accepting the same
value formats as the `size` field.
  • Loading branch information
jbowens committed Jun 28, 2021
1 parent a2ad6c7 commit 95a9a3a
Show file tree
Hide file tree
Showing 10 changed files with 507 additions and 13 deletions.
3 changes: 3 additions & 0 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -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).
Expand Down
40 changes: 36 additions & 4 deletions pkg/base/store_spec.go
Original file line number Diff line number Diff line change
Expand Up @@ -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]
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -315,6 +324,18 @@ func NewStoreSpec(value string) (StoreSpec, error) {
if err != nil {
return StoreSpec{}, err
}
case "ballast-size":
var minBytesAllowed int64
var maxPercent float64 = 50
ballastSize, err := NewSizeSpec(
value,
&intInterval{min: &minBytesAllowed},
&floatInterval{min: nil, 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{})
Expand Down Expand Up @@ -384,6 +405,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")
}
Expand Down Expand Up @@ -417,6 +441,14 @@ 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. The returned path must be stable across
// releases (eg, we cannot change these constants), otherwise we may duplicate
// ballasts.
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 {
Expand Down
4 changes: 4 additions & 0 deletions pkg/cli/exit/codes.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
18 changes: 18 additions & 0 deletions pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -331,6 +332,23 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr
}()
}

// Check for stores with full disks and exit with an informative exit
// code.
var err error
for _, spec := range serverCfg.Stores.Specs {
if isDiskFull, err2 := storage.IsDiskFull(vfs.Default, spec); err2 != nil {
return err2
} else if isDiskFull {
err = errors.CombineErrors(err, &cliError{
exitCode: exit.DiskFull(),
cause: errors.Errorf("store %s: out of disk space", spec.Path),
})
}
}
if err != nil {
return 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())
Expand Down
12 changes: 8 additions & 4 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -514,11 +514,14 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {
engines = append(engines, storage.NewInMem(ctx, spec.Attributes, cfg.CacheSize, sizeInBytes, cfg.Settings))
}
} else {
if err := vfs.Default.MkdirAll(spec.Path, 0755); err != nil {
return Engines{}, errors.Wrap(err, "creating store directory")
}
du, err := vfs.Default.GetDiskUsage(spec.Path)
if err != nil {
return Engines{}, errors.Wrap(err, "retrieving disk usage")
}
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 {
Expand All @@ -532,6 +535,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,
Expand Down
5 changes: 5 additions & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_library(
srcs = [
"array_32bit.go",
"array_64bit.go",
"ballast.go",
"batch.go",
"disk_map.go",
"doc.go",
Expand Down Expand Up @@ -73,6 +74,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/cli/exit",
"//pkg/clusterversion",
"//pkg/keys",
"//pkg/kv/kvserver/concurrency/lock",
Expand All @@ -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",
Expand All @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
145 changes: 145 additions & 0 deletions pkg/storage/ballast.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,145 @@
// 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) {
// The store directory might not exist yet. We don't want to try to create
// it yet, because there might not be any disk space to do so. Check the
// disk usage on the first parent that exists.
path := spec.Path
diskUsage, err := fs.GetDiskUsage(path)
for oserror.IsNotExist(err) {
if parentPath := fs.PathDir(path); parentPath == path {
break
} else {
path = parentPath
}
diskUsage, err = fs.GetDiskUsage(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
}
}
Loading

0 comments on commit 95a9a3a

Please sign in to comment.