From 6a0055b013f249f6f1b525bcbb91637a8b87e8b5 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Thu, 7 Sep 2023 14:21:02 -0700 Subject: [PATCH] cli: fix debug pebble commands on encrypted stores Currently the debug pebble commands only work correctly on an encrypted store if the encrypted store's path is `cockroach-data` or the store directory is passed using `--store` (in addition to being passed to the pebble subcommand itself). What's worse, knowledge of this subtle fact was lost among team members. The root cause is that we are trying to resolve encryption options using the server config. The difficulty is that there are a bunch of different commands and there is no unified way to obtain the store directory of interest To fix this, we create `autoDecryptFS`. This is a `vfs.FS` implementation which is able to automatically detect encrypted paths and use the correct unencrypted FS. It does this by having a list of known encrypted stores (the ones in the `--enterprise-encryption` flag), and looking for any of these paths as ancestors of any path in an operation. This new implementation replaces `swappableFS` and `absoluteFS`. We also improve the error message when we try to open an encrypted store without setting up the key correctly. Fixes: #110121 Release note (bug fix): `cockroach debug pebble` commands now work correctly with encrypted stores which don't use the default `cockroach-data` path without having to also pass `--store`. --- pkg/ccl/cliccl/debug.go | 7 + pkg/cli/BUILD.bazel | 5 +- pkg/cli/absolute_fs.go | 139 ---------- pkg/cli/auto_decrypt_fs.go | 259 ++++++++++++++++++ pkg/cli/auto_decrypt_fs_test.go | 195 +++++++++++++ pkg/cli/debug.go | 68 +++-- pkg/cli/interactive_tests/test_encryption.tcl | 8 +- pkg/cli/swappable_fs.go | 23 -- 8 files changed, 510 insertions(+), 194 deletions(-) delete mode 100644 pkg/cli/absolute_fs.go create mode 100644 pkg/cli/auto_decrypt_fs.go create mode 100644 pkg/cli/auto_decrypt_fs_test.go delete mode 100644 pkg/cli/swappable_fs.go 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 -}