Skip to content

Commit

Permalink
sstable/block: move Compression enum from sstable
Browse files Browse the repository at this point in the history
Move the sstable.Compression enum into the sstable/block package.
  • Loading branch information
jbowens committed Jul 31, 2024
1 parent e3782de commit c73ffdb
Show file tree
Hide file tree
Showing 19 changed files with 122 additions and 105 deletions.
3 changes: 2 additions & 1 deletion ingest_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/cockroachdb/pebble/objstorage/remote"
"github.com/cockroachdb/pebble/record"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/sstable/block"
"github.com/cockroachdb/pebble/vfs"
"github.com/cockroachdb/pebble/vfs/errorfs"
"github.com/kr/pretty"
Expand Down Expand Up @@ -172,7 +173,7 @@ func TestIngestLoadRand(t *testing.T) {
},
path: paths[i],
}
expected[i].fileMetadata.Stats.CompressionType = sstable.SnappyCompression
expected[i].fileMetadata.Stats.CompressionType = block.SnappyCompression
expected[i].StatsMarkValid()

func() {
Expand Down
3 changes: 2 additions & 1 deletion internal/manifest/version.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/sstable/block"
)

// Compare exports the base.Compare type.
Expand Down Expand Up @@ -73,7 +74,7 @@ type TableStats struct {
// Total size of value blocks and value index block.
ValueBlocksSize uint64
// CompressionType is the compression type of the table.
CompressionType sstable.Compression
CompressionType block.Compression
}

// boundType represents the type of key (point or range) present as the smallest
Expand Down
7 changes: 4 additions & 3 deletions metamorphic/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/objstorage/remote"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/sstable/block"
"github.com/cockroachdb/pebble/vfs"
"github.com/cockroachdb/pebble/wal"
"golang.org/x/exp/rand"
Expand Down Expand Up @@ -730,11 +731,11 @@ func RandomOptions(
// We use either no compression, snappy compression or zstd compression.
switch rng.Intn(3) {
case 0:
lopts.Compression = func() sstable.Compression { return pebble.NoCompression }
lopts.Compression = func() block.Compression { return pebble.NoCompression }
case 1:
lopts.Compression = func() sstable.Compression { return pebble.ZstdCompression }
lopts.Compression = func() block.Compression { return pebble.ZstdCompression }
default:
lopts.Compression = func() sstable.Compression { return pebble.SnappyCompression }
lopts.Compression = func() block.Compression { return pebble.SnappyCompression }
}
opts.Levels = []pebble.LevelOptions{lopts}

Expand Down
21 changes: 11 additions & 10 deletions options.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/pebble/objstorage/remote"
"github.com/cockroachdb/pebble/rangekey"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/sstable/block"
"github.com/cockroachdb/pebble/vfs"
"github.com/cockroachdb/pebble/wal"
)
Expand All @@ -35,14 +36,14 @@ const (
)

// Compression exports the base.Compression type.
type Compression = sstable.Compression
type Compression = block.Compression

// Exported Compression constants.
const (
DefaultCompression = sstable.DefaultCompression
NoCompression = sstable.NoCompression
SnappyCompression = sstable.SnappyCompression
ZstdCompression = sstable.ZstdCompression
DefaultCompression = block.DefaultCompression
NoCompression = block.NoCompression
SnappyCompression = block.SnappyCompression
ZstdCompression = block.ZstdCompression
)

// FilterType exports the base.FilterType type.
Expand Down Expand Up @@ -1808,13 +1809,13 @@ func (o *Options) Parse(s string, hooks *ParseHooks) error {
case "compression":
switch value {
case "Default":
l.Compression = func() sstable.Compression { return DefaultCompression }
l.Compression = func() Compression { return DefaultCompression }
case "NoCompression":
l.Compression = func() sstable.Compression { return NoCompression }
l.Compression = func() Compression { return NoCompression }
case "Snappy":
l.Compression = func() sstable.Compression { return SnappyCompression }
l.Compression = func() Compression { return SnappyCompression }
case "ZSTD":
l.Compression = func() sstable.Compression { return ZstdCompression }
l.Compression = func() Compression { return ZstdCompression }
default:
return errors.Errorf("pebble: unknown compression: %q", errors.Safe(value))
}
Expand Down Expand Up @@ -1988,7 +1989,7 @@ func (o *Options) MakeWriterOptions(level int, format sstable.TableFormat) sstab
}

