Skip to content

Commit 43b243f

Browse files
craig[bot]jbowens
andcommitted
Merge #156476
156476: cli: delete temporary directories before determining ENOSPC r=RaduBerinde a=jbowens Before determining whether there's sufficient disk space available to start the node, remove the temporary directories. If SQL-spilling created a substantial volume of data on the filesystem, it's possible cleaning up this ephemeral data will allow the node to start. Release note: none Fixes: #85147. Epic: none Co-authored-by: Jackson Owens <[email protected]>
2 parents a4e8d27 + a8cb283 commit 43b243f

File tree

8 files changed

+176
-37
lines changed

8 files changed

+176
-37
lines changed

pkg/cli/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -457,6 +457,7 @@ go_test(
457457
"//pkg/ts/tspb",
458458
"//pkg/util",
459459
"//pkg/util/envutil",
460+
"//pkg/util/humanizeutil",
460461
"//pkg/util/ioctx",
461462
"//pkg/util/leaktest",
462463
"//pkg/util/log",
@@ -470,6 +471,7 @@ go_test(
470471
"//pkg/util/tracing",
471472
"//pkg/util/uuid",
472473
"//pkg/workload/examples",
474+
"@com_github_cockroachdb_crlib//crstrings",
473475
"@com_github_cockroachdb_datadriven//:datadriven",
474476
"@com_github_cockroachdb_errors//:errors",
475477
"@com_github_cockroachdb_errors//oserror",

pkg/cli/interactive_tests/test_temp_dir.tcl

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -66,7 +66,7 @@ interrupt_and_wait
6666
# Verify the temp directory is removed.
6767
glob_not_exists "$tempdir/$tempprefix*"
6868
# Verify temp directory path is removed from record file.
69-
if {[file size "$storedir/$recordfile"] > 0} {
69+
if {[file exists "$storedir/$recordfile"] && [file size "$storedir/$recordfile"] > 0} {
7070
report "RECORD FILE NOT EMPTY"
7171
exit 1
7272
}
@@ -120,7 +120,7 @@ send "$argv start-single-node --insecure --store=$storedir --background\r"
120120
eexpect ":/# "
121121
# Try to start up a second cockroach instance with the same store path.
122122
send "$argv start-single-node --insecure --store=$storedir\r"
123-
eexpect "ERROR: could not cleanup temporary directories from record file: could not lock temporary directory $cwd/$storedir/$tempprefix*"
123+
eexpect "ERROR: could not lock temporary directory $cwd/$storedir/$tempprefix*"
124124
# Verify the temp directory still exists.
125125
glob_exists "$storedir/$tempprefix*"
126126
send "pkill -9 cockroach\r"

pkg/cli/start.go

Lines changed: 34 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -193,9 +193,9 @@ func initTempStorageConfig(
193193
// target, if any. If we can't find one, we use the first StoreSpec in the
194194
// list.
195195
//
196-
// While we look, we also clean up any abandoned temporary directories. We
197-
// don't know which store spec was used previously—and it may change if
198-
// encryption gets enabled after the fact—so we check each store.
196+
// Note that we already cleaned up any abandoned temporary directories from
197+
// the previous process earlier in the startup sequence (see
198+
// reclaimDiskSpace).
199199
specIdxDisk := -1
200200
specIdxEncrypted := -1
201201
for i, spec := range stores.Specs {
@@ -211,11 +211,6 @@ func initTempStorageConfig(
211211
if specIdxDisk == -1 {
212212
specIdxDisk = i
213213
}
214-
recordPath := filepath.Join(spec.Path, server.TempDirsRecordFilename)
215-
if err := fs.CleanupTempDirs(recordPath); err != nil {
216-
return base.TempStorageConfig{}, errors.Wrap(err,
217-
"could not cleanup temporary directories from record file")
218-
}
219214
}
220215

221216
// Use first store by default. This might be an in-memory store.
@@ -311,7 +306,7 @@ func initTempStorageConfig(
311306
// Remove temporary directory on shutdown.
312307
stopper.AddCloser(stop.CloserFn(func() {
313308
unlockDirFn()
314-
if err := fs.CleanupTempDirs(recordPath); err != nil {
309+
if err := fs.CleanupTempDirs(ctx, vfs.Default, recordPath); err != nil {
315310
log.Dev.Errorf(ctx, "could not remove temporary store directory: %v", err.Error())
316311
}
317312
}))
@@ -435,13 +430,18 @@ func runStartInternal(
435430
signal.Notify(signalCh, exitAbruptlySignal)
436431
}
437432

433+
// Set up a cancellable context for the entire start command.
434+
// The context will be canceled at the end.
435+
ctx, cancel := context.WithCancel(context.Background())
436+
defer cancel()
437+
438438
// Check for stores with full disks and exit with an informative exit
439439
// code. This needs to happen early during start, before we perform any
440440
// writes to the filesystem including log rotation. We need to guarantee
441441
// that the process continues to exit with the Disk Full exit code. A
442442
// flapping exit code can affect alerting, including the alerting
443443
// performed within CockroachCloud.
444-
if err := exitIfDiskFull(vfs.Default, serverCfg.Stores.Specs); err != nil {
444+
if err := exitIfDiskFull(ctx, vfs.Default, serverCfg.Stores.Specs); err != nil {
445445
return err
446446
}
447447

@@ -456,11 +456,6 @@ func runStartInternal(
456456
// making progress.
457457
log.SetMakeProcessUnavailableFunc(closeAllSockets)
458458

459-
// Set up a cancellable context for the entire start command.
460-
// The context will be canceled at the end.
461-
ctx, cancel := context.WithCancel(context.Background())
462-
defer cancel()
463-
464459
// The context annotation ensures that server identifiers show up
465460
// in the logging metadata as soon as they are known.
466461
ambientCtx := serverCfg.AmbientCtx
@@ -1398,7 +1393,14 @@ func maybeWarnMemorySizes(ctx context.Context) {
13981393
}
13991394
}
14001395

1401-
func exitIfDiskFull(fs vfs.FS, specs []base.StoreSpec) error {
1396+
func exitIfDiskFull(ctx context.Context, fs vfs.FS, specs []base.StoreSpec) error {
1397+
// First try to reclaim disk space by cleaning up obsolete files. It's
1398+
// possible this will free up enough space to allow us to start when we
1399+
// otherwise would not be able to.
1400+
if err := reclaimDiskSpace(ctx, fs, specs); err != nil {
1401+
return err
1402+
}
1403+
14021404
var cause error
14031405
var ballastPaths []string
14041406
var ballastMissing bool
@@ -1438,6 +1440,22 @@ disk space exhaustion may result in node loss.`, ballastPathsStr)
14381440
return err
14391441
}
14401442

1443+
// reclaimDiskSpace attempts to reclaim disk space by cleaning up obsolete files
1444+
// leftover from the previous process. For example, SQL temporary directories
1445+
// become obsolete when a process exits and can be reclaimed.
1446+
func reclaimDiskSpace(ctx context.Context, rootFS vfs.FS, specs []base.StoreSpec) error {
1447+
var err error
1448+
// Reclaim any temporary directories.
1449+
for _, spec := range specs {
1450+
if spec.InMemory {
1451+
continue
1452+
}
1453+
recordPath := filepath.Join(spec.Path, server.TempDirsRecordFilename)
1454+
err = errors.CombineErrors(err, fs.CleanupTempDirs(ctx, rootFS, recordPath))
1455+
}
1456+
return err
1457+
}
1458+
14411459
// setupAndInitializeLoggingAndProfiling does what it says on the label.
14421460
// Prior to this however it determines suitable defaults for the
14431461
// logging output directory and the verbosity level of stderr logging.

pkg/cli/start_test.go

Lines changed: 85 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -6,19 +6,25 @@
66
package cli
77

88
import (
9+
"bytes"
910
"errors"
11+
"fmt"
1012
"os"
1113
"path/filepath"
14+
"strings"
1215
"testing"
1316

1417
"github.com/cockroachdb/cockroach/pkg/base"
1518
"github.com/cockroachdb/cockroach/pkg/cli/clierror"
1619
"github.com/cockroachdb/cockroach/pkg/cli/exit"
1720
"github.com/cockroachdb/cockroach/pkg/server"
1821
"github.com/cockroachdb/cockroach/pkg/testutils"
22+
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
1923
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2024
"github.com/cockroachdb/cockroach/pkg/util/log"
2125
"github.com/cockroachdb/cockroach/pkg/util/netutil/addr"
26+
"github.com/cockroachdb/crlib/crstrings"
27+
"github.com/cockroachdb/datadriven"
2228
"github.com/cockroachdb/pebble/vfs"
2329
"github.com/stretchr/testify/require"
2430
)
@@ -215,9 +221,8 @@ func TestExitIfDiskFull(t *testing.T) {
215221
defer leaktest.AfterTest(t)()
216222
defer log.Scope(t).Close(t)
217223

218-
err := exitIfDiskFull(mockDiskSpaceFS{FS: vfs.NewMem()}, []base.StoreSpec{
219-
{},
220-
})
224+
err := exitIfDiskFull(t.Context(), &mockDiskSpaceFS{FS: vfs.NewMem()},
225+
[]base.StoreSpec{{}})
221226
require.Error(t, err)
222227
var cliErr *clierror.Error
223228
require.True(t, errors.As(err, &cliErr))
@@ -226,12 +231,88 @@ func TestExitIfDiskFull(t *testing.T) {
226231

227232
type mockDiskSpaceFS struct {
228233
vfs.FS
234+
diskUsages map[string]vfs.DiskUsage
229235
}
230236

231-
func (fs mockDiskSpaceFS) GetDiskUsage(path string) (vfs.DiskUsage, error) {
237+
func (fs *mockDiskSpaceFS) Unwrap() vfs.FS { return fs.FS }
238+
239+
func (fs *mockDiskSpaceFS) GetDiskUsage(path string) (vfs.DiskUsage, error) {
240+
if fs.diskUsages != nil {
241+
for usagePath, usage := range fs.diskUsages {
242+
if strings.HasPrefix(path, usagePath) {
243+
return usage, nil
244+
}
245+
}
246+
}
232247
return vfs.DiskUsage{
233248
AvailBytes: 10 << 20,
234249
TotalBytes: 100 << 30,
235250
UsedBytes: 100<<30 - 10<<20,
236251
}, nil
237252
}
253+
254+
func TestExitIfDiskFullDatadriven(t *testing.T) {
255+
defer leaktest.AfterTest(t)()
256+
defer log.Scope(t).Close(t)
257+
258+
var buf bytes.Buffer
259+
var fs vfs.FS = vfs.NewMem()
260+
require.NoError(t, fs.MkdirAll("/mnt", 0755))
261+
mockDiskUsage := &mockDiskSpaceFS{FS: fs, diskUsages: make(map[string]vfs.DiskUsage)}
262+
fs = vfs.WithLogging(mockDiskUsage, func(format string, args ...interface{}) {
263+
fmt.Fprint(&buf, "# ")
264+
fmt.Fprintf(&buf, format, args...)
265+
fmt.Fprintln(&buf)
266+
})
267+
datadriven.RunTestAny(t, "testdata/exit_if_disk_full", func(t testing.TB, td *datadriven.TestData) string {
268+
buf.Reset()
269+
switch td.Cmd {
270+
case "run":
271+
var specs []base.StoreSpec
272+
clear(mockDiskUsage.diskUsages)
273+
for _, line := range crstrings.Lines(td.Input) {
274+
switch {
275+
case strings.HasPrefix(line, "disk-usage:"):
276+
// disk-usage: <path> <avail> <total>
277+
line = strings.TrimSpace(strings.TrimPrefix(line, "disk-usage:"))
278+
fields := strings.Fields(line)
279+
if len(fields) != 3 {
280+
return fmt.Sprintf("invalid disk-usage line: %q", line)
281+
}
282+
path := fields[0]
283+
avail, err := humanizeutil.ParseBytes(fields[1])
284+
if err != nil {
285+
return fmt.Sprintf("invalid disk-usage line: %q", line)
286+
}
287+
total, err := humanizeutil.ParseBytes(fields[2])
288+
if err != nil {
289+
return fmt.Sprintf("invalid disk-usage line: %q", line)
290+
}
291+
mockDiskUsage.diskUsages[path] = vfs.DiskUsage{
292+
AvailBytes: uint64(avail),
293+
TotalBytes: uint64(total),
294+
UsedBytes: uint64(total - avail),
295+
}
296+
case strings.HasPrefix(line, "store:"):
297+
line = strings.TrimSpace(strings.TrimPrefix(line, "store:"))
298+
spec, err := base.NewStoreSpec(line)
299+
if err != nil {
300+
return fmt.Sprintf("failed to parse store spec: %v", err)
301+
}
302+
specs = append(specs, spec)
303+
default:
304+
return fmt.Sprintf("unknown line: %q", line)
305+
}
306+
}
307+
err := exitIfDiskFull(t.Context(), fs, specs)
308+
if err == nil {
309+
fmt.Fprint(&buf, "<no-error>")
310+
} else {
311+
fmt.Fprintf(&buf, "<error:%s>", err)
312+
}
313+
return buf.String()
314+
default:
315+
return fmt.Sprintf("unknown command: %s", td.Cmd)
316+
}
317+
})
318+
}

pkg/cli/testdata/exit_if_disk_full

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,11 @@
1+
run
2+
disk-usage: /mnt/foo 1KiB 300GiB
3+
disk-usage: /mnt/bar 1KiB 300GiB
4+
store: path=/mnt/foo
5+
store: path=/mnt/bar
6+
----
7+
# open: /mnt/foo/temp-dirs-record.txt
8+
# open: /mnt/bar/temp-dirs-record.txt
9+
# get-disk-usage: /mnt/foo
10+
# get-disk-usage: /mnt/bar
11+
<error:store /mnt/foo: out of disk space>

pkg/storage/ballast.go

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -123,6 +123,14 @@ func maybeEstablishBallast(
123123
currentSizeBytes = fi.Size()
124124
}
125125

126+
// MemFS doesn't support resizing, so just return false. This exists
127+
// primarily to ease testing. If starting a real cockroach binary with a
128+
// store with a MemFS, we shouldn't even get this far because the StoreSpec
129+
// is marked as InMemory.
130+
if _, isMem := vfs.Root(fs).(*vfs.MemFS); isMem {
131+
return false, nil
132+
}
133+
126134
switch {
127135
case currentSizeBytes > ballastSizeBytes:
128136
// If the current ballast is too big, shrink it regardless of current

pkg/storage/fs/temp_dir.go

Lines changed: 31 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -25,8 +25,8 @@ type lockStruct struct {
2525
}
2626

2727
// lockFile sets a lock on the specified file, using flock.
28-
func lockFile(filename string) (lockStruct, error) {
29-
closer, err := vfs.Default.Lock(filename)
28+
func lockFile(fs vfs.FS, filename string) (lockStruct, error) {
29+
closer, err := fs.Lock(filename)
3030
if err != nil {
3131
return lockStruct{}, err
3232
}
@@ -64,7 +64,7 @@ func CreateTempDir(parentDir, prefix string) (_ string, unlockDirFn func(), _ er
6464
}
6565

6666
// Create a lock file.
67-
flock, err := lockFile(filepath.Join(absPath, lockFilename))
67+
flock, err := lockFile(vfs.Default, filepath.Join(absPath, lockFilename))
6868
if err != nil {
6969
return "", nil, errors.Wrapf(err, "could not create lock on new temporary directory")
7070
}
@@ -98,19 +98,23 @@ func RecordTempDir(recordPath, tempPath string) error {
9898
// up abandoned temporary directories.
9999
// It should also be invoked when a newly created temporary directory is no
100100
// longer needed and needs to be removed from the record file.
101-
func CleanupTempDirs(recordPath string) error {
101+
func CleanupTempDirs(ctx context.Context, fs vfs.FS, recordPath string) error {
102102
// Reading the entire file into memory shouldn't be a problem since
103103
// it is extremely rare for this record file to contain more than a few
104104
// entries.
105-
f, err := os.OpenFile(recordPath, os.O_RDWR, 0644)
105+
f, err := fs.Open(recordPath)
106106
// There is no existing record file and thus nothing to clean up.
107107
if oserror.IsNotExist(err) {
108108
return nil
109109
}
110110
if err != nil {
111111
return err
112112
}
113-
defer f.Close()
113+
defer func() {
114+
if f != nil {
115+
f.Close()
116+
}
117+
}()
114118

115119
scanner := bufio.NewScanner(f)
116120
// Iterate through each temporary directory path and remove the
@@ -122,14 +126,14 @@ func CleanupTempDirs(recordPath string) error {
122126
}
123127

124128
// Check if the temporary directory exists; if it does not, skip over it.
125-
if _, err := os.Stat(path); oserror.IsNotExist(err) {
126-
log.Dev.Warningf(context.Background(), "could not locate previous temporary directory %s, might require manual cleanup, or might have already been cleaned up.", path)
129+
if _, err := fs.Stat(path); oserror.IsNotExist(err) {
130+
log.Dev.Warningf(ctx, "could not locate previous temporary directory %s, might require manual cleanup, or might have already been cleaned up.", path)
127131
continue
128132
}
129133

130134
// Check if another Cockroach instance is using this temporary
131135
// directory i.e. has a lock on the temp dir lock file.
132-
flock, err := lockFile(filepath.Join(path, lockFilename))
136+
flock, err := lockFile(fs, fs.PathJoin(path, lockFilename))
133137
if err != nil {
134138
return errors.Wrapf(err, "could not lock temporary directory %s, may still be in use", path)
135139
}
@@ -143,15 +147,29 @@ func CleanupTempDirs(recordPath string) error {
143147
// process is dead because we were able to acquire the lock in the first
144148
// place.
145149
if err := unlockFile(flock); err != nil {
146-
log.Dev.Errorf(context.TODO(), "could not unlock file lock when removing temporary directory: %s", err.Error())
150+
log.Dev.Errorf(ctx, "could not unlock file lock when removing temporary directory: %s", err.Error())
147151
}
148152

149153
// If path/directory does not exist, error is nil.
150-
if err := os.RemoveAll(path); err != nil {
154+
if err := fs.RemoveAll(path); err != nil {
151155
return err
152156
}
153157
}
158+
// If the scanner ecnoutners an error, we may not have been able to remove
159+
// all the temporary directories.
160+
if err := scanner.Err(); err != nil {
161+
err = errors.CombineErrors(err, f.Close())
162+
f = nil
163+
return err
164+
}
154165

155-
// Clear out the record file now that we're done.
156-
return f.Truncate(0)
166+
// Close and remove the record file now that we're done. We need to close
167+
// the file first to accommodate Windows.
168+
err = f.Close()
169+
f = nil
170+
rmErr := fs.Remove(recordPath)
171+
if rmErr != nil && !oserror.IsNotExist(rmErr) {
172+
err = errors.CombineErrors(err, rmErr)
173+
}
174+
return err
157175
}

0 commit comments

Comments
 (0)