Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
66893: cli,storage: add emergency ballast  r=jbowens a=jbowens

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 #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. Also, add a new Disk Full (10) exit
code that indicates that the node exited because disk space on at least
one store is exhausted. On node start, if any store has less than half
the ballast's size bytes available, the node immediately exits with the
Disk Full (10) exit code. The operator may manually remove the
configured ballast (assuming they haven't already) to allow the node to
start, and they can take action to remedy the disk space exhaustion. The
ballast will automatically be recreated when available disk space is 4x
the ballast size, or at least 10 GiB is available after the ballast is
created.

68645: keys/kvprober: introduce a range-local key for probing, use from kvprober r=tbg a=joshimhoff

This work sets the stage for extending `kvprober` to do writes as is discussed in detail with @tbg at #67112.

**keys: add a range-local key for probing**

This commit introduces a range-local key for probing. The key will
only be used by probing components like kvprober. This means no
contention with user-traffic or other CRDB components. This key also provides
a safe place to write to in order to test write availabilty. A kvprober that
does writes is coming soon.

Release note: None.

**kvprober: probe the range-local key dedicated to probing**

Before this commit, kvprober probed the start key of a range. This worked okay,
as kvprober only did reads, and contention issues leading to false positive
pages haven't happened in practice. But contention issues are possible,
as there may be data located at the start key of the range.

With this commit, kvprober probes the range-local key dedicated to
probing. No contention issues are possible, as that key is only for
probing. This key is also needed for write probes, which are coming soon.

Release note: None.

69164: Revert "backupccl: protect entire keyspan during cluster backup" r=dt a=adityamaru

This reverts commit 1b5fd4f.

The commit above laid a pts record over the entire table keyspace.
This did not account for two things (with the potential of there being
more):

1. System tables that we do not backup could have a short GC TTL, and
so incremental backups that attempt to protect from `StartTime` of the
previous backup would fail.

2. Dropped tables often have a short GC TTL to clear data once they have
been dropped. This change would also attempt to protect "dropped but not
gc'ed tables" even though we exclude them from the backup, and fail on
pts verification.

One suggested approach is to exclude all objects we do not backup by
subtracting these spans from {TableDataMin, TableDataMax}. This works
for system tables, and dropped but not gc'ed tables, but breaks for
dropped and gc'ed tables. A pts verification would still find the leaseholder
of the empty span and attempt to protect below the gc threshold.

In conclusion, we need to think about the semantics a little more before
we rush to protect a single key span.

Co-authored-by: Jackson Owens <[email protected]>
Co-authored-by: Josh Imhoff <[email protected]>
Co-authored-by: Aditya Maru <[email protected]>
  • Loading branch information
4 people committed Aug 19, 2021
4 parents 65457f9 + 54282ad + f41ac24 + 00dccf5 commit 24036ac
Show file tree
Hide file tree
Showing 21 changed files with 630 additions and 107 deletions.
6 changes: 6 additions & 0 deletions build/teamcity-support.sh
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,12 @@ function run_json_test() {
rm -f "${fullfile}"
fi
rm -f "${tmpfile}" artifacts/stripped.txt

# Some unit tests test automatic ballast creation. These ballasts can be
# larger than the maximum artifact size. Remove any artifacts with the
# EMERGENCY_BALLAST filename.
find artifacts -name "EMERGENCY_BALLAST" -delete

tc_end_block "artifacts"

# Make it easier to figure out whether we're exiting because of a test failure
Expand Down
3 changes: 3 additions & 0 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -510,6 +510,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
66 changes: 52 additions & 14 deletions pkg/base/store_spec.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/oserror"
"github.com/cockroachdb/pebble"
"github.com/cockroachdb/redact"
humanize "github.com/dustin/go-humanize"
"github.com/spf13/pflag"
)
Expand Down Expand Up @@ -79,7 +80,7 @@ type floatInterval struct {
// NewSizeSpec parses the string passed into a --size flag and returns a
// SizeSpec if it is correctly parsed.
func NewSizeSpec(
value string, bytesRange *intInterval, percentRange *floatInterval,
field redact.SafeString, value string, bytesRange *intInterval, percentRange *floatInterval,
) (SizeSpec, error) {
var size SizeSpec
if fractionRegex.MatchString(value) {
Expand All @@ -93,13 +94,14 @@ func NewSizeSpec(
size.Percent, err = strconv.ParseFloat(factorValue, 64)
size.Percent *= percentFactor
if err != nil {
return SizeSpec{}, fmt.Errorf("could not parse store size (%s) %s", value, err)
return SizeSpec{}, errors.Newf("could not parse %s size (%s) %s", field, value, err)
}
if percentRange != nil {
if (percentRange.min != nil && size.Percent < *percentRange.min) ||
(percentRange.max != nil && size.Percent > *percentRange.max) {
return SizeSpec{}, fmt.Errorf(
"store size (%s) must be between %f%% and %f%%",
return SizeSpec{}, errors.Newf(
"%s size (%s) must be between %f%% and %f%%",
field,
value,
*percentRange.min,
*percentRange.max,
Expand All @@ -110,16 +112,16 @@ func NewSizeSpec(
var err error
size.InBytes, err = humanizeutil.ParseBytes(value)
if err != nil {
return SizeSpec{}, fmt.Errorf("could not parse store size (%s) %s", value, err)
return SizeSpec{}, errors.Newf("could not parse %s size (%s) %s", field, value, err)
}
if bytesRange != nil {
if bytesRange.min != nil && size.InBytes < *bytesRange.min {
return SizeSpec{}, fmt.Errorf("store size (%s) must be larger than %s", value,
humanizeutil.IBytes(*bytesRange.min))
return SizeSpec{}, errors.Newf("%s size (%s) must be larger than %s",
field, value, humanizeutil.IBytes(*bytesRange.min))
}
if bytesRange.max != nil && size.InBytes > *bytesRange.max {
return SizeSpec{}, fmt.Errorf("store size (%s) must be smaller than %s", value,
humanizeutil.IBytes(*bytesRange.max))
return SizeSpec{}, errors.Newf("%s size (%s) must be smaller than %s",
field, value, humanizeutil.IBytes(*bytesRange.max))
}
}
}
Expand Down Expand Up @@ -150,7 +152,7 @@ var _ pflag.Value = &SizeSpec{}
// Set adds a new value to the StoreSpecValue. It is the important part of
// pflag's value interface.
func (ss *SizeSpec) Set(value string) error {
spec, err := NewSizeSpec(value, nil, nil)
spec, err := NewSizeSpec("specified", value, nil, nil)
if err != nil {
return err
}
Expand All @@ -162,10 +164,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 All @@ -190,6 +193,7 @@ type StoreSpec struct {

// String returns a fully parsable version of the store spec.
func (ss StoreSpec) String() string {
// TODO(jackson): Implement redact.SafeFormatter
var buffer bytes.Buffer
if len(ss.Path) != 0 {
fmt.Fprintf(&buffer, "path=%s,", ss.Path)
Expand All @@ -203,6 +207,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 @@ -308,13 +320,28 @@ func NewStoreSpec(value string) (StoreSpec, error) {
var minPercent float64 = 1
var maxPercent float64 = 100
ss.Size, err = NewSizeSpec(
"store",
value,
&intInterval{min: &minBytesAllowed},
&floatInterval{min: &minPercent, max: &maxPercent},
)
if err != nil {
return StoreSpec{}, err
}
case "ballast-size":
var minBytesAllowed int64
var minPercent float64 = 0
var maxPercent float64 = 50
ballastSize, err := NewSizeSpec(
"ballast",
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{})
Expand Down Expand Up @@ -384,6 +411,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 +447,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
7 changes: 7 additions & 0 deletions pkg/base/store_spec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,13 @@ target_file_size=2097152`
{"size=20GiB,path=/mnt/hda1,size=20GiB", "size field was used twice in store definition", StoreSpec{}},
{"size=123TB", "no path specified", StoreSpec{}},

// ballast size
{"path=/mnt/hda1,ballast-size=671088640", "", StoreSpec{Path: "/mnt/hda1", BallastSize: &SizeSpec{InBytes: 671088640}}},
{"path=/mnt/hda1,ballast-size=20GB", "", StoreSpec{Path: "/mnt/hda1", BallastSize: &SizeSpec{InBytes: 20000000000}}},
{"path=/mnt/hda1,ballast-size=1%", "", StoreSpec{Path: "/mnt/hda1", BallastSize: &SizeSpec{Percent: 1}}},
{"path=/mnt/hda1,ballast-size=100.000%", "ballast size (100.000%) must be between 0.000000% and 50.000000%", StoreSpec{}},
{"ballast-size=20GiB,path=/mnt/hda1,ballast-size=20GiB", "ballast-size field was used twice in store definition", StoreSpec{}},

// type
{"type=mem,size=20GiB", "", StoreSpec{Size: SizeSpec{InBytes: 21474836480}, InMemory: true}},
{"size=20GiB,type=mem", "", StoreSpec{Size: SizeSpec{InBytes: 21474836480}, InMemory: true}},
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -169,7 +169,6 @@ go_test(
"//pkg/kv/kvclient/kvcoord:with-mocks",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/kvserverbase",
"//pkg/kv/kvserver/protectedts/ptstorage",
"//pkg/roachpb:with-mocks",
"//pkg/scheduledjobs",
"//pkg/security",
Expand Down
17 changes: 2 additions & 15 deletions pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -1312,18 +1312,6 @@ func backupPlanHook(
CreatedBy: backupStmt.CreatedByInfo,
}

spansToProtect := spans
// If this is a full cluster backup from the system tenant then we write a
// single protected timestamp record spanning the entire keyspace.
if backupStmt.Coverage() == tree.AllDescriptors {
if p.ExecCfg().Codec.ForSystemTenant() {
spansToProtect = []roachpb.Span{{
Key: keys.TableDataMin,
EndKey: keys.TableDataMax,
}}
}
}

if backupStmt.Options.Detached {
// When running inside an explicit transaction, we simply create the job
// record. We do not wait for the job to finish.
Expand All @@ -1341,7 +1329,7 @@ func backupPlanHook(
// The protect timestamp logic for a DETACHED BACKUP can be run within the
// same txn as the BACKUP is being planned in, because we do not wait for
// the BACKUP job to complete.
err = protectTimestampForBackup(ctx, p, p.ExtendedEvalContext().Txn, jobID, spansToProtect,
err = protectTimestampForBackup(ctx, p, p.ExtendedEvalContext().Txn, jobID, spans,
startTime, endTime, backupDetails)
if err != nil {
return err
Expand Down Expand Up @@ -1375,8 +1363,7 @@ func backupPlanHook(
if err := doWriteBackupManifestCheckpoint(ctx, jobID); err != nil {
return err
}
if err := protectTimestampForBackup(ctx, p, plannerTxn, jobID, spansToProtect, startTime,
endTime, backupDetails); err != nil {
if err := protectTimestampForBackup(ctx, p, plannerTxn, jobID, spans, startTime, endTime, backupDetails); err != nil {
return err
}

Expand Down
55 changes: 0 additions & 55 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
"regexp"
"strconv"
"strings"
"sync"
"sync/atomic"
"testing"
"time"
Expand All @@ -50,7 +49,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptstorage"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -6182,59 +6180,6 @@ func getTableID(db *kv.DB, dbName, tableName string) descpb.ID {
return desc.GetID()
}

func TestProtectedTimestampSpanSelectionClusterBackup(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

const numAccounts = 1
serverArgs := base.TestServerArgs{Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}}
params := base.TestClusterArgs{ServerArgs: serverArgs}
allowRequest := make(chan struct{})
params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{
TestingRequestFilter: func(ctx context.Context, request roachpb.BatchRequest) *roachpb.Error {
for _, ru := range request.Requests {
switch ru.GetInner().(type) {
case *roachpb.ExportRequest:
<-allowRequest
}
}
return nil
},
}

_, tc, sqlDB, _, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, numAccounts,
InitManualReplication, params)
defer cleanupFn()

var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
sqlDB.Exec(t, `BACKUP TO 'nodelocal://self/foo'`)
}()

var jobID string
conn := tc.Conns[0]
testutils.SucceedsSoon(t, func() error {
row := conn.QueryRow("SELECT job_id FROM [SHOW JOBS] ORDER BY created DESC LIMIT 1")
return row.Scan(&jobID)
})

// Check protected timestamp record to ensure we are protecting the clusters
// key-span.
var spans []byte
sqlDB.QueryRow(t, `SELECT spans FROM system.protected_ts_records LIMIT 1`).Scan(&spans)
var protectedSpans ptstorage.Spans
require.NoError(t, protoutil.Unmarshal(spans, &protectedSpans))

expectedSpans := ptstorage.Spans{
Spans: []roachpb.Span{{Key: keys.TableDataMin, EndKey: keys.TableDataMax}},
}
require.Equal(t, expectedSpans, protectedSpans)
close(allowRequest)
wg.Wait()
}

// TestSpanSelectionDuringBackup tests the method spansForAllTableIndexes which
// is used to resolve the spans which will be backed up, and spans for which
// protected ts records will be created.
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
Loading

0 comments on commit 24036ac

Please sign in to comment.