Skip to content

Commit

Permalink
Merge #67723 #67944
Browse files Browse the repository at this point in the history
67723: storage: refactor interfaces for constructing new engines r=sumeerbhola a=jbowens

Refactor several of the various methods for constructing Engines into a
single Open method that supports configuration through the functional
options pattern. This removes some boilerplate configuration.

An eventual goal of this refactor is to tighten the storage package
interface to a narrower, more controlled interface.

Release note: None

67944: sql,server: refactor SQL Stats control to sqlstats package r=Azhng a=Azhng

sql,server: refactor SQL Stats control to sqlstats package

Previsouly, the administrative actions on SQL Stats subsystem
(e.g. reset sql stats) is directly implements on the sql.Server.
The implication of this is that various parts of the system
needs to maintain a reference of sql.Server one way or another.
As SQL Stats subsystem grows in complexity, it's no longer
feasible to piggyback off sql.Server for implementing the
control logic.
This commit refactors those logics into a new sslocal.Controller
struct. This struct will be reponsible for all the control actions.

Release note: None

Co-authored-by: Jackson Owens <[email protected]>
Co-authored-by: Azhng <[email protected]>
  • Loading branch information
3 people committed Aug 4, 2021
3 parents e0c8f31 + 0202227 + 251b4bd commit 8455178
Show file tree
Hide file tree
Showing 58 changed files with 507 additions and 483 deletions.
1 change: 0 additions & 1 deletion pkg/ccl/importccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -191,7 +191,6 @@ go_test(
"//pkg/workload/workloadsql",
"@com_github_cockroachdb_cockroach_go//crdb",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_pebble//:pebble",
"@com_github_go_sql_driver_mysql//:mysql",
"@com_github_gogo_protobuf//proto",
"@com_github_jackc_pgx//:pgx",
Expand Down
13 changes: 4 additions & 9 deletions pkg/ccl/importccl/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/workload"
"github.com/cockroachdb/cockroach/pkg/workload/tpcc"
"github.com/cockroachdb/pebble"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -92,18 +91,14 @@ func benchmarkWriteAndLink(b *testing.B, dir string, tables []tableSSTable) {
b.SetBytes(bytes)

ctx := context.Background()
cache := pebble.NewCache(server.DefaultCacheSize)
defer cache.Unref()

b.ResetTimer()
for i := 0; i < b.N; i++ {
b.StopTimer()
cfg := storage.PebbleConfig{
StorageConfig: base.StorageConfig{
Dir: filepath.Join(dir, `pebble`, timeutil.Now().String())}}
cfg.Opts = storage.DefaultPebbleOptions()
cfg.Opts.Cache = cache
db, err := storage.NewPebble(context.Background(), cfg)
db, err := storage.Open(
context.Background(),
storage.Filesystem(filepath.Join(dir, `pebble`, timeutil.Now().String())),
storage.CacheSize(server.DefaultCacheSize))
if err != nil {
b.Fatal(err)
}
Expand Down
12 changes: 3 additions & 9 deletions pkg/ccl/storageccl/engineccl/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ import (
"path/filepath"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -53,15 +52,10 @@ func loadTestData(
exists = false
}

eng, err := storage.NewPebble(
eng, err := storage.Open(
ctx,
storage.PebbleConfig{
StorageConfig: base.StorageConfig{
Settings: cluster.MakeTestingClusterSettings(),
Dir: dir,
},
},
)
storage.Filesystem(dir),
storage.Settings(cluster.MakeTestingClusterSettings()))
if err != nil {
return nil, err
}
Expand Down
10 changes: 4 additions & 6 deletions pkg/ccl/storageccl/export_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -646,12 +646,10 @@ func TestRandomKeyAndTimestampExport(t *testing.T) {

mkEngine := func(t *testing.T) (e storage.Engine, cleanup func()) {
dir, cleanupDir := testutils.TempDir(t)
e, err := storage.NewDefaultEngine(
0,
base.StorageConfig{
Settings: cluster.MakeTestingClusterSettings(),
Dir: dir,
})
e, err := storage.Open(ctx,
storage.Filesystem(dir),
storage.CacheSize(0),
storage.Settings(cluster.MakeTestingClusterSettings()))
if err != nil {
t.Fatal(err)
}
Expand Down
2 changes: 0 additions & 2 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -206,7 +206,6 @@ go_library(
"@com_github_cockroachdb_errors//:errors",
"@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",
Expand Down Expand Up @@ -356,7 +355,6 @@ go_test(
"//pkg/workload/examples",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_pebble//:pebble",
"@com_github_spf13_cobra//:cobra",
"@com_github_spf13_pflag//:pflag",
"@com_github_stretchr_testify//assert",
Expand Down
45 changes: 18 additions & 27 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,6 @@ 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"
humanize "github.com/dustin/go-humanize"
Expand Down Expand Up @@ -131,6 +130,17 @@ type OpenEngineOptions struct {
MustExist bool
}

func (opts OpenEngineOptions) configOptions() []storage.ConfigOption {
var cfgOpts []storage.ConfigOption
if opts.ReadOnly {
cfgOpts = append(cfgOpts, storage.ReadOnly)
}
if opts.MustExist {
cfgOpts = append(cfgOpts, storage.MustExist)
}
return cfgOpts
}

// OpenExistingStore opens the rocksdb engine rooted at 'dir'.
// If 'readOnly' is true, opens the store in read-only mode.
func OpenExistingStore(dir string, stopper *stop.Stopper, readOnly bool) (storage.Engine, error) {
Expand All @@ -144,32 +154,13 @@ func OpenEngine(dir string, stopper *stop.Stopper, opts OpenEngineOptions) (stor
if err != nil {
return nil, err
}

storageConfig := base.StorageConfig{
Settings: serverCfg.Settings,
Dir: dir,
MustExist: opts.MustExist,
}
if PopulateRocksDBConfigHook != nil {
if err := PopulateRocksDBConfigHook(&storageConfig); err != nil {
return nil, err
}
}

var db storage.Engine

cfg := storage.PebbleConfig{
StorageConfig: storageConfig,
Opts: storage.DefaultPebbleOptions(),
}
cfg.Opts.Cache = pebble.NewCache(server.DefaultCacheSize)
defer cfg.Opts.Cache.Unref()

cfg.Opts.MaxOpenFiles = int(maxOpenFiles)
cfg.Opts.ReadOnly = opts.ReadOnly

db, err = storage.NewPebble(context.Background(), cfg)

db, err := storage.Open(context.Background(),
storage.Filesystem(dir),
storage.MaxOpenFiles(int(maxOpenFiles)),
storage.CacheSize(server.DefaultCacheSize),
storage.Settings(serverCfg.Settings),
storage.Hook(PopulateRocksDBConfigHook),
storage.CombineOptions(opts.configOptions()...))
if err != nil {
return nil, err
}
Expand Down
10 changes: 4 additions & 6 deletions pkg/cli/debug_check_store_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,12 +81,10 @@ func TestDebugCheckStore(t *testing.T) {

// Introduce a stats divergence on s1.
func() {
eng, err := storage.NewDefaultEngine(
10<<20, /* 10mb */
base.StorageConfig{
Dir: storePaths[0],
MustExist: true,
})
eng, err := storage.Open(ctx,
storage.Filesystem(storePaths[0]),
storage.CacheSize(10<<20 /* 10 MiB */),
storage.MustExist)
require.NoError(t, err)
defer eng.Close()
sl := stateloader.Make(1)
Expand Down
16 changes: 4 additions & 12 deletions pkg/cli/debug_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,22 +39,14 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
)

func createStore(t *testing.T, path string) {
t.Helper()
cache := pebble.NewCache(server.DefaultCacheSize)
defer cache.Unref()
cfg := storage.PebbleConfig{
StorageConfig: base.StorageConfig{
Dir: path,
MustExist: false,
},
}
cfg.Opts = storage.DefaultPebbleOptions()
cfg.Opts.Cache = cache
db, err := storage.NewPebble(context.Background(), cfg)
db, err := storage.Open(
context.Background(),
storage.Filesystem(path),
storage.CacheSize(server.DefaultCacheSize))
if err != nil {
t.Fatal(err)
}
Expand Down
1 change: 0 additions & 1 deletion pkg/cli/syncbench/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,6 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/cli/syncbench",
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/settings/cluster",
"//pkg/storage",
"//pkg/util/encoding",
Expand Down
18 changes: 8 additions & 10 deletions pkg/cli/syncbench/syncbench.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"sync/atomic"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand Down Expand Up @@ -122,10 +121,10 @@ type Options struct {
LogOnly bool
}

// Run a test of writing synchronously to the RocksDB WAL.
// Run a test of writing synchronously to the Pebble WAL.
//
// TODO(tschottdorf): this should receive a RocksDB instance so that the caller
// in cli can use OpenEngine (which in turn allows to use encryption, etc).
// TODO(tschottdorf): this should receive an engine so that the caller in cli
// can use OpenEngine (which in turn allows to use encryption, etc).
func Run(opts Options) error {
// Check if the directory exists.
_, err := os.Stat(opts.Dir)
Expand All @@ -139,12 +138,11 @@ func Run(opts Options) error {

fmt.Printf("writing to %s\n", opts.Dir)

db, err := storage.NewDefaultEngine(
0,
base.StorageConfig{
Settings: cluster.MakeTestingClusterSettings(),
Dir: opts.Dir,
})
db, err := storage.Open(
context.Background(),
storage.Filesystem(opts.Dir),
storage.CacheSize(0),
storage.Settings(cluster.MakeTestingClusterSettings()))
if err != nil {
return err
}
Expand Down
1 change: 0 additions & 1 deletion pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -387,7 +387,6 @@ go_test(
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//oserror",
"@com_github_cockroachdb_logtags//:logtags",
"@com_github_cockroachdb_pebble//:pebble",
"@com_github_cockroachdb_redact//:redact",
"@com_github_gogo_protobuf//proto",
"@com_github_google_btree//:btree",
Expand Down
7 changes: 1 addition & 6 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,16 +37,11 @@ import (
"github.com/kr/pretty"
)

// createTestPebbleEngine returns a new in-memory Pebble storage engine.
func createTestPebbleEngine() storage.Engine {
return storage.NewInMemForTesting(context.Background(), roachpb.Attributes{}, 1<<20)
}

var engineImpls = []struct {
name string
create func() storage.Engine
}{
{"pebble", createTestPebbleEngine},
{"pebble", storage.NewDefaultInMemForTesting},
}

func singleKVSSTable(key storage.MVCCKey, value []byte) ([]byte, error) {
Expand Down
19 changes: 8 additions & 11 deletions pkg/kv/kvserver/batcheval/cmd_revert_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,22 +52,16 @@ func getStats(t *testing.T, reader storage.Reader) enginepb.MVCCStats {
return s
}

// createTestRocksDBEngine returns a new in-memory RocksDB engine with 1MB of
// storage capacity.
func createTestRocksDBEngine(ctx context.Context) storage.Engine {
return storage.NewInMemForTesting(ctx, roachpb.Attributes{}, 1<<20)
}

// createTestPebbleEngine returns a new in-memory Pebble storage engine.
func createTestPebbleEngine(ctx context.Context) storage.Engine {
return storage.NewInMemForTesting(ctx, roachpb.Attributes{}, 1<<20)
func createTestPebbleEngine(ctx context.Context) (storage.Engine, error) {
return storage.Open(ctx, storage.InMemory(),
storage.MaxSize(1<<20), storage.SettingsForTesting())
}

var engineImpls = []struct {
name string
create func(context.Context) storage.Engine
create func(context.Context) (storage.Engine, error)
}{
{"rocksdb", createTestRocksDBEngine},
{"pebble", createTestPebbleEngine},
}

Expand All @@ -85,7 +79,10 @@ func TestCmdRevertRange(t *testing.T) {
// https://github.com/cockroachdb/cockroach/pull/42386
for _, engineImpl := range engineImpls {
t.Run(engineImpl.name, func(t *testing.T) {
eng := engineImpl.create(ctx)
eng, err := engineImpl.create(ctx)
if err != nil {
t.Fatal(err)
}
defer eng.Close()

baseTime := hlc.Timestamp{WallTime: 1000}
Expand Down
18 changes: 5 additions & 13 deletions pkg/kv/kvserver/consistency_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/pebble"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -444,7 +443,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) {
// VFS to verify its contents.
fs, err := stickyEngineRegistry.GetUnderlyingFS(base.StoreSpec{StickyInMemoryEngineID: strconv.FormatInt(int64(i), 10)})
assert.NoError(t, err)
cpEng := storage.InMemFromFS(context.Background(), roachpb.Attributes{}, 1<<20, 0, fs, cps[0], nil)
cpEng := storage.InMemFromFS(context.Background(), fs, cps[0], storage.CacheSize(1<<20))
defer cpEng.Close()

iter := cpEng.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: []byte("\xff")})
Expand Down Expand Up @@ -577,17 +576,10 @@ func testConsistencyQueueRecomputeStatsImpl(t *testing.T, hadEstimates bool) {
const sysCountGarbage = 123000

func() {
cache := pebble.NewCache(1 << 20)
defer cache.Unref()
opts := storage.DefaultPebbleOptions()
opts.Cache = cache
eng, err := storage.NewPebble(ctx, storage.PebbleConfig{
StorageConfig: base.StorageConfig{
Dir: path,
MustExist: true,
},
Opts: opts,
})
eng, err := storage.Open(ctx,
storage.Filesystem(path),
storage.CacheSize(1<<20 /* 1 MiB */),
storage.MustExist)
if err != nil {
t.Fatal(err)
}
Expand Down
10 changes: 3 additions & 7 deletions pkg/kv/kvserver/raft_log_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand All @@ -33,7 +32,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
"github.com/stretchr/testify/assert"
"go.etcd.io/etcd/raft/v3"
"go.etcd.io/etcd/raft/v3/tracker"
Expand Down Expand Up @@ -862,11 +860,9 @@ func TestTruncateLogRecompute(t *testing.T) {
dir, cleanup := testutils.TempDir(t)
defer cleanup()

cache := pebble.NewCache(1 << 20)
defer cache.Unref()
opts := storage.DefaultPebbleOptions()
opts.Cache = cache
eng, err := storage.NewPebble(ctx, storage.PebbleConfig{StorageConfig: base.StorageConfig{Dir: dir}, Opts: opts})
eng, err := storage.Open(ctx,
storage.Filesystem(dir),
storage.CacheSize(1<<20 /* 1 MiB */))
if err != nil {
t.Fatal(err)
}
Expand Down
Loading

0 comments on commit 8455178

Please sign in to comment.