diff --git a/batch.go b/batch.go index 998d453961..2fe3e1dc43 100644 --- a/batch.go +++ b/batch.go @@ -64,13 +64,13 @@ type DeferredBatchOp struct { // have been filled into Key and Value. Not calling Finish or not // copying/encoding keys will result in an incomplete index, and calling Finish // twice may result in a panic. -func (d DeferredBatchOp) Finish() { +func (d DeferredBatchOp) Finish() error { if d.index != nil { if err := d.index.Add(d.offset); err != nil { - // We never add duplicate entries, so an error should never occur. - panic(err) + return err } } + return nil } // A Batch is a sequence of Sets, Merges, Deletes, DeleteRanges, RangeKeySets, @@ -394,8 +394,7 @@ func (b *Batch) Apply(batch *Batch, _ *WriteOptions) error { err = b.index.Add(uint32(offset)) } if err != nil { - // We never add duplicate entries, so an error should never occur. - panic(err) + return err } } b.memTableSize += memTableEntrySize(len(key), len(value)) @@ -512,8 +511,7 @@ func (b *Batch) Set(key, value []byte, _ *WriteOptions) error { // in go1.13 will remove the need for this. if b.index != nil { if err := b.index.Add(deferredOp.offset); err != nil { - // We never add duplicate entries, so an error should never occur. - panic(err) + return err } } return nil @@ -542,8 +540,7 @@ func (b *Batch) Merge(key, value []byte, _ *WriteOptions) error { // in go1.13 will remove the need for this. if b.index != nil { if err := b.index.Add(deferredOp.offset); err != nil { - // We never add duplicate entries, so an error should never occur. - panic(err) + return err } } return nil @@ -569,8 +566,7 @@ func (b *Batch) Delete(key []byte, _ *WriteOptions) error { // in go1.13 will remove the need for this. if b.index != nil { if err := b.index.Add(deferredOp.offset); err != nil { - // We never add duplicate entries, so an error should never occur. - panic(err) + return err } } return nil @@ -597,8 +593,7 @@ func (b *Batch) SingleDelete(key []byte, _ *WriteOptions) error { // in go1.13 will remove the need for this. if b.index != nil { if err := b.index.Add(deferredOp.offset); err != nil { - // We never add duplicate entries, so an error should never occur. - panic(err) + return err } } return nil @@ -628,8 +623,7 @@ func (b *Batch) DeleteRange(start, end []byte, _ *WriteOptions) error { // in go1.13 will remove the need for this. if deferredOp.index != nil { if err := deferredOp.index.Add(deferredOp.offset); err != nil { - // We never add duplicate entries, so an error should never occur. - panic(err) + return err } } return nil @@ -679,8 +673,7 @@ func (b experimentalBatch) RangeKeySet(start, end, suffix, value []byte, _ *Writ // Manually inline DeferredBatchOp.Finish(). if deferredOp.index != nil { if err := deferredOp.index.Add(deferredOp.offset); err != nil { - // We never add duplicate entries, so an error should never occur. - panic(err) + return err } } return nil @@ -719,8 +712,7 @@ func (b experimentalBatch) RangeKeyUnset(start, end, suffix []byte, _ *WriteOpti // Manually inline DeferredBatchOp.Finish() if deferredOp.index != nil { if err := deferredOp.index.Add(deferredOp.offset); err != nil { - // We never add duplicate entries, so an error should never occur. - panic(err) + return err } } return nil @@ -740,8 +732,7 @@ func (b experimentalBatch) RangeKeyDelete(start, end []byte, _ *WriteOptions) er // Manually inline DeferredBatchOp.Finish(). if deferredOp.index != nil { if err := deferredOp.index.Add(deferredOp.offset); err != nil { - // We never add duplicate entries, so an error should never occur. - panic(err) + return err } } return nil diff --git a/internal/batchskl/skl.go b/internal/batchskl/skl.go index a364682480..a5bc96f1e7 100644 --- a/internal/batchskl/skl.go +++ b/internal/batchskl/skl.go @@ -67,15 +67,23 @@ import ( ) const ( - maxHeight = 20 - maxNodeSize = int(unsafe.Sizeof(node{})) - linksSize = int(unsafe.Sizeof(links{})) + maxHeight = 20 + maxNodeSize = int(unsafe.Sizeof(node{})) + linksSize = int(unsafe.Sizeof(links{})) + maxNodesSize = math.MaxUint32 ) -// ErrExists indicates that a duplicate record was inserted. This should never -// happen for normal usage of batchskl as every key should have a unique -// sequence number. -var ErrExists = errors.New("record with this key already exists") +var ( + // ErrExists indicates that a duplicate record was inserted. This should never + // happen for normal usage of batchskl as every key should have a unique + // sequence number. + ErrExists = errors.New("record with this key already exists") + + // ErrTooManyRecords is a sentinel error returned when the size of the raw + // nodes slice exceeds the maximum allowed size (currently 1 << 32 - 1). This + // corresponds to ~117 M skiplist entries. + ErrTooManyRecords = errors.New("too many records") +) type links struct { next uint32 @@ -171,9 +179,17 @@ func (s *Skiplist) Init(storage *[]byte, cmp base.Compare, abbreviatedKey base.A s.nodes = make([]byte, 0, initBufSize) } - // Allocate head and tail nodes. - s.head = s.newNode(maxHeight, 0, 0, 0, 0) - s.tail = s.newNode(maxHeight, 0, 0, 0, 0) + // Allocate head and tail nodes. While allocating a new node can fail, in the + // context of initializing the skiplist we consider it unrecoverable. + var err error + s.head, err = s.newNode(maxHeight, 0, 0, 0, 0) + if err != nil { + panic(err) + } + s.tail, err = s.newNode(maxHeight, 0, 0, 0, 0) + if err != nil { + panic(err) + } // Link all head/tail levels together. headNode := s.node(s.head) @@ -230,7 +246,10 @@ func (s *Skiplist) Add(keyOffset uint32) error { // We always insert from the base level and up. After you add a node in base // level, we cannot create a node in the level above because it would have // discovered the node in the base level. - nd := s.newNode(height, keyOffset, keyStart, keyEnd, abbreviatedKey) + nd, err := s.newNode(height, keyOffset, keyStart, keyEnd, abbreviatedKey) + if err != nil { + return err + } newNode := s.node(nd) for level := uint32(0); level < height; level++ { next := spl[level].next @@ -255,23 +274,26 @@ func (s *Skiplist) NewIter(lower, upper []byte) Iterator { } func (s *Skiplist) newNode(height, - offset, keyStart, keyEnd uint32, abbreviatedKey uint64) uint32 { + offset, keyStart, keyEnd uint32, abbreviatedKey uint64) (uint32, error) { if height < 1 || height > maxHeight { panic("height cannot be less than one or greater than the max height") } unusedSize := (maxHeight - int(height)) * linksSize - nodeOffset := s.alloc(uint32(maxNodeSize - unusedSize)) + nodeOffset, err := s.alloc(uint32(maxNodeSize - unusedSize)) + if err != nil { + return 0, err + } nd := s.node(nodeOffset) nd.offset = offset nd.keyStart = keyStart nd.keyEnd = keyEnd nd.abbreviatedKey = abbreviatedKey - return nodeOffset + return nodeOffset, nil } -func (s *Skiplist) alloc(size uint32) uint32 { +func (s *Skiplist) alloc(size uint32) (uint32, error) { offset := len(s.nodes) // We only have a need for memory up to offset + size, but we never want @@ -282,6 +304,19 @@ func (s *Skiplist) alloc(size uint32) uint32 { if allocSize < minAllocSize { allocSize = minAllocSize } + // Cap the allocation at the max allowed size to avoid wasted capacity. + if allocSize > maxNodesSize { + // The new record may still not fit within the allocation, in which case + // we return early with an error. This avoids the panic below when we + // resize the slice. It also avoids the allocation and copy. + if uint64(offset)+uint64(size) > maxNodesSize { + return 0, errors.Wrapf(ErrTooManyRecords, + "alloc of new record (size=%d) would overflow uint32 (current size=%d)", + uint64(offset)+uint64(size), offset, + ) + } + allocSize = maxNodesSize + } tmp := make([]byte, len(s.nodes), allocSize) copy(tmp, s.nodes) s.nodes = tmp @@ -289,7 +324,7 @@ func (s *Skiplist) alloc(size uint32) uint32 { newSize := uint32(offset) + size s.nodes = s.nodes[:newSize] - return uint32(offset) + return uint32(offset), nil } func (s *Skiplist) node(offset uint32) *node { diff --git a/internal/batchskl/skl_test.go b/internal/batchskl/skl_test.go index e261bed086..4dad141235 100644 --- a/internal/batchskl/skl_test.go +++ b/internal/batchskl/skl_test.go @@ -24,6 +24,7 @@ import ( "testing" "time" + "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/internal/base" "github.com/stretchr/testify/require" "golang.org/x/exp/rand" @@ -215,6 +216,24 @@ func TestSkiplistAdd(t *testing.T) { require.Equal(t, 6, lengthRev(l)) } +func TestSkiplistAdd_Overflow(t *testing.T) { + // Regression test for cockroachdb/pebble#1258. The length of the nodes buffer + // cannot exceed the maximum allowable size. + d := &testStorage{} + l := newTestSkiplist(d) + + // Simulate a full nodes slice. This speeds up the test significantly, as + // opposed to adding data to the list. + l.nodes = make([]byte, maxNodesSize) + + // Adding a new node to the list would overflow the nodes slice. Note that it + // is the size of a new node struct that is relevant here, rather than the + // size of the data being added to the list. + err := l.Add(d.add("too much!")) + require.Error(t, err) + require.True(t, errors.Is(err, ErrTooManyRecords)) +} + // TestIteratorNext tests a basic iteration over all nodes from the beginning. func TestIteratorNext(t *testing.T) { const n = 100 @@ -480,7 +499,8 @@ func BenchmarkIterNext(b *testing.B) { for len(d.data)+20 < cap(d.data) { key := randomKey(rng, buf[:]) offset := d.addBytes(key) - _ = l.Add(offset) + err := l.Add(offset) + require.NoError(b, err) } it := l.NewIter(nil, nil) @@ -504,7 +524,8 @@ func BenchmarkIterPrev(b *testing.B) { for len(d.data)+20 < cap(d.data) { key := randomKey(rng, buf[:]) offset := d.addBytes(key) - _ = l.Add(offset) + err := l.Add(offset) + require.NoError(b, err) } it := l.NewIter(nil, nil)