func resolveDefaultCompression(c Compression) Compression {
if c <= DefaultCompression || c >= sstable.NCompression {
if c <= DefaultCompression || c >= block.NCompression {
c = SnappyCompression
}
return c
Expand Down
51 changes: 51 additions & 0 deletions sstable/block/compression.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
// Copyright 2024 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.

package block

// Compression is the per-block compression algorithm to use.
type Compression int

// The available compression types.
const (
DefaultCompression Compression = iota
NoCompression
SnappyCompression
ZstdCompression
NCompression
)

// String implements fmt.Stringer, returning a human-readable name for the
// compression algorithm.
func (c Compression) String() string {
switch c {
case DefaultCompression:
return "Default"
case NoCompression:
return "NoCompression"
case SnappyCompression:
return "Snappy"
case ZstdCompression:
return "ZSTD"
default:
return "Unknown"
}
}

// CompressionFromString returns an sstable.Compression from its
// string representation. Inverse of c.String() above.
func CompressionFromString(s string) Compression {
switch s {
case "Default":
return DefaultCompression
case "NoCompression":
return NoCompression
case "Snappy":
return SnappyCompression
case "ZSTD":
return ZstdCompression
default:
return DefaultCompression
}
}
9 changes: 5 additions & 4 deletions sstable/compression.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/cache"
"github.com/cockroachdb/pebble/sstable/block"
"github.com/golang/snappy"
)

Expand Down Expand Up @@ -81,12 +82,12 @@ func decompressBlock(blockType blockType, b []byte) (*cache.Value, error) {

// compressBlock compresses an SST block, using compressBuf as the desired destination.
func compressBlock(
compression Compression, b []byte, compressedBuf []byte,
compression block.Compression, b []byte, compressedBuf []byte,
) (blockType blockType, compressed []byte) {
switch compression {
case SnappyCompression:
case block.SnappyCompression:
return snappyCompressionBlockType, snappy.Encode(compressedBuf, b)
case NoCompression:
case block.NoCompression:
return noCompressionBlockType, b
}

Expand All @@ -95,7 +96,7 @@ func compressBlock(
}
varIntLen := binary.PutUvarint(compressedBuf, uint64(len(b)))
switch compression {
case ZstdCompression:
case block.ZstdCompression:
return zstdCompressionBlockType, encodeZstd(compressedBuf, varIntLen, b)
default:
return noCompressionBlockType, b
Expand Down
3 changes: 2 additions & 1 deletion sstable/compression_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"time"

"github.com/cockroachdb/pebble/internal/cache"
"github.com/cockroachdb/pebble/sstable/block"
"github.com/stretchr/testify/require"
)

Expand All @@ -19,7 +20,7 @@ func TestCompressionRoundtrip(t *testing.T) {
t.Logf("seed %d", seed)
rng := rand.New(rand.NewSource(seed))

for compression := DefaultCompression + 1; compression < NCompression; compression++ {
for compression := block.DefaultCompression + 1; compression < block.NCompression; compression++ {
t.Run(compression.String(), func(t *testing.T) {
payload := make([]byte, rng.Intn(10<<10 /* 10 KiB */))
rng.Read(payload)
Expand Down
8 changes: 4 additions & 4 deletions sstable/layout.go
Original file line number Diff line number Diff line change
Expand Up @@ -298,7 +298,7 @@ type layoutWriter struct {

// options copied from WriterOptions
tableFormat TableFormat
compression Compression
compression block.Compression
checksumType block.ChecksumType

// offset tracks the current write offset within the writable.
Expand Down Expand Up @@ -412,7 +412,7 @@ func (w *layoutWriter) WriteRangeDeletionBlock(b []byte) (block.Handle, error) {
}

func (w *layoutWriter) writeNamedBlock(b []byte, name string) (bh block.Handle, err error) {
bh, err = w.writeBlock(b, NoCompression, &w.buf)
bh, err = w.writeBlock(b, block.NoCompression, &w.buf)
if err == nil {
w.recordToMetaindex(name, bh)
}
Expand Down Expand Up @@ -459,7 +459,7 @@ func (w *layoutWriter) WriteValueIndexBlock(
}

func (w *layoutWriter) writeBlock(
b []byte, compression Compression, buf *blockBuf,
b []byte, compression block.Compression, buf *blockBuf,
) (block.Handle, error) {
blk, trailer := compressAndChecksum(b, compression, buf)
bh := block.Handle{Offset: w.offset, Length: uint64(len(blk))}
Expand Down Expand Up @@ -526,7 +526,7 @@ func (w *layoutWriter) Finish() (size uint64, err error) {
for _, h := range w.handles {
bw.AddRaw(unsafe.Slice(unsafe.StringData(h.key), len(h.key)), h.encodedBlockHandle)
}
metaIndexHandle, err := w.writeBlock(bw.Finish(), NoCompression, &w.buf)
metaIndexHandle, err := w.writeBlock(bw.Finish(), block.NoCompression, &w.buf)
if err != nil {
return 0, err
}
Expand Down
50 changes: 3 additions & 47 deletions sstable/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,18 +15,6 @@ import (
// MaximumBlockSize is the maximum permissible size of a block.
const MaximumBlockSize = rowblk.MaximumSize

// Compression is the per-block compression algorithm to use.
type Compression int

// The available compression types.
const (
DefaultCompression Compression = iota
NoCompression
SnappyCompression
ZstdCompression
NCompression
)

var ignoredInternalProperties = map[string]struct{}{
"rocksdb.column.family.id": {},
"rocksdb.fixed.key.length": {},
Expand All @@ -38,38 +26,6 @@ var ignoredInternalProperties = map[string]struct{}{
"rocksdb.format.version": {},
}

func (c Compression) String() string {
switch c {
case DefaultCompression:
return "Default"
case NoCompression:
return "NoCompression"
case SnappyCompression:
return "Snappy"
case ZstdCompression:
return "ZSTD"
default:
return "Unknown"
}
}

// CompressionFromString returns an sstable.Compression from its
// string representation. Inverse of c.String() above.
func CompressionFromString(s string) Compression {
switch s {
case "Default":
return DefaultCompression
case "NoCompression":
return NoCompression
case "Snappy":
return SnappyCompression
case "ZSTD":
return ZstdCompression
default:
return DefaultCompression
}
}

// FilterType exports the base.FilterType type.
type FilterType = base.FilterType

Expand Down Expand Up @@ -199,7 +155,7 @@ type WriterOptions struct {
// Compression defines the per-block compression to use.
//
// The default value (DefaultCompression) uses snappy compression.
Compression Compression
Compression block.Compression

// FilterPolicy defines a filter algorithm (such as a Bloom filter) that can
// reduce disk reads for Get calls.
Expand Down Expand Up @@ -313,8 +269,8 @@ func (o WriterOptions) ensureDefaults() WriterOptions {
if o.Comparer == nil {
o.Comparer = base.DefaultComparer
}
if o.Compression <= DefaultCompression || o.Compression >= NCompression {
o.Compression = SnappyCompression
if o.Compression <= block.DefaultCompression || o.Compression >= block.NCompression {
o.Compression = block.SnappyCompression
}
if o.IndexBlockSize <= 0 {
o.IndexBlockSize = o.BlockSize
Expand Down
6 changes: 3 additions & 3 deletions sstable/raw_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ type RawWriter struct {
compare Compare
split Split
formatKey base.FormatKey
compression Compression
compression block.Compression
separator Separator
successor Successor
tableFormat TableFormat
Expand Down Expand Up @@ -622,7 +622,7 @@ func (d *dataBlockBuf) finish() {
d.uncompressed = d.dataBlock.Finish()
}

func (d *dataBlockBuf) compressAndChecksum(c Compression) {
func (d *dataBlockBuf) compressAndChecksum(c block.Compression) {
d.compressed, d.trailer = compressAndChecksum(d.uncompressed, c, &d.blockBuf)
}

Expand Down Expand Up @@ -1584,7 +1584,7 @@ func (w *RawWriter) writeTwoLevelIndex() (block.Handle, error) {
}

func compressAndChecksum(
b []byte, compression Compression, blockBuf *blockBuf,
b []byte, compression block.Compression, blockBuf *blockBuf,
) (compressed []byte, trailer block.Trailer) {
// Compress the buffer, discarding the result if the improvement isn't at
// least 12.5%.
Expand Down
Loading

0 comments on commit c73ffdb

Please sign in to comment.