Skip to content

Commit

Permalink
base: add AssertionFailedf wrapper
Browse files Browse the repository at this point in the history
We add a wrapper for `errors.AssertionFailedf` which panics in
`invariants.Enabled` mode. This makes these errors easier to debug and
less likely to be ignored.
  • Loading branch information
RaduBerinde committed Mar 28, 2024
1 parent 016ef88 commit e962431
Show file tree
Hide file tree
Showing 19 changed files with 80 additions and 43 deletions.
2 changes: 1 addition & 1 deletion batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -1914,7 +1914,7 @@ func newFlushableBatch(batch *Batch, comparer *Comparer) (*flushableBatch, error
// we'll begin assigning keys sequence numbers that weren't
// allocated.
if index >= uint32(batch.count) {
return nil, errors.AssertionFailedf("pebble: batch entry index %d ≥ batch.count %d", index, batch.count)
return nil, base.AssertionFailedf("pebble: batch entry index %d ≥ batch.count %d", index, batch.count)
}
index++
}
Expand Down
18 changes: 9 additions & 9 deletions ingest.go
Original file line number Diff line number Diff line change
Expand Up @@ -490,11 +490,11 @@ func ingestLoad(
})
if external[i].Level > 0 {
if i != 0 && !result.externalFilesHaveLevel {
return ingestLoadResult{}, errors.AssertionFailedf("pebble: external sstables must all have level set or unset")
return ingestLoadResult{}, base.AssertionFailedf("pebble: external sstables must all have level set or unset")
}
result.externalFilesHaveLevel = true
} else if result.externalFilesHaveLevel {
return ingestLoadResult{}, errors.AssertionFailedf("pebble: external sstables must all have level set or unset")
return ingestLoadResult{}, base.AssertionFailedf("pebble: external sstables must all have level set or unset")
}
}
return result, nil
Expand All @@ -512,7 +512,7 @@ func ingestSortAndVerify(cmp Compare, lr ingestLoadResult, exciseSpan KeyRange)
if lr.externalFilesHaveLevel {
for _, f := range lr.external {
if !exciseSpan.Contains(cmp, f.Smallest) || !exciseSpan.Contains(cmp, f.Largest) {
return errors.AssertionFailedf("pebble: external file outside of excise span, span [%s-%s), file = %s", exciseSpan.Start, exciseSpan.End, f.String())
return base.AssertionFailedf("pebble: external file outside of excise span, span [%s-%s), file = %s", exciseSpan.Start, exciseSpan.End, f.String())
}
}
}
Expand All @@ -521,7 +521,7 @@ func ingestSortAndVerify(cmp Compare, lr ingestLoadResult, exciseSpan KeyRange)
if len(lr.shared) > 0 {
// If external files are present alongside shared files,
// return an error.
return errors.AssertionFailedf("pebble: external files cannot be ingested atomically alongside shared files")
return base.AssertionFailedf("pebble: external files cannot be ingested atomically alongside shared files")
}

