diff --git a/cmd/pebble/db.go b/cmd/pebble/db.go index ace743e5b8..aec4ddefc5 100644 --- a/cmd/pebble/db.go +++ b/cmd/pebble/db.go @@ -12,6 +12,7 @@ import ( "github.com/cockroachdb/pebble/internal/bytealloc" "github.com/cockroachdb/pebble/internal/crdbtest" "github.com/cockroachdb/pebble/objstorage/remote" + "github.com/cockroachdb/pebble/sstable" "github.com/cockroachdb/pebble/vfs" ) @@ -61,7 +62,8 @@ func newPebbleDB(dir string) DB { Comparer: &crdbtest.Comparer, DisableWAL: disableWAL, FormatMajorVersion: pebble.FormatNewest, - KeySchema: crdbtest.KeySchema, + KeySchema: crdbtest.KeySchema.Name, + KeySchemas: sstable.MakeKeySchemas(crdbtest.KeySchema), L0CompactionThreshold: 2, L0StopWritesThreshold: 1000, LBaseMaxBytes: 64 << 20, // 64 MB diff --git a/compaction_test.go b/compaction_test.go index 16cba2ebce..bce7f96252 100644 --- a/compaction_test.go +++ b/compaction_test.go @@ -33,7 +33,6 @@ import ( "github.com/cockroachdb/pebble/objstorage/objstorageprovider" "github.com/cockroachdb/pebble/objstorage/remote" "github.com/cockroachdb/pebble/sstable" - "github.com/cockroachdb/pebble/sstable/colblk" "github.com/cockroachdb/pebble/vfs" "github.com/cockroachdb/pebble/vfs/errorfs" "github.com/stretchr/testify/require" @@ -617,7 +616,6 @@ func TestCompaction(t *testing.T) { Comparer: testkeys.Comparer, DebugCheck: DebugCheckLevels, FS: mem, - KeySchema: colblk.DefaultKeySchema(testkeys.Comparer, 16), L0CompactionThreshold: 8, MemTableSize: memTableSize, } diff --git a/ingest.go b/ingest.go index 3e8f762ef5..d96640c992 100644 --- a/ingest.go +++ b/ingest.go @@ -273,6 +273,13 @@ func ingestLoad1( tf, fmv, fmv.MinTableFormat(), fmv.MaxTableFormat(), ) } + if tf.BlockColumnar() { + if _, ok := opts.KeySchemas[r.Properties.KeySchemaName]; !ok { + return nil, errors.Newf( + "pebble: table uses key schema %q unknown to the database", + r.Properties.KeySchemaName) + } + } meta := &fileMetadata{} meta.FileNum = fileNum diff --git a/ingest_test.go b/ingest_test.go index 9d27adce06..4cd922c2db 100644 --- a/ingest_test.go +++ b/ingest_test.go @@ -38,7 +38,6 @@ import ( "github.com/cockroachdb/pebble/record" "github.com/cockroachdb/pebble/sstable" "github.com/cockroachdb/pebble/sstable/block" - "github.com/cockroachdb/pebble/sstable/colblk" "github.com/cockroachdb/pebble/vfs" "github.com/cockroachdb/pebble/vfs/errorfs" "github.com/kr/pretty" @@ -218,10 +217,9 @@ func TestIngestLoadRand(t *testing.T) { } opts := (&Options{ - Comparer: base.DefaultComparer, - KeySchema: colblk.DefaultKeySchema(base.DefaultComparer, 16), - FS: mem, - }).WithFSDefaults() + Comparer: base.DefaultComparer, + FS: mem, + }).WithFSDefaults().EnsureDefaults() lr, err := ingestLoad(context.Background(), opts, version, paths, nil, nil, 0, pending) require.NoError(t, err) @@ -3671,7 +3669,7 @@ func TestIngestValidation(t *testing.T) { BlockSize: blockSize, // Create many smaller blocks. Comparer: opts.Comparer, Compression: NoCompression, // For simpler debugging. - KeySchema: opts.KeySchema, + KeySchema: opts.KeySchemas[opts.KeySchema], }) for _, kv := range keyVals { require.NoError(t, w.Set(kv.key, kv.val)) diff --git a/internal/compact/spans_test.go b/internal/compact/spans_test.go index dc3874a920..c881a2a30c 100644 --- a/internal/compact/spans_test.go +++ b/internal/compact/spans_test.go @@ -132,8 +132,8 @@ func TestSplitAndEncodeSpan(t *testing.T) { require.NoError(t, SplitAndEncodeSpan(base.DefaultComparer.Compare, &span, upToKey, tw)) require.NoError(t, tw.Close()) _, rangeDels, rangeKeys := sstable.ReadAll(obj, sstable.ReaderOptions{ - Comparer: wo.Comparer, - KeySchema: wo.KeySchema, + Comparer: wo.Comparer, + KeySchemas: sstable.MakeKeySchemas(wo.KeySchema), }) require.LessOrEqual(t, len(rangeDels)+len(rangeKeys), 1) s := "." diff --git a/internal/crdbtest/crdbtest.go b/internal/crdbtest/crdbtest.go index 8b1cc33a37..daa605ab1c 100644 --- a/internal/crdbtest/crdbtest.go +++ b/internal/crdbtest/crdbtest.go @@ -501,6 +501,7 @@ const ( ) var KeySchema = colblk.KeySchema{ + Name: "crdb1", ColumnTypes: []colblk.DataType{ cockroachColRoachKey: colblk.DataTypePrefixBytes, cockroachColMVCCWallTime: colblk.DataTypeUint, diff --git a/level_checker_test.go b/level_checker_test.go index 03c97ee5b9..0291f36834 100644 --- a/level_checker_test.go +++ b/level_checker_test.go @@ -236,8 +236,8 @@ func TestCheckLevelsCornerCases(t *testing.T) { } // Set FileNum for logging purposes. readerOpts := sstable.ReaderOptions{ - Comparer: testkeys.Comparer, - KeySchema: writerOpts.KeySchema, + Comparer: testkeys.Comparer, + KeySchemas: sstable.KeySchemas{writerOpts.KeySchema.Name: writerOpts.KeySchema}, } readerOpts.SetInternalCacheOpts(sstableinternal.CacheOptions{FileNum: base.DiskFileNum(fileNum - 1)}) r, err := sstable.NewReader(context.Background(), readable, readerOpts) diff --git a/metamorphic/options_test.go b/metamorphic/options_test.go index 63a1bdc68f..a897d1173b 100644 --- a/metamorphic/options_test.go +++ b/metamorphic/options_test.go @@ -65,6 +65,7 @@ func TestOptionsRoundtrip(t *testing.T) { "Cache:", "Cache.", "FS:", + "KeySchemas[", "TableCache:", // Function pointers "BlockPropertyCollectors:", diff --git a/options.go b/options.go index ca10379619..ae251b9ddd 100644 --- a/options.go +++ b/options.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/pebble/internal/humanize" "github.com/cockroachdb/pebble/internal/keyspan" "github.com/cockroachdb/pebble/internal/manifest" + "github.com/cockroachdb/pebble/internal/testkeys" "github.com/cockroachdb/pebble/objstorage/objstorageprovider" "github.com/cockroachdb/pebble/objstorage/remote" "github.com/cockroachdb/pebble/rangekey" @@ -61,6 +62,9 @@ type FilterWriter = base.FilterWriter // FilterPolicy exports the base.FilterPolicy type. type FilterPolicy = base.FilterPolicy +// KeySchema exports the colblk.KeySchema type. +type KeySchema = colblk.KeySchema + // BlockPropertyCollector exports the sstable.BlockPropertyCollector type. type BlockPropertyCollector = sstable.BlockPropertyCollector @@ -890,11 +894,22 @@ type Options struct { // The default value uses the underlying operating system's file system. FS vfs.FS - // KeySchema defines the schema of a user key. When columnar blocks are in - // use (see FormatColumnarBlocks), the user may specify how a key should be - // decomposed into columns. If not set, colblk.DefaultKeySchema is used to - // construct a default key schema. - KeySchema colblk.KeySchema + // KeySchema is the name of the key schema that should be used when writing + // new sstables. There must be a key schema with this name defined in + // KeySchemas. If not set, colblk.DefaultKeySchema is used to construct a + // default key schema. + KeySchema string + + // KeySchemas defines the set of known schemas of user keys. When columnar + // blocks are in use (see FormatColumnarBlocks), the user may specify how a + // key should be decomposed into columns. Each KeySchema must have a unique + // name. The schema named by Options.KeySchema is used while writing + // sstables during flushes and compactions. + // + // Multiple KeySchemas may be used over the lifetime of a database. Once a + // KeySchema is used, it must be provided in KeySchemas in subsequent calls + // to Open for perpetuity. + KeySchemas sstable.KeySchemas // Lock, if set, must be a database lock acquired through LockDirectory for // the same directory passed to Open. If provided, Open will skip locking @@ -1207,8 +1222,10 @@ func (o *Options) EnsureDefaults() *Options { if o.Experimental.KeyValidationFunc == nil { o.Experimental.KeyValidationFunc = func([]byte) error { return nil } } - if len(o.KeySchema.ColumnTypes) == 0 { - o.KeySchema = colblk.DefaultKeySchema(o.Comparer, 16 /* bundleSize */) + if o.KeySchema == "" && len(o.KeySchemas) == 0 { + ks := colblk.DefaultKeySchema(o.Comparer, 16 /* bundleSize */) + o.KeySchema = ks.Name + o.KeySchemas = sstable.MakeKeySchemas(ks) } if o.L0CompactionThreshold <= 0 { o.L0CompactionThreshold = 4 @@ -1433,6 +1450,7 @@ func (o *Options) String() string { fmt.Fprintf(&buf, " flush_delay_range_key=%s\n", o.FlushDelayRangeKey) fmt.Fprintf(&buf, " flush_split_bytes=%d\n", o.FlushSplitBytes) fmt.Fprintf(&buf, " format_major_version=%d\n", o.FormatMajorVersion) + fmt.Fprintf(&buf, " key_schema=%s\n", o.KeySchema) fmt.Fprintf(&buf, " l0_compaction_concurrency=%d\n", o.Experimental.L0CompactionConcurrency) fmt.Fprintf(&buf, " l0_compaction_file_threshold=%d\n", o.L0CompactionFileThreshold) fmt.Fprintf(&buf, " l0_compaction_threshold=%d\n", o.L0CompactionThreshold) @@ -1576,6 +1594,7 @@ type ParseHooks struct { NewCleaner func(name string) (Cleaner, error) NewComparer func(name string) (*Comparer, error) NewFilterPolicy func(name string) (FilterPolicy, error) + NewKeySchema func(name string) (KeySchema, error) NewMerger func(name string) (*Merger, error) SkipUnknown func(name, value string) bool } @@ -1590,6 +1609,20 @@ func (o *Options) Parse(s string, hooks *ParseHooks) error { // a backwards incompatible change. Instead, leave in support for parsing the // key but simply don't parse the value. + parseComparer := func(name string) (*Comparer, error) { + switch name { + case DefaultComparer.Name: + return DefaultComparer, nil + case testkeys.Comparer.Name: + return testkeys.Comparer, nil + default: + if hooks != nil && hooks.NewComparer != nil { + return hooks.NewComparer(name) + } + return nil, nil + } + } + switch { case section == "Version": switch key { @@ -1631,14 +1664,7 @@ func (o *Options) Parse(s string, hooks *ParseHooks) error { } } case "comparer": - switch value { - case "leveldb.BytewiseComparator": - o.Comparer = DefaultComparer - default: - if hooks != nil && hooks.NewComparer != nil { - o.Comparer, err = hooks.NewComparer(value) - } - } + o.Comparer, err = parseComparer(value) case "compaction_debt_concurrency": o.Experimental.CompactionDebtConcurrency, err = strconv.ParseUint(value, 10, 64) case "delete_range_flush_delay": @@ -1678,6 +1704,32 @@ func (o *Options) Parse(s string, hooks *ParseHooks) error { if err == nil { o.FormatMajorVersion = FormatMajorVersion(v) } + case "key_schema": + o.KeySchema = value + if o.KeySchemas == nil { + o.KeySchemas = make(map[string]KeySchema) + } + if _, ok := o.KeySchemas[o.KeySchema]; !ok { + if strings.HasPrefix(value, "DefaultKeySchema(") && strings.HasSuffix(value, ")") { + argsStr := strings.TrimSuffix(strings.TrimPrefix(value, "DefaultKeySchema("), ")") + args := strings.FieldsFunc(argsStr, func(r rune) bool { + return unicode.IsSpace(r) || r == ',' + }) + var comparer *base.Comparer + var bundleSize int + comparer, err = parseComparer(args[0]) + if err == nil { + bundleSize, err = strconv.Atoi(args[1]) + } + if err == nil { + schema := colblk.DefaultKeySchema(comparer, bundleSize) + o.KeySchema = schema.Name + o.KeySchemas[o.KeySchema] = schema + } + } else if hooks != nil && hooks.NewKeySchema != nil { + o.KeySchemas[value], err = hooks.NewKeySchema(value) + } + } case "l0_compaction_concurrency": o.Experimental.L0CompactionConcurrency, err = strconv.Atoi(value) case "l0_compaction_file_threshold": @@ -2006,6 +2058,14 @@ func (o *Options) Validate() error { if o.TableCache != nil && o.Cache != o.TableCache.cache { fmt.Fprintf(&buf, "underlying cache in the TableCache and the Cache dont match\n") } + if len(o.KeySchemas) > 0 { + if o.KeySchema == "" { + fmt.Fprintf(&buf, "KeySchemas is set but KeySchema is not\n") + } + if _, ok := o.KeySchemas[o.KeySchema]; !ok { + fmt.Fprintf(&buf, "KeySchema %q not found in KeySchemas\n", o.KeySchema) + } + } if buf.Len() == 0 { return nil } @@ -2019,7 +2079,7 @@ func (o *Options) MakeReaderOptions() sstable.ReaderOptions { if o != nil { readerOpts.Comparer = o.Comparer readerOpts.Filters = o.Filters - readerOpts.KeySchema = o.KeySchema + readerOpts.KeySchemas = o.KeySchemas readerOpts.LoadBlockSema = o.LoadBlockSema readerOpts.LoggerAndTracer = o.LoggerAndTracer readerOpts.Merger = o.Merger @@ -2054,7 +2114,7 @@ func (o *Options) MakeWriterOptions(level int, format sstable.TableFormat) sstab writerOpts.FilterPolicy = levelOpts.FilterPolicy writerOpts.FilterType = levelOpts.FilterType writerOpts.IndexBlockSize = levelOpts.IndexBlockSize - writerOpts.KeySchema = o.KeySchema + writerOpts.KeySchema = o.KeySchemas[o.KeySchema] writerOpts.AllocatorSizeClasses = o.AllocatorSizeClasses writerOpts.NumDeletionsThreshold = o.Experimental.NumDeletionsThreshold writerOpts.DeletionSizeRatioThreshold = o.Experimental.DeletionSizeRatioThreshold diff --git a/options_test.go b/options_test.go index f239b12e03..2ea4895bfe 100644 --- a/options_test.go +++ b/options_test.go @@ -90,6 +90,7 @@ func TestOptionsString(t *testing.T) { flush_delay_range_key=0s flush_split_bytes=4194304 format_major_version=13 + key_schema=DefaultKeySchema(leveldb.BytewiseComparator,16) l0_compaction_concurrency=10 l0_compaction_file_threshold=500 l0_compaction_threshold=4 diff --git a/replay/testdata/replay b/replay/testdata/replay index 7ba447b567..c4b7c7409c 100644 --- a/replay/testdata/replay +++ b/replay/testdata/replay @@ -11,7 +11,7 @@ tree 614 000007.sst 0 LOCK 133 MANIFEST-000001 - 1359 OPTIONS-000003 + 1418 OPTIONS-000003 0 marker.format-version.000001.013 0 marker.manifest.000001.MANIFEST-000001 simple/ @@ -21,7 +21,7 @@ tree 25 000004.log 586 000005.sst 85 MANIFEST-000001 - 1359 OPTIONS-000003 + 1418 OPTIONS-000003 0 marker.format-version.000001.013 0 marker.manifest.000001.MANIFEST-000001 @@ -42,6 +42,7 @@ cat build/OPTIONS-000003 flush_delay_range_key=0s flush_split_bytes=4194304 format_major_version=13 + key_schema=DefaultKeySchema(pebble.internal.testkeys,16) l0_compaction_concurrency=10 l0_compaction_file_threshold=500 l0_compaction_threshold=4 diff --git a/replay/testdata/replay_paced b/replay/testdata/replay_paced index 560ab347a2..c6799f7cfe 100644 --- a/replay/testdata/replay_paced +++ b/replay/testdata/replay_paced @@ -14,7 +14,7 @@ tree 0 LOCK 133 MANIFEST-000001 205 MANIFEST-000010 - 1359 OPTIONS-000003 + 1418 OPTIONS-000003 0 marker.format-version.000001.013 0 marker.manifest.000002.MANIFEST-000010 high_read_amp/ @@ -26,7 +26,7 @@ tree 39 000008.log 560 000009.sst 157 MANIFEST-000010 - 1359 OPTIONS-000003 + 1418 OPTIONS-000003 0 marker.format-version.000001.013 0 marker.manifest.000001.MANIFEST-000010 diff --git a/sstable/colblk/data_block.go b/sstable/colblk/data_block.go index a976f7581d..193627c2e1 100644 --- a/sstable/colblk/data_block.go +++ b/sstable/colblk/data_block.go @@ -34,6 +34,7 @@ import ( // attributes inlined within the data block. For inlined-values, the // user-defined value columns would be implicitly null. type KeySchema struct { + Name string ColumnTypes []DataType NewKeyWriter func() KeyWriter NewKeySeeker func() KeySeeker @@ -161,6 +162,7 @@ var defaultKeySeekerPool = sync.Pool{ // into its prefix and suffix. Prefixes are sorted in lexicographical order. func DefaultKeySchema(comparer *base.Comparer, prefixBundleSize int) KeySchema { return KeySchema{ + Name: fmt.Sprintf("DefaultKeySchema(%s,%d)", comparer.Name, prefixBundleSize), ColumnTypes: defaultSchemaColumnTypes, NewKeyWriter: func() KeyWriter { kw := &defaultKeyWriter{comparer: comparer} diff --git a/sstable/colblk_writer.go b/sstable/colblk_writer.go index fade02238b..fb2d613779 100644 --- a/sstable/colblk_writer.go +++ b/sstable/colblk_writer.go @@ -178,6 +178,7 @@ func newColumnarWriter(writable objstorage.Writable, o WriterOptions) *RawColumn w.props.ComparerName = o.Comparer.Name w.props.CompressionName = o.Compression.String() + w.props.KeySchemaName = o.KeySchema.Name w.props.MergerName = o.MergerName w.writeQueue.ch = make(chan *compressedBlock) diff --git a/sstable/colblk_writer_test.go b/sstable/colblk_writer_test.go index 60efdabd9b..ec6d30bbcb 100644 --- a/sstable/colblk_writer_test.go +++ b/sstable/colblk_writer_test.go @@ -57,8 +57,8 @@ func TestColumnarWriter(t *testing.T) { } var err error r, err = NewReader(context.Background(), obj, ReaderOptions{ - Comparer: testkeys.Comparer, - KeySchema: keySchema, + Comparer: testkeys.Comparer, + KeySchemas: KeySchemas{keySchema.Name: keySchema}, }) require.NoError(t, err) return "ok" diff --git a/sstable/copier_test.go b/sstable/copier_test.go index 7c893cddd1..4830430116 100644 --- a/sstable/copier_test.go +++ b/sstable/copier_test.go @@ -24,6 +24,8 @@ func TestCopySpan(t *testing.T) { fs := vfs.NewMem() blockCache := cache.New(1 << 20 /* 1 MB */) defer blockCache.Unref() + + keySchema := colblk.DefaultKeySchema(testkeys.Comparer, 16) datadriven.RunTest(t, "testdata/copy_span", func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "build": @@ -48,7 +50,7 @@ func TestCopySpan(t *testing.T) { BlockSize: 1, TableFormat: tableFormat, Comparer: testkeys.Comparer, - KeySchema: colblk.DefaultKeySchema(testkeys.Comparer, 16), + KeySchema: keySchema, }) for _, key := range strings.Split(d.Input, "\n") { j := strings.Index(key, ":") @@ -75,8 +77,8 @@ func TestCopySpan(t *testing.T) { return err.Error() } r, err := NewReader(context.TODO(), readable, ReaderOptions{ - Comparer: testkeys.Comparer, - KeySchema: colblk.DefaultKeySchema(testkeys.Comparer, 16), + Comparer: testkeys.Comparer, + KeySchemas: KeySchemas{keySchema.Name: keySchema}, }) defer r.Close() if err != nil { @@ -118,8 +120,8 @@ func TestCopySpan(t *testing.T) { return err.Error() } rOpts := ReaderOptions{ - Comparer: testkeys.Comparer, - KeySchema: colblk.DefaultKeySchema(testkeys.Comparer, 16), + Comparer: testkeys.Comparer, + KeySchemas: KeySchemas{keySchema.Name: keySchema}, } rOpts.internal.CacheOpts.Cache = blockCache r, err := NewReader(context.TODO(), readable, rOpts) @@ -156,8 +158,8 @@ func TestCopySpan(t *testing.T) { return err.Error() } r, err := NewReader(context.TODO(), readable, ReaderOptions{ - Comparer: testkeys.Comparer, - KeySchema: colblk.DefaultKeySchema(testkeys.Comparer, 16), + Comparer: testkeys.Comparer, + KeySchemas: KeySchemas{keySchema.Name: keySchema}, }) if err != nil { return err.Error() diff --git a/sstable/data_test.go b/sstable/data_test.go index a7e3d4c166..d135721411 100644 --- a/sstable/data_test.go +++ b/sstable/data_test.go @@ -163,8 +163,8 @@ func runBuildCmd( func openReader(obj *objstorage.MemObj, writerOpts *WriterOptions, cacheSize int) (*Reader, error) { readerOpts := ReaderOptions{ - Comparer: writerOpts.Comparer, - KeySchema: writerOpts.KeySchema, + Comparer: writerOpts.Comparer, + KeySchemas: KeySchemas{writerOpts.KeySchema.Name: writerOpts.KeySchema}, } if writerOpts.FilterPolicy != nil { readerOpts.Filters = map[string]FilterPolicy{ @@ -240,8 +240,8 @@ func runBuildRawCmd( return nil, nil, err } r, err := NewReader(context.Background(), f1, ReaderOptions{ - Comparer: opts.Comparer, - KeySchema: opts.KeySchema, + Comparer: opts.Comparer, + KeySchemas: KeySchemas{opts.KeySchema.Name: opts.KeySchema}, }) if err != nil { return nil, nil, err @@ -513,8 +513,8 @@ func runRewriteCmd( return nil, r, errors.Wrap(err, "rewrite failed") } readerOpts := ReaderOptions{ - Comparer: opts.Comparer, - KeySchema: opts.KeySchema, + Comparer: opts.Comparer, + KeySchemas: KeySchemas{opts.KeySchema.Name: opts.KeySchema}, } if opts.FilterPolicy != nil { readerOpts.Filters = map[string]FilterPolicy{ diff --git a/sstable/options.go b/sstable/options.go index ca68081e4c..ce9b284d0c 100644 --- a/sstable/options.go +++ b/sstable/options.go @@ -5,6 +5,8 @@ package sstable import ( + "fmt" + "github.com/cockroachdb/fifo" "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/sstableinternal" @@ -61,6 +63,22 @@ type Comparers map[string]*base.Comparer // mergers. type Mergers map[string]*base.Merger +// KeySchemas is a map from key schema name to key schema. A single database may +// contain sstables with multiple key schemas. +type KeySchemas map[string]colblk.KeySchema + +// MakeKeySchemas constructs a KeySchemas from a slice of key schemas. +func MakeKeySchemas(keySchemas ...colblk.KeySchema) KeySchemas { + m := make(KeySchemas, len(keySchemas)) + for _, keySchema := range keySchemas { + if _, ok := m[keySchema.Name]; ok { + panic(fmt.Sprintf("duplicate key schemas with name %q", keySchema.Name)) + } + m[keySchema.Name] = keySchema + } + return m +} + // ReaderOptions holds the parameters needed for reading an sstable. type ReaderOptions struct { // LoadBlockSema, if set, is used to limit the number of blocks that can be @@ -83,11 +101,10 @@ type ReaderOptions struct { Comparers Comparers Mergers Mergers - - // KeySchema describes the schema to use when interpreting columnar data - // blocks. Only used for sstables encoded in format TableFormatPebblev5 or - // higher. - KeySchema colblk.KeySchema + // KeySchemas contains the set of known key schemas to use when interpreting + // columnar data blocks. Only used for sstables encoded in format + // TableFormatPebblev5 or higher. + KeySchemas KeySchemas // Filters is a map from filter policy name to filter policy. Filters with // policies that are not in this map will be ignored. @@ -130,9 +147,16 @@ func (o ReaderOptions) ensureDefaults() ReaderOptions { if o.DeniedUserProperties == nil { o.DeniedUserProperties = ignoredInternalProperties } + if o.KeySchemas == nil { + o.KeySchemas = defaultKeySchemas + } return o } +var defaultKeySchemas = MakeKeySchemas( + colblk.DefaultKeySchema(base.DefaultComparer, 16), +) + // WriterOptions holds the parameters used to control building an sstable. type WriterOptions struct { // BlockRestartInterval is the number of keys between restart points diff --git a/sstable/properties.go b/sstable/properties.go index 45eef946c3..bf65b8c83b 100644 --- a/sstable/properties.go +++ b/sstable/properties.go @@ -156,6 +156,9 @@ type Properties struct { // For formats >= TableFormatPebblev4, this is set to true if the obsolete // bit is strict for all the point keys. IsStrictObsolete bool `prop:"pebble.obsolete.is_strict"` + // The name of the key schema used in this table. Empty for formats <= + // TableFormatPebblev4. + KeySchemaName string `prop:"pebble.colblk.schema"` // The name of the merger used in this table. Empty if no merger is used. MergerName string `prop:"rocksdb.merge.operator"` // The number of merge operands in the table. @@ -366,6 +369,9 @@ func (p *Properties) save(tblFormat TableFormat, w *rowblk.Writer) { if p.IsStrictObsolete { p.saveBool(m, unsafe.Offsetof(p.IsStrictObsolete), p.IsStrictObsolete) } + if p.KeySchemaName != "" { + p.saveString(m, unsafe.Offsetof(p.KeySchemaName), p.KeySchemaName) + } if p.MergerName != "" { p.saveString(m, unsafe.Offsetof(p.MergerName), p.MergerName) } diff --git a/sstable/properties_test.go b/sstable/properties_test.go index 766c54ca4a..3c7dc2a9e9 100644 --- a/sstable/properties_test.go +++ b/sstable/properties_test.go @@ -80,6 +80,7 @@ var testProps = Properties{ IndexSize: 11, IndexType: 12, IsStrictObsolete: true, + KeySchemaName: "key schema name", MergerName: "merge operator name", NumMergeOperands: 17, NumRangeKeyUnsets: 21, diff --git a/sstable/random_test.go b/sstable/random_test.go index 937dcd9c26..25f6d12559 100644 --- a/sstable/random_test.go +++ b/sstable/random_test.go @@ -275,9 +275,9 @@ type randomTableConfig struct { func (cfg *randomTableConfig) readerOpts() ReaderOptions { rOpts := ReaderOptions{ - Comparer: testkeys.Comparer, - KeySchema: cfg.wopts.KeySchema, - Filters: map[string]FilterPolicy{}, + Comparer: testkeys.Comparer, + KeySchemas: map[string]colblk.KeySchema{cfg.wopts.KeySchema.Name: cfg.wopts.KeySchema}, + Filters: map[string]FilterPolicy{}, } if cfg.wopts.FilterPolicy != nil { rOpts.Filters[cfg.wopts.FilterPolicy.Name()] = cfg.wopts.FilterPolicy diff --git a/sstable/reader.go b/sstable/reader.go index 77a5eec788..7c74c8cc70 100644 --- a/sstable/reader.go +++ b/sstable/reader.go @@ -8,6 +8,7 @@ import ( "cmp" "context" "encoding/binary" + "fmt" "io" "path/filepath" "runtime" @@ -927,7 +928,6 @@ func NewReader(ctx context.Context, f objstorage.Readable, o ReaderOptions) (*Re r := &Reader{ readable: f, cacheOpts: o.internal.CacheOpts, - keySchema: o.KeySchema, loadBlockSema: o.LoadBlockSema, deniedUserProperties: o.DeniedUserProperties, filterMetricsTracker: o.FilterMetricsTracker, @@ -963,13 +963,6 @@ func NewReader(ctx context.Context, f objstorage.Readable, o ReaderOptions) (*Re return nil, r.Close() } - // If the table format indicates that blocks are encoded within the columnar - // format, we require a key schema to interpret it correctly. - if r.tableFormat.BlockColumnar() && len(r.keySchema.ColumnTypes) == 0 { - r.err = errors.Newf("pebble/table: key schema required for reading tables of format %s", r.tableFormat) - return nil, r.Close() - } - if r.Properties.ComparerName == "" || o.Comparer.Name == r.Properties.ComparerName { r.Comparer = o.Comparer r.Compare = o.Comparer.Compare @@ -998,6 +991,22 @@ func NewReader(ctx context.Context, f objstorage.Readable, o ReaderOptions) (*Re } } + if r.tableFormat.BlockColumnar() { + if ks, ok := o.KeySchemas[r.Properties.KeySchemaName]; ok { + r.keySchema = ks + } else { + var known []string + for name := range o.KeySchemas { + known = append(known, fmt.Sprintf("%q", name)) + } + slices.Sort(known) + + r.err = errors.Newf("pebble/table: %d: unknown key schema %q; known key schemas: %s", + errors.Safe(r.cacheOpts.FileNum), errors.Safe(r.Properties.KeySchemaName), errors.Safe(known)) + panic(r.err) + } + } + if r.err != nil { return nil, r.Close() } diff --git a/sstable/reader_test.go b/sstable/reader_test.go index d736828bfb..a31137c62a 100644 --- a/sstable/reader_test.go +++ b/sstable/reader_test.go @@ -1688,8 +1688,8 @@ func TestReader_TableFormat(t *testing.T) { f, err = fs.Open("test") require.NoError(t, err) r, err := newReader(f, ReaderOptions{ - Comparer: opts.Comparer, - KeySchema: opts.KeySchema, + Comparer: opts.Comparer, + KeySchemas: MakeKeySchemas(opts.KeySchema), }) require.NoError(t, err) defer r.Close() diff --git a/sstable/suffix_rewriter_test.go b/sstable/suffix_rewriter_test.go index 93776e8861..48711df083 100644 --- a/sstable/suffix_rewriter_test.go +++ b/sstable/suffix_rewriter_test.go @@ -81,9 +81,9 @@ func TestRewriteSuffixProps(t *testing.T) { // Rewrite the SST using updated options and check the returned props. readerOpts := ReaderOptions{ - Comparer: wOpts.Comparer, - KeySchema: wOpts.KeySchema, - Filters: map[string]base.FilterPolicy{wOpts.FilterPolicy.Name(): wOpts.FilterPolicy}, + Comparer: wOpts.Comparer, + KeySchemas: KeySchemas{wOpts.KeySchema.Name: wOpts.KeySchema}, + Filters: map[string]base.FilterPolicy{wOpts.FilterPolicy.Name(): wOpts.FilterPolicy}, } r, err := NewMemReader(sst, readerOpts) require.NoError(t, err) diff --git a/sstable/testdata/columnar_writer/simple_binary b/sstable/testdata/columnar_writer/simple_binary index bb705cf66e..e3a7881b4b 100644 --- a/sstable/testdata/columnar_writer/simple_binary +++ b/sstable/testdata/columnar_writer/simple_binary @@ -80,37 +80,38 @@ sstable │ ├── 00000 block:0/88 │ │ │ └── trailer [compression=none checksum=0x6147006e] - ├── properties offset: 149 length: 561 + ├── properties offset: 149 length: 622 │ ├── 00000 obsolete-key (16) [restart] - │ ├── 00016 pebble.internal.testkeys.suffixes (48) - │ ├── 00064 pebble.raw.point-tombstone.key.size (32) - │ ├── 00096 rocksdb.block.based.table.index.type (43) - │ ├── 00139 rocksdb.comparator (37) - │ ├── 00176 rocksdb.compression (23) - │ ├── 00199 rocksdb.compression_options (106) - │ ├── 00305 rocksdb.data.size (13) - │ ├── 00318 rocksdb.deleted.keys (15) - │ ├── 00333 rocksdb.filter.size (15) - │ ├── 00348 rocksdb.index.size (14) - │ ├── 00362 rocksdb.merge.operands (18) - │ ├── 00380 rocksdb.merge.operator (24) - │ ├── 00404 rocksdb.num.data.blocks (19) - │ ├── 00423 rocksdb.num.entries (11) - │ ├── 00434 rocksdb.num.range-deletions (19) - │ ├── 00453 rocksdb.property.collectors (70) - │ ├── 00523 rocksdb.raw.key.size (16) - │ ├── 00539 rocksdb.raw.value.size (14) + │ ├── 00016 pebble.colblk.schema (68) + │ ├── 00084 pebble.internal.testkeys.suffixes (41) + │ ├── 00125 pebble.raw.point-tombstone.key.size (32) + │ ├── 00157 rocksdb.block.based.table.index.type (43) + │ ├── 00200 rocksdb.comparator (37) + │ ├── 00237 rocksdb.compression (23) + │ ├── 00260 rocksdb.compression_options (106) + │ ├── 00366 rocksdb.data.size (13) + │ ├── 00379 rocksdb.deleted.keys (15) + │ ├── 00394 rocksdb.filter.size (15) + │ ├── 00409 rocksdb.index.size (14) + │ ├── 00423 rocksdb.merge.operands (18) + │ ├── 00441 rocksdb.merge.operator (24) + │ ├── 00465 rocksdb.num.data.blocks (19) + │ ├── 00484 rocksdb.num.entries (11) + │ ├── 00495 rocksdb.num.range-deletions (19) + │ ├── 00514 rocksdb.property.collectors (70) + │ ├── 00584 rocksdb.raw.key.size (16) + │ ├── 00600 rocksdb.raw.value.size (14) │ ├── restart points - │ │ └── 00553 [restart 0] - │ └── trailer [compression=none checksum=0xf181bf0d] - ├── meta-index offset: 715 length: 33 - │ ├── 0000 rocksdb.properties block:149/561 [restart] + │ │ └── 00614 [restart 0] + │ └── trailer [compression=none checksum=0xf75fa767] + ├── meta-index offset: 776 length: 33 + │ ├── 0000 rocksdb.properties block:149/622 [restart] │ ├── restart points │ │ └── 00025 [restart 0] - │ └── trailer [compression=none checksum=0x742412ac] - └── footer offset: 753 length: 53 + │ └── trailer [compression=none checksum=0x816ce1ee] + └── footer offset: 814 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=715, length=33 + ├── 001 meta: offset=776, length=33 ├── 004 index: offset=93, length=51 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -131,6 +132,7 @@ rocksdb.data.size: 93 rocksdb.filter.size: 0 rocksdb.index.size: 56 rocksdb.block.based.table.index.type: 0 +pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) rocksdb.merge.operator: pebble.concatenate rocksdb.merge.operands: 0 rocksdb.property.collectors: [pebble.internal.testkeys.suffixes,obsolete-key] @@ -226,36 +228,37 @@ sstable │ ├── 00000 block:0/116 │ │ │ └── trailer [compression=none checksum=0xcf86ef66] - ├── properties offset: 177 length: 529 + ├── properties offset: 177 length: 590 │ ├── 00000 obsolete-key (16) [restart] - │ ├── 00016 pebble.internal.testkeys.suffixes (48) - │ ├── 00064 rocksdb.block.based.table.index.type (43) - │ ├── 00107 rocksdb.comparator (37) - │ ├── 00144 rocksdb.compression (23) - │ ├── 00167 rocksdb.compression_options (106) - │ ├── 00273 rocksdb.data.size (13) - │ ├── 00286 rocksdb.deleted.keys (15) - │ ├── 00301 rocksdb.filter.size (15) - │ ├── 00316 rocksdb.index.size (14) - │ ├── 00330 rocksdb.merge.operands (18) - │ ├── 00348 rocksdb.merge.operator (24) - │ ├── 00372 rocksdb.num.data.blocks (19) - │ ├── 00391 rocksdb.num.entries (11) - │ ├── 00402 rocksdb.num.range-deletions (19) - │ ├── 00421 rocksdb.property.collectors (70) - │ ├── 00491 rocksdb.raw.key.size (16) - │ ├── 00507 rocksdb.raw.value.size (14) + │ ├── 00016 pebble.colblk.schema (68) + │ ├── 00084 pebble.internal.testkeys.suffixes (41) + │ ├── 00125 rocksdb.block.based.table.index.type (43) + │ ├── 00168 rocksdb.comparator (37) + │ ├── 00205 rocksdb.compression (23) + │ ├── 00228 rocksdb.compression_options (106) + │ ├── 00334 rocksdb.data.size (13) + │ ├── 00347 rocksdb.deleted.keys (15) + │ ├── 00362 rocksdb.filter.size (15) + │ ├── 00377 rocksdb.index.size (14) + │ ├── 00391 rocksdb.merge.operands (18) + │ ├── 00409 rocksdb.merge.operator (24) + │ ├── 00433 rocksdb.num.data.blocks (19) + │ ├── 00452 rocksdb.num.entries (11) + │ ├── 00463 rocksdb.num.range-deletions (19) + │ ├── 00482 rocksdb.property.collectors (70) + │ ├── 00552 rocksdb.raw.key.size (16) + │ ├── 00568 rocksdb.raw.value.size (14) │ ├── restart points - │ │ └── 00521 [restart 0] - │ └── trailer [compression=none checksum=0xd5071efb] - ├── meta-index offset: 711 length: 33 - │ ├── 0000 rocksdb.properties block:177/529 [restart] + │ │ └── 00582 [restart 0] + │ └── trailer [compression=none checksum=0x54772987] + ├── meta-index offset: 772 length: 33 + │ ├── 0000 rocksdb.properties block:177/590 [restart] │ ├── restart points │ │ └── 00025 [restart 0] - │ └── trailer [compression=none checksum=0x34a4c633] - └── footer offset: 749 length: 53 + │ └── trailer [compression=none checksum=0x3febf671] + └── footer offset: 810 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=711, length=33 + ├── 001 meta: offset=772, length=33 ├── 004 index: offset=121, length=51 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -919,38 +922,39 @@ sstable │ ├── 00002 block:1246/53 │ │ │ └── trailer [compression=none checksum=0x70310e3a] - ├── properties offset: 1400 length: 571 + ├── properties offset: 1400 length: 632 │ ├── 00000 obsolete-key (16) [restart] - │ ├── 00016 pebble.internal.testkeys.suffixes (48) - │ ├── 00064 rocksdb.block.based.table.index.type (43) - │ ├── 00107 rocksdb.comparator (37) - │ ├── 00144 rocksdb.compression (23) - │ ├── 00167 rocksdb.compression_options (106) - │ ├── 00273 rocksdb.data.size (14) - │ ├── 00287 rocksdb.deleted.keys (15) - │ ├── 00302 rocksdb.filter.size (15) - │ ├── 00317 rocksdb.index.partitions (20) - │ ├── 00337 rocksdb.index.size (9) - │ ├── 00346 rocksdb.merge.operands (18) - │ ├── 00364 rocksdb.merge.operator (24) - │ ├── 00388 rocksdb.num.data.blocks (19) - │ ├── 00407 rocksdb.num.entries (11) - │ ├── 00418 rocksdb.num.range-deletions (19) - │ ├── 00437 rocksdb.property.collectors (70) - │ ├── 00507 rocksdb.raw.key.size (17) - │ ├── 00524 rocksdb.raw.value.size (15) - │ ├── 00539 rocksdb.top-level.index.size (24) + │ ├── 00016 pebble.colblk.schema (68) + │ ├── 00084 pebble.internal.testkeys.suffixes (41) + │ ├── 00125 rocksdb.block.based.table.index.type (43) + │ ├── 00168 rocksdb.comparator (37) + │ ├── 00205 rocksdb.compression (23) + │ ├── 00228 rocksdb.compression_options (106) + │ ├── 00334 rocksdb.data.size (14) + │ ├── 00348 rocksdb.deleted.keys (15) + │ ├── 00363 rocksdb.filter.size (15) + │ ├── 00378 rocksdb.index.partitions (20) + │ ├── 00398 rocksdb.index.size (9) + │ ├── 00407 rocksdb.merge.operands (18) + │ ├── 00425 rocksdb.merge.operator (24) + │ ├── 00449 rocksdb.num.data.blocks (19) + │ ├── 00468 rocksdb.num.entries (11) + │ ├── 00479 rocksdb.num.range-deletions (19) + │ ├── 00498 rocksdb.property.collectors (70) + │ ├── 00568 rocksdb.raw.key.size (17) + │ ├── 00585 rocksdb.raw.value.size (15) + │ ├── 00600 rocksdb.top-level.index.size (24) │ ├── restart points - │ │ └── 00563 [restart 0] - │ └── trailer [compression=none checksum=0x8bcc1b94] - ├── meta-index offset: 1976 length: 33 - │ ├── 0000 rocksdb.properties block:1400/571 [restart] + │ │ └── 00624 [restart 0] + │ └── trailer [compression=none checksum=0x16dee43f] + ├── meta-index offset: 2037 length: 33 + │ ├── 0000 rocksdb.properties block:1400/632 [restart] │ ├── restart points │ │ └── 00025 [restart 0] - │ └── trailer [compression=none checksum=0xdc5c5254] - └── footer offset: 2014 length: 53 + │ └── trailer [compression=none checksum=0x8499e3b5] + └── footer offset: 2075 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=1976, length=33 + ├── 001 meta: offset=2037, length=33 ├── 004 index: offset=1304, length=91 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -975,6 +979,7 @@ rocksdb.filter.size: 0 rocksdb.index.partitions: 3 rocksdb.index.size: 384 rocksdb.block.based.table.index.type: 2 +pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) rocksdb.merge.operator: pebble.concatenate rocksdb.merge.operands: 0 rocksdb.property.collectors: [pebble.internal.testkeys.suffixes,obsolete-key] @@ -1611,38 +1616,39 @@ sstable │ ├── 00002 block:1246/53 │ │ │ └── trailer [compression=none checksum=0x70310e3a] - ├── properties offset: 1400 length: 571 + ├── properties offset: 1400 length: 632 │ ├── 00000 obsolete-key (16) [restart] - │ ├── 00016 pebble.internal.testkeys.suffixes (48) - │ ├── 00064 rocksdb.block.based.table.index.type (43) - │ ├── 00107 rocksdb.comparator (37) - │ ├── 00144 rocksdb.compression (23) - │ ├── 00167 rocksdb.compression_options (106) - │ ├── 00273 rocksdb.data.size (14) - │ ├── 00287 rocksdb.deleted.keys (15) - │ ├── 00302 rocksdb.filter.size (15) - │ ├── 00317 rocksdb.index.partitions (20) - │ ├── 00337 rocksdb.index.size (9) - │ ├── 00346 rocksdb.merge.operands (18) - │ ├── 00364 rocksdb.merge.operator (24) - │ ├── 00388 rocksdb.num.data.blocks (19) - │ ├── 00407 rocksdb.num.entries (11) - │ ├── 00418 rocksdb.num.range-deletions (19) - │ ├── 00437 rocksdb.property.collectors (70) - │ ├── 00507 rocksdb.raw.key.size (17) - │ ├── 00524 rocksdb.raw.value.size (15) - │ ├── 00539 rocksdb.top-level.index.size (24) + │ ├── 00016 pebble.colblk.schema (68) + │ ├── 00084 pebble.internal.testkeys.suffixes (41) + │ ├── 00125 rocksdb.block.based.table.index.type (43) + │ ├── 00168 rocksdb.comparator (37) + │ ├── 00205 rocksdb.compression (23) + │ ├── 00228 rocksdb.compression_options (106) + │ ├── 00334 rocksdb.data.size (14) + │ ├── 00348 rocksdb.deleted.keys (15) + │ ├── 00363 rocksdb.filter.size (15) + │ ├── 00378 rocksdb.index.partitions (20) + │ ├── 00398 rocksdb.index.size (9) + │ ├── 00407 rocksdb.merge.operands (18) + │ ├── 00425 rocksdb.merge.operator (24) + │ ├── 00449 rocksdb.num.data.blocks (19) + │ ├── 00468 rocksdb.num.entries (11) + │ ├── 00479 rocksdb.num.range-deletions (19) + │ ├── 00498 rocksdb.property.collectors (70) + │ ├── 00568 rocksdb.raw.key.size (17) + │ ├── 00585 rocksdb.raw.value.size (15) + │ ├── 00600 rocksdb.top-level.index.size (24) │ ├── restart points - │ │ └── 00563 [restart 0] - │ └── trailer [compression=none checksum=0x8bcc1b94] - ├── meta-index offset: 1976 length: 33 - │ ├── 0000 rocksdb.properties block:1400/571 [restart] + │ │ └── 00624 [restart 0] + │ └── trailer [compression=none checksum=0x16dee43f] + ├── meta-index offset: 2037 length: 33 + │ ├── 0000 rocksdb.properties block:1400/632 [restart] │ ├── restart points │ │ └── 00025 [restart 0] - │ └── trailer [compression=none checksum=0xdc5c5254] - └── footer offset: 2014 length: 53 + │ └── trailer [compression=none checksum=0x8499e3b5] + └── footer offset: 2075 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=1976, length=33 + ├── 001 meta: offset=2037, length=33 ├── 004 index: offset=1304, length=91 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -1709,38 +1715,39 @@ sstable │ │ │ └── 50-50: x # data[0]: │ │ └── 50-51: x 00 # block padding byte │ └── trailer [compression=none checksum=0xb1e3982b] - ├── properties offset: 89 length: 519 + ├── properties offset: 89 length: 580 │ ├── 00000 obsolete-key (17) [restart] - │ ├── 00017 pebble.internal.testkeys.suffixes (37) - │ ├── 00054 rocksdb.block.based.table.index.type (43) - │ ├── 00097 rocksdb.comparator (37) - │ ├── 00134 rocksdb.compression (23) - │ ├── 00157 rocksdb.compression_options (106) - │ ├── 00263 rocksdb.data.size (13) - │ ├── 00276 rocksdb.deleted.keys (15) - │ ├── 00291 rocksdb.filter.size (15) - │ ├── 00306 rocksdb.index.size (14) - │ ├── 00320 rocksdb.merge.operands (18) - │ ├── 00338 rocksdb.merge.operator (24) - │ ├── 00362 rocksdb.num.data.blocks (19) - │ ├── 00381 rocksdb.num.entries (11) - │ ├── 00392 rocksdb.num.range-deletions (19) - │ ├── 00411 rocksdb.property.collectors (70) - │ ├── 00481 rocksdb.raw.key.size (16) - │ ├── 00497 rocksdb.raw.value.size (14) + │ ├── 00017 pebble.colblk.schema (68) + │ ├── 00085 pebble.internal.testkeys.suffixes (30) + │ ├── 00115 rocksdb.block.based.table.index.type (43) + │ ├── 00158 rocksdb.comparator (37) + │ ├── 00195 rocksdb.compression (23) + │ ├── 00218 rocksdb.compression_options (106) + │ ├── 00324 rocksdb.data.size (13) + │ ├── 00337 rocksdb.deleted.keys (15) + │ ├── 00352 rocksdb.filter.size (15) + │ ├── 00367 rocksdb.index.size (14) + │ ├── 00381 rocksdb.merge.operands (18) + │ ├── 00399 rocksdb.merge.operator (24) + │ ├── 00423 rocksdb.num.data.blocks (19) + │ ├── 00442 rocksdb.num.entries (11) + │ ├── 00453 rocksdb.num.range-deletions (19) + │ ├── 00472 rocksdb.property.collectors (70) + │ ├── 00542 rocksdb.raw.key.size (16) + │ ├── 00558 rocksdb.raw.value.size (14) │ ├── restart points - │ │ └── 00511 [restart 0] - │ └── trailer [compression=none checksum=0x3c8fb979] - ├── meta-index offset: 613 length: 59 - │ ├── 0000 rocksdb.properties block:89/519 [restart] + │ │ └── 00572 [restart 0] + │ └── trailer [compression=none checksum=0xd4a2d9f6] + ├── meta-index offset: 674 length: 59 + │ ├── 0000 rocksdb.properties block:89/580 [restart] │ ├── 0024 rocksdb.range_del2 block:33/51 [restart] │ ├── restart points │ │ ├── 00047 [restart 0] │ │ └── 00051 [restart 24] - │ └── trailer [compression=none checksum=0xbabe7d85] - └── footer offset: 677 length: 53 + │ └── trailer [compression=none checksum=0xb95e4caa] + └── footer offset: 738 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=613, length=59 + ├── 001 meta: offset=674, length=59 ├── 004 index: offset=0, length=28 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -1821,43 +1828,44 @@ sstable │ │ │ └── 67-67: x # data[1]: │ │ └── 67-68: x 00 # block padding byte │ └── trailer [compression=none checksum=0x45325be7] - ├── properties offset: 106 length: 601 + ├── properties offset: 106 length: 662 │ ├── 00000 obsolete-key (17) [restart] - │ ├── 00017 pebble.internal.testkeys.suffixes (39) - │ ├── 00056 pebble.num.range-key-dels (22) - │ ├── 00078 pebble.num.range-key-sets (8) - │ ├── 00086 pebble.num.range-key-unsets (10) - │ ├── 00096 pebble.raw.range-key.key.size (26) - │ ├── 00122 pebble.raw.range-key.value.size (14) - │ ├── 00136 rocksdb.block.based.table.index.type (43) - │ ├── 00179 rocksdb.comparator (37) - │ ├── 00216 rocksdb.compression (23) - │ ├── 00239 rocksdb.compression_options (106) - │ ├── 00345 rocksdb.data.size (13) - │ ├── 00358 rocksdb.deleted.keys (15) - │ ├── 00373 rocksdb.filter.size (15) - │ ├── 00388 rocksdb.index.size (14) - │ ├── 00402 rocksdb.merge.operands (18) - │ ├── 00420 rocksdb.merge.operator (24) - │ ├── 00444 rocksdb.num.data.blocks (19) - │ ├── 00463 rocksdb.num.entries (11) - │ ├── 00474 rocksdb.num.range-deletions (19) - │ ├── 00493 rocksdb.property.collectors (70) - │ ├── 00563 rocksdb.raw.key.size (16) - │ ├── 00579 rocksdb.raw.value.size (14) + │ ├── 00017 pebble.colblk.schema (68) + │ ├── 00085 pebble.internal.testkeys.suffixes (32) + │ ├── 00117 pebble.num.range-key-dels (22) + │ ├── 00139 pebble.num.range-key-sets (8) + │ ├── 00147 pebble.num.range-key-unsets (10) + │ ├── 00157 pebble.raw.range-key.key.size (26) + │ ├── 00183 pebble.raw.range-key.value.size (14) + │ ├── 00197 rocksdb.block.based.table.index.type (43) + │ ├── 00240 rocksdb.comparator (37) + │ ├── 00277 rocksdb.compression (23) + │ ├── 00300 rocksdb.compression_options (106) + │ ├── 00406 rocksdb.data.size (13) + │ ├── 00419 rocksdb.deleted.keys (15) + │ ├── 00434 rocksdb.filter.size (15) + │ ├── 00449 rocksdb.index.size (14) + │ ├── 00463 rocksdb.merge.operands (18) + │ ├── 00481 rocksdb.merge.operator (24) + │ ├── 00505 rocksdb.num.data.blocks (19) + │ ├── 00524 rocksdb.num.entries (11) + │ ├── 00535 rocksdb.num.range-deletions (19) + │ ├── 00554 rocksdb.property.collectors (70) + │ ├── 00624 rocksdb.raw.key.size (16) + │ ├── 00640 rocksdb.raw.value.size (14) │ ├── restart points - │ │ └── 00593 [restart 0] - │ └── trailer [compression=none checksum=0x4bab3a75] - ├── meta-index offset: 712 length: 57 + │ │ └── 00654 [restart 0] + │ └── trailer [compression=none checksum=0xbdc87030] + ├── meta-index offset: 773 length: 57 │ ├── 0000 pebble.range_key block:33/68 [restart] - │ ├── 0021 rocksdb.properties block:106/601 [restart] + │ ├── 0021 rocksdb.properties block:106/662 [restart] │ ├── restart points │ │ ├── 00045 [restart 0] │ │ └── 00049 [restart 21] - │ └── trailer [compression=none checksum=0x8f7219ca] - └── footer offset: 774 length: 53 + │ └── trailer [compression=none checksum=0x1a6d95f4] + └── footer offset: 835 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=712, length=57 + ├── 001 meta: offset=773, length=57 ├── 004 index: offset=0, length=28 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 diff --git a/sstable/testdata/copy_span b/sstable/testdata/copy_span index e939a50c60..f68cea1485 100644 --- a/sstable/testdata/copy_span +++ b/sstable/testdata/copy_span @@ -53,7 +53,7 @@ d#0,SET: foobar copy-span test3 test4 b.SET.10 cc.SET.0 ---- -copied 821 bytes +copied 889 bytes iter test4 ---- @@ -63,7 +63,7 @@ d#0,SET: foobar copy-span test3 test5 a.SET.10 bb.SET.0 ---- -copied 829 bytes +copied 897 bytes iter test5 ---- diff --git a/sstable/testdata/rewriter_v5 b/sstable/testdata/rewriter_v5 index 44ac7b8a90..47a9fc078e 100644 --- a/sstable/testdata/rewriter_v5 +++ b/sstable/testdata/rewriter_v5 @@ -45,9 +45,9 @@ sstable ├── index offset: 363 length: 37 ├── top-index offset: 405 length: 52 ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 462 length: 69 - ├── properties offset: 536 length: 515 - ├── meta-index offset: 1056 length: 80 - └── footer offset: 1141 length: 53 + ├── properties offset: 536 length: 583 + ├── meta-index offset: 1124 length: 80 + └── footer offset: 1209 length: 53 scan ---- @@ -80,9 +80,9 @@ sstable ├── index offset: 367 length: 37 ├── top-index offset: 409 length: 56 ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 470 length: 69 - ├── properties offset: 544 length: 515 - ├── meta-index offset: 1064 length: 80 - └── footer offset: 1149 length: 53 + ├── properties offset: 544 length: 583 + ├── meta-index offset: 1132 length: 80 + └── footer offset: 1217 length: 53 scan ---- @@ -115,9 +115,9 @@ sstable ├── index offset: 367 length: 37 ├── top-index offset: 409 length: 56 ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 470 length: 69 - ├── properties offset: 544 length: 515 - ├── meta-index offset: 1064 length: 80 - └── footer offset: 1149 length: 53 + ├── properties offset: 544 length: 583 + ├── meta-index offset: 1132 length: 80 + └── footer offset: 1217 length: 53 scan ---- @@ -150,9 +150,9 @@ sstable ├── index offset: 367 length: 37 ├── top-index offset: 409 length: 56 ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 470 length: 69 - ├── properties offset: 544 length: 515 - ├── meta-index offset: 1064 length: 80 - └── footer offset: 1149 length: 53 + ├── properties offset: 544 length: 583 + ├── meta-index offset: 1132 length: 80 + └── footer offset: 1217 length: 53 scan ---- @@ -186,9 +186,9 @@ sstable ├── index offset: 363 length: 37 ├── top-index offset: 405 length: 52 ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 462 length: 69 - ├── properties offset: 536 length: 515 - ├── meta-index offset: 1056 length: 80 - └── footer offset: 1141 length: 53 + ├── properties offset: 536 length: 583 + ├── meta-index offset: 1124 length: 80 + └── footer offset: 1209 length: 53 scan ---- diff --git a/sstable/testdata/virtual_reader_props b/sstable/testdata/virtual_reader_props index 1bbc77f1c2..606806b2b8 100644 --- a/sstable/testdata/virtual_reader_props +++ b/sstable/testdata/virtual_reader_props @@ -44,7 +44,7 @@ bounds: [b#1,SET-c#1,SET] filenum: 000002 props: rocksdb.num.entries: 1 - rocksdb.raw.key.size: 5 + rocksdb.raw.key.size: 4 rocksdb.raw.value.size: 1 rocksdb.num.data.blocks: 1 @@ -300,6 +300,6 @@ bounds: [dd#5,SET-ddd#6,SET] filenum: 000008 props: rocksdb.num.entries: 2 - rocksdb.raw.key.size: 11 + rocksdb.raw.key.size: 10 rocksdb.raw.value.size: 2 rocksdb.num.data.blocks: 1 diff --git a/sstable/testdata/writer_v5 b/sstable/testdata/writer_v5 index 0801d12849..f825bdb95c 100644 --- a/sstable/testdata/writer_v5 +++ b/sstable/testdata/writer_v5 @@ -103,6 +103,7 @@ rocksdb.data.size: 0 rocksdb.filter.size: 0 rocksdb.index.size: 33 rocksdb.block.based.table.index.type: 0 +pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) rocksdb.merge.operator: pebble.concatenate rocksdb.merge.operands: 0 rocksdb.property.collectors: [obsolete-key] @@ -234,9 +235,9 @@ sstable ├── index offset: 278 length: 37 ├── index offset: 320 length: 37 ├── top-index offset: 362 length: 48 - ├── properties offset: 415 length: 480 - ├── meta-index offset: 900 length: 33 - └── footer offset: 938 length: 53 + ├── properties offset: 415 length: 548 + ├── meta-index offset: 968 length: 33 + └── footer offset: 1006 length: 53 # Exercise the non-Reader layout-decoding codepath. @@ -250,9 +251,9 @@ sstable ├── index offset: 278 length: 37 ├── index offset: 320 length: 37 ├── top-index offset: 362 length: 48 - ├── properties offset: 415 length: 480 - ├── meta-index offset: 900 length: 33 - └── footer offset: 938 length: 53 + ├── properties offset: 415 length: 548 + ├── meta-index offset: 968 length: 33 + └── footer offset: 1006 length: 53 scan ---- @@ -297,9 +298,9 @@ layout sstable ├── index offset: 0 length: 28 ├── range-key offset: 33 length: 84 - ├── properties offset: 122 length: 528 - ├── meta-index offset: 655 length: 57 - └── footer offset: 717 length: 53 + ├── properties offset: 122 length: 589 + ├── meta-index offset: 716 length: 57 + └── footer offset: 778 length: 53 props ---- @@ -318,6 +319,7 @@ rocksdb.data.size: 0 rocksdb.filter.size: 0 rocksdb.index.size: 33 rocksdb.block.based.table.index.type: 0 +pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) rocksdb.merge.operator: pebble.concatenate rocksdb.merge.operands: 0 pebble.num.range-key-unsets: 0 diff --git a/sstable/testdata/writer_value_blocks b/sstable/testdata/writer_value_blocks index 04029b641b..8ac47df2f5 100644 --- a/sstable/testdata/writer_value_blocks +++ b/sstable/testdata/writer_value_blocks @@ -643,42 +643,43 @@ sstable │ └── trailer [compression=none checksum=0x60e7fb82] ├── value-index offset: 680 length: 8 │ └── trailer [compression=none checksum=0xb327e021] - ├── properties offset: 693 length: 549 + ├── properties offset: 693 length: 610 │ ├── 00000 obsolete-key (16) [restart] - │ ├── 00016 pebble.num.value-blocks (27) - │ ├── 00043 pebble.num.values.in.value-blocks (21) - │ ├── 00064 pebble.value-blocks.size (21) - │ ├── 00085 rocksdb.block.based.table.index.type (43) - │ ├── 00128 rocksdb.comparator (37) - │ ├── 00165 rocksdb.compression (16) - │ ├── 00181 rocksdb.compression_options (106) - │ ├── 00287 rocksdb.data.size (14) - │ ├── 00301 rocksdb.deleted.keys (15) - │ ├── 00316 rocksdb.filter.size (15) - │ ├── 00331 rocksdb.index.partitions (20) - │ ├── 00351 rocksdb.index.size (9) - │ ├── 00360 rocksdb.merge.operands (18) - │ ├── 00378 rocksdb.merge.operator (24) - │ ├── 00402 rocksdb.num.data.blocks (19) - │ ├── 00421 rocksdb.num.entries (11) - │ ├── 00432 rocksdb.num.range-deletions (19) - │ ├── 00451 rocksdb.property.collectors (36) - │ ├── 00487 rocksdb.raw.key.size (16) - │ ├── 00503 rocksdb.raw.value.size (14) - │ ├── 00517 rocksdb.top-level.index.size (24) + │ ├── 00016 pebble.colblk.schema (68) + │ ├── 00084 pebble.num.value-blocks (20) + │ ├── 00104 pebble.num.values.in.value-blocks (21) + │ ├── 00125 pebble.value-blocks.size (21) + │ ├── 00146 rocksdb.block.based.table.index.type (43) + │ ├── 00189 rocksdb.comparator (37) + │ ├── 00226 rocksdb.compression (16) + │ ├── 00242 rocksdb.compression_options (106) + │ ├── 00348 rocksdb.data.size (14) + │ ├── 00362 rocksdb.deleted.keys (15) + │ ├── 00377 rocksdb.filter.size (15) + │ ├── 00392 rocksdb.index.partitions (20) + │ ├── 00412 rocksdb.index.size (9) + │ ├── 00421 rocksdb.merge.operands (18) + │ ├── 00439 rocksdb.merge.operator (24) + │ ├── 00463 rocksdb.num.data.blocks (19) + │ ├── 00482 rocksdb.num.entries (11) + │ ├── 00493 rocksdb.num.range-deletions (19) + │ ├── 00512 rocksdb.property.collectors (36) + │ ├── 00548 rocksdb.raw.key.size (16) + │ ├── 00564 rocksdb.raw.value.size (14) + │ ├── 00578 rocksdb.top-level.index.size (24) │ ├── restart points - │ │ └── 00541 [restart 0] - │ └── trailer [compression=none checksum=0x74ea3d7a] - ├── meta-index offset: 1247 length: 64 + │ │ └── 00602 [restart 0] + │ └── trailer [compression=none checksum=0x307b671a] + ├── meta-index offset: 1308 length: 64 │ ├── 0000 pebble.value_index block:680/8 value-blocks-index-lengths: 1(num), 2(offset), 1(length) [restart] - │ ├── 0027 rocksdb.properties block:693/549 [restart] + │ ├── 0027 rocksdb.properties block:693/610 [restart] │ ├── restart points │ │ ├── 00052 [restart 0] │ │ └── 00056 [restart 27] - │ └── trailer [compression=none checksum=0x5a3ff93a] - └── footer offset: 1316 length: 53 + │ └── trailer [compression=none checksum=0x28531dc] + └── footer offset: 1377 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=1247, length=64 + ├── 001 meta: offset=1308, length=64 ├── 004 index: offset=562, length=77 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -938,36 +939,37 @@ sstable │ ├── 00000 block:0/100 │ │ │ └── trailer [compression=none checksum=0x760132f1] - ├── properties offset: 146 length: 479 + ├── properties offset: 146 length: 540 │ ├── 00000 obsolete-key (16) [restart] - │ ├── 00016 pebble.raw.point-tombstone.key.size (39) - │ ├── 00055 rocksdb.block.based.table.index.type (43) - │ ├── 00098 rocksdb.comparator (37) - │ ├── 00135 rocksdb.compression (16) - │ ├── 00151 rocksdb.compression_options (106) - │ ├── 00257 rocksdb.data.size (13) - │ ├── 00270 rocksdb.deleted.keys (15) - │ ├── 00285 rocksdb.filter.size (15) - │ ├── 00300 rocksdb.index.size (14) - │ ├── 00314 rocksdb.merge.operands (18) - │ ├── 00332 rocksdb.merge.operator (24) - │ ├── 00356 rocksdb.num.data.blocks (19) - │ ├── 00375 rocksdb.num.entries (11) - │ ├── 00386 rocksdb.num.range-deletions (19) - │ ├── 00405 rocksdb.property.collectors (36) - │ ├── 00441 rocksdb.raw.key.size (16) - │ ├── 00457 rocksdb.raw.value.size (14) + │ ├── 00016 pebble.colblk.schema (68) + │ ├── 00084 pebble.raw.point-tombstone.key.size (32) + │ ├── 00116 rocksdb.block.based.table.index.type (43) + │ ├── 00159 rocksdb.comparator (37) + │ ├── 00196 rocksdb.compression (16) + │ ├── 00212 rocksdb.compression_options (106) + │ ├── 00318 rocksdb.data.size (13) + │ ├── 00331 rocksdb.deleted.keys (15) + │ ├── 00346 rocksdb.filter.size (15) + │ ├── 00361 rocksdb.index.size (14) + │ ├── 00375 rocksdb.merge.operands (18) + │ ├── 00393 rocksdb.merge.operator (24) + │ ├── 00417 rocksdb.num.data.blocks (19) + │ ├── 00436 rocksdb.num.entries (11) + │ ├── 00447 rocksdb.num.range-deletions (19) + │ ├── 00466 rocksdb.property.collectors (36) + │ ├── 00502 rocksdb.raw.key.size (16) + │ ├── 00518 rocksdb.raw.value.size (14) │ ├── restart points - │ │ └── 00471 [restart 0] - │ └── trailer [compression=none checksum=0x6d52d476] - ├── meta-index offset: 630 length: 33 - │ ├── 0000 rocksdb.properties block:146/479 [restart] + │ │ └── 00532 [restart 0] + │ └── trailer [compression=none checksum=0x62079e3b] + ├── meta-index offset: 691 length: 33 + │ ├── 0000 rocksdb.properties block:146/540 [restart] │ ├── restart points │ │ └── 00025 [restart 0] - │ └── trailer [compression=none checksum=0x95dbb0f4] - └── footer offset: 668 length: 53 + │ └── trailer [compression=none checksum=0xda289a22] + └── footer offset: 729 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=630, length=33 + ├── 001 meta: offset=691, length=33 ├── 004 index: offset=105, length=36 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 diff --git a/table_stats_test.go b/table_stats_test.go index ef1362da8f..872ebb7378 100644 --- a/table_stats_test.go +++ b/table_stats_test.go @@ -193,6 +193,7 @@ func TestTableStats(t *testing.T) { func TestTableRangeDeletionIter(t *testing.T) { var m *fileMetadata cmp := testkeys.Comparer + keySchema := colblk.DefaultKeySchema(cmp, 16) fs := vfs.NewMem() datadriven.RunTest(t, "testdata/table_stats_deletion_iter", func(t *testing.T, td *datadriven.TestData) string { switch cmd := td.Cmd; cmd { @@ -203,7 +204,7 @@ func TestTableRangeDeletionIter(t *testing.T) { } w := sstable.NewRawWriter(objstorageprovider.NewFileWritable(f), sstable.WriterOptions{ Comparer: cmp, - KeySchema: colblk.DefaultKeySchema(cmp, 16), + KeySchema: keySchema, TableFormat: sstable.TableFormatMax, }) m = &fileMetadata{} @@ -241,8 +242,8 @@ func TestTableRangeDeletionIter(t *testing.T) { return err.Error() } r, err = sstable.NewReader(context.Background(), readable, sstable.ReaderOptions{ - Comparer: cmp, - KeySchema: colblk.DefaultKeySchema(cmp, 16), + Comparer: cmp, + KeySchemas: sstable.KeySchemas{keySchema.Name: keySchema}, }) if err != nil { return err.Error() diff --git a/testdata/checkpoint b/testdata/checkpoint index 66a896cb16..22e5b7399d 100644 --- a/testdata/checkpoint +++ b/testdata/checkpoint @@ -218,17 +218,17 @@ close: db/000009.sst sync: db sync: db/MANIFEST-000001 open: db/000005.sst (options: *vfs.randomReadsOption) -read-at(617, 53): db/000005.sst -read-at(579, 38): db/000005.sst -read-at(132, 447): db/000005.sst +read-at(687, 53): db/000005.sst +read-at(649, 38): db/000005.sst +read-at(132, 517): db/000005.sst open: db/000009.sst (options: *vfs.randomReadsOption) -read-at(621, 53): db/000009.sst -read-at(583, 38): db/000009.sst -read-at(136, 447): db/000009.sst +read-at(691, 53): db/000009.sst +read-at(653, 38): db/000009.sst +read-at(136, 517): db/000009.sst open: db/000007.sst (options: *vfs.randomReadsOption) -read-at(617, 53): db/000007.sst -read-at(579, 38): db/000007.sst -read-at(132, 447): db/000007.sst +read-at(687, 53): db/000007.sst +read-at(649, 38): db/000007.sst +read-at(132, 517): db/000007.sst read-at(91, 41): db/000005.sst open: db/000005.sst (options: *vfs.sequentialReadsOption) read-at(0, 91): db/000005.sst @@ -298,15 +298,15 @@ close: checkpoints/checkpoint1/000006.log scan checkpoints/checkpoint1 ---- open: checkpoints/checkpoint1/000007.sst (options: *vfs.randomReadsOption) -read-at(617, 53): checkpoints/checkpoint1/000007.sst -read-at(579, 38): checkpoints/checkpoint1/000007.sst -read-at(132, 447): checkpoints/checkpoint1/000007.sst +read-at(687, 53): checkpoints/checkpoint1/000007.sst +read-at(649, 38): checkpoints/checkpoint1/000007.sst +read-at(132, 517): checkpoints/checkpoint1/000007.sst read-at(91, 41): checkpoints/checkpoint1/000007.sst read-at(0, 91): checkpoints/checkpoint1/000007.sst open: checkpoints/checkpoint1/000005.sst (options: *vfs.randomReadsOption) -read-at(617, 53): checkpoints/checkpoint1/000005.sst -read-at(579, 38): checkpoints/checkpoint1/000005.sst -read-at(132, 447): checkpoints/checkpoint1/000005.sst +read-at(687, 53): checkpoints/checkpoint1/000005.sst +read-at(649, 38): checkpoints/checkpoint1/000005.sst +read-at(132, 517): checkpoints/checkpoint1/000005.sst read-at(91, 41): checkpoints/checkpoint1/000005.sst read-at(0, 91): checkpoints/checkpoint1/000005.sst a 1 @@ -321,9 +321,9 @@ g 10 scan db ---- open: db/000010.sst (options: *vfs.randomReadsOption) -read-at(626, 53): db/000010.sst -read-at(588, 38): db/000010.sst -read-at(141, 447): db/000010.sst +read-at(696, 53): db/000010.sst +read-at(658, 38): db/000010.sst +read-at(141, 517): db/000010.sst read-at(100, 41): db/000010.sst read-at(0, 100): db/000010.sst a 1 @@ -365,9 +365,9 @@ close: checkpoints/checkpoint2/000006.log scan checkpoints/checkpoint2 ---- open: checkpoints/checkpoint2/000007.sst (options: *vfs.randomReadsOption) -read-at(617, 53): checkpoints/checkpoint2/000007.sst -read-at(579, 38): checkpoints/checkpoint2/000007.sst -read-at(132, 447): checkpoints/checkpoint2/000007.sst +read-at(687, 53): checkpoints/checkpoint2/000007.sst +read-at(649, 38): checkpoints/checkpoint2/000007.sst +read-at(132, 517): checkpoints/checkpoint2/000007.sst read-at(91, 41): checkpoints/checkpoint2/000007.sst read-at(0, 91): checkpoints/checkpoint2/000007.sst b 5 @@ -407,15 +407,15 @@ close: checkpoints/checkpoint3/000006.log scan checkpoints/checkpoint3 ---- open: checkpoints/checkpoint3/000007.sst (options: *vfs.randomReadsOption) -read-at(617, 53): checkpoints/checkpoint3/000007.sst -read-at(579, 38): checkpoints/checkpoint3/000007.sst -read-at(132, 447): checkpoints/checkpoint3/000007.sst +read-at(687, 53): checkpoints/checkpoint3/000007.sst +read-at(649, 38): checkpoints/checkpoint3/000007.sst +read-at(132, 517): checkpoints/checkpoint3/000007.sst read-at(91, 41): checkpoints/checkpoint3/000007.sst read-at(0, 91): checkpoints/checkpoint3/000007.sst open: checkpoints/checkpoint3/000005.sst (options: *vfs.randomReadsOption) -read-at(617, 53): checkpoints/checkpoint3/000005.sst -read-at(579, 38): checkpoints/checkpoint3/000005.sst -read-at(132, 447): checkpoints/checkpoint3/000005.sst +read-at(687, 53): checkpoints/checkpoint3/000005.sst +read-at(649, 38): checkpoints/checkpoint3/000005.sst +read-at(132, 517): checkpoints/checkpoint3/000005.sst read-at(91, 41): checkpoints/checkpoint3/000005.sst read-at(0, 91): checkpoints/checkpoint3/000005.sst a 1 @@ -546,9 +546,9 @@ close: checkpoints/checkpoint4/000008.log scan checkpoints/checkpoint4 ---- open: checkpoints/checkpoint4/000010.sst (options: *vfs.randomReadsOption) -read-at(626, 53): checkpoints/checkpoint4/000010.sst -read-at(588, 38): checkpoints/checkpoint4/000010.sst -read-at(141, 447): checkpoints/checkpoint4/000010.sst +read-at(696, 53): checkpoints/checkpoint4/000010.sst +read-at(658, 38): checkpoints/checkpoint4/000010.sst +read-at(141, 517): checkpoints/checkpoint4/000010.sst read-at(100, 41): checkpoints/checkpoint4/000010.sst read-at(0, 100): checkpoints/checkpoint4/000010.sst a 1 diff --git a/testdata/compaction_delete_only_hints b/testdata/compaction_delete_only_hints index fe0a0fa498..52668e5653 100644 --- a/testdata/compaction_delete_only_hints +++ b/testdata/compaction_delete_only_hints @@ -88,7 +88,7 @@ maybe-compact Deletion hints: L0.000004 b-r seqnums(tombstone=200-230, file-smallest=30, type=point-key-only) Compactions: - [JOB 100] compacted(delete-only) multilevel L2 [000005] (673B) Score=0.00 + L3 [000006] (673B) Score=0.00 + L4 [000007] (673B) Score=0.00 -> L6 [000008] (94B), in 1.0s (2.0s total), output rate 94B/s + [JOB 100] compacted(delete-only) multilevel L2 [000005] (743B) Score=0.00 + L3 [000006] (743B) Score=0.00 + L4 [000007] (743B) Score=0.00 -> L6 [000008] (94B), in 1.0s (2.0s total), output rate 94B/s # Verify that compaction correctly handles the presence of multiple # overlapping hints which might delete a file multiple times. All of the @@ -127,7 +127,7 @@ maybe-compact Deletion hints: L1.000005 b-r seqnums(tombstone=200-230, file-smallest=30, type=point-key-only) Compactions: - [JOB 100] compacted(delete-only) multilevel L2 [000005] (673B) Score=0.00 + L3 [000006] (673B) Score=0.00 + L4 [000007] (673B) Score=0.00 -> L6 [000008] (94B), in 1.0s (2.0s total), output rate 94B/s + [JOB 100] compacted(delete-only) multilevel L2 [000005] (743B) Score=0.00 + L3 [000006] (743B) Score=0.00 + L4 [000007] (743B) Score=0.00 -> L6 [000008] (94B), in 1.0s (2.0s total), output rate 94B/s # Test a range tombstone that is already compacted into L6. @@ -206,7 +206,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) multilevel L2 [000005] (673B) Score=0.00 + L3 [000006] (673B) Score=0.00 + L4 [000007] (673B) Score=0.00 -> L6 [000009] (94B), in 1.0s (2.0s total), output rate 94B/s + [JOB 100] compacted(delete-only) multilevel L2 [000005] (743B) Score=0.00 + L3 [000006] (743B) Score=0.00 + L4 [000007] (743B) Score=0.00 -> L6 [000009] (94B), in 1.0s (2.0s total), output rate 94B/s # A deletion hint present on an sstable in a higher level should NOT result in a # deletion-only compaction incorrectly removing an sstable in L6 following an @@ -255,7 +255,7 @@ L0.000001 a-z seqnums(tombstone=5-27, file-smallest=0, type=point-key-only) close-snapshot 10 ---- -[JOB 100] compacted(elision-only) L6 [000004] (743B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (662B), in 1.0s (2.0s total), output rate 662B/s +[JOB 100] compacted(elision-only) L6 [000004] (813B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (732B), in 1.0s (2.0s total), output rate 732B/s # In previous versions of the code, the deletion hint was removed by the # elision-only compaction because it zeroed sequence numbers of keys with @@ -430,7 +430,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) L6 [000006 000007 000008 000009 000011] (3.6KB) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s + [JOB 100] compacted(delete-only) L6 [000006 000007 000008 000009 000011] (3.9KB) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s # Verify that a delete-only compaction can partially excise a file. @@ -474,7 +474,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) multilevel L1 [000005] (660B) Score=0.00 + L2 [000006] (673B) Score=0.00 + L3 [000007] (673B) Score=0.00 + L4 [000008] (673B) Score=0.00 -> L6 [000009 000010] (95B), in 1.0s (2.0s total), output rate 95B/s + [JOB 100] compacted(delete-only) multilevel L1 [000005] (730B) Score=0.00 + L2 [000006] (743B) Score=0.00 + L3 [000007] (743B) Score=0.00 + L4 [000008] (743B) Score=0.00 -> L6 [000009 000010] (95B), in 1.0s (2.0s total), output rate 95B/s describe-lsm ---- @@ -542,7 +542,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) L6 [000005] (660B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s + [JOB 100] compacted(delete-only) L6 [000005] (730B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s describe-lsm ---- @@ -608,7 +608,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) L6 [000004] (849B) Score=0.00 -> L6 [000007 000008] (186B), in 1.0s (2.0s total), output rate 186B/s + [JOB 100] compacted(delete-only) L6 [000004] (912B) Score=0.00 -> L6 [000007 000008] (186B), in 1.0s (2.0s total), output rate 186B/s describe-lsm ---- diff --git a/testdata/compaction_picker_pick_file b/testdata/compaction_picker_pick_file index 17bc14bc79..d9991fd759 100644 --- a/testdata/compaction_picker_pick_file +++ b/testdata/compaction_picker_pick_file @@ -171,7 +171,7 @@ L5: 000005:[f#11,SET-f#11,SET]: 57942 bytes (57KB) L6: 000006:[c#0,SET-c#0,SET]: 66134 bytes (65KB) - 000009:[d#13,SET-d#13,SET]: 660 bytes (660B) + 000009:[d#13,SET-d#13,SET]: 728 bytes (728B) 000007:[e#0,SET-e#0,SET]: 66134 bytes (65KB) 000008:[f#0,SET-f#0,SET]: 66134 bytes (65KB) @@ -209,8 +209,8 @@ L5: 000011:[e#11,SET-e#11,SET]: 126 bytes (126B) 000005:[f#11,SET-f#11,SET]: 57942 bytes (57KB) L6: - 000012:[c#15,SET-c#15,SET]: 660 bytes (660B) - 000009:[d#13,SET-d#13,SET]: 660 bytes (660B) + 000012:[c#15,SET-c#15,SET]: 728 bytes (728B) + 000009:[d#13,SET-d#13,SET]: 728 bytes (728B) 000007:[e#0,SET-e#0,SET]: 66134 bytes (65KB) 000008:[f#0,SET-f#0,SET]: 66134 bytes (65KB) diff --git a/testdata/compaction_picker_scores b/testdata/compaction_picker_scores index f0cb04d5da..bc04732377 100644 --- a/testdata/compaction_picker_scores +++ b/testdata/compaction_picker_scores @@ -145,7 +145,7 @@ wait-pending-table-stats num-entries: 5 num-deletions: 5 num-range-key-sets: 0 -point-deletions-bytes-estimate: 164552 +point-deletions-bytes-estimate: 164583 range-deletions-bytes-estimate: 0 maybe-compact diff --git a/testdata/compaction_tombstones b/testdata/compaction_tombstones index 82904edad0..c5257259e0 100644 --- a/testdata/compaction_tombstones +++ b/testdata/compaction_tombstones @@ -41,7 +41,7 @@ range-deletions-bytes-estimate: 0 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (660B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s +[JOB 100] compacted(elision-only) L6 [000004] (730B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s # Test a table that straddles a snapshot. It should not be compacted. define snapshots=(50) auto-compactions=off @@ -80,12 +80,12 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 93 +point-deletions-bytes-estimate: 101 range-deletions-bytes-estimate: 0 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (711B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (662B), in 1.0s (2.0s total), output rate 662B/s +[JOB 100] compacted(elision-only) L6 [000004] (774B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (732B), in 1.0s (2.0s total), output rate 732B/s version ---- @@ -119,7 +119,7 @@ wait-pending-table-stats num-entries: 6 num-deletions: 2 num-range-key-sets: 0 -point-deletions-bytes-estimate: 45 +point-deletions-bytes-estimate: 49 range-deletions-bytes-estimate: 101 maybe-compact @@ -134,7 +134,7 @@ close-snapshot close-snapshot 103 ---- -[JOB 100] compacted(elision-only) L6 [000004] (875B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s +[JOB 100] compacted(elision-only) L6 [000004] (938B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s # Test a table that contains both deletions and non-deletions, but whose # non-deletions well outnumber its deletions. The table should not be @@ -152,7 +152,7 @@ wait-pending-table-stats num-entries: 11 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 22 +point-deletions-bytes-estimate: 23 range-deletions-bytes-estimate: 0 close-snapshot @@ -208,7 +208,7 @@ range-deletions-bytes-estimate: 16824 maybe-compact ---- -[JOB 100] compacted(default) L5 [000004 000005] (26KB) Score=88.38 + L6 [000007] (17KB) Score=0.73 -> L6 [000009] (25KB), in 1.0s (2.0s total), output rate 25KB/s +[JOB 100] compacted(default) L5 [000004 000005] (26KB) Score=88.86 + L6 [000007] (17KB) Score=0.73 -> L6 [000009] (25KB), in 1.0s (2.0s total), output rate 25KB/s define level-max-bytes=(L5 : 1000) auto-compactions=off L5 @@ -233,7 +233,7 @@ wait-pending-table-stats num-entries: 3 num-deletions: 3 num-range-key-sets: 0 -point-deletions-bytes-estimate: 6917 +point-deletions-bytes-estimate: 6983 range-deletions-bytes-estimate: 0 # By plain file size, 000005 should be picked because it is larger and @@ -243,7 +243,7 @@ range-deletions-bytes-estimate: 0 maybe-compact ---- -[JOB 100] compacted(default) L5 [000004] (715B) Score=13.49 + L6 [000006] (13KB) Score=0.92 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s +[JOB 100] compacted(default) L5 [000004] (778B) Score=13.75 + L6 [000006] (13KB) Score=0.92 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s # A table containing only range keys is not eligible for elision. # RANGEKEYDEL or RANGEKEYUNSET. @@ -323,7 +323,7 @@ range-deletions-bytes-estimate: 94 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (926B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (660B), in 1.0s (2.0s total), output rate 660B/s +[JOB 100] compacted(elision-only) L6 [000004] (989B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (730B), in 1.0s (2.0s total), output rate 730B/s # Close the DB, asserting that the reference counts balance. close @@ -359,7 +359,7 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 2766 +point-deletions-bytes-estimate: 2793 range-deletions-bytes-estimate: 0 wait-pending-table-stats @@ -402,7 +402,7 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 2766 +point-deletions-bytes-estimate: 2793 range-deletions-bytes-estimate: 0 wait-pending-table-stats diff --git a/testdata/event_listener b/testdata/event_listener index 31963e91e2..a66a55a61e 100644 --- a/testdata/event_listener +++ b/testdata/event_listener @@ -84,7 +84,7 @@ close: db/marker.manifest.000002.MANIFEST-000006 remove: db/marker.manifest.000001.MANIFEST-000001 sync: db [JOB 3] MANIFEST created 000006 -[JOB 3] flushed 1 memtable (100B) to L0 [000005] (656B), in 1.0s (2.0s total), output rate 656B/s +[JOB 3] flushed 1 memtable (100B) to L0 [000005] (726B), in 1.0s (2.0s total), output rate 726B/s compact ---- @@ -108,18 +108,18 @@ close: db/marker.manifest.000003.MANIFEST-000009 remove: db/marker.manifest.000002.MANIFEST-000006 sync: db [JOB 5] MANIFEST created 000009 -[JOB 5] flushed 1 memtable (100B) to L0 [000008] (656B), in 1.0s (2.0s total), output rate 656B/s +[JOB 5] flushed 1 memtable (100B) to L0 [000008] (726B), in 1.0s (2.0s total), output rate 726B/s remove: db/MANIFEST-000001 [JOB 5] MANIFEST deleted 000001 -[JOB 6] compacting(default) L0 [000005 000008] (1.3KB) Score=0.00 + L6 [] (0B) Score=0.00; OverlappingRatio: Single 0.00, Multi 0.00 +[JOB 6] compacting(default) L0 [000005 000008] (1.4KB) Score=0.00 + L6 [] (0B) Score=0.00; OverlappingRatio: Single 0.00, Multi 0.00 open: db/000005.sst (options: *vfs.randomReadsOption) -read-at(603, 53): db/000005.sst -read-at(566, 37): db/000005.sst -read-at(119, 447): db/000005.sst +read-at(673, 53): db/000005.sst +read-at(636, 37): db/000005.sst +read-at(119, 517): db/000005.sst open: db/000008.sst (options: *vfs.randomReadsOption) -read-at(603, 53): db/000008.sst -read-at(566, 37): db/000008.sst -read-at(119, 447): db/000008.sst +read-at(673, 53): db/000008.sst +read-at(636, 37): db/000008.sst +read-at(119, 517): db/000008.sst read-at(78, 41): db/000005.sst open: db/000005.sst (options: *vfs.sequentialReadsOption) read-at(0, 78): db/000005.sst @@ -141,7 +141,7 @@ close: db/marker.manifest.000004.MANIFEST-000011 remove: db/marker.manifest.000003.MANIFEST-000009 sync: db [JOB 6] MANIFEST created 000011 -[JOB 6] compacted(default) L0 [000005 000008] (1.3KB) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000010] (659B), in 1.0s (3.0s total), output rate 659B/s +[JOB 6] compacted(default) L0 [000005 000008] (1.4KB) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000010] (729B), in 1.0s (3.0s total), output rate 729B/s close: db/000005.sst close: db/000008.sst remove: db/000005.sst @@ -176,7 +176,7 @@ close: db/marker.manifest.000005.MANIFEST-000014 remove: db/marker.manifest.000004.MANIFEST-000011 sync: db [JOB 8] MANIFEST created 000014 -[JOB 8] flushed 1 memtable (100B) to L0 [000013] (656B), in 1.0s (2.0s total), output rate 656B/s +[JOB 8] flushed 1 memtable (100B) to L0 [000013] (726B), in 1.0s (2.0s total), output rate 726B/s enable-file-deletions ---- @@ -186,9 +186,9 @@ remove: db/MANIFEST-000009 ingest ---- open: ext/0 -read-at(606, 53): ext/0 -read-at(569, 37): ext/0 -read-at(122, 447): ext/0 +read-at(676, 53): ext/0 +read-at(639, 37): ext/0 +read-at(122, 517): ext/0 read-at(81, 41): ext/0 read-at(0, 81): ext/0 close: ext/0 @@ -206,31 +206,31 @@ sync: db remove: db/MANIFEST-000011 [JOB 10] MANIFEST deleted 000011 remove: ext/0 -[JOB 10] ingested L0:000015 (659B) +[JOB 10] ingested L0:000015 (729B) metrics ---- | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 2 1.3KB 0B 0 | 0.40 | 81B | 1 659B | 0 0B | 3 1.9KB | 0B | 2 24.3 + 0 | 2 1.4KB 0B 0 | 0.40 | 81B | 1 729B | 0 0B | 3 2.1KB | 0B | 2 26.9 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 - 6 | 1 659B 0B 0 | - | 1.3KB | 0 0B | 0 0B | 1 659B | 1.3KB | 1 0.5 -total | 3 1.9KB 0B 0 | - | 740B | 1 659B | 0 0B | 4 3.3KB | 1.3KB | 3 4.5 + 6 | 1 729B 0B 0 | - | 1.4KB | 0 0B | 0 0B | 1 729B | 1.4KB | 1 0.5 +total | 3 2.1KB 0B 0 | - | 810B | 1 729B | 0 0B | 4 3.6KB | 1.4KB | 3 4.6 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 48B written: 81B (69% overhead) Flushes: 3 -Compactions: 1 estimated debt: 1.9KB in progress: 0 (0B) +Compactions: 1 estimated debt: 2.1KB in progress: 0 (0B) default: 1 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 0 multi-level: 0 MemTables: 1 (256KB) zombie: 1 (256KB) Zombie tables: 0 (0B, local: 0B) Backing tables: 0 (0B) Virtual tables: 0 (0B) -Local tables size: 1.9KB +Local tables size: 2.1KB Compression types: snappy: 3 Block cache: 2 entries (112B) hit rate: 0.0% Table cache: 0 entries (0B) hit rate: 50.0% @@ -249,16 +249,16 @@ ingest-flushable ---- sync-data: wal/000012.log open: ext/a -read-at(606, 53): ext/a -read-at(569, 37): ext/a -read-at(122, 447): ext/a +read-at(676, 53): ext/a +read-at(639, 37): ext/a +read-at(122, 517): ext/a read-at(81, 41): ext/a read-at(0, 81): ext/a close: ext/a open: ext/b -read-at(606, 53): ext/b -read-at(569, 37): ext/b -read-at(122, 447): ext/b +read-at(676, 53): ext/b +read-at(639, 37): ext/b +read-at(122, 517): ext/b read-at(81, 41): ext/b read-at(0, 81): ext/b close: ext/b @@ -280,7 +280,7 @@ sync: wal [JOB 13] WAL created 000020 remove: ext/a remove: ext/b -[JOB 11] ingested as flushable 000017 (659B), 000018 (659B) +[JOB 11] ingested as flushable 000017 (729B), 000018 (729B) sync-data: wal/000020.log close: wal/000020.log create: wal/000021.log @@ -293,7 +293,7 @@ sync-data: db/000022.sst close: db/000022.sst sync: db sync: db/MANIFEST-000016 -[JOB 15] flushed 1 memtable (100B) to L0 [000022] (656B), in 1.0s (2.0s total), output rate 656B/s +[JOB 15] flushed 1 memtable (100B) to L0 [000022] (726B), in 1.0s (2.0s total), output rate 726B/s [JOB 16] flushing 2 ingested tables create: db/MANIFEST-000023 close: db/MANIFEST-000016 @@ -303,7 +303,7 @@ close: db/marker.manifest.000007.MANIFEST-000023 remove: db/marker.manifest.000006.MANIFEST-000016 sync: db [JOB 16] MANIFEST created 000023 -[JOB 16] flushed 2 ingested flushables L0:000017 (659B) + L6:000018 (659B) in 1.0s (2.0s total), output rate 1.3KB/s +[JOB 16] flushed 2 ingested flushables L0:000017 (729B) + L6:000018 (729B) in 1.0s (2.0s total), output rate 1.4KB/s remove: db/MANIFEST-000014 [JOB 16] MANIFEST deleted 000014 [JOB 17] flushing 1 memtable (100B) to L0 @@ -315,24 +315,24 @@ metrics | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 4 2.6KB 0B 0 | 0.80 | 108B | 2 1.3KB | 0 0B | 4 2.6KB | 0B | 4 24.3 + 0 | 4 2.8KB 0B 0 | 0.80 | 108B | 2 1.4KB | 0 0B | 4 2.8KB | 0B | 4 26.9 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 - 6 | 2 1.3KB 0B 0 | - | 1.3KB | 1 659B | 0 0B | 1 659B | 1.3KB | 1 0.5 -total | 6 3.9KB 0B 0 | - | 2.0KB | 3 1.9KB | 0 0B | 5 5.2KB | 1.3KB | 5 2.6 + 6 | 2 1.4KB 0B 0 | - | 1.4KB | 1 729B | 0 0B | 1 729B | 1.4KB | 1 0.5 +total | 6 4.3KB 0B 0 | - | 2.2KB | 3 2.1KB | 0 0B | 5 5.8KB | 1.4KB | 5 2.6 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 82B written: 108B (32% overhead) Flushes: 6 -Compactions: 1 estimated debt: 3.9KB in progress: 0 (0B) +Compactions: 1 estimated debt: 4.3KB in progress: 0 (0B) default: 1 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 0 multi-level: 0 MemTables: 1 (512KB) zombie: 1 (512KB) Zombie tables: 0 (0B, local: 0B) Backing tables: 0 (0B) Virtual tables: 0 (0B) -Local tables size: 3.9KB +Local tables size: 4.3KB Compression types: snappy: 6 Block cache: 6 entries (336B) hit rate: 0.0% Table cache: 0 entries (0B) hit rate: 50.0% @@ -340,7 +340,7 @@ Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 Filter utility: 0.0% -Ingestions: 2 as flushable: 1 (1.3KB in 2 tables) +Ingestions: 2 as flushable: 1 (1.4KB in 2 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B sstables diff --git a/testdata/ingest b/testdata/ingest index 9eccab8dbc..494098d820 100644 --- a/testdata/ingest +++ b/testdata/ingest @@ -54,7 +54,7 @@ Virtual tables: 0 (0B) Local tables size: 569B Compression types: snappy: 1 Block cache: 3 entries (84B) hit rate: 18.2% -Table cache: 1 entries (784B) hit rate: 50.0% +Table cache: 1 entries (816B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -359,7 +359,7 @@ num-entries: 2 num-deletions: 2 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 1229 +range-deletions-bytes-estimate: 1299 # A set operation takes precedence over a range deletion at the same # sequence number as can occur during ingestion. diff --git a/testdata/ingest_external b/testdata/ingest_external index 1e675bfba8..5f5bca5b18 100644 --- a/testdata/ingest_external +++ b/testdata/ingest_external @@ -330,8 +330,8 @@ gi: (foo, .) lsm verbose ---- L6: - 000004(000004):[gc#10,DELSIZED-gf#inf,RANGEDEL] seqnums:[10-10] points:[gc#10,DELSIZED-gf#inf,RANGEDEL] size:1261 - 000005(000005):[gg#11,DELSIZED-gj#inf,RANGEDEL] seqnums:[11-11] points:[gg#11,DELSIZED-gj#inf,RANGEDEL] size:1016 + 000004(000004):[gc#10,DELSIZED-gf#inf,RANGEDEL] seqnums:[10-10] points:[gc#10,DELSIZED-gf#inf,RANGEDEL] size:1329 + 000005(000005):[gg#11,DELSIZED-gj#inf,RANGEDEL] seqnums:[11-11] points:[gg#11,DELSIZED-gj#inf,RANGEDEL] size:1084 download g h via-backing-file-download ---- @@ -341,8 +341,8 @@ ok lsm verbose ---- L6: - 000006(000006):[gc#10,DELSIZED-gf#inf,RANGEDEL] seqnums:[10-10] points:[gc#10,DELSIZED-gf#inf,RANGEDEL] size:908 - 000007(000007):[gg#11,DELSIZED-gj#inf,RANGEDEL] seqnums:[11-11] points:[gg#11,DELSIZED-gj#inf,RANGEDEL] size:916 + 000006(000006):[gc#10,DELSIZED-gf#inf,RANGEDEL] seqnums:[10-10] points:[gc#10,DELSIZED-gf#inf,RANGEDEL] size:976 + 000007(000007):[gg#11,DELSIZED-gj#inf,RANGEDEL] seqnums:[11-11] points:[gg#11,DELSIZED-gj#inf,RANGEDEL] size:984 reopen ---- diff --git a/testdata/manual_compaction_set_with_del_sstable_Pebblev5 b/testdata/manual_compaction_set_with_del_sstable_Pebblev5 index 145bdcdbbf..ea38a49d18 100644 --- a/testdata/manual_compaction_set_with_del_sstable_Pebblev5 +++ b/testdata/manual_compaction_set_with_del_sstable_Pebblev5 @@ -88,7 +88,7 @@ num-entries: 1 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 1320 +range-deletions-bytes-estimate: 1460 compact a-e L1 ---- @@ -106,7 +106,7 @@ num-entries: 2 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 660 +range-deletions-bytes-estimate: 730 # Same as above, except range tombstone covers multiple grandparent file boundaries. diff --git a/testdata/marked_for_compaction b/testdata/marked_for_compaction index 5d4a9966a6..42f6bb7213 100644 --- a/testdata/marked_for_compaction +++ b/testdata/marked_for_compaction @@ -6,9 +6,9 @@ L1 d.SET.0:foo ---- L0.0: - 000004:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:653 + 000004:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:723 L1: - 000005:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:664 + 000005:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:734 mark-for-compaction file=000005 ---- @@ -20,9 +20,9 @@ marked L0.000004 maybe-compact ---- -[JOB 100] compacted(rewrite) L1 [000005] (664B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000006] (664B), in 1.0s (2.0s total), output rate 664B/s -[JOB 100] compacted(rewrite) L0 [000004] (653B) Score=0.00 + L0 [] (0B) Score=0.00 -> L0 [000007] (653B), in 1.0s (2.0s total), output rate 653B/s +[JOB 100] compacted(rewrite) L1 [000005] (734B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000006] (734B), in 1.0s (2.0s total), output rate 734B/s +[JOB 100] compacted(rewrite) L0 [000004] (723B) Score=0.00 + L0 [] (0B) Score=0.00 -> L0 [000007] (723B), in 1.0s (2.0s total), output rate 723B/s L0.0: - 000007:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:653 + 000007:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:723 L1: - 000006:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:664 + 000006:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:734 diff --git a/testdata/metrics b/testdata/metrics index 22c6a878df..c6f42ede44 100644 --- a/testdata/metrics +++ b/testdata/metrics @@ -76,7 +76,7 @@ Virtual tables: 0 (0B) Local tables size: 589B Compression types: snappy: 1 Block cache: 2 entries (44B) hit rate: 0.0% -Table cache: 1 entries (784B) hit rate: 0.0% +Table cache: 1 entries (816B) hit rate: 0.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 1 @@ -133,7 +133,7 @@ Virtual tables: 0 (0B) Local tables size: 595B Compression types: snappy: 1 Block cache: 2 entries (44B) hit rate: 33.3% -Table cache: 2 entries (1.5KB) hit rate: 66.7% +Table cache: 2 entries (1.6KB) hit rate: 66.7% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 2 @@ -177,7 +177,7 @@ Virtual tables: 0 (0B) Local tables size: 595B Compression types: snappy: 1 Block cache: 2 entries (44B) hit rate: 33.3% -Table cache: 2 entries (1.5KB) hit rate: 66.7% +Table cache: 2 entries (1.6KB) hit rate: 66.7% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 2 @@ -218,7 +218,7 @@ Virtual tables: 0 (0B) Local tables size: 595B Compression types: snappy: 1 Block cache: 2 entries (44B) hit rate: 33.3% -Table cache: 1 entries (784B) hit rate: 66.7% +Table cache: 1 entries (816B) hit rate: 66.7% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 1 @@ -231,7 +231,7 @@ Iter category stats: disk-usage ---- -2.7KB +2.8KB # Closing iter b will release the last zombie sstable and the last zombie memtable. @@ -277,7 +277,7 @@ Iter category stats: disk-usage ---- -2.1KB +2.2KB additional-metrics ---- @@ -476,32 +476,32 @@ metrics | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 4 2.5KB 0B 0 | 0.50 | 187B | 3 1.9KB | 0 0B | 6 3.8KB | 0B | 2 20.6 + 0 | 4 2.7KB 0B 0 | 0.50 | 187B | 3 2.1KB | 0 0B | 6 3.8KB | 0B | 2 20.6 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 6 | 3 2.0KB 41B 0 | - | 3.2KB | 0 0B | 0 0B | 3 2.0KB | 3.2KB | 1 0.6 -total | 7 4.5KB 41B 0 | - | 2.1KB | 3 1.9KB | 0 0B | 9 7.9KB | 3.2KB | 3 3.7 +total | 7 4.7KB 41B 0 | - | 2.3KB | 3 2.1KB | 0 0B | 9 8.1KB | 3.2KB | 3 3.5 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 176B written: 187B (6% overhead) Flushes: 8 -Compactions: 2 estimated debt: 4.5KB in progress: 0 (0B) +Compactions: 2 estimated debt: 4.7KB in progress: 0 (0B) default: 2 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 0 multi-level: 0 MemTables: 1 (1.0MB) zombie: 1 (1.0MB) Zombie tables: 0 (0B, local: 0B) Backing tables: 0 (0B) Virtual tables: 0 (0B) -Local tables size: 4.5KB +Local tables size: 4.7KB Compression types: snappy: 7 Block cache: 8 entries (417B) hit rate: 9.1% -Table cache: 1 entries (784B) hit rate: 53.8% +Table cache: 1 entries (816B) hit rate: 53.8% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 Filter utility: 0.0% -Ingestions: 2 as flushable: 2 (1.9KB in 3 tables) +Ingestions: 2 as flushable: 2 (2.1KB in 3 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} @@ -540,32 +540,32 @@ metrics | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 7 4.3KB 0B 0 | 0.50 | 245B | 3 1.9KB | 0 0B | 9 5.5KB | 0B | 2 23.1 + 0 | 7 4.5KB 0B 0 | 0.50 | 245B | 3 2.1KB | 0 0B | 9 5.5KB | 0B | 2 23.1 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 6 | 3 2.0KB 41B 0 | - | 3.2KB | 0 0B | 0 0B | 3 2.0KB | 3.2KB | 1 0.6 -total | 10 6.3KB 41B 0 | - | 2.2KB | 3 1.9KB | 0 0B | 12 9.7KB | 3.2KB | 3 4.5 +total | 10 6.5KB 41B 0 | - | 2.4KB | 3 2.1KB | 0 0B | 12 9.9KB | 3.2KB | 3 4.2 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 223B written: 245B (10% overhead) Flushes: 9 -Compactions: 2 estimated debt: 6.3KB in progress: 0 (0B) +Compactions: 2 estimated debt: 6.5KB in progress: 0 (0B) default: 2 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 0 multi-level: 0 MemTables: 1 (1.0MB) zombie: 1 (1.0MB) Zombie tables: 0 (0B, local: 0B) Backing tables: 0 (0B) Virtual tables: 0 (0B) -Local tables size: 6.3KB +Local tables size: 6.5KB Compression types: snappy: 10 Block cache: 8 entries (417B) hit rate: 9.1% -Table cache: 1 entries (784B) hit rate: 53.8% +Table cache: 1 entries (816B) hit rate: 53.8% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 Filter utility: 0.0% -Ingestions: 2 as flushable: 2 (1.9KB in 3 tables) +Ingestions: 2 as flushable: 2 (2.1KB in 3 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} @@ -618,24 +618,24 @@ metrics zero-cache-hits-misses | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 7 3.2KB 0B 2 | 0.50 | 245B | 3 1.9KB | 0 0B | 9 5.5KB | 0B | 2 23.1 + 0 | 7 3.4KB 0B 2 | 0.50 | 245B | 3 2.1KB | 0 0B | 9 5.5KB | 0B | 2 23.1 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 - 6 | 4 2.7KB 41B 0 | - | 3.2KB | 1 660B | 0 0B | 3 2.0KB | 3.2KB | 1 0.6 -total | 11 5.9KB 41B 2 | - | 2.8KB | 4 2.6KB | 0 0B | 12 10KB | 3.2KB | 3 3.7 + 6 | 4 2.7KB 41B 0 | - | 3.2KB | 1 728B | 0 0B | 3 2.0KB | 3.2KB | 1 0.6 +total | 11 6.1KB 41B 2 | - | 3.1KB | 4 2.8KB | 0 0B | 12 11KB | 3.2KB | 3 3.5 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 223B written: 245B (10% overhead) Flushes: 9 -Compactions: 2 estimated debt: 5.9KB in progress: 0 (0B) +Compactions: 2 estimated debt: 6.1KB in progress: 0 (0B) default: 2 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 0 multi-level: 0 MemTables: 1 (1.0MB) zombie: 1 (1.0MB) Zombie tables: 0 (0B, local: 0B) Backing tables: 2 (1.2KB) Virtual tables: 2 (102B) -Local tables size: 7.0KB +Local tables size: 7.2KB Compression types: snappy: 9 unknown: 2 Block cache: 0 entries (0B) hit rate: 0.0% Table cache: 0 entries (0B) hit rate: 0.0% @@ -643,7 +643,7 @@ Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 Filter utility: 0.0% -Ingestions: 3 as flushable: 2 (1.9KB in 3 tables) +Ingestions: 3 as flushable: 2 (2.1KB in 3 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} @@ -721,14 +721,14 @@ metrics zero-cache-hits-misses | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 0 0B 0B 0 | 0.00 | 245B | 3 1.9KB | 0 0B | 9 5.5KB | 0B | 0 23.1 + 0 | 0 0B 0B 0 | 0.00 | 245B | 3 2.1KB | 0 0B | 9 5.5KB | 0B | 0 23.1 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 - 6 | 6 3.9KB 41B 0 | - | 6.4KB | 2 1.3KB | 0 0B | 4 2.6KB | 6.4KB | 1 0.4 -total | 6 3.9KB 41B 0 | - | 3.5KB | 5 3.2KB | 0 0B | 13 12KB | 6.4KB | 1 3.4 + 6 | 6 4.1KB 41B 0 | - | 6.6KB | 2 1.4KB | 0 0B | 4 2.6KB | 6.6KB | 1 0.4 +total | 6 4.1KB 41B 0 | - | 3.8KB | 5 3.5KB | 0 0B | 13 12KB | 6.6KB | 1 3.2 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 223B written: 245B (10% overhead) Flushes: 9 @@ -738,7 +738,7 @@ MemTables: 1 (1.0MB) zombie: 1 (1.0MB) Zombie tables: 0 (0B, local: 0B) Backing tables: 0 (0B) Virtual tables: 0 (0B) -Local tables size: 3.9KB +Local tables size: 4.1KB Compression types: snappy: 6 Block cache: 0 entries (0B) hit rate: 0.0% Table cache: 0 entries (0B) hit rate: 0.0% @@ -746,7 +746,7 @@ Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 Filter utility: 0.0% -Ingestions: 4 as flushable: 2 (1.9KB in 3 tables) +Ingestions: 4 as flushable: 2 (2.1KB in 3 tables) Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B Iter category stats: b, latency: {BlockBytes:44 BlockBytesInCache:0 BlockReadDuration:10ms} @@ -835,7 +835,7 @@ Virtual tables: 0 (0B) Local tables size: 0B Compression types: snappy: 1 Block cache: 0 entries (0B) hit rate: 0.0% -Table cache: 1 entries (784B) hit rate: 0.0% +Table cache: 1 entries (816B) hit rate: 0.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -863,18 +863,18 @@ metrics | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 1 660B 0B 0 | 0.25 | 38B | 1 660B | 0 0B | 1 604B | 0B | 1 15.9 + 0 | 1 728B 0B 0 | 0.25 | 38B | 1 728B | 0 0B | 1 604B | 0B | 1 15.9 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 6 | 1 604B 0B 0 | - | 604B | 0 0B | 0 0B | 1 604B | 0B | 1 1.0 -total | 2 1.2KB 0B 0 | - | 698B | 1 660B | 0 0B | 2 1.9KB | 0B | 2 2.7 +total | 2 1.3KB 0B 0 | - | 766B | 1 728B | 0 0B | 2 1.9KB | 0B | 2 2.6 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 27B written: 38B (41% overhead) Flushes: 1 -Compactions: 1 estimated debt: 1.2KB in progress: 0 (0B) +Compactions: 1 estimated debt: 1.3KB in progress: 0 (0B) default: 0 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 1 multi-level: 0 MemTables: 1 (256KB) zombie: 1 (256KB) Zombie tables: 0 (0B, local: 0B) @@ -883,7 +883,7 @@ Virtual tables: 0 (0B) Local tables size: 0B Compression types: snappy: 2 Block cache: 4 entries (187B) hit rate: 0.0% -Table cache: 1 entries (784B) hit rate: 50.0% +Table cache: 1 entries (816B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -912,18 +912,18 @@ metrics | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 2 1.2KB 0B 0 | 0.50 | 66B | 1 660B | 0 0B | 2 1.2KB | 0B | 2 18.1 + 0 | 2 1.3KB 0B 0 | 0.50 | 66B | 1 728B | 0 0B | 2 1.2KB | 0B | 2 18.1 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 6 | 1 604B 0B 0 | - | 604B | 0 0B | 0 0B | 1 604B | 0B | 1 1.0 -total | 3 1.8KB 0B 0 | - | 726B | 1 660B | 0 0B | 3 2.5KB | 0B | 3 3.5 +total | 3 1.9KB 0B 0 | - | 794B | 1 728B | 0 0B | 3 2.5KB | 0B | 3 3.3 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 44B written: 66B (50% overhead) Flushes: 2 -Compactions: 1 estimated debt: 1.8KB in progress: 0 (0B) +Compactions: 1 estimated debt: 1.9KB in progress: 0 (0B) default: 0 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 1 multi-level: 0 MemTables: 1 (256KB) zombie: 1 (256KB) Zombie tables: 0 (0B, local: 0B) @@ -932,7 +932,7 @@ Virtual tables: 0 (0B) Local tables size: 589B Compression types: snappy: 3 Block cache: 4 entries (187B) hit rate: 0.0% -Table cache: 1 entries (784B) hit rate: 50.0% +Table cache: 1 entries (816B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -952,18 +952,18 @@ metrics zero-cache-hits-misses | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 2 1.2KB 0B 0 | 0.50 | 0B | 0 0B | 0 0B | 0 0B | 0B | 2 0.0 + 0 | 2 1.3KB 0B 0 | 0.50 | 0B | 0 0B | 0 0B | 0 0B | 0B | 2 0.0 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 6 | 1 604B 0B 0 | - | 0B | 0 0B | 0 0B | 0 0B | 0B | 1 0.0 -total | 3 1.8KB 0B 0 | - | 0B | 0 0B | 0 0B | 0 0B | 0B | 3 0.0 +total | 3 1.9KB 0B 0 | - | 0B | 0 0B | 0 0B | 0 0B | 0B | 3 0.0 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 0B written: 0B (0% overhead) Flushes: 1 -Compactions: 0 estimated debt: 1.8KB in progress: 0 (0B) +Compactions: 0 estimated debt: 1.9KB in progress: 0 (0B) default: 0 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 0 multi-level: 0 MemTables: 1 (512KB) zombie: 1 (256KB) Zombie tables: 0 (0B, local: 0B) @@ -997,8 +997,8 @@ level | tables size val-bl vtables | score | in | tables size | tables siz 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 - 6 | 1 603B 0B 0 | - | 1.2KB | 0 0B | 0 0B | 1 603B | 1.8KB | 1 0.5 -total | 1 603B 0B 0 | - | 0B | 0 0B | 0 0B | 1 603B | 1.8KB | 1 0.0 + 6 | 1 603B 0B 0 | - | 1.3KB | 0 0B | 0 0B | 1 603B | 1.9KB | 1 0.5 +total | 1 603B 0B 0 | - | 0B | 0 0B | 0 0B | 1 603B | 1.9KB | 1 0.0 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 0B written: 0B (0% overhead) Flushes: 1 diff --git a/tool/sstable.go b/tool/sstable.go index 91c76a673a..ae8f1d2d3e 100644 --- a/tool/sstable.go +++ b/tool/sstable.go @@ -148,11 +148,11 @@ func (s *sstableT) newReader(f vfs.File) (*sstable.Reader, error) { return nil, err } o := sstable.ReaderOptions{ - Comparer: s.opts.Comparer, - Comparers: s.comparers, - KeySchema: s.opts.KeySchema, - Filters: s.opts.Filters, - Mergers: s.mergers, + Comparer: s.opts.Comparer, + Comparers: s.comparers, + KeySchemas: s.opts.KeySchemas, + Filters: s.opts.Filters, + Mergers: s.mergers, } c := pebble.NewCache(128 << 20 /* 128 MB */) defer c.Unref() @@ -339,6 +339,7 @@ func (s *sstableT) runProperties(cmd *cobra.Command, args []string) { fmt.Fprintf(tw, " key\t") fmt.Fprintf(tw, " value\t") fmt.Fprintf(tw, "comparer\t%s\n", r.Properties.ComparerName) + fmt.Fprintf(tw, "key-schema\t%s\n", formatNull(r.Properties.KeySchemaName)) fmt.Fprintf(tw, "merger\t%s\n", formatNull(r.Properties.MergerName)) fmt.Fprintf(tw, "filter\t%s\n", formatNull(r.Properties.FilterPolicyName)) fmt.Fprintf(tw, "compression\t%s\n", r.Properties.CompressionName) diff --git a/tool/testdata/sstable_properties b/tool/testdata/sstable_properties index cbc9f541a6..76888870e4 100644 Binary files a/tool/testdata/sstable_properties and b/tool/testdata/sstable_properties differ diff --git a/tool/tool.go b/tool/tool.go index 055b438bc8..8d94180396 100644 --- a/tool/tool.go +++ b/tool/tool.go @@ -10,6 +10,7 @@ import ( "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/objstorage/remote" "github.com/cockroachdb/pebble/sstable" + "github.com/cockroachdb/pebble/sstable/colblk" "github.com/cockroachdb/pebble/vfs" "github.com/spf13/cobra" ) @@ -55,6 +56,16 @@ func Comparers(cmps ...*Comparer) Option { } } +// KeySchemas may be passed to New to register key schemas for use by the +// introspection tools. +func KeySchemas(schemas ...colblk.KeySchema) Option { + return func(t *T) { + for _, s := range schemas { + t.opts.KeySchemas[s.Name] = s + } + } +} + // DefaultComparer registers a comparer for use by the introspection tools and // sets it as the default. func DefaultComparer(c *Comparer) Option {