Skip to content

Commit

Permalink
sstable: show filenum when metadata is corrupted
Browse files Browse the repository at this point in the history
Fixes #3804.
  • Loading branch information
RaduBerinde committed Jul 29, 2024
1 parent f8e117d commit 368636f
Show file tree
Hide file tree
Showing 11 changed files with 37 additions and 23 deletions.
8 changes: 4 additions & 4 deletions sstable/format.go
Original file line number Diff line number Diff line change
Expand Up @@ -210,14 +210,14 @@ const (

// ParseTableFormat parses the given magic bytes and version into its
// corresponding internal TableFormat.
func ParseTableFormat(magic []byte, version uint32) (TableFormat, error) {
func ParseTableFormat(magic []byte, version uint32, fileNum base.DiskFileNum) (TableFormat, error) {
switch string(magic) {
case levelDBMagic:
return TableFormatLevelDB, nil
case rocksDBMagic:
if version != rocksDBFormatVersion2 {
return TableFormatUnspecified, base.CorruptionErrorf(
"pebble/table: unsupported rocksdb format version %d", errors.Safe(version),
"pebble/table: invalid table %s (unsupported rocksdb format version %d)", errors.Safe(fileNum), errors.Safe(version),
)
}
return TableFormatRocksDBv2, nil
Expand All @@ -233,12 +233,12 @@ func ParseTableFormat(magic []byte, version uint32) (TableFormat, error) {
return TableFormatPebblev4, nil
default:
return TableFormatUnspecified, base.CorruptionErrorf(
"pebble/table: unsupported pebble format version %d", errors.Safe(version),
"pebble/table: invalid table %s (unsupported pebble format version %d)", errors.Safe(fileNum), errors.Safe(version),
)
}
default:
return TableFormatUnspecified, base.CorruptionErrorf(
"pebble/table: invalid table (bad magic number: 0x%x)", magic,
"pebble/table: invalid table %s (bad magic number: 0x%x)", errors.Safe(fileNum), magic,
)
}
}
Expand Down
8 changes: 4 additions & 4 deletions sstable/format_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,25 +60,25 @@ func TestTableFormat_RoundTrip(t *testing.T) {
name: "Invalid RocksDB version",
magic: rocksDBMagic,
version: 1,
wantErr: "pebble/table: unsupported rocksdb format version 1",
wantErr: "pebble/table: invalid table 000001 (unsupported rocksdb format version 1)",
},
{
name: "Invalid PebbleDB version",
magic: pebbleDBMagic,
version: 5,
wantErr: "pebble/table: unsupported pebble format version 5",
wantErr: "pebble/table: invalid table 000001 (unsupported pebble format version 5)",
},
{
name: "Unknown magic string",
magic: "foo",
wantErr: "pebble/table: invalid table (bad magic number: 0x666f6f)",
wantErr: "pebble/table: invalid table 000001 (bad magic number: 0x666f6f)",
},
}

for _, tc := range tcs {
t.Run(tc.name, func(t *testing.T) {
// Tuple -> TableFormat.
f, err := ParseTableFormat([]byte(tc.magic), tc.version)
f, err := ParseTableFormat([]byte(tc.magic), tc.version, 1)
if tc.wantErr != "" {
require.Error(t, err)
require.Equal(t, tc.wantErr, err.Error())
Expand Down
2 changes: 1 addition & 1 deletion sstable/reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -979,7 +979,7 @@ func NewReader(ctx context.Context, f objstorage.Readable, o ReaderOptions) (*Re
r.readable, objstorage.ReadBeforeForNewReader, &preallocRH)
defer rh.Close()

footer, err := readFooter(ctx, f, rh, r.logger)
footer, err := readFooter(ctx, f, rh, r.logger, r.cacheOpts.FileNum)
if err != nil {
r.err = err
return nil, r.Close()
Expand Down
17 changes: 9 additions & 8 deletions sstable/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -307,11 +307,12 @@ func readFooter(
f objstorage.Readable,
readHandle objstorage.ReadHandle,
logger base.LoggerAndTracer,
fileNum base.DiskFileNum,
) (footer, error) {
var footer footer
size := f.Size()
if size < minFooterLen {
return footer, base.CorruptionErrorf("pebble/table: invalid table (file size is too small)")
return footer, base.CorruptionErrorf("pebble/table: invalid table %s (file size is too small)", errors.Safe(fileNum))
}

buf := make([]byte, maxFooterLen)
Expand Down Expand Up @@ -342,7 +343,7 @@ func readFooter(
case levelDBMagic:
if len(buf) < levelDBFooterLen {
return footer, base.CorruptionErrorf(
"pebble/table: invalid table (footer too short): %d", errors.Safe(len(buf)))
"pebble/table: invalid table %s (footer too short): %d", errors.Safe(fileNum), errors.Safe(len(buf)))
}
footer.footerBH.Offset = uint64(off+int64(len(buf))) - levelDBFooterLen
buf = buf[len(buf)-levelDBFooterLen:]
Expand All @@ -354,14 +355,14 @@ func readFooter(
// NOTE: The Pebble magic string implies the same footer format as that used
// by the RocksDBv2 table format.
if len(buf) < rocksDBFooterLen {
return footer, base.CorruptionErrorf("pebble/table: invalid table (footer too short): %d", errors.Safe(len(buf)))
return footer, base.CorruptionErrorf("pebble/table: invalid table %s (footer too short): %d", errors.Safe(fileNum), errors.Safe(len(buf)))
}
footer.footerBH.Offset = uint64(off+int64(len(buf))) - rocksDBFooterLen
buf = buf[len(buf)-rocksDBFooterLen:]
footer.footerBH.Length = uint64(len(buf))
version := binary.LittleEndian.Uint32(buf[rocksDBVersionOffset:rocksDBMagicOffset])

format, err := ParseTableFormat(magic, version)
format, err := ParseTableFormat(magic, version, fileNum)
if err != nil {
return footer, err
}
Expand All @@ -373,26 +374,26 @@ func readFooter(
case block.ChecksumTypeXXHash64:
footer.checksum = block.ChecksumTypeXXHash64
default:
return footer, base.CorruptionErrorf("pebble/table: unsupported checksum type %d", errors.Safe(footer.checksum))
return footer, base.CorruptionErrorf("pebble/table: invalid table %s (unsupported checksum type %d)", errors.Safe(fileNum), errors.Safe(footer.checksum))
}
buf = buf[1:]

default:
return footer, base.CorruptionErrorf("pebble/table: invalid table (bad magic number: 0x%x)", magic)
return footer, base.CorruptionErrorf("pebble/table: invalid table %s (bad magic number: 0x%x)", errors.Safe(fileNum), magic)
}

{
end := uint64(size)
var n int
footer.metaindexBH, n = decodeBlockHandle(buf)
if n == 0 || footer.metaindexBH.Offset+footer.metaindexBH.Length > end {
return footer, base.CorruptionErrorf("pebble/table: invalid table (bad metaindex block handle)")
return footer, base.CorruptionErrorf("pebble/table: invalid table %s (bad metaindex block handle)", errors.Safe(fileNum))
}
buf = buf[n:]

footer.indexBH, n = decodeBlockHandle(buf)
if n == 0 || footer.indexBH.Offset+footer.indexBH.Length > end {
return footer, base.CorruptionErrorf("pebble/table: invalid table (bad index block handle)")
return footer, base.CorruptionErrorf("pebble/table: invalid table %s (bad index block handle)", errors.Safe(fileNum))
}
}

Expand Down
6 changes: 4 additions & 2 deletions sstable/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -588,7 +588,7 @@ func TestFooterRoundTrip(t *testing.T) {
readable, err := NewSimpleReadable(f)
require.NoError(t, err)

result, err := readFooter(context.Background(), readable, nil, base.NoopLoggerAndTracer{})
result, err := readFooter(context.Background(), readable, nil, base.NoopLoggerAndTracer{}, 1)
require.NoError(t, err)
require.NoError(t, readable.Close())

Expand Down Expand Up @@ -641,10 +641,12 @@ func TestReadFooter(t *testing.T) {
readable, err := NewSimpleReadable(f)
require.NoError(t, err)

if _, err := readFooter(context.Background(), readable, nil, base.NoopLoggerAndTracer{}); err == nil {
if _, err := readFooter(context.Background(), readable, nil, base.NoopLoggerAndTracer{}, 1); err == nil {
t.Fatalf("expected %q, but found success", c.expected)
} else if !strings.Contains(err.Error(), c.expected) {
t.Fatalf("expected %q, but found %v", c.expected, err)
} else if !strings.Contains(err.Error(), "table 000001") {
t.Fatalf("expected error to contain table number: %q", err)
}
})
}
Expand Down
2 changes: 1 addition & 1 deletion table_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1025,7 +1025,7 @@ func TestTableCacheErrorBadMagicNumber(t *testing.T) {
t.Fatalf("expected failure, but found success")
}
require.Equal(t,
"pebble: backing file 000003 error: pebble/table: invalid table (bad magic number: 0xf09faab3f09faa00)",
"pebble: backing file 000003 error: pebble/table: invalid table 000003 (bad magic number: 0xf09faab3f09faa00)",
err.Error())
}

Expand Down
10 changes: 10 additions & 0 deletions tool/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/humanize"
"github.com/cockroachdb/pebble/internal/manifest"
"github.com/cockroachdb/pebble/internal/sstableinternal"
"github.com/cockroachdb/pebble/objstorage"
"github.com/cockroachdb/pebble/objstorage/objstorageprovider"
"github.com/cockroachdb/pebble/record"
Expand Down Expand Up @@ -883,6 +884,15 @@ func (d *dbT) addProps(
if err != nil {
return err
}
opts := sstable.ReaderOptions{
Mergers: d.mergers,
Comparers: d.comparers,
}
opts.SetInternal(sstableinternal.ReaderOptions{
CacheOpts: sstableinternal.CacheOptions{
FileNum: m.FileBacking.DiskFileNum,
},
})
r, err := sstable.NewReader(ctx, f, sstable.ReaderOptions{
Mergers: d.mergers,
Comparers: d.comparers,
Expand Down
3 changes: 2 additions & 1 deletion tool/find.go
Original file line number Diff line number Diff line change
Expand Up @@ -446,7 +446,8 @@ func (f *findT) searchTables(stdout io.Writer, searchKey []byte, refs []findRef)
}
opts.SetInternal(sstableinternal.ReaderOptions{
CacheOpts: sstableinternal.CacheOptions{
Cache: cache,
Cache: cache,
FileNum: fl.DiskFileNum,
},
})
readable, err := sstable.NewSimpleReadable(tf)
Expand Down
2 changes: 1 addition & 1 deletion tool/testdata/find
Original file line number Diff line number Diff line change
Expand Up @@ -137,4 +137,4 @@ hex:636363
----
000002.sst
test formatter: ccc#14,MERGE
Unable to decode sstable find-mixed/000001.sst, pebble/table: invalid table (file size is too small)
Unable to decode sstable find-mixed/000001.sst, pebble/table: invalid table 000001 (file size is too small)
2 changes: 1 addition & 1 deletion tool/testdata/sstable_check
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ sstable check
testdata/bad-magic.sst
----
bad-magic.sst
pebble/table: invalid table (bad magic number: 0xf6cff485b741e288)
pebble/table: invalid table 000000 (bad magic number: 0xf6cff485b741e288)

sstable check
./testdata/mixed/000005.sst
Expand Down
Binary file modified tool/testdata/sstable_properties
Binary file not shown.

0 comments on commit 368636f

Please sign in to comment.