diff --git a/pkg/ccl/cliccl/debug.go b/pkg/ccl/cliccl/debug.go index a30de500cec6..3fa0464cef8e 100644 --- a/pkg/ccl/cliccl/debug.go +++ b/pkg/ccl/cliccl/debug.go @@ -136,6 +136,13 @@ with their env type and encryption settings (if applicable). &storeEncryptionSpecs, cliflagsccl.EnterpriseEncryption) cli.PopulateStorageConfigHook = fillEncryptionOptionsForStore + cli.EncryptedStorePathsHook = func() []string { + var res []string + for _, spec := range storeEncryptionSpecs.Specs { + res = append(res, spec.Path) + } + return res + } } // fillEncryptionOptionsForStore fills the StorageConfig fields diff --git a/pkg/cli/BUILD.bazel b/pkg/cli/BUILD.bazel index 039b6bd8592d..a0b0c266cc7e 100644 --- a/pkg/cli/BUILD.bazel +++ b/pkg/cli/BUILD.bazel @@ -6,8 +6,8 @@ load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "cli", srcs = [ - "absolute_fs.go", "auth.go", + "auto_decrypt_fs.go", "cert.go", "cli.go", "client_url.go", @@ -59,7 +59,6 @@ go_library( "start_windows.go", "statement_bundle.go", "statement_diag.go", - "swappable_fs.go", "testutils.go", "tsdump.go", "userfile.go", @@ -234,6 +233,7 @@ go_library( "@com_github_cockroachdb_errors//hintdetail", "@com_github_cockroachdb_errors//oserror", "@com_github_cockroachdb_logtags//:logtags", + "@com_github_cockroachdb_pebble//:pebble", "@com_github_cockroachdb_pebble//tool", "@com_github_cockroachdb_pebble//vfs", "@com_github_cockroachdb_redact//:redact", @@ -303,6 +303,7 @@ go_test( name = "cli_test", size = "large", srcs = [ + "auto_decrypt_fs_test.go", "cert_test.go", "cli_debug_test.go", "cli_test.go", diff --git a/pkg/cli/absolute_fs.go b/pkg/cli/absolute_fs.go deleted file mode 100644 index 8b3ba2fe8a65..000000000000 --- a/pkg/cli/absolute_fs.go +++ /dev/null @@ -1,139 +0,0 @@ -// 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 cli - -import ( - "io" - "os" - "path/filepath" - - "github.com/cockroachdb/pebble/vfs" -) - -// absoluteFS is a wrapper vfs.FS for an encryptedFS that is used only -// by the Pebble tool. It converts filepath names to absolute paths -// before calling the underlying interface implementation for functions -// that make use of the PebbleFileRegistry. -// -// This is needed when using encryptedFS, since the PebbleFileRegistry used in -// that context attempts to convert function input paths to relative paths using -// the DBDir. Both the DBDir and function input paths in a CockroachDB node are -// absolute paths, but when using the Pebble tool, the function input paths are -// based on what the cli user passed to the pebble command. We do not wish for a -// user using the cli to remember to pass an absolute path to the various pebble -// tool commands that accept paths. Note that the pebble tool commands taking a -// path parameter are quite varied: ranging from "pebble db" to "pebble lsm", -// so it is simplest to intercept the function input paths here. -// -// Note that absoluteFS assumes the wrapped vfs.FS corresponds to the underlying -// OS filesystem and will not work for the general case of a vfs.FS. -// This limitation is acceptable for this tool's use cases. -type absoluteFS struct { - fs vfs.FS -} - -var _ vfs.FS = &absoluteFS{} - -func (fs *absoluteFS) Create(name string) (vfs.File, error) { - name, err := filepath.Abs(name) - if err != nil { - return nil, err - } - return fs.fs.Create(name) -} - -func (fs *absoluteFS) Link(oldname, newname string) error { - return wrapWithAbsolute(fs.fs.Link, oldname, newname) -} - -func (fs *absoluteFS) Open(name string, opts ...vfs.OpenOption) (vfs.File, error) { - name, err := filepath.Abs(name) - if err != nil { - return nil, err - } - return fs.fs.Open(name, opts...) -} - -func (fs *absoluteFS) OpenDir(name string) (vfs.File, error) { - return fs.fs.OpenDir(name) -} - -func (fs *absoluteFS) Remove(name string) error { - name, err := filepath.Abs(name) - if err != nil { - return err - } - return fs.fs.Remove(name) -} - -func (fs *absoluteFS) RemoveAll(name string) error { - return fs.fs.RemoveAll(name) -} - -func (fs *absoluteFS) Rename(oldname, newname string) error { - return wrapWithAbsolute(fs.fs.Rename, oldname, newname) -} - -func (fs *absoluteFS) ReuseForWrite(oldname, newname string) (vfs.File, error) { - oldname, err := filepath.Abs(oldname) - if err != nil { - return nil, err - } - newname, err = filepath.Abs(newname) - if err != nil { - return nil, err - } - return fs.fs.ReuseForWrite(oldname, newname) -} - -func (fs *absoluteFS) MkdirAll(dir string, perm os.FileMode) error { - return fs.fs.MkdirAll(dir, perm) -} - -func (fs *absoluteFS) Lock(name string) (io.Closer, error) { - return fs.fs.Lock(name) -} - -func (fs *absoluteFS) List(dir string) ([]string, error) { - return fs.fs.List(dir) -} - -func (fs *absoluteFS) Stat(name string) (os.FileInfo, error) { - return fs.fs.Stat(name) -} - -func (fs *absoluteFS) PathBase(path string) string { - return fs.fs.PathBase(path) -} - -func (fs *absoluteFS) PathJoin(elem ...string) string { - return fs.fs.PathJoin(elem...) -} - -func (fs *absoluteFS) PathDir(path string) string { - return fs.fs.PathDir(path) -} - -func (fs *absoluteFS) GetDiskUsage(path string) (vfs.DiskUsage, error) { - return fs.fs.GetDiskUsage(path) -} - -func wrapWithAbsolute(fn func(string, string) error, oldname, newname string) error { - oldname, err := filepath.Abs(oldname) - if err != nil { - return err - } - newname, err = filepath.Abs(newname) - if err != nil { - return err - } - return fn(oldname, newname) -} diff --git a/pkg/cli/auto_decrypt_fs.go b/pkg/cli/auto_decrypt_fs.go new file mode 100644 index 000000000000..462fd7258597 --- /dev/null +++ b/pkg/cli/auto_decrypt_fs.go @@ -0,0 +1,259 @@ +// Copyright 2023 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 cli + +import ( + "io" + "os" + "path/filepath" + "sync" + + "github.com/cockroachdb/pebble/vfs" +) + +// autoDecryptFS is a filesystem which automatically detects paths that are +// registered as encrypted stores and uses the appropriate encryptedFS. +type autoDecryptFS struct { + encryptedDirs map[string]*encryptedDir + resolveFn resolveEncryptedDirFn +} + +var _ vfs.FS = (*autoDecryptFS)(nil) + +type encryptedDir struct { + once sync.Once + fs vfs.FS + resolveErr error +} + +type resolveEncryptedDirFn func(dir string) (vfs.FS, error) + +// Init sets up the given paths as encrypted and provides a callback that can +// resolve an encrypted directory path into an FS. +// +// Any FS operations inside encrypted paths will use the corresponding resolved FS. +// +// resolveFN is lazily called at most once for each encrypted dir. +func (afs *autoDecryptFS) Init(encryptedDirs []string, resolveFn resolveEncryptedDirFn) { + afs.resolveFn = resolveFn + + afs.encryptedDirs = make(map[string]*encryptedDir) + for _, dir := range encryptedDirs { + if absDir, err := filepath.Abs(dir); err == nil { + dir = absDir + } + afs.encryptedDirs[dir] = &encryptedDir{} + } +} + +func (afs *autoDecryptFS) Create(name string) (vfs.File, error) { + name, err := filepath.Abs(name) + if err != nil { + return nil, err + } + fs, err := afs.maybeSwitchFS(name) + if err != nil { + return nil, err + } + return fs.Create(name) +} + +func (afs *autoDecryptFS) Link(oldname, newname string) error { + oldname, err := filepath.Abs(oldname) + if err != nil { + return err + } + newname, err = filepath.Abs(newname) + if err != nil { + return err + } + fs, err := afs.maybeSwitchFS(oldname) + if err != nil { + return err + } + return fs.Link(oldname, newname) +} + +func (afs *autoDecryptFS) Open(name string, opts ...vfs.OpenOption) (vfs.File, error) { + name, err := filepath.Abs(name) + if err != nil { + return nil, err + } + fs, err := afs.maybeSwitchFS(name) + if err != nil { + return nil, err + } + return fs.Open(name, opts...) +} + +func (afs *autoDecryptFS) OpenDir(name string) (vfs.File, error) { + name, err := filepath.Abs(name) + if err != nil { + return nil, err + } + fs, err := afs.maybeSwitchFS(name) + if err != nil { + return nil, err + } + return fs.OpenDir(name) +} + +func (afs *autoDecryptFS) Remove(name string) error { + name, err := filepath.Abs(name) + if err != nil { + return err + } + fs, err := afs.maybeSwitchFS(name) + if err != nil { + return err + } + return fs.Remove(name) +} + +func (afs *autoDecryptFS) RemoveAll(name string) error { + name, err := filepath.Abs(name) + if err != nil { + return err + } + fs, err := afs.maybeSwitchFS(name) + if err != nil { + return err + } + return fs.RemoveAll(name) +} + +func (afs *autoDecryptFS) Rename(oldname, newname string) error { + fs, err := afs.maybeSwitchFS(oldname) + if err != nil { + return err + } + return fs.Rename(oldname, newname) +} + +func (afs *autoDecryptFS) ReuseForWrite(oldname, newname string) (vfs.File, error) { + oldname, err := filepath.Abs(oldname) + if err != nil { + return nil, err + } + newname, err = filepath.Abs(newname) + if err != nil { + return nil, err + } + fs, err := afs.maybeSwitchFS(oldname) + if err != nil { + return nil, err + } + return fs.ReuseForWrite(oldname, newname) +} + +func (afs *autoDecryptFS) MkdirAll(dir string, perm os.FileMode) error { + dir, err := filepath.Abs(dir) + if err != nil { + return err + } + fs, err := afs.maybeSwitchFS(dir) + if err != nil { + return err + } + return fs.MkdirAll(dir, perm) +} + +func (afs *autoDecryptFS) Lock(name string) (io.Closer, error) { + name, err := filepath.Abs(name) + if err != nil { + return nil, err + } + fs, err := afs.maybeSwitchFS(name) + if err != nil { + return nil, err + } + return fs.Lock(name) +} + +func (afs *autoDecryptFS) List(dir string) ([]string, error) { + dir, err := filepath.Abs(dir) + if err != nil { + return nil, err + } + fs, err := afs.maybeSwitchFS(dir) + if err != nil { + return nil, err + } + return fs.List(dir) +} + +func (afs *autoDecryptFS) Stat(name string) (os.FileInfo, error) { + name, err := filepath.Abs(name) + if err != nil { + return nil, err + } + fs, err := afs.maybeSwitchFS(name) + if err != nil { + return nil, err + } + return fs.Stat(name) +} + +func (afs *autoDecryptFS) PathBase(path string) string { + return filepath.Base(path) +} + +func (afs *autoDecryptFS) PathJoin(elem ...string) string { + return filepath.Join(elem...) +} + +func (afs *autoDecryptFS) PathDir(path string) string { + return filepath.Dir(path) +} + +func (afs *autoDecryptFS) GetDiskUsage(path string) (vfs.DiskUsage, error) { + path, err := filepath.Abs(path) + if err != nil { + return vfs.DiskUsage{}, err + } + fs, err := afs.maybeSwitchFS(path) + if err != nil { + return vfs.DiskUsage{}, err + } + return fs.GetDiskUsage(path) +} + +// maybeSwitchFS finds the first ancestor of path that is registered as an +// encrypted FS; if there is such a path, returns the decrypted FS for that +// path. Otherwise, returns the default FS. +// +// Assumes that path is absolute and clean (i.e. filepath.Abs was run on it); +// the returned FS also assumes that any paths used are absolute and clean. This +// is needed when using encryptedFS, since the PebbleFileRegistry used in that +// context attempts to convert function input paths to relative paths using the +// DBDir. Both the DBDir and function input paths in a CockroachDB node are +// absolute paths, but when using the Pebble tool, the function input paths are +// based on what the cli user passed to the pebble command. We do not wish for a +// user using the cli to remember to pass an absolute path to the various pebble +// tool commands that accept paths. Note that the pebble tool commands taking a +// path parameter are quite varied: ranging from "pebble db" to "pebble lsm", so +// it is simplest to intercept the function input paths here. +func (afs *autoDecryptFS) maybeSwitchFS(path string) (vfs.FS, error) { + for { + if e := afs.encryptedDirs[path]; e != nil { + e.once.Do(func() { + e.fs, e.resolveErr = afs.resolveFn(path) + }) + return e.fs, e.resolveErr + } + parent := filepath.Dir(path) + if path == parent { + break + } + path = parent + } + return vfs.Default, nil +} diff --git a/pkg/cli/auto_decrypt_fs_test.go b/pkg/cli/auto_decrypt_fs_test.go new file mode 100644 index 000000000000..565a631b7d8f --- /dev/null +++ b/pkg/cli/auto_decrypt_fs_test.go @@ -0,0 +1,195 @@ +// Copyright 2023 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 cli + +import ( + "bytes" + "fmt" + "io" + "os" + "path/filepath" + "runtime" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/pebble/vfs" + "github.com/stretchr/testify/require" +) + +func TestAutoDecryptFS(t *testing.T) { + defer leaktest.AfterTest(t)() + + if runtime.GOOS == "windows" { + skip.IgnoreLint(t, "expected output uses unix paths") + } + dir, err := os.MkdirTemp("", "auto-decrypt-test") + require.NoError(t, err) + defer func() { _ = os.RemoveAll(dir) }() + + path1 := filepath.Join(dir, "path1") + path2 := filepath.Join(dir, "foo", "path2") + + var buf bytes.Buffer + resolveFn := func(dir string) (vfs.FS, error) { + if dir != path1 && dir != path2 { + t.Fatalf("unexpected dir %s", dir) + } + fs := vfs.NewMem() + require.NoError(t, fs.MkdirAll(dir, 0755)) + return WithLogging(fs, func(format string, args ...interface{}) { + fmt.Fprintf(&buf, dir+": "+format+"\n", args...) + }), nil + } + + var fs autoDecryptFS + fs.Init([]string{path1, path2}, resolveFn) + + create := func(pathElems ...string) { + file, err := fs.Create(filepath.Join(pathElems...)) + require.NoError(t, err) + file.Close() + } + + create(dir, "foo") + create(path1, "bar") + create(path2, "baz") + require.NoError(t, fs.MkdirAll(filepath.Join(path2, "a", "b"), 0755)) + create(path2, "a", "b", "xx") + + // Check that operations inside the two paths happen using the resolved FSes. + output := strings.ReplaceAll(buf.String(), dir, "$TMPDIR") + expected := + `$TMPDIR/path1: create: $TMPDIR/path1/bar +$TMPDIR/path1: close: $TMPDIR/path1/bar +$TMPDIR/foo/path2: create: $TMPDIR/foo/path2/baz +$TMPDIR/foo/path2: close: $TMPDIR/foo/path2/baz +$TMPDIR/foo/path2: mkdir-all: $TMPDIR/foo/path2/a/b 0755 +$TMPDIR/foo/path2: create: $TMPDIR/foo/path2/a/b/xx +$TMPDIR/foo/path2: close: $TMPDIR/foo/path2/a/b/xx +` + require.Equal(t, expected, output) +} + +// WithLogging wraps an FS and logs filesystem modification operations to the +// given logFn. +func WithLogging(fs vfs.FS, logFn LogFn) vfs.FS { + return &loggingFS{ + FS: fs, + logFn: logFn, + } +} + +// LogFn is a function that is used to capture a log when WithLogging is used. +type LogFn func(fmt string, args ...interface{}) + +type loggingFS struct { + vfs.FS + logFn LogFn +} + +var _ vfs.FS = (*loggingFS)(nil) + +func (fs *loggingFS) Create(name string) (vfs.File, error) { + fs.logFn("create: %s", name) + f, err := fs.FS.Create(name) + if err != nil { + return nil, err + } + return newLoggingFile(f, name, fs.logFn), nil +} + +func (fs *loggingFS) Link(oldname, newname string) error { + fs.logFn("link: %s -> %s", oldname, newname) + return fs.FS.Link(oldname, newname) +} + +func (fs *loggingFS) OpenDir(name string) (vfs.File, error) { + fs.logFn("open-dir: %s", name) + f, err := fs.FS.OpenDir(name) + if err != nil { + return nil, err + } + return newLoggingFile(f, name, fs.logFn), nil +} + +func (fs *loggingFS) Rename(oldname, newname string) error { + fs.logFn("rename: %s -> %s", oldname, newname) + return fs.FS.Rename(oldname, newname) +} + +func (fs *loggingFS) ReuseForWrite(oldname, newname string) (vfs.File, error) { + fs.logFn("reuseForWrite: %s -> %s", oldname, newname) + f, err := fs.FS.ReuseForWrite(oldname, newname) + if err != nil { + return nil, err + } + return newLoggingFile(f, newname, fs.logFn), nil +} + +func (fs *loggingFS) MkdirAll(dir string, perm os.FileMode) error { + fs.logFn("mkdir-all: %s %#o", dir, perm) + return fs.FS.MkdirAll(dir, perm) +} + +func (fs *loggingFS) Lock(name string) (io.Closer, error) { + fs.logFn("lock: %s", name) + return fs.FS.Lock(name) +} + +func (fs loggingFS) Remove(name string) error { + fs.logFn("remove: %s", name) + err := fs.FS.Remove(name) + return err +} + +func (fs loggingFS) RemoveAll(name string) error { + fs.logFn("remove-all: %s", name) + err := fs.FS.RemoveAll(name) + return err +} + +type loggingFile struct { + vfs.File + name string + logFn LogFn +} + +var _ vfs.File = (*loggingFile)(nil) + +func newLoggingFile(f vfs.File, name string, logFn LogFn) *loggingFile { + return &loggingFile{ + File: f, + name: name, + logFn: logFn, + } +} + +func (f *loggingFile) Close() error { + f.logFn("close: %s", f.name) + return f.File.Close() +} + +func (f *loggingFile) Sync() error { + f.logFn("sync: %s", f.name) + return f.File.Sync() +} + +func (f *loggingFile) SyncData() error { + f.logFn("sync-data: %s", f.name) + return f.File.SyncData() +} + +func (f *loggingFile) SyncTo(length int64) (fullSync bool, err error) { + f.logFn("sync-to(%d): %s", length, f.name) + return f.File.SyncTo(length) +} diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 2cd90caaeb0f..eb146e4652fc 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -65,6 +65,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/oserror" + "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/tool" "github.com/cockroachdb/pebble/vfs" "github.com/cockroachdb/ttycolor" @@ -96,9 +97,14 @@ Create a ballast file to fill the store directory up to a given amount // PopulateStorageConfigHook is a callback set by CCL code. // It populates any needed fields in the StorageConfig. -// It must do nothing in OSS code. +// It must stay unset in OSS code. var PopulateStorageConfigHook func(*base.StorageConfig) error +// EncryptedStorePathsHook is a callback set by CCL code. +// It returns the store paths that are encrypted. +// It must stay unset in OSS code. +var EncryptedStorePathsHook func() []string + func parsePositiveInt(arg string) (int64, error) { i, err := strconv.ParseInt(arg, 10, 64) if err != nil { @@ -1712,7 +1718,7 @@ func (m lockValueFormatter) Format(f fmt.State, c rune) { // It is necessary because an FS must be passed to tool.New before // the command line flags are parsed (i.e. before we can determine // if we have an encrypted FS). -var pebbleToolFS = &swappableFS{vfs.Default} +var pebbleToolFS = &autoDecryptFS{} func init() { DebugCmd.AddCommand(debugCmds...) @@ -1739,7 +1745,16 @@ func init() { // to write those files. pebbleTool := tool.New(tool.Mergers(storage.MVCCMerger), tool.DefaultComparer(storage.EngineComparer), - tool.FS(&absoluteFS{pebbleToolFS}), + tool.FS(pebbleToolFS), + tool.OpenErrEnhancer(func(err error) error { + if pebble.IsCorruptionError(err) { + // None of the wrappers provided by the error library allow adding a + // message that shows up with "%s" after the original message. + // nolint:errwrap + return errors.Newf("%v\nIf this is an encrypted store, make sure the correct encryption key is set.", err) + } + return err + }), ) DebugPebbleCmd.AddCommand(pebbleTool.Commands...) initPebbleCmds(DebugPebbleCmd) @@ -1857,35 +1872,36 @@ func initPebbleCmds(cmd *cobra.Command) { return err } } - return pebbleCryptoInitializer() + pebbleCryptoInitializer(cmd.Context()) + return nil } initPebbleCmds(c) } } -func pebbleCryptoInitializer() error { - storageConfig := base.StorageConfig{ - Settings: serverCfg.Settings, - Dir: serverCfg.Stores.Specs[0].Path, - } +func pebbleCryptoInitializer(ctx context.Context) { + if EncryptedStorePathsHook != nil && PopulateStorageConfigHook != nil { + encryptedPaths := EncryptedStorePathsHook() + resolveFn := func(dir string) (vfs.FS, error) { + storageConfig := base.StorageConfig{ + Settings: serverCfg.Settings, + Dir: dir, + } + if err := PopulateStorageConfigHook(&storageConfig); err != nil { + return nil, err + } + cfg := storage.PebbleConfig{ + StorageConfig: storageConfig, + Opts: storage.DefaultPebbleOptions(), + } - if PopulateStorageConfigHook != nil { - if err := PopulateStorageConfigHook(&storageConfig); err != nil { - return err + // This has the side effect of storing the encrypted FS into cfg.Opts.FS. + _, _, err := storage.ResolveEncryptedEnvOptions(&cfg) + if err != nil { + return nil, err + } + return cfg.Opts.FS, nil } + pebbleToolFS.Init(encryptedPaths, resolveFn) } - - cfg := storage.PebbleConfig{ - StorageConfig: storageConfig, - Opts: storage.DefaultPebbleOptions(), - } - - // This has the side effect of storing the encrypted FS into cfg.Opts.FS. - _, _, err := storage.ResolveEncryptedEnvOptions(&cfg) - if err != nil { - return err - } - - pebbleToolFS.set(cfg.Opts.FS) - return nil } diff --git a/pkg/cli/interactive_tests/test_encryption.tcl b/pkg/cli/interactive_tests/test_encryption.tcl index a415bcc234bc..0465247d03b0 100644 --- a/pkg/cli/interactive_tests/test_encryption.tcl +++ b/pkg/cli/interactive_tests/test_encryption.tcl @@ -51,7 +51,7 @@ eexpect "" end_test start_test "Run pebble debug tool." -send "$argv debug pebble db lsm $storedir --store=$storedir\r" +send "$argv debug pebble db lsm $storedir\r" eexpect "__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp\r" end_test @@ -105,9 +105,9 @@ eexpect "store key ID * was not found" end_test start_test "Run pebble debug tool with AES-256." -send "$argv debug pebble db lsm $storedir --store=$storedir --enterprise-encryption=path=$storedir,key=$keydir/aes-256.key,old-key=$keydir/aes-256.key\r" +send "$argv debug pebble db lsm $storedir --enterprise-encryption=path=$storedir,key=$keydir/aes-256.key,old-key=$keydir/aes-256.key\r" eexpect "__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp\r" # Try running without the encryption flag. -send "$argv debug pebble db lsm $storedir --store=$storedir\r" -eexpect "encryption was used on this store before, but no encryption flags specified." +send "$argv debug pebble db lsm $storedir\r" +eexpect "If this is an encrypted store, make sure the correct encryption key is set." end_test diff --git a/pkg/cli/swappable_fs.go b/pkg/cli/swappable_fs.go deleted file mode 100644 index 232fa0aa52b4..000000000000 --- a/pkg/cli/swappable_fs.go +++ /dev/null @@ -1,23 +0,0 @@ -// 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 cli - -import "github.com/cockroachdb/pebble/vfs" - -// swappableFS is a vfs.FS that can be swapped out at a future time. -type swappableFS struct { - vfs.FS -} - -// set replaces the FS in a swappableFS. -func (s *swappableFS) set(fs vfs.FS) { - s.FS = fs -}