Skip to content

Commit 8d3363b

Browse files
committed
sstable/block: move category stats, read env
1 parent a18792e commit 8d3363b

25 files changed

+157
-144
lines changed

db.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ import (
3030
"github.com/cockroachdb/pebble/rangekey"
3131
"github.com/cockroachdb/pebble/record"
3232
"github.com/cockroachdb/pebble/sstable"
33+
"github.com/cockroachdb/pebble/sstable/block"
3334
"github.com/cockroachdb/pebble/vfs"
3435
"github.com/cockroachdb/pebble/vfs/atomicfs"
3536
"github.com/cockroachdb/pebble/wal"
@@ -1260,7 +1261,7 @@ func finishInitializingIter(ctx context.Context, buf *iterAlloc) *Iterator {
12601261
// iteration is invalid in those cases.
12611262
func (d *DB) ScanInternal(
12621263
ctx context.Context,
1263-
category sstable.Category,
1264+
category block.Category,
12641265
lower, upper []byte,
12651266
visitPointKey func(key *InternalKey, value LazyValue, iterInfo IteratorLevel) error,
12661267
visitRangeDel func(start, end []byte, seqNum SeqNum) error,

file_cache.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import (
2525
"github.com/cockroachdb/pebble/objstorage"
2626
"github.com/cockroachdb/pebble/objstorage/objstorageprovider/objiotracing"
2727
"github.com/cockroachdb/pebble/sstable"
28+
"github.com/cockroachdb/pebble/sstable/block"
2829
"github.com/cockroachdb/pebble/sstable/valblk"
2930
)
3031

@@ -101,7 +102,7 @@ type fileCacheOpts struct {
101102
cacheID cache.ID
102103
objProvider objstorage.Provider
103104
readerOpts sstable.ReaderOptions
104-
sstStatsCollector *sstable.CategoryStatsCollector
105+
sstStatsCollector *block.CategoryStatsCollector
105106
}
106107

107108
// fileCacheContainer contains the file cache and fields which are unique to the
@@ -122,7 +123,7 @@ func newFileCacheContainer(
122123
objProvider objstorage.Provider,
123124
opts *Options,
124125
size int,
125-
sstStatsCollector *sstable.CategoryStatsCollector,
126+
sstStatsCollector *block.CategoryStatsCollector,
126127
) *fileCacheContainer {
127128
// We will release a ref to the file cache acquired here when
128129
// fileCacheContainer.close is called.

file_cache_test.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import (
2525
"github.com/cockroachdb/pebble/objstorage"
2626
"github.com/cockroachdb/pebble/objstorage/objstorageprovider"
2727
"github.com/cockroachdb/pebble/sstable"
28+
"github.com/cockroachdb/pebble/sstable/block"
2829
"github.com/cockroachdb/pebble/vfs"
2930
"github.com/stretchr/testify/require"
3031
)
@@ -202,7 +203,7 @@ func newFileCacheContainerTest(
202203
}
203204

204205
c := newFileCacheContainer(tc, opts.Cache.NewID(), objProvider, opts, fileCacheTestCacheSize,
205-
&sstable.CategoryStatsCollector{})
206+
&block.CategoryStatsCollector{})
206207
return c, fs, nil
207208
}
208209

@@ -1014,7 +1015,7 @@ func TestFileCacheErrorBadMagicNumber(t *testing.T) {
10141015
opts.Cache = NewCache(8 << 20) // 8 MB
10151016
defer opts.Cache.Unref()
10161017
c := newFileCacheContainer(nil, opts.Cache.NewID(), objProvider, opts, fileCacheTestCacheSize,
1017-
&sstable.CategoryStatsCollector{})
1018+
&block.CategoryStatsCollector{})
10181019
require.NoError(t, err)
10191020
defer c.close()
10201021

ingest_test.go

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1133,7 +1133,7 @@ func testIngestSharedImpl(
11331133
w := sstable.NewRawWriter(objstorageprovider.NewFileWritable(f), writeOpts)
11341134

11351135
var sharedSSTs []SharedSSTMeta
1136-
err = from.ScanInternal(context.TODO(), sstable.CategoryUnknown, startKey, endKey,
1136+
err = from.ScanInternal(context.TODO(), block.CategoryUnknown, startKey, endKey,
11371137
func(key *InternalKey, value LazyValue, _ IteratorLevel) error {
11381138
val, _, err := value.Value(nil)
11391139
require.NoError(t, err)
@@ -1634,7 +1634,7 @@ func TestConcurrentExcise(t *testing.T) {
16341634
w := sstable.NewRawWriter(objstorageprovider.NewFileWritable(f), writeOpts)
16351635

16361636
var sharedSSTs []SharedSSTMeta
1637-
err = from.ScanInternal(context.TODO(), sstable.CategoryUnknown, startKey, endKey,
1637+
err = from.ScanInternal(context.TODO(), block.CategoryUnknown, startKey, endKey,
16381638
func(key *InternalKey, value LazyValue, _ IteratorLevel) error {
16391639
val, _, err := value.Value(nil)
16401640
require.NoError(t, err)
@@ -2071,7 +2071,7 @@ func TestIngestExternal(t *testing.T) {
20712071
w := sstable.NewRawWriter(objstorageprovider.NewFileWritable(f), writeOpts)
20722072

20732073
var externalFiles []ExternalFile
2074-
err = from.ScanInternal(context.TODO(), sstable.CategoryUnknown, startKey, endKey,
2074+
err = from.ScanInternal(context.TODO(), block.CategoryUnknown, startKey, endKey,
20752075
func(key *InternalKey, value LazyValue, _ IteratorLevel) error {
20762076
val, _, err := value.Value(nil)
20772077
require.NoError(t, err)

level_iter.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,16 +16,17 @@ import (
1616
"github.com/cockroachdb/pebble/internal/manifest"
1717
"github.com/cockroachdb/pebble/internal/treeprinter"
1818
"github.com/cockroachdb/pebble/sstable"
19+
"github.com/cockroachdb/pebble/sstable/block"
1920
)
2021

2122
type internalIterOpts struct {
2223
// if compaction is set, sstable-level iterators will be created using
2324
// NewCompactionIter; these iterators have a more constrained interface
2425
// and are optimized for the sequential scan of a compaction.
2526
compaction bool
26-
bufferPool *sstable.BufferPool
27+
bufferPool *block.BufferPool
2728
stats *base.InternalIteratorStats
28-
iterStatsAccumulator sstable.IterStatsAccumulator
29+
iterStatsAccumulator block.IterStatsAccumulator
2930
boundLimitedFilter sstable.BoundLimitedBlockPropertyFilter
3031
}
3132

metamorphic/ops.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import (
2424
"github.com/cockroachdb/pebble/internal/rangekey"
2525
"github.com/cockroachdb/pebble/objstorage/objstorageprovider"
2626
"github.com/cockroachdb/pebble/sstable"
27+
"github.com/cockroachdb/pebble/sstable/block"
2728
"github.com/cockroachdb/pebble/vfs"
2829
"github.com/cockroachdb/pebble/vfs/errorfs"
2930
)
@@ -1980,7 +1981,7 @@ func (r *replicateOp) runSharedReplicate(
19801981
) {
19811982
var sharedSSTs []pebble.SharedSSTMeta
19821983
var err error
1983-
err = source.ScanInternal(context.TODO(), sstable.CategoryUnknown, r.start, r.end,
1984+
err = source.ScanInternal(context.TODO(), block.CategoryUnknown, r.start, r.end,
19841985
func(key *pebble.InternalKey, value pebble.LazyValue, _ pebble.IteratorLevel) error {
19851986
val, _, err := value.Value(nil)
19861987
if err != nil {
@@ -2043,7 +2044,7 @@ func (r *replicateOp) runExternalReplicate(
20432044
) {
20442045
var externalSSTs []pebble.ExternalFile
20452046
var err error
2046-
err = source.ScanInternal(context.TODO(), sstable.CategoryUnknown, r.start, r.end,
2047+
err = source.ScanInternal(context.TODO(), block.CategoryUnknown, r.start, r.end,
20472048
func(key *pebble.InternalKey, value pebble.LazyValue, _ pebble.IteratorLevel) error {
20482049
val, _, err := value.Value(nil)
20492050
if err != nil {

metrics.go

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@ import (
1616
"github.com/cockroachdb/pebble/objstorage/objstorageprovider/sharedcache"
1717
"github.com/cockroachdb/pebble/record"
1818
"github.com/cockroachdb/pebble/sstable"
19+
"github.com/cockroachdb/pebble/sstable/block"
1920
"github.com/cockroachdb/pebble/wal"
2021
"github.com/cockroachdb/redact"
2122
"github.com/prometheus/client_golang/prometheus"
@@ -147,9 +148,9 @@ func (m *LevelMetrics) WriteAmp() float64 {
147148
return float64(m.BytesFlushed+m.BytesCompacted) / float64(m.BytesIn)
148149
}
149150

150-
var categoryCompaction = sstable.RegisterCategory("pebble-compaction", sstable.NonLatencySensitiveQoSLevel)
151-
var categoryIngest = sstable.RegisterCategory("pebble-ingest", sstable.LatencySensitiveQoSLevel)
152-
var categoryGet = sstable.RegisterCategory("pebble-get", sstable.LatencySensitiveQoSLevel)
151+
var categoryCompaction = block.RegisterCategory("pebble-compaction", block.NonLatencySensitiveQoSLevel)
152+
var categoryIngest = block.RegisterCategory("pebble-ingest", block.LatencySensitiveQoSLevel)
153+
var categoryGet = block.RegisterCategory("pebble-get", block.LatencySensitiveQoSLevel)
153154

154155
// Metrics holds metrics for various subsystems of the DB such as the Cache,
155156
// Compactions, WAL, and per-Level metrics.
@@ -338,7 +339,7 @@ type Metrics struct {
338339
record.LogWriterMetrics
339340
}
340341

341-
CategoryStats []sstable.CategoryStatsAggregate
342+
CategoryStats []block.CategoryStatsAggregate
342343

343344
SecondaryCacheMetrics SecondaryCacheMetrics
344345

metrics_test.go

Lines changed: 8 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import (
2222
"github.com/cockroachdb/pebble/internal/testkeys"
2323
"github.com/cockroachdb/pebble/objstorage/remote"
2424
"github.com/cockroachdb/pebble/sstable"
25+
"github.com/cockroachdb/pebble/sstable/block"
2526
"github.com/cockroachdb/pebble/vfs"
2627
"github.com/cockroachdb/pebble/vfs/errorfs"
2728
"github.com/cockroachdb/redact"
@@ -109,9 +110,9 @@ func exampleMetrics() Metrics {
109110

110111
func init() {
111112
// Register some categories for the purposes of the test.
112-
sstable.RegisterCategory("a", sstable.NonLatencySensitiveQoSLevel)
113-
sstable.RegisterCategory("b", sstable.LatencySensitiveQoSLevel)
114-
sstable.RegisterCategory("c", sstable.NonLatencySensitiveQoSLevel)
113+
block.RegisterCategory("a", block.NonLatencySensitiveQoSLevel)
114+
block.RegisterCategory("b", block.LatencySensitiveQoSLevel)
115+
block.RegisterCategory("c", block.NonLatencySensitiveQoSLevel)
115116
}
116117

117118
func TestMetrics(t *testing.T) {
@@ -313,11 +314,11 @@ func TestMetrics(t *testing.T) {
313314
return err.Error()
314315
}
315316
}
316-
category := sstable.CategoryUnknown
317+
category := block.CategoryUnknown
317318
if td.HasArg("category") {
318319
var s string
319320
td.ScanArgs(t, "category", &s)
320-
category = sstable.StringToCategoryForTesting(s)
321+
category = block.StringToCategoryForTesting(s)
321322
}
322323
iter, _ := d.NewIter(&IterOptions{Category: category})
323324
// Some iterators (eg. levelIter) do not instantiate the underlying
@@ -344,8 +345,8 @@ func TestMetrics(t *testing.T) {
344345
m.FileCache = cache.Metrics{}
345346
m.BlockCache = cache.Metrics{}
346347
// Empirically, the unknown stats are also non-deterministic.
347-
if len(m.CategoryStats) > 0 && m.CategoryStats[0].Category == sstable.CategoryUnknown {
348-
m.CategoryStats[0].CategoryStats = sstable.CategoryStats{}
348+
if len(m.CategoryStats) > 0 && m.CategoryStats[0].Category == block.CategoryUnknown {
349+
m.CategoryStats[0].CategoryStats = block.CategoryStats{}
349350
}
350351
}
351352
var buf strings.Builder

open.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ import (
3232
"github.com/cockroachdb/pebble/objstorage/objstorageprovider"
3333
"github.com/cockroachdb/pebble/objstorage/remote"
3434
"github.com/cockroachdb/pebble/record"
35-
"github.com/cockroachdb/pebble/sstable"
35+
"github.com/cockroachdb/pebble/sstable/block"
3636
"github.com/cockroachdb/pebble/vfs"
3737
"github.com/cockroachdb/pebble/wal"
3838
"github.com/prometheus/client_golang/prometheus"
@@ -411,7 +411,7 @@ func Open(dirname string, opts *Options) (db *DB, err error) {
411411
fileCacheSize := FileCacheSize(opts.MaxOpenFiles)
412412
d.fileCache = newFileCacheContainer(
413413
opts.FileCache, d.cacheID, d.objProvider, d.opts, fileCacheSize,
414-
&sstable.CategoryStatsCollector{})
414+
&block.CategoryStatsCollector{})
415415
d.newIters = d.fileCache.newIters
416416
d.tableNewRangeKeyIter = tableNewRangeKeyIter(d.newIters)
417417

options.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -197,7 +197,7 @@ type IterOptions struct {
197197
UseL6Filters bool
198198
// Category is used for categorized iterator stats. This should not be
199199
// changed by calling SetOptions.
200-
Category sstable.Category
200+
Category block.Category
201201

202202
DebugRangeKeyStack bool
203203

@@ -270,7 +270,7 @@ func (o *IterOptions) SpanIterOptions() keyspan.SpanIterOptions {
270270
type scanInternalOptions struct {
271271
IterOptions
272272

273-
category sstable.Category
273+
category block.Category
274274

275275
visitPointKey func(key *InternalKey, value LazyValue, iterInfo IteratorLevel) error
276276
visitRangeDel func(start, end []byte, seqNum SeqNum) error

0 commit comments

Comments
 (0)