// Sort according to the smallest key.
Expand Down Expand Up @@ -570,7 +570,7 @@ func ingestSortAndVerify(cmp Compare, lr ingestLoadResult, exciseSpan KeyRange)
})
for i := 1; i < len(filesInLevel); i++ {
if sstableKeyCompare(cmp, filesInLevel[i-1].Largest, filesInLevel[i].Smallest) >= 0 {
return errors.AssertionFailedf("pebble: external shared sstables have overlapping ranges")
return base.AssertionFailedf("pebble: external shared sstables have overlapping ranges")
}
}
}
Expand Down Expand Up @@ -988,7 +988,7 @@ func ingestTargetLevel(
// This assertion implicitly checks that we have the current version of
// the metadata.
if v.L0Sublevels == nil {
return 0, nil, errors.AssertionFailedf("could not read L0 sublevels")
return 0, nil, base.AssertionFailedf("could not read L0 sublevels")
}
iterOps.CategoryAndQoS = sstable.CategoryAndQoS{
Category: "pebble-ingest",
Expand Down Expand Up @@ -2022,7 +2022,7 @@ func (d *DB) excise(
key, _ := iter.SeekGE(exciseSpan.End.Key, base.SeekGEFlagsNone)
if key != nil {
if exciseSpan.End.Kind == base.Inclusive && d.equal(exciseSpan.End.Key, key.UserKey) {
return nil, errors.AssertionFailedf("cannot excise with an inclusive end key and data overlap at end key")
return nil, base.AssertionFailedf("cannot excise with an inclusive end key and data overlap at end key")
}
rightFile.ExtendPointKeyBounds(d.cmp, key.Clone(), largestPointKey)
}
Expand All @@ -2037,7 +2037,7 @@ func (d *DB) excise(
if d.cmp(firstRangeDel, exciseSpan.End.Key) < 0 {
// NB: This can only be done if the end bound is exclusive.
if exciseSpan.End.Kind != base.Exclusive {
return nil, errors.AssertionFailedf("cannot truncate rangedel during excise with an inclusive upper bound")
return nil, base.AssertionFailedf("cannot truncate rangedel during excise with an inclusive upper bound")
}
firstRangeDel = exciseSpan.End.Key
}
Expand Down Expand Up @@ -2069,7 +2069,7 @@ func (d *DB) excise(
firstRangeKey = append(firstRangeKey[:0], rkey.Start...)
if d.cmp(firstRangeKey, exciseSpan.End.Key) < 0 {
if exciseSpan.End.Kind != base.Exclusive {
return nil, errors.AssertionFailedf("cannot truncate range key during excise with an inclusive upper bound")
return nil, base.AssertionFailedf("cannot truncate range key during excise with an inclusive upper bound")
}
firstRangeKey = exciseSpan.End.Key
}
Expand Down
31 changes: 30 additions & 1 deletion internal/base/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,10 @@

package base

import "github.com/cockroachdb/errors"
import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/invariants"
)

// ErrNotFound means that a get or delete call did not find the requested key.
var ErrNotFound = errors.New("pebble: not found")
Expand All @@ -26,3 +29,29 @@ func MarkCorruptionError(err error) error {
func CorruptionErrorf(format string, args ...interface{}) error {
return errors.Mark(errors.Newf(format, args...), ErrCorruption)
}

// AssertionFailedf creates an assertion error and panics in invariants.Enabled
// builds. It should only be used when it indicates a bug.
func AssertionFailedf(format string, args ...interface{}) error {
err := errors.AssertionFailedf(format, args...)
if invariants.Enabled {
panic(err)
}
return err
}

// CatchErrorPanic runs a function and catches any panic that contains an
// error, returning that error. Used in tests, in particular to catch panics
// threw by AssertionFailedf.
func CatchErrorPanic(f func() error) (err error) {
defer func() {
if r := recover(); r != nil {
if e, ok := r.(error); ok {
err = e
} else {
panic(r)
}
}
}()
return f()
}
2 changes: 1 addition & 1 deletion internal/keyspan/interleaving_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1069,7 +1069,7 @@ func (i *InterleavingIter) savePoint(key *base.InternalKey, value base.LazyValue
if invariants.Enabled {
if err := i.pointIter.Error(); key != nil && err != nil {
panic(errors.WithSecondaryError(
errors.AssertionFailedf("pebble: %T point iterator returned non-nil key %q while iter has error", i.pointIter, key),
base.AssertionFailedf("pebble: %T point iterator returned non-nil key %q while iter has error", i.pointIter, key),
err))
}
}
Expand Down
3 changes: 1 addition & 2 deletions internal/keyspan/truncate.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,6 @@
package keyspan

import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
)
Expand Down Expand Up @@ -127,7 +126,7 @@ func (i *truncatingIter) nextSpanWithinBounds(
var err error
for span != nil {
if i.bounds.End.Kind == base.Inclusive && span.Contains(i.cmp, i.bounds.End.Key) {
err := errors.AssertionFailedf("inclusive upper bound %q inside span %s", i.bounds.End.Key, span)
err := base.AssertionFailedf("inclusive upper bound %q inside span %s", i.bounds.End.Key, span)
if invariants.Enabled {
panic(err)
}
Expand Down
2 changes: 1 addition & 1 deletion objstorage/objstorageprovider/provider.go
Original file line number Diff line number Diff line change
Expand Up @@ -403,7 +403,7 @@ func (p *provider) Lookup(
)
}
if meta.FileType != fileType {
return objstorage.ObjectMetadata{}, errors.AssertionFailedf(
return objstorage.ObjectMetadata{}, base.AssertionFailedf(
"file %s type mismatch (known type %d, expected type %d)",
fileNum, errors.Safe(meta.FileType), errors.Safe(fileType),
)
Expand Down
8 changes: 6 additions & 2 deletions objstorage/objstorageprovider/provider_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -222,8 +222,12 @@ func TestProvider(t *testing.T) {
scanArgs("<key> <file-num>", &key, &fileNum)
meta, err := curProvider.Lookup(base.FileTypeTable, fileNum)
require.NoError(t, err)
handle, err := curProvider.RemoteObjectBacking(&meta)
if err != nil {
var handle objstorage.RemoteObjectBackingHandle
if err := base.CatchErrorPanic(func() error {
var err error
handle, err = curProvider.RemoteObjectBacking(&meta)
return err
}); err != nil {
return err.Error()
}
backing, err := handle.Get()
Expand Down
2 changes: 1 addition & 1 deletion objstorage/objstorageprovider/remote.go
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,7 @@ func (p *provider) sharedClose() error {
// SetCreatorID is part of the objstorage.Provider interface.
func (p *provider) SetCreatorID(creatorID objstorage.CreatorID) error {
if p.st.Remote.StorageFactory == nil {
return errors.AssertionFailedf("attempt to set CreatorID but remote storage not enabled")
return base.AssertionFailedf("attempt to set CreatorID but remote storage not enabled")
}
// Note: this call is a cheap no-op if the creator ID was already set. This
// call also checks if we are trying to change the ID.
Expand Down
2 changes: 1 addition & 1 deletion objstorage/objstorageprovider/remote_backing.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ func (p *provider) encodeRemoteObjectBacking(
meta *objstorage.ObjectMetadata,
) (objstorage.RemoteObjectBacking, error) {
if !meta.IsRemote() {
return nil, errors.AssertionFailedf("object %s not on remote storage", meta.DiskFileNum)
return nil, base.AssertionFailedf("object %s not on remote storage", meta.DiskFileNum)
}

buf := make([]byte, 0, binary.MaxVarintLen64*4)
Expand Down
10 changes: 5 additions & 5 deletions objstorage/objstorageprovider/remoteobjcat/catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,15 +125,15 @@ func Open(fs vfs.FS, dirname string) (*Catalog, CatalogContents, error) {
// SetCreatorID sets the creator ID. If it is already set, it must match.
func (c *Catalog) SetCreatorID(id objstorage.CreatorID) error {
if !id.IsSet() {
return errors.AssertionFailedf("attempt to unset CreatorID")
return base.AssertionFailedf("attempt to unset CreatorID")
}

c.mu.Lock()
defer c.mu.Unlock()

if c.mu.creatorID.IsSet() {
if c.mu.creatorID != id {
return errors.AssertionFailedf("attempt to change CreatorID from %s to %s", c.mu.creatorID, id)
return base.AssertionFailedf("attempt to change CreatorID from %s to %s", c.mu.creatorID, id)
}
return nil
}
Expand Down Expand Up @@ -236,13 +236,13 @@ func (c *Catalog) ApplyBatch(b Batch) error {
}
for _, meta := range b.ve.NewObjects {
if exists(meta.FileNum) {
return errors.AssertionFailedf("adding existing object %s", meta.FileNum)
return base.AssertionFailedf("adding existing object %s", meta.FileNum)
}
toAdd[meta.FileNum] = struct{}{}
}
for _, n := range b.ve.DeletedObjects {
if !exists(n) {
return errors.AssertionFailedf("deleting non-existent object %s", n)
return base.AssertionFailedf("deleting non-existent object %s", n)
}
}

Expand Down Expand Up @@ -331,7 +331,7 @@ func makeCatalogFilename(iter uint64) string {
// current catalog and sets c.mu.catalogFile and c.mu.catalogRecWriter.
func (c *Catalog) createNewCatalogFileLocked() (outErr error) {
if c.mu.catalogFile != nil {
return errors.AssertionFailedf("catalogFile already open")
return base.AssertionFailedf("catalogFile already open")
}
filename := makeCatalogFilename(c.mu.marker.NextIter())
filepath := c.fs.PathJoin(c.dirname, filename)
Expand Down
8 changes: 6 additions & 2 deletions objstorage/objstorageprovider/remoteobjcat/catalog_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,9 @@ func TestCatalog(t *testing.T) {
td.Fatalf(t, "set-creator-id <id>")
}
id := objstorage.CreatorID(toUInt64(td.CmdArgs[0].String())[0])
if err := cat.SetCreatorID(id); err != nil {
if err := base.CatchErrorPanic(func() error {
return cat.SetCreatorID(id)
}); err != nil {
return fmt.Sprintf("error setting creator ID: %v", err)
}
return memLog.String()
Expand All @@ -115,7 +117,9 @@ func TestCatalog(t *testing.T) {
td.Fatalf(t, "unknown batch command: %s", tokens[0])
}
}
if err := cat.ApplyBatch(b); err != nil {
if err := base.CatchErrorPanic(func() error {
return cat.ApplyBatch(b)
}); err != nil {
return fmt.Sprintf("error applying batch: %v", err)
}
b.Reset()
Expand Down
4 changes: 2 additions & 2 deletions objstorage/objstorageprovider/remoteobjcat/version_edit.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,15 +237,15 @@ func (v *VersionEdit) Apply(
for _, meta := range v.NewObjects {
if invariants.Enabled {
if _, exists := objects[meta.FileNum]; exists {
return errors.AssertionFailedf("version edit adds existing object %s", meta.FileNum)
return base.AssertionFailedf("version edit adds existing object %s", meta.FileNum)
}
}
objects[meta.FileNum] = meta
}
for _, fileNum := range v.DeletedObjects {
if invariants.Enabled {
if _, exists := objects[fileNum]; !exists {
return errors.AssertionFailedf("version edit deletes non-existent object %s", fileNum)
return base.AssertionFailedf("version edit deletes non-existent object %s", fileNum)
}
}
delete(objects, fileNum)
Expand Down
6 changes: 3 additions & 3 deletions scan_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -494,11 +494,11 @@ func (d *DB) truncateExternalFile(
}

if cmp(sst.StartKey, sst.EndKey) > 0 {
return nil, errors.AssertionFailedf("pebble: invalid external file bounds after truncation [%q, %q)", sst.StartKey, sst.EndKey)
return nil, base.AssertionFailedf("pebble: invalid external file bounds after truncation [%q, %q)", sst.StartKey, sst.EndKey)
}

if cmp(sst.StartKey, sst.EndKey) == 0 && !sst.EndKeyIsInclusive {
return nil, errors.AssertionFailedf("pebble: invalid external file bounds after truncation [%q, %q)", sst.StartKey, sst.EndKey)
return nil, base.AssertionFailedf("pebble: invalid external file bounds after truncation [%q, %q)", sst.StartKey, sst.EndKey)
}

return sst, nil
Expand Down Expand Up @@ -683,7 +683,7 @@ func scanInternalImpl(
panic("lower and upper bounds must be specified in skip-shared iteration mode")
}
if opts.visitSharedFile != nil && opts.visitExternalFile != nil {
return errors.AssertionFailedf("cannot provide both a shared-file and external-file visitor")
return base.AssertionFailedf("cannot provide both a shared-file and external-file visitor")
}

// Before starting iteration, check if any files in levels sharedLevelsStart
Expand Down
3 changes: 1 addition & 2 deletions sstable/block_property.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@ import (
"sync"
"unsafe"

"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/rangekey"
)
Expand Down Expand Up @@ -478,7 +477,7 @@ func (b *BlockIntervalFilter) Intersects(prop []byte) (bool, error) {
// SyntheticSuffixIntersects implements the BlockPropertyFilter interface.
func (b *BlockIntervalFilter) SyntheticSuffixIntersects(prop []byte, suffix []byte) (bool, error) {
if b.syntheticReplacer == nil {
return false, errors.AssertionFailedf("missing SyntheticReplacer for SyntheticSuffixIntersects()")
return false, base.AssertionFailedf("missing SyntheticReplacer for SyntheticSuffixIntersects()")
}
var i interval
if err := i.decode(prop); err != nil {
Expand Down
7 changes: 5 additions & 2 deletions sstable/block_property_obsolete.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,10 @@

package sstable

import "github.com/cockroachdb/errors"
import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
)

// obsoleteKeyBlockPropertyCollector is a block property collector used to
// implement obsoleteKeyBlockPropertyFilter - a filter that excludes blocks
Expand Down Expand Up @@ -117,7 +120,7 @@ func (o obsoleteKeyBlockPropertyFilter) SyntheticSuffixIntersects(
}
// A block with suffix replacement should never be obsolete.
if isObsolete {
return false, errors.AssertionFailedf("block with synthetic suffix is obsolete")
return false, base.AssertionFailedf("block with synthetic suffix is obsolete")
}
return true, nil
}
Expand Down
2 changes: 1 addition & 1 deletion sstable/copier.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ func CopySpan(
length := blocks[len(blocks)-1].bh.Offset + blocks[len(blocks)-1].bh.Length + blockTrailerLen - offset

if spanEnd := length + offset; spanEnd < offset {
return 0, errors.AssertionFailedf("invalid intersecting span for CopySpan [%d, %d)", offset, spanEnd)
return 0, base.AssertionFailedf("invalid intersecting span for CopySpan [%d, %d)", offset, spanEnd)
}

if err := objstorage.Copy(ctx, r.readable, w.writable, offset, length); err != nil {
Expand Down
8 changes: 4 additions & 4 deletions sstable/reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -451,10 +451,10 @@ func (r *Reader) NewRawRangeDelIter(transforms IterTransforms) (keyspan.Fragment
return nil, nil
}
if transforms.SyntheticSuffix.IsSet() {
return nil, errors.AssertionFailedf("synthetic suffix not supported with range del iterator")
return nil, base.AssertionFailedf("synthetic suffix not supported with range del iterator")
}
if transforms.SyntheticPrefix.IsSet() {
return nil, errors.AssertionFailedf("synthetic prefix not supported with range del iterator")
return nil, base.AssertionFailedf("synthetic prefix not supported with range del iterator")
}
h, err := r.readRangeDel(nil /* stats */, nil /* iterStats */)
if err != nil {
Expand Down Expand Up @@ -482,10 +482,10 @@ func (r *Reader) NewRawRangeKeyIter(transforms IterTransforms) (keyspan.Fragment
return nil, nil
}
if transforms.SyntheticSuffix.IsSet() {
return nil, errors.AssertionFailedf("synthetic suffix not supported with range key iterator")
return nil, base.AssertionFailedf("synthetic suffix not supported with range key iterator")
}
if transforms.SyntheticPrefix.IsSet() {
return nil, errors.AssertionFailedf("synthetic prefix not supported with range key iterator")
return nil, base.AssertionFailedf("synthetic prefix not supported with range key iterator")
}
h, err := r.readRangeKey(nil /* stats */, nil /* iterStats */)
if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion version_set.go
Original file line number Diff line number Diff line change
Expand Up @@ -550,7 +550,7 @@ func (vs *versionSet) logAndApply(
defer vs.mu.Lock()

if vs.getFormatMajorVersion() < FormatVirtualSSTables && len(ve.CreatedBackingTables) > 0 {
return errors.AssertionFailedf("MANIFEST cannot contain virtual sstable records due to format major version")
return base.AssertionFailedf("MANIFEST cannot contain virtual sstable records due to format major version")
}
var b bulkVersionEdit
err := b.Accumulate(ve)
Expand Down

0 comments on commit e962431

Please sign in to comment.