From aec0ab0ff703d65e4e25b1d43629fcb0999684e6 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Tue, 17 May 2022 13:40:54 -0400 Subject: [PATCH] vendor: bump Pebble to e567fec84c6e This commit includes non-trivial changes to account for the change in the `vfs.WithDiskHealthChecks` function signature change. ``` e567fec8 db: ensure Open closes opened directories on error b8c9a560 internal/metamorphic: overwrite unused bounds buffers 7c5f0cbb db: copy user-provided bounds and optimize unchanged bounds f8897076 *: Add IterOption to optionally read L6 filter blocks. d79f9617 vfs: extend disk-health checking to filesystem metadata operations 5ae21746 db: remove newRangeKeyIter closure 6d975489 db: add BenchmarkIteratorScan 782d102e sstable: fix invariant check for sstable size estimation 738a7f0b db: fix NewIter regression resulting in extra memtable levels 37558663 *: Use keyspan.LevelIter for rangedels in compactions e6c60c71 db: use sublevel level iters for all compactions out of L0 d8f63bef db: extend documentation on ingested file overlap; improve test cases b9e970a8 internal/keyspan: correct and document MergingIter key stability 498177bb internal/keyspan: collapse fragmentBoundIterator into MergingIter ``` Release note (bug fix): Fix a gap in disk-stall detection. Previously, disk stalls during filesystem metadata operations could go undetected, inducing deadlocks. Now stalls during these types of operations will correctly fatal the process. --- DEPS.bzl | 6 ++-- build/bazelutil/distdir_files.bzl | 2 +- go.mod | 2 +- go.sum | 4 +-- pkg/kv/kvserver/client_metrics_test.go | 12 ++++--- pkg/server/config.go | 1 + pkg/storage/disk_map_test.go | 1 - pkg/storage/engine_test.go | 20 +++++------ pkg/storage/open.go | 19 +++++----- pkg/storage/pebble.go | 49 +++++++++++++++++++++----- pkg/storage/temp_engine.go | 16 ++++++--- vendor | 2 +- 12 files changed, 86 insertions(+), 48 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 77e1c9a60df8..ddb322c68332 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -1347,10 +1347,10 @@ def go_deps(): patches = [ "@com_github_cockroachdb_cockroach//build/patches:com_github_cockroachdb_pebble.patch", ], - sha256 = "dfa5ce136f7d8d40ddf24077323df27de81f0e1889c1058e4453c77cd8c2cb75", - strip_prefix = "github.com/cockroachdb/pebble@v0.0.0-20220426173801-b33d6e173cae", + sha256 = "65c359674e777445a63c2268e62d8fc740992c1aa86f042a07344371ba01e46b", + strip_prefix = "github.com/cockroachdb/pebble@v0.0.0-20220517003944-e567fec84c6e", urls = [ - "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220426173801-b33d6e173cae.zip", + "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220517003944-e567fec84c6e.zip", ], ) go_repository( diff --git a/build/bazelutil/distdir_files.bzl b/build/bazelutil/distdir_files.bzl index a695791587e9..60997e35127c 100644 --- a/build/bazelutil/distdir_files.bzl +++ b/build/bazelutil/distdir_files.bzl @@ -179,7 +179,7 @@ DISTDIR_FILES = { "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/go-test-teamcity/com_github_cockroachdb_go_test_teamcity-v0.0.0-20191211140407-cff980ad0a55.zip": "bac30148e525b79d004da84d16453ddd2d5cd20528e9187f1d7dac708335674b", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/gostdlib/com_github_cockroachdb_gostdlib-v1.13.0.zip": "b3d43d8f95edf65f73a5348f29e1159823cac64b148f8d3bb48340bf55d70872", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/logtags/com_github_cockroachdb_logtags-v0.0.0-20211118104740-dabe8e521a4f.zip": "1972c3f171f118add3fd9e64bcea6cbb9959a3b7fa0ada308e8a7310813fea74", - "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220426173801-b33d6e173cae.zip": "dfa5ce136f7d8d40ddf24077323df27de81f0e1889c1058e4453c77cd8c2cb75", + "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/pebble/com_github_cockroachdb_pebble-v0.0.0-20220517003944-e567fec84c6e.zip": "65c359674e777445a63c2268e62d8fc740992c1aa86f042a07344371ba01e46b", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/redact/com_github_cockroachdb_redact-v1.1.3.zip": "7778b1e4485e4f17f35e5e592d87eb99c29e173ac9507801d000ad76dd0c261e", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/returncheck/com_github_cockroachdb_returncheck-v0.0.0-20200612231554-92cdbca611dd.zip": "ce92ba4352deec995b1f2eecf16eba7f5d51f5aa245a1c362dfe24c83d31f82b", "https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/sentry-go/com_github_cockroachdb_sentry_go-v0.6.1-cockroachdb.2.zip": "fbb2207d02aecfdd411b1357efe1192dbb827959e36b7cab7491731ac55935c9", diff --git a/go.mod b/go.mod index f38b10b9db00..4a18cd6dc432 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55 github.com/cockroachdb/gostdlib v1.13.0 github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f - github.com/cockroachdb/pebble v0.0.0-20220426173801-b33d6e173cae + github.com/cockroachdb/pebble v0.0.0-20220517003944-e567fec84c6e github.com/cockroachdb/redact v1.1.3 github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd github.com/cockroachdb/stress v0.0.0-20220310203902-58fb4627376e diff --git a/go.sum b/go.sum index 90fb6b96e6cb..7fdf03c60f06 100644 --- a/go.sum +++ b/go.sum @@ -453,8 +453,8 @@ github.com/cockroachdb/gostdlib v1.13.0/go.mod h1:eXX95p9QDrYwJfJ6AgeN9QnRa/lqqi github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f h1:6jduT9Hfc0njg5jJ1DdKCFPdMBrp/mdZfCpa5h+WM74= github.com/cockroachdb/logtags v0.0.0-20211118104740-dabe8e521a4f/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= -github.com/cockroachdb/pebble v0.0.0-20220426173801-b33d6e173cae h1:7lGpwt2wTBh7FApXTEdTDX5OFWsEg9CCe8wMlHJZDwA= -github.com/cockroachdb/pebble v0.0.0-20220426173801-b33d6e173cae/go.mod h1:buxOO9GBtOcq1DiXDpIPYrmxY020K2A8lOrwno5FetU= +github.com/cockroachdb/pebble v0.0.0-20220517003944-e567fec84c6e h1:PU73bIcAcMerOI+xzYa4f3Grrd4I5cxO2ffT9+OcRt0= +github.com/cockroachdb/pebble v0.0.0-20220517003944-e567fec84c6e/go.mod h1:buxOO9GBtOcq1DiXDpIPYrmxY020K2A8lOrwno5FetU= github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= github.com/cockroachdb/redact v1.1.3/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= diff --git a/pkg/kv/kvserver/client_metrics_test.go b/pkg/kv/kvserver/client_metrics_test.go index 28575385104d..a11bdba9a7b5 100644 --- a/pkg/kv/kvserver/client_metrics_test.go +++ b/pkg/kv/kvserver/client_metrics_test.go @@ -132,11 +132,15 @@ func verifyStats(t *testing.T, tc *testcluster.TestCluster, storeIdxSlice ...int } } -func verifyRocksDBStats(t *testing.T, s *kvserver.Store) { +func verifyStorageStats(t *testing.T, s *kvserver.Store) { if err := s.ComputeMetrics(context.Background(), 0); err != nil { t.Fatal(err) } + // TODO(jackson): Adjust the test to ensure there are non-L6 files, and + // assert that RdbBloomFilter{PrefixChecked,PrefixUseful} are nonzero. Since + // filters are not consulted for L6 files, TestStoreMetrics is flaky if we + // assert that the bloom filter is consulted. m := s.Metrics() testcases := []struct { gauge *metric.Gauge @@ -146,8 +150,6 @@ func verifyRocksDBStats(t *testing.T, s *kvserver.Store) { {m.RdbBlockCacheMisses, 0}, {m.RdbBlockCacheUsage, 0}, {m.RdbBlockCachePinnedUsage, 0}, - {m.RdbBloomFilterPrefixChecked, 20}, - {m.RdbBloomFilterPrefixUseful, 20}, {m.RdbMemtableTotalSize, 5000}, {m.RdbFlushes, 1}, {m.RdbCompactions, 0}, @@ -356,8 +358,8 @@ func TestStoreMetrics(t *testing.T) { // Verify all stats on all stores after range is removed. verifyStats(t, tc, 1, 2) - verifyRocksDBStats(t, tc.GetFirstStoreFromServer(t, 1)) - verifyRocksDBStats(t, tc.GetFirstStoreFromServer(t, 2)) + verifyStorageStats(t, tc.GetFirstStoreFromServer(t, 1)) + verifyStorageStats(t, tc.GetFirstStoreFromServer(t, 2)) } // TestStoreMaxBehindNanosOnlyTracksEpochBasedLeases ensures that the metric diff --git a/pkg/server/config.go b/pkg/server/config.go index ce427f3f2f4b..4d6af93ffc8a 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -584,6 +584,7 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) { storage.CacheSize(cfg.CacheSize), storage.MaxSize(sizeInBytes), storage.EncryptionAtRest(spec.EncryptionOptions), + storage.DisableFilesystemMiddlewareTODO, storage.Settings(cfg.Settings)) if err != nil { return Engines{}, err diff --git a/pkg/storage/disk_map_test.go b/pkg/storage/disk_map_test.go index 4bc4bade791d..afde670d624d 100644 --- a/pkg/storage/disk_map_test.go +++ b/pkg/storage/disk_map_test.go @@ -179,7 +179,6 @@ func TestPebbleMap(t *testing.T) { defer e.Close() runTestForEngine(ctx, t, testutils.TestDataPath(t, "diskmap"), e) - } func TestPebbleMultiMap(t *testing.T) { diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 82fdf731ca1d..03fec4795853 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -574,20 +574,16 @@ func TestEngineMustExist(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - test := func(errStr string) { - tempDir, dirCleanupFn := testutils.TempDir(t) - defer dirCleanupFn() + tempDir, dirCleanupFn := testutils.TempDir(t) + defer dirCleanupFn() - _, err := Open(context.Background(), Filesystem(tempDir), MustExist) - if err == nil { - t.Fatal("expected error related to missing directory") - } - if !strings.Contains(fmt.Sprint(err), errStr) { - t.Fatal(err) - } + _, err := Open(context.Background(), Filesystem(tempDir), MustExist) + if err == nil { + t.Fatal("expected error related to missing directory") + } + if !strings.Contains(fmt.Sprint(err), "no such file or directory") { + t.Fatal(err) } - - test("no such file or directory") } func TestEngineTimeBound(t *testing.T) { diff --git a/pkg/storage/open.go b/pkg/storage/open.go index e5943a0441ca..90fef7a5cc66 100644 --- a/pkg/storage/open.go +++ b/pkg/storage/open.go @@ -55,6 +55,15 @@ var DisableAutomaticCompactions ConfigOption = func(cfg *engineConfig) error { return nil } +// DisableFilesystemMiddlewareTODO configures an engine to not include +// filesystem middleware like disk-health checking and ENOSPC-detection. This is +// a temporary option while some units leak file descriptors, and by extension, +// disk-health checking goroutines. +var DisableFilesystemMiddlewareTODO = func(cfg *engineConfig) error { + cfg.DisableFilesystemMiddlewareTODO = true + return nil +} + // ForTesting configures the engine for use in testing. It may randomize some // config options to improve test coverage. var ForTesting ConfigOption = func(cfg *engineConfig) error { @@ -164,11 +173,7 @@ type Location struct { func Filesystem(dir string) Location { return Location{ dir: dir, - // fs is left nil intentionally, so that it will be left as the - // default of vfs.Default wrapped in vfs.WithDiskHealthChecks - // (initialized by DefaultPebbleOptions). - // TODO(jackson): Refactor to make it harder to accidentally remove - // disk health checks by setting your own VFS in a call to NewPebble. + fs: vfs.Default, } } @@ -196,9 +201,7 @@ func Open(ctx context.Context, loc Location, opts ...ConfigOption) (*Pebble, err var cfg engineConfig cfg.Dir = loc.dir cfg.Opts = DefaultPebbleOptions() - if loc.fs != nil { - cfg.Opts.FS = loc.fs - } + cfg.Opts.FS = loc.fs for _, opt := range opts { if err := opt(&cfg); err != nil { return nil, err diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index f28bb73e732b..7011a28a511e 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -579,6 +579,7 @@ func DefaultPebbleOptions() *pebble.Options { opts := &pebble.Options{ Comparer: EngineComparer, + FS: vfs.Default, L0CompactionThreshold: 2, L0StopWritesThreshold: 1000, LBaseMaxBytes: 64 << 20, // 64 MB @@ -635,8 +636,15 @@ func DefaultPebbleOptions() *pebble.Options { // of the benefit of having bloom filters on every level for only 10% of the // memory cost. opts.Levels[6].FilterPolicy = nil + return opts +} - // Set disk health check interval to min(5s, maxSyncDurationDefault). This +// wrapFilesystemMiddleware wraps the Option's vfs.FS with disk-health checking +// and ENOSPC detection. It mutates the provided options to set the FS and +// returns a Closer that should be invoked when the filesystem will no longer be +// used. +func wrapFilesystemMiddleware(opts *pebble.Options) io.Closer { + // Set disk-health check interval to min(5s, maxSyncDurationDefault). This // is mostly to ease testing; the default of 5s is too infrequent to test // conveniently. See the disk-stalled roachtest for an example of how this // is used. @@ -644,9 +652,11 @@ func DefaultPebbleOptions() *pebble.Options { if diskHealthCheckInterval.Seconds() > maxSyncDurationDefault.Seconds() { diskHealthCheckInterval = maxSyncDurationDefault } - // Instantiate a file system with disk health checking enabled. This FS wraps - // vfs.Default, and can be wrapped for encryption-at-rest. - opts.FS = vfs.WithDiskHealthChecks(vfs.Default, diskHealthCheckInterval, + // Instantiate a file system with disk health checking enabled. This FS + // wraps the filesystem with a layer that times all write-oriented + // operations. + var closer io.Closer + opts.FS, closer = vfs.WithDiskHealthChecks(opts.FS, diskHealthCheckInterval, func(name string, duration time.Duration) { opts.EventListener.DiskSlow(pebble.DiskSlowInfo{ Path: name, @@ -657,7 +667,7 @@ func DefaultPebbleOptions() *pebble.Options { opts.FS = vfs.OnDiskFull(opts.FS, func() { exit.WithCode(exit.DiskFull()) }) - return opts + return closer } type pebbleLogger struct { @@ -682,6 +692,9 @@ type PebbleConfig struct { base.StorageConfig // Pebble specific options. Opts *pebble.Options + // Temporary option while there exist file descriptor leaks. See the + // DisableFilesystemMiddlewareTODO ConfigOption that sets this, and #81389. + DisableFilesystemMiddlewareTODO bool } // EncryptionStatsHandler provides encryption related stats. @@ -733,6 +746,9 @@ type Pebble struct { syncutil.Mutex flushCompletedCallback func() } + // closer is populated when the database is opened. The closer is associated + // with the filesyetem + closer io.Closer wrappedIntentWriter intentDemuxWriter @@ -825,13 +841,26 @@ func ResolveEncryptedEnvOptions(cfg *PebbleConfig) (*PebbleFileRegistry, *Encryp } // NewPebble creates a new Pebble instance, at the specified path. -func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { +func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { // pebble.Open also calls EnsureDefaults, but only after doing a clone. Call // EnsureDefaults beforehand so we have a matching cfg here for when we save // cfg.FS and cfg.ReadOnly later on. if cfg.Opts == nil { cfg.Opts = DefaultPebbleOptions() } + + // Initialize the FS, wrapping it with disk health-checking and + // ENOSPC-detection. + var filesystemCloser io.Closer + if !cfg.DisableFilesystemMiddlewareTODO { + filesystemCloser = wrapFilesystemMiddleware(cfg.Opts) + defer func() { + if err != nil { + filesystemCloser.Close() + } + }() + } + cfg.Opts.EnsureDefaults() cfg.Opts.ErrorIfNotExists = cfg.MustExist if settings := cfg.Settings; settings != nil { @@ -853,8 +882,6 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { // FS for those that need it. Some call sites need the unencrypted // FS for the purpose of atomic renames. unencryptedFS := cfg.Opts.FS - // TODO(jackson): Assert that unencryptedFS provides atomic renames. - fileRegistry, env, err := ResolveEncryptedEnvOptions(&cfg) if err != nil { return nil, err @@ -899,7 +926,7 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { storeProps := computeStoreProperties(ctx, cfg.Dir, cfg.Opts.ReadOnly, env != nil /* encryptionEnabled */) - p := &Pebble{ + p = &Pebble{ readOnly: cfg.Opts.ReadOnly, path: cfg.Dir, auxDir: auxDir, @@ -915,6 +942,7 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { unencryptedFS: unencryptedFS, logger: cfg.Opts.Logger, storeIDPebbleLog: storeIDContainer, + closer: filesystemCloser, } cfg.Opts.EventListener = pebble.TeeEventListener( pebble.MakeLoggingEventListener(pebbleLogger{ @@ -1030,6 +1058,9 @@ func (p *Pebble) Close() { if p.encryption != nil { _ = p.encryption.Closer.Close() } + if p.closer != nil { + _ = p.closer.Close() + } } // Closed implements the Engine interface. diff --git a/pkg/storage/temp_engine.go b/pkg/storage/temp_engine.go index e5fadfada634..bbdae5baf9ca 100644 --- a/pkg/storage/temp_engine.go +++ b/pkg/storage/temp_engine.go @@ -12,6 +12,7 @@ package storage import ( "context" + "io" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/diskmap" @@ -29,13 +30,16 @@ func NewTempEngine( } type pebbleTempEngine struct { - db *pebble.DB + db *pebble.DB + closer io.Closer } // Close implements the diskmap.Factory interface. func (r *pebbleTempEngine) Close() { - err := r.db.Close() - if err != nil { + if err := r.db.Close(); err != nil { + log.Fatalf(context.TODO(), "%v", err) + } + if err := r.closer.Close(); err != nil { log.Fatalf(context.TODO(), "%v", err) } } @@ -93,6 +97,8 @@ func newPebbleTempEngine( // Set store ID for the pebble engine. p.SetStoreID(ctx, base.TempStoreID) - - return &pebbleTempEngine{db: p.db}, p, nil + return &pebbleTempEngine{ + db: p.db, + closer: p.closer, + }, p, nil } diff --git a/vendor b/vendor index 73c75fbaf35e..c404bd45895f 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit 73c75fbaf35e68220c2f5d3a2acfed24dcd9fbef +Subproject commit c404bd45895f1a010fb547a4f6c833d392fc4062