Skip to content

Commit

Permalink
cli: support encrypted fs with pebble debug tool
Browse files Browse the repository at this point in the history
Release note (cli change): The `cockroach debug pebble` tool
can now be used with encrypted stores.
  • Loading branch information
Andy Yang committed May 14, 2021
1 parent dd1bf20 commit 29c4422
Show file tree
Hide file tree
Showing 8 changed files with 221 additions and 6 deletions.
6 changes: 6 additions & 0 deletions pkg/ccl/cliccl/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,12 @@ AES128_CTR:be235... # AES-128 encryption with store key ID
cli.VarFlag(cmd.Flags(), &storeEncryptionSpecs, cliflagsccl.EnterpriseEncryption)
}

// init has already run in cli/debug.go since this package imports it, so
// DebugPebbleCmd already has all its subcommands. We could traverse those
// here. But we don't need to by using PersistentFlags.
cli.VarFlag(cli.DebugPebbleCmd.PersistentFlags(),
&storeEncryptionSpecs, cliflagsccl.EnterpriseEncryption)

cli.PopulateRocksDBConfigHook = fillEncryptionOptionsForStore
}

Expand Down
3 changes: 3 additions & 0 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ load("//build:STRINGER.bzl", "stringer")
go_library(
name = "cli",
srcs = [
"absolute_fs.go",
"auth.go",
"cert.go",
"cli.go",
Expand Down Expand Up @@ -49,6 +50,7 @@ go_library(
"start_unix.go",
"start_windows.go",
"statement_diag.go",
"swappable_fs.go",
"testutils.go",
"tsdump.go",
"userfile.go",
Expand Down Expand Up @@ -194,6 +196,7 @@ go_library(
"@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",
"@com_github_dustin_go_humanize//:go-humanize",
"@com_github_elastic_gosigar//:gosigar",
Expand Down
107 changes: 107 additions & 0 deletions pkg/cli/absolute_fs.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
// 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 (
"path/filepath"

"github.com/cockroachdb/pebble/vfs"
)

// absoluteFS is a wrapper vfs.FS that converts filepath names to absolute paths
// before calling the underlying interface implementation for each function.
type absoluteFS struct {
vfs.FS
}

func (fs *absoluteFS) Create(name string) (vfs.File, error) {
return wrapWithAbsolute1(fs.FS.Create, name)
}

func (fs *absoluteFS) Link(oldname, newname string) error {
return wrapWithAbsolute3(fs.FS.Link, oldname, newname)
}

func (fs *absoluteFS) Open(name string, opts ...vfs.OpenOption) (vfs.File, error) {
return wrapWithAbsolute2(fs.FS.Open, name, opts...)
}

func (fs *absoluteFS) OpenDir(name string) (vfs.File, error) {
return wrapWithAbsolute1(fs.FS.OpenDir, name)
}

func (fs *absoluteFS) Remove(name string) error {
return wrapWithAbsolute0(fs.FS.Remove, name)
}

func (fs *absoluteFS) RemoveAll(name string) error {
return wrapWithAbsolute0(fs.FS.RemoveAll, name)
}

func (fs *absoluteFS) Rename(oldname, newname string) error {
return wrapWithAbsolute3(fs.FS.Rename, oldname, newname)
}

func (fs *absoluteFS) ReuseForWrite(oldname, newname string) (vfs.File, error) {
return wrapWithAbsolute4(fs.FS.ReuseForWrite, oldname, newname)
}

func wrapWithAbsolute0(fn func(string) error, name string) error {
name, err := filepath.Abs(name)
if err != nil {
return err
}
return fn(name)
}

func wrapWithAbsolute1(fn func(string) (vfs.File, error), name string) (vfs.File, error) {
name, err := filepath.Abs(name)
if err != nil {
return nil, err
}
return fn(name)
}

func wrapWithAbsolute2(
fn func(string, ...vfs.OpenOption) (vfs.File, error), name string, opts ...vfs.OpenOption,
) (vfs.File, error) {
name, err := filepath.Abs(name)
if err != nil {
return nil, err
}
return fn(name, opts...)
}

func wrapWithAbsolute3(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)
}

func wrapWithAbsolute4(
fn func(string, string) (vfs.File, error), 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 fn(oldname, newname)
}
8 changes: 6 additions & 2 deletions pkg/cli/cli.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,14 +99,18 @@ func doMain(cmd *cobra.Command, cmdName string) error {
// and PersistentPreRun in `(*cobra.Command) execute()`.)
wrapped := cmd.PreRunE
cmd.PreRunE = func(cmd *cobra.Command, args []string) error {
// We call setupLogging before the PreRunE function since
// that function may perform logging.
err := setupLogging(context.Background(), cmd,
false /* isServerCmd */, true /* applyConfig */)

if wrapped != nil {
if err := wrapped(cmd, args); err != nil {
return err
}
}

return setupLogging(context.Background(), cmd,
false /* isServerCmd */, true /* applyConfig */)
return err
}
}
}
Expand Down
62 changes: 58 additions & 4 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
"github.com/cockroachdb/pebble/tool"
"github.com/cockroachdb/pebble/vfs"
"github.com/gogo/protobuf/jsonpb"
"github.com/kr/pretty"
"github.com/spf13/cobra"
Expand Down Expand Up @@ -689,7 +690,9 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error {
return nil
}

var debugPebbleCmd = &cobra.Command{
// DebugPebbleCmd is the root of all debug pebble commands.
// Exported to allow modification by CCL code.
var DebugPebbleCmd = &cobra.Command{
Use: "pebble [command]",
Short: "run a Pebble introspection tool command",
Long: `
Expand Down Expand Up @@ -1291,6 +1294,12 @@ func (m lockValueFormatter) Format(f fmt.State, c rune) {
fmt.Fprint(f, kvserver.SprintIntent(m.value))
}

// pebbleToolFS is the vfs.FS that the pebble tool should use.
// 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}

func init() {
DebugCmd.AddCommand(debugCmds...)

Expand All @@ -1315,9 +1324,12 @@ func init() {
// and merger functions must be specified to pebble that match the ones used
// to write those files.
pebbleTool := tool.New(tool.Mergers(storage.MVCCMerger),
tool.DefaultComparer(storage.EngineComparer))
debugPebbleCmd.AddCommand(pebbleTool.Commands...)
DebugCmd.AddCommand(debugPebbleCmd)
tool.DefaultComparer(storage.EngineComparer),
tool.FS(&absoluteFS{pebbleToolFS}),
)
DebugPebbleCmd.AddCommand(pebbleTool.Commands...)
initPebbleCmds(DebugPebbleCmd)
DebugCmd.AddCommand(DebugPebbleCmd)

doctorExamineCmd.AddCommand(doctorExamineClusterCmd, doctorExamineZipDirCmd)
doctorRecreateCmd.AddCommand(doctorRecreateClusterCmd, doctorRecreateZipDirCmd)
Expand Down Expand Up @@ -1365,3 +1377,45 @@ func init() {
f = debugCheckLogConfigCmd.Flags()
f.Var(&debugLogChanSel, "only-channels", "selection of channels to include in the output diagram.")
}

func initPebbleCmds(cmd *cobra.Command) {
for _, c := range cmd.Commands() {
wrapped := c.PreRunE
c.PreRunE = func(cmd *cobra.Command, args []string) error {
if wrapped != nil {
if err := wrapped(cmd, args); err != nil {
return err
}
}
return pebbleCryptoInitializer()
}
initPebbleCmds(c)
}
}

func pebbleCryptoInitializer() error {
storageConfig := base.StorageConfig{
Settings: serverCfg.Settings,
Dir: serverCfg.Stores.Specs[0].Path,
}

if PopulateRocksDBConfigHook != nil {
if err := PopulateRocksDBConfigHook(&storageConfig); err != nil {
return err
}
}

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
}
5 changes: 5 additions & 0 deletions pkg/cli/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -857,6 +857,11 @@ func init() {
f := debugBallastCmd.Flags()
varFlag(f, &debugCtx.ballastSize, cliflags.Size)
}
{
// TODO(ayang): clean up so dir isn't passed to both pebble and --store
f := DebugPebbleCmd.PersistentFlags()
varFlag(f, &serverCfg.Stores, cliflags.Store)
}
{
for _, c := range []*cobra.Command{
doctorExamineClusterCmd,
Expand Down
13 changes: 13 additions & 0 deletions pkg/cli/interactive_tests/test_encryption.tcl
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,11 @@ send "$argv debug encryption-status $storedir\r"
eexpect ""
end_test

start_test "Run pebble debug tool."
send "$argv debug pebble db lsm $storedir --store=$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

start_test "Restart with plaintext."
send "$argv start-single-node --insecure --store=$storedir --enterprise-encryption=path=$storedir,key=plain,old-key=plain\r"
eexpect "node starting"
Expand Down Expand Up @@ -88,3 +93,11 @@ eexpect "encryption was used on this store before, but no encryption flags speci
send "$argv start-single-node --insecure --store=$storedir --enterprise-encryption=path=$storedir,key=$keydir/aes-192.key,old-key=plain\r"
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"
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."
end_test
23 changes: 23 additions & 0 deletions pkg/cli/swappable_fs.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
// 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
}

0 comments on commit 29c4422

Please sign in to comment.