diff --git a/checkpoint.go b/checkpoint.go index 62f4b6e060..fb42114914 100644 --- a/checkpoint.go +++ b/checkpoint.go @@ -5,6 +5,7 @@ package pebble import ( + "bytes" "io" "os" @@ -239,10 +240,10 @@ func (d *DB) Checkpoint( } { - // Link or copy the OPTIONS. + // Copy the OPTIONS. srcPath := base.MakeFilepath(fs, d.dirname, fileTypeOptions, optionsFileNum) destPath := fs.PathJoin(destDir, fs.PathBase(srcPath)) - ckErr = vfs.LinkOrCopy(fs, srcPath, destPath) + ckErr = copyCheckpointOptions(fs, srcPath, destPath) if ckErr != nil { return ckErr } @@ -373,6 +374,58 @@ func (d *DB) Checkpoint( return ckErr } +// copyCheckpointOptions copies an OPTIONS file, commenting out some options +// that existed on the original database but no longer apply to the checkpointed +// database. For example, the entire [WAL Failover] stanza is commented out +// because Checkpoint will copy all WAL segment files from both the primary and +// secondary WAL directories into the checkpoint. +func copyCheckpointOptions(fs vfs.FS, srcPath, dstPath string) error { + var buf bytes.Buffer + f, err := fs.Open(srcPath) + if err != nil { + return err + } + defer f.Close() + b, err := io.ReadAll(f) + if err != nil { + return err + } + // Copy the OPTIONS file verbatim, but commenting out the [WAL Failover] + // section. + err = parseOptions(string(b), parseOptionsFuncs{ + visitNewSection: func(startOff, endOff int, section string) error { + if section == "WAL Failover" { + buf.WriteString("# ") + } + buf.Write(b[startOff:endOff]) + return nil + }, + visitKeyValue: func(startOff, endOff int, section, key, value string) error { + if section == "WAL Failover" { + buf.WriteString("# ") + } + buf.Write(b[startOff:endOff]) + return nil + }, + visitCommentOrWhitespace: func(startOff, endOff int, line string) error { + buf.Write(b[startOff:endOff]) + return nil + }, + }) + if err != nil { + return err + } + nf, err := fs.Create(dstPath, vfs.WriteCategoryUnspecified) + if err != nil { + return err + } + _, err = io.Copy(nf, &buf) + if err != nil { + return err + } + return errors.CombineErrors(nf.Sync(), nf.Close()) +} + func (d *DB) writeCheckpointManifest( fs vfs.FS, formatVers FormatMajorVersion, diff --git a/checkpoint_test.go b/checkpoint_test.go index b688ac7387..112d37b738 100644 --- a/checkpoint_test.go +++ b/checkpoint_test.go @@ -8,6 +8,7 @@ import ( "bytes" "context" "fmt" + "io" "math/rand/v2" "runtime" "sort" @@ -35,22 +36,25 @@ func testCheckpointImpl(t *testing.T, ddFile string, createOnShared bool) { mem := vfs.NewMem() var memLog base.InMemLogger remoteMem := remote.NewInMem() - opts := &Options{ - FS: vfs.WithLogging(mem, memLog.Infof), - FormatMajorVersion: internalFormatNewest, - L0CompactionThreshold: 10, - DisableAutomaticCompactions: true, - Logger: testLogger{t}, - } - opts.Experimental.EnableColumnarBlocks = func() bool { return true } - opts.Experimental.RemoteStorage = remote.MakeSimpleFactory(map[remote.Locator]remote.Storage{ - "": remoteMem, - }) - if createOnShared { - opts.Experimental.CreateOnShared = remote.CreateOnSharedAll + makeOptions := func() *Options { + opts := &Options{ + FS: vfs.WithLogging(mem, memLog.Infof), + FormatMajorVersion: internalFormatNewest, + L0CompactionThreshold: 10, + DisableAutomaticCompactions: true, + Logger: testLogger{t}, + } + opts.Experimental.EnableColumnarBlocks = func() bool { return true } + opts.Experimental.RemoteStorage = remote.MakeSimpleFactory(map[remote.Locator]remote.Storage{ + "": remoteMem, + }) + if createOnShared { + opts.Experimental.CreateOnShared = remote.CreateOnSharedAll + } + opts.DisableTableStats = true + opts.private.testingAlwaysWaitForCleanup = true + return opts } - opts.DisableTableStats = true - opts.private.testingAlwaysWaitForCleanup = true datadriven.RunTest(t, ddFile, func(t *testing.T, td *datadriven.TestData) string { switch td.Cmd { @@ -70,7 +74,7 @@ func testCheckpointImpl(t *testing.T, ddFile string, createOnShared bool) { return memLog.String() case "checkpoint": - if !(len(td.CmdArgs) == 2 || (len(td.CmdArgs) == 3 && td.CmdArgs[2].Key == "restrict")) { + if len(td.CmdArgs) < 2 { return "checkpoint [restrict=(start-end, ...)]" } var opts []CheckpointOption @@ -93,6 +97,10 @@ func testCheckpointImpl(t *testing.T, ddFile string, createOnShared bool) { if err := d.Checkpoint(td.CmdArgs[1].String(), opts...); err != nil { return err.Error() } + if td.HasArg("nondeterministic") { + memLog.Reset() + return "" + } return memLog.String() case "ingest-and-excise": @@ -184,19 +192,19 @@ func testCheckpointImpl(t *testing.T, ddFile string, createOnShared bool) { return fmt.Sprintf("%s\n", strings.Join(paths, "\n")) case "open": - if len(td.CmdArgs) != 1 && len(td.CmdArgs) != 2 { + if len(td.CmdArgs) < 1 { return "open [readonly]" } - opts.ReadOnly = false - if len(td.CmdArgs) == 2 { - if td.CmdArgs[1].String() != "readonly" { - return "open [readonly]" - } - opts.ReadOnly = true - } + opts := makeOptions() + require.NoError(t, parseDBOptionsArgs(opts, td.CmdArgs[1:])) memLog.Reset() dir := td.CmdArgs[0].String() + if _, ok := dbs[dir]; ok { + require.NoError(t, dbs[dir].Close()) + dbs[dir] = nil + } + d, err := Open(dir, opts) if err != nil { return err.Error() @@ -208,6 +216,12 @@ func testCheckpointImpl(t *testing.T, ddFile string, createOnShared bool) { return err.Error() } } + waitForCompactionsAndTableStats(d) + + if td.HasArg("nondeterministic") { + memLog.Reset() + return "" + } return memLog.String() case "scan": @@ -232,6 +246,33 @@ func testCheckpointImpl(t *testing.T, ddFile string, createOnShared bool) { }) } +func TestCopyCheckpointOptions(t *testing.T) { + fs := vfs.NewMem() + datadriven.RunTest(t, "testdata/copy_checkpoint_options", func(t *testing.T, td *datadriven.TestData) string { + switch td.Cmd { + case "copy": + f, err := fs.Create("old", vfs.WriteCategoryUnspecified) + require.NoError(t, err) + _, err = io.WriteString(f, td.Input) + require.NoError(t, err) + require.NoError(t, f.Close()) + + if err := copyCheckpointOptions(fs, "old", "new"); err != nil { + return err.Error() + } + + f, err = fs.Open("new") + require.NoError(t, err) + newFile, err := io.ReadAll(f) + require.NoError(t, err) + require.NoError(t, f.Close()) + return string(newFile) + default: + panic(fmt.Sprintf("unrecognized command %q", td.Cmd)) + } + }) +} + func TestCheckpoint(t *testing.T) { t.Run("shared=false", func(t *testing.T) { testCheckpointImpl(t, "testdata/checkpoint", false /* createOnShared */) diff --git a/data_test.go b/data_test.go index b6bfb5a511..c43aff8e1e 100644 --- a/data_test.go +++ b/data_test.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/pebble/sstable" "github.com/cockroachdb/pebble/vfs" "github.com/cockroachdb/pebble/vfs/errorfs" + "github.com/cockroachdb/pebble/wal" "github.com/ghemawat/stream" "github.com/stretchr/testify/require" ) @@ -1473,6 +1474,8 @@ func parseDBOptionsArgs(opts *Options, args []datadriven.CmdArg) error { default: return errors.Newf("unrecognized Merger %q\n", cmdArg.Vals[0]) } + case "readonly": + opts.ReadOnly = true case "target-file-sizes": if len(opts.Levels) < len(cmdArg.Vals) { opts.Levels = slices.Grow(opts.Levels, len(cmdArg.Vals)-len(opts.Levels))[0:len(cmdArg.Vals)] @@ -1484,6 +1487,15 @@ func parseDBOptionsArgs(opts *Options, args []datadriven.CmdArg) error { } opts.Levels[i].TargetFileSize = size } + case "wal-failover": + if v := cmdArg.Vals[0]; v == "off" || v == "disabled" { + opts.WALFailover = nil + continue + } + opts.WALFailover = &WALFailoverOptions{ + Secondary: wal.Dir{FS: opts.FS, Dirname: cmdArg.Vals[0]}, + } + opts.WALFailover.EnsureDefaults() } } return nil diff --git a/open.go b/open.go index 31e344f441..2ab0b6cde1 100644 --- a/open.go +++ b/open.go @@ -743,17 +743,19 @@ func GetVersion(dir string, fs vfs.FS) (string, error) { if err != nil { return "", err } - err = parseOptions(string(data), func(section, key, value string) error { - switch { - case section == "Version": - switch key { - case "pebble_version": - version = value - case "rocksdb_version": - version = fmt.Sprintf("rocksdb v%s", value) + err = parseOptions(string(data), parseOptionsFuncs{ + visitKeyValue: func(i, j int, section, key, value string) error { + switch { + case section == "Version": + switch key { + case "pebble_version": + version = value + case "rocksdb_version": + version = fmt.Sprintf("rocksdb v%s", value) + } } - } - return nil + return nil + }, }) if err != nil { return "", err diff --git a/options.go b/options.go index e871e9ced7..cd8215622b 100644 --- a/options.go +++ b/options.go @@ -1546,24 +1546,56 @@ func (o *Options) String() string { return buf.String() } -// parseOptions takes options serialized by Options.String() and parses them into -// keys and values, calling fn for each one. -func parseOptions(s string, fn func(section, key, value string) error) error { - var section string - for _, line := range strings.Split(s, "\n") { - line = strings.TrimSpace(line) - if len(line) == 0 { - // Skip blank lines. - continue +type parseOptionsFuncs struct { + visitNewSection func(i, j int, section string) error + visitKeyValue func(i, j int, section, key, value string) error + visitCommentOrWhitespace func(i, j int, whitespace string) error +} + +// parseOptions takes options serialized by Options.String() and parses them +// into keys and values. It calls fns.visitNewSection for the beginning of each +// new section, fns.visitKeyValue for each key-value pair, and +// visitCommentOrWhitespace for comments and whitespace between key-value pairs. +func parseOptions(s string, fns parseOptionsFuncs) error { + var section, mappedSection string + i := 0 + for i < len(s) { + rem := s[i:] + j := strings.IndexByte(rem, '\n') + if j < 0 { + j = len(rem) + } else { + j += 1 // Include the newline. } - if line[0] == ';' || line[0] == '#' { - // Skip comments. + line := strings.TrimSpace(s[i : i+j]) + startOff, endOff := i, i+j + i += j + + if len(line) == 0 || line[0] == ';' || line[0] == '#' { + // Skip blank lines and comments. + if fns.visitCommentOrWhitespace != nil { + if err := fns.visitCommentOrWhitespace(startOff, endOff, line); err != nil { + return err + } + } continue } n := len(line) if line[0] == '[' && line[n-1] == ']' { // Parse section. section = line[1 : n-1] + // RocksDB uses a similar (INI-style) syntax for the OPTIONS file, but + // different section names and keys. The "CFOptions ..." paths are the + // RocksDB versions which we map to the Pebble paths. + mappedSection = section + if section == `CFOptions "default"` { + mappedSection = "Options" + } + if fns.visitNewSection != nil { + if err := fns.visitNewSection(startOff, endOff, mappedSection); err != nil { + return err + } + } continue } @@ -1579,12 +1611,7 @@ func parseOptions(s string, fn func(section, key, value string) error) error { key := strings.TrimSpace(line[:pos]) value := strings.TrimSpace(line[pos+1:]) - // RocksDB uses a similar (INI-style) syntax for the OPTIONS file, but - // different section names and keys. The "CFOptions ..." paths are the - // RocksDB versions which we map to the Pebble paths. - mappedSection := section if section == `CFOptions "default"` { - mappedSection = "Options" switch key { case "comparator": key = "comparer" @@ -1592,9 +1619,10 @@ func parseOptions(s string, fn func(section, key, value string) error) error { key = "merger" } } - - if err := fn(mappedSection, key, value); err != nil { - return err + if fns.visitKeyValue != nil { + if err := fns.visitKeyValue(startOff, endOff, mappedSection, key, value); err != nil { + return err + } } } return nil @@ -1616,7 +1644,7 @@ type ParseHooks struct { // options cannot be parsed into populated fields. For example, comparer and // merger. func (o *Options) Parse(s string, hooks *ParseHooks) error { - return parseOptions(s, func(section, key, value string) error { + visitKeyValue := func(i, j int, section, key, value string) error { // WARNING: DO NOT remove entries from the switches below because doing so // causes a key previously written to the OPTIONS file to be considered unknown, // a backwards incompatible change. Instead, leave in support for parsing the @@ -1994,6 +2022,9 @@ func (o *Options) Parse(s string, hooks *ParseHooks) error { return nil } return errors.Errorf("pebble: unknown section: %q", errors.Safe(section)) + } + return parseOptions(s, parseOptionsFuncs{ + visitKeyValue: visitKeyValue, }) } @@ -2016,7 +2047,7 @@ func (e ErrMissingWALRecoveryDir) Error() string { // This function only looks at specific keys and does not error out if the // options are newer and contain unknown keys. func (o *Options) CheckCompatibility(previousOptions string) error { - return parseOptions(previousOptions, func(section, key, value string) error { + visitKeyValue := func(i, j int, section, key, value string) error { switch section + "." + key { case "Options.comparer": if value != o.Comparer.Name { @@ -2046,7 +2077,8 @@ func (o *Options) CheckCompatibility(previousOptions string) error { } } return nil - }) + } + return parseOptions(previousOptions, parseOptionsFuncs{visitKeyValue: visitKeyValue}) } // Validate verifies that the options are mutually consistent. For example, diff --git a/testdata/checkpoint b/testdata/checkpoint index 22e5b7399d..9140cec42d 100644 --- a/testdata/checkpoint +++ b/testdata/checkpoint @@ -103,7 +103,11 @@ open-dir: . sync: . close: . open-dir: checkpoints/checkpoint1 -link: db/OPTIONS-000003 -> checkpoints/checkpoint1/OPTIONS-000003 +open: db/OPTIONS-000003 +create: checkpoints/checkpoint1/OPTIONS-000003 +sync-data: checkpoints/checkpoint1/OPTIONS-000003 +close: checkpoints/checkpoint1/OPTIONS-000003 +close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint1 create: checkpoints/checkpoint1/marker.format-version.000001.019 sync-data: checkpoints/checkpoint1/marker.format-version.000001.019 @@ -143,7 +147,11 @@ open-dir: checkpoints sync: checkpoints close: checkpoints open-dir: checkpoints/checkpoint2 -link: db/OPTIONS-000003 -> checkpoints/checkpoint2/OPTIONS-000003 +open: db/OPTIONS-000003 +create: checkpoints/checkpoint2/OPTIONS-000003 +sync-data: checkpoints/checkpoint2/OPTIONS-000003 +close: checkpoints/checkpoint2/OPTIONS-000003 +close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint2 create: checkpoints/checkpoint2/marker.format-version.000001.019 sync-data: checkpoints/checkpoint2/marker.format-version.000001.019 @@ -178,7 +186,11 @@ open-dir: checkpoints sync: checkpoints close: checkpoints open-dir: checkpoints/checkpoint3 -link: db/OPTIONS-000003 -> checkpoints/checkpoint3/OPTIONS-000003 +open: db/OPTIONS-000003 +create: checkpoints/checkpoint3/OPTIONS-000003 +sync-data: checkpoints/checkpoint3/OPTIONS-000003 +close: checkpoints/checkpoint3/OPTIONS-000003 +close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint3 create: checkpoints/checkpoint3/marker.format-version.000001.019 sync-data: checkpoints/checkpoint3/marker.format-version.000001.019 @@ -498,7 +510,11 @@ open-dir: checkpoints sync: checkpoints close: checkpoints open-dir: checkpoints/checkpoint4 -link: db/OPTIONS-000003 -> checkpoints/checkpoint4/OPTIONS-000003 +open: db/OPTIONS-000003 +create: checkpoints/checkpoint4/OPTIONS-000003 +sync-data: checkpoints/checkpoint4/OPTIONS-000003 +close: checkpoints/checkpoint4/OPTIONS-000003 +close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint4 create: checkpoints/checkpoint4/marker.format-version.000001.019 sync-data: checkpoints/checkpoint4/marker.format-version.000001.019 @@ -603,7 +619,11 @@ open-dir: checkpoints sync: checkpoints close: checkpoints open-dir: checkpoints/checkpoint5 -link: db/OPTIONS-000003 -> checkpoints/checkpoint5/OPTIONS-000003 +open: db/OPTIONS-000003 +create: checkpoints/checkpoint5/OPTIONS-000003 +sync-data: checkpoints/checkpoint5/OPTIONS-000003 +close: checkpoints/checkpoint5/OPTIONS-000003 +close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint5 create: checkpoints/checkpoint5/marker.format-version.000001.019 sync-data: checkpoints/checkpoint5/marker.format-version.000001.019 @@ -701,7 +721,11 @@ open-dir: checkpoints sync: checkpoints close: checkpoints open-dir: checkpoints/checkpoint6 -link: db/OPTIONS-000003 -> checkpoints/checkpoint6/OPTIONS-000003 +open: db/OPTIONS-000003 +create: checkpoints/checkpoint6/OPTIONS-000003 +sync-data: checkpoints/checkpoint6/OPTIONS-000003 +close: checkpoints/checkpoint6/OPTIONS-000003 +close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint6 create: checkpoints/checkpoint6/marker.format-version.000001.019 sync-data: checkpoints/checkpoint6/marker.format-version.000001.019 @@ -782,3 +806,29 @@ L6: 000015(000011):[i#20,SET-i#20,SET] 000016(000011):[k#20,SET-k#20,SET] 000014:[z#22,SET-z#22,SET] + +# Close and re-open db, this time with WAL failover configured and the secondary +# pointing to /failover-logs. + +open db wal-failover=/failover-logs nondeterministic +---- + +# Perform a checkpoint. +checkpoint db checkpoints/checkpoint7 nondeterministic +---- + +# Validate that we can open the checkpoint. + +open checkpoints/checkpoint7 readonly nondeterministic +---- + +lsm checkpoints/checkpoint7 +---- +L0.0: + 000018:[h#18,SET-h#18,SET] +L6: + 000012(000010):[a#0,SET-b#0,SET] + 000013(000010):[d#0,SET-g#0,SET] + 000015(000011):[i#20,SET-i#20,SET] + 000016(000011):[k#20,SET-k#20,SET] + 000014:[z#22,SET-z#22,SET] diff --git a/testdata/checkpoint_shared b/testdata/checkpoint_shared index 9a0024e5d1..9836137967 100644 --- a/testdata/checkpoint_shared +++ b/testdata/checkpoint_shared @@ -91,7 +91,11 @@ open-dir: . sync: . close: . open-dir: checkpoints/checkpoint1 -link: db/OPTIONS-000003 -> checkpoints/checkpoint1/OPTIONS-000003 +open: db/OPTIONS-000003 +create: checkpoints/checkpoint1/OPTIONS-000003 +sync-data: checkpoints/checkpoint1/OPTIONS-000003 +close: checkpoints/checkpoint1/OPTIONS-000003 +close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint1 create: checkpoints/checkpoint1/marker.format-version.000001.019 sync-data: checkpoints/checkpoint1/marker.format-version.000001.019 @@ -140,7 +144,11 @@ open-dir: checkpoints sync: checkpoints close: checkpoints open-dir: checkpoints/checkpoint2 -link: db/OPTIONS-000003 -> checkpoints/checkpoint2/OPTIONS-000003 +open: db/OPTIONS-000003 +create: checkpoints/checkpoint2/OPTIONS-000003 +sync-data: checkpoints/checkpoint2/OPTIONS-000003 +close: checkpoints/checkpoint2/OPTIONS-000003 +close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint2 create: checkpoints/checkpoint2/marker.format-version.000001.019 sync-data: checkpoints/checkpoint2/marker.format-version.000001.019 @@ -185,7 +193,11 @@ open-dir: checkpoints sync: checkpoints close: checkpoints open-dir: checkpoints/checkpoint3 -link: db/OPTIONS-000003 -> checkpoints/checkpoint3/OPTIONS-000003 +open: db/OPTIONS-000003 +create: checkpoints/checkpoint3/OPTIONS-000003 +sync-data: checkpoints/checkpoint3/OPTIONS-000003 +close: checkpoints/checkpoint3/OPTIONS-000003 +close: db/OPTIONS-000003 open-dir: checkpoints/checkpoint3 create: checkpoints/checkpoint3/marker.format-version.000001.019 sync-data: checkpoints/checkpoint3/marker.format-version.000001.019 diff --git a/testdata/copy_checkpoint_options b/testdata/copy_checkpoint_options new file mode 100644 index 0000000000..59818c53f4 --- /dev/null +++ b/testdata/copy_checkpoint_options @@ -0,0 +1,349 @@ +copy +[Version] + pebble_version=0.1 + +[Options] + bytes_per_sync=524288 + cache_size=8388608 + cleaner=delete + compaction_debt_concurrency=1073741824 + comparer=leveldb.BytewiseComparator + disable_wal=false + flush_delay_delete_range=0s + flush_delay_range_key=0s + flush_split_bytes=4194304 + format_major_version=13 + l0_compaction_concurrency=10 + l0_compaction_file_threshold=500 + l0_compaction_threshold=4 + l0_stop_writes_threshold=12 + lbase_max_bytes=67108864 + max_concurrent_compactions=1 + max_manifest_file_size=134217728 + max_open_files=1000 + mem_table_size=4194304 + mem_table_stop_writes_threshold=2 + min_deletion_rate=0 + merger=pebble.concatenate + read_compaction_rate=16000 + read_sampling_multiplier=16 + strict_wal_tail=true + table_cache_shards=10 + table_property_collectors=[] + validate_on_ingest=false + wal_dir= + wal_bytes_per_sync=0 + max_writer_concurrency=0 + force_writer_parallelism=false + secondary_cache_size_bytes=0 + create_on_shared=0 + +[Level "0"] + block_restart_interval=16 + block_size=4096 + block_size_threshold=90 + compression=Snappy + filter_policy=none + filter_type=table + index_block_size=4096 + target_file_size=2097152 +---- +---- +[Version] + pebble_version=0.1 + +[Options] + bytes_per_sync=524288 + cache_size=8388608 + cleaner=delete + compaction_debt_concurrency=1073741824 + comparer=leveldb.BytewiseComparator + disable_wal=false + flush_delay_delete_range=0s + flush_delay_range_key=0s + flush_split_bytes=4194304 + format_major_version=13 + l0_compaction_concurrency=10 + l0_compaction_file_threshold=500 + l0_compaction_threshold=4 + l0_stop_writes_threshold=12 + lbase_max_bytes=67108864 + max_concurrent_compactions=1 + max_manifest_file_size=134217728 + max_open_files=1000 + mem_table_size=4194304 + mem_table_stop_writes_threshold=2 + min_deletion_rate=0 + merger=pebble.concatenate + read_compaction_rate=16000 + read_sampling_multiplier=16 + strict_wal_tail=true + table_cache_shards=10 + table_property_collectors=[] + validate_on_ingest=false + wal_dir= + wal_bytes_per_sync=0 + max_writer_concurrency=0 + force_writer_parallelism=false + secondary_cache_size_bytes=0 + create_on_shared=0 + +[Level "0"] + block_restart_interval=16 + block_size=4096 + block_size_threshold=90 + compression=Snappy + filter_policy=none + filter_type=table + index_block_size=4096 + target_file_size=2097152 +---- +---- + +copy +[Version] + pebble_version=0.1 + +[Options] + bytes_per_sync=524288 + cache_size=16010668032 + cleaner=delete + compaction_debt_concurrency=1073741824 + comparer=cockroach_comparator + disable_wal=false + flush_delay_delete_range=10s + flush_delay_range_key=10s + flush_split_bytes=4194304 + format_major_version=17 + l0_compaction_concurrency=2 + l0_compaction_file_threshold=500 + l0_compaction_threshold=2 + l0_stop_writes_threshold=1000 + lbase_max_bytes=67108864 + max_concurrent_compactions=3 + max_concurrent_downloads=8 + max_manifest_file_size=134217728 + max_open_files=131870 + mem_table_size=67108864 + mem_table_stop_writes_threshold=4 + min_deletion_rate=134217728 + merger=cockroach_merge_operator + multilevel_compaction_heuristic=wamp(0.00, false) + read_compaction_rate=16000 + read_sampling_multiplier=16 + num_deletions_threshold=100 + deletion_size_ratio_threshold=0.500000 + tombstone_dense_compaction_threshold=0.050000 + strict_wal_tail=true + table_cache_shards=16 + validate_on_ingest=false + wal_dir= + wal_bytes_per_sync=0 + max_writer_concurrency=2 + force_writer_parallelism=false + secondary_cache_size_bytes=0 + create_on_shared=0 + +[WAL Failover] + secondary_dir=/mnt/data2/cockroach/auxiliary/wals-among-stores + primary_dir_probe_interval=1s + healthy_probe_latency_threshold=25ms + healthy_interval=15s + unhealthy_sampling_interval=100ms + unhealthy_operation_latency_threshold=100ms + elevated_write_stall_threshold_lag=1m0s + +[Level "0"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=2097152 + +[Level "1"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=4194304 + +[Level "2"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=8388608 + +[Level "3"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=16777216 + +[Level "4"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=33554432 + +[Level "5"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=67108864 + +[Level "6"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=134217728 +---- +---- +[Version] + pebble_version=0.1 + +[Options] + bytes_per_sync=524288 + cache_size=16010668032 + cleaner=delete + compaction_debt_concurrency=1073741824 + comparer=cockroach_comparator + disable_wal=false + flush_delay_delete_range=10s + flush_delay_range_key=10s + flush_split_bytes=4194304 + format_major_version=17 + l0_compaction_concurrency=2 + l0_compaction_file_threshold=500 + l0_compaction_threshold=2 + l0_stop_writes_threshold=1000 + lbase_max_bytes=67108864 + max_concurrent_compactions=3 + max_concurrent_downloads=8 + max_manifest_file_size=134217728 + max_open_files=131870 + mem_table_size=67108864 + mem_table_stop_writes_threshold=4 + min_deletion_rate=134217728 + merger=cockroach_merge_operator + multilevel_compaction_heuristic=wamp(0.00, false) + read_compaction_rate=16000 + read_sampling_multiplier=16 + num_deletions_threshold=100 + deletion_size_ratio_threshold=0.500000 + tombstone_dense_compaction_threshold=0.050000 + strict_wal_tail=true + table_cache_shards=16 + validate_on_ingest=false + wal_dir= + wal_bytes_per_sync=0 + max_writer_concurrency=2 + force_writer_parallelism=false + secondary_cache_size_bytes=0 + create_on_shared=0 + +# [WAL Failover] +# secondary_dir=/mnt/data2/cockroach/auxiliary/wals-among-stores +# primary_dir_probe_interval=1s +# healthy_probe_latency_threshold=25ms +# healthy_interval=15s +# unhealthy_sampling_interval=100ms +# unhealthy_operation_latency_threshold=100ms +# elevated_write_stall_threshold_lag=1m0s + +[Level "0"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=2097152 + +[Level "1"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=4194304 + +[Level "2"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=8388608 + +[Level "3"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=16777216 + +[Level "4"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=33554432 + +[Level "5"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=67108864 + +[Level "6"] + block_restart_interval=16 + block_size=32768 + block_size_threshold=90 + compression=Snappy + filter_policy=rocksdb.BuiltinBloomFilter + filter_type=table + index_block_size=262144 + target_file_size=134217728 +---- +---- \ No newline at end of file diff --git a/testdata/event_listener b/testdata/event_listener index cd326ee930..07428769a3 100644 --- a/testdata/event_listener +++ b/testdata/event_listener @@ -361,7 +361,11 @@ open-dir: . sync: . close: . open-dir: checkpoint -link: db/OPTIONS-000003 -> checkpoint/OPTIONS-000003 +open: db/OPTIONS-000003 +create: checkpoint/OPTIONS-000003 +sync-data: checkpoint/OPTIONS-000003 +close: checkpoint/OPTIONS-000003 +close: db/OPTIONS-000003 open-dir: checkpoint create: checkpoint/marker.format-version.000001.019 sync-data: checkpoint/marker.format-version.000001.019