Skip to content

Commit

Permalink
storage/pebbleiter: guard against double-closed Iterators
Browse files Browse the repository at this point in the history
Replace direct uses of the `*pebble.Iterator` type with a `pebbleiter.Iterator`
type definition. In production builds, this type is a simple type definition for
`*pebble.Iterator` which suffers no overhead. In `crdb_test` builds, the
`pebbleiter.Iterator` is defined as a `*pebbleiter.AssertionIter` type that can
assert proper use of the `*pebble.Iterator` type and its contract.

Initially, use this new assertion iterator to ensure that a `*pebble.Iterator`
is never double-closed, which may cause the Iterator to be added to a sync pool
twice leading to races. This type of assertion cannot be done by
`*pebble.Iterator`, because the iterator may already be reused by the time the
second Close is run.

In the future, I'd like to also introduce assertions that pkg/storage always
checks validity before calling HasPointAndRange and RangeKeyChanged (see
cockroachdb/pebble#2205).

Release note: None
Epic: None

Close #90643.
  • Loading branch information
jbowens committed Jan 5, 2023
1 parent c925e38 commit 7e098bf
Show file tree
Hide file tree
Showing 14 changed files with 191 additions and 16 deletions.
2 changes: 2 additions & 0 deletions build/bazelutil/check.sh
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,8 @@ pkg/util/buildutil/crdb_test_off.go://go:build !crdb_test || crdb_test_off
pkg/util/buildutil/crdb_test_on.go://go:build crdb_test && !crdb_test_off
pkg/kv/kvnemesis/kvnemesisutil/crdb_test_off.go://go:build !crdb_test || crdb_test_off
pkg/kv/kvnemesis/kvnemesisutil/crdb_test_on.go://go:build crdb_test && !crdb_test_off
pkg/storage/pebbleiter/crdb_test_off.go://go:build !crdb_test || crdb_test_off
pkg/storage/pebbleiter/crdb_test_on.go://go:build crdb_test && !crdb_test_off
"

if [ -z "${COCKROACH_BAZEL_CHECK_FAST:-}" ]; then
Expand Down
4 changes: 4 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -519,6 +519,7 @@ ALL_TESTS = [
"//pkg/storage/enginepb:enginepb_test",
"//pkg/storage/fs:fs_test",
"//pkg/storage/metamorphic:metamorphic_test",
"//pkg/storage/pebbleiter:pebbleiter_test",
"//pkg/storage:storage_test",
"//pkg/testutils/docker:docker_test",
"//pkg/testutils/echotest:echotest_test",
Expand Down Expand Up @@ -1875,6 +1876,8 @@ GO_TARGETS = [
"//pkg/storage/fs:fs_test",
"//pkg/storage/metamorphic:metamorphic",
"//pkg/storage/metamorphic:metamorphic_test",
"//pkg/storage/pebbleiter:pebbleiter",
"//pkg/storage/pebbleiter:pebbleiter_test",
"//pkg/storage:storage",
"//pkg/storage:storage_test",
"//pkg/testutils/buildutil:buildutil",
Expand Down Expand Up @@ -2977,6 +2980,7 @@ GET_X_DATA_TARGETS = [
"//pkg/storage/enginepb:get_x_data",
"//pkg/storage/fs:get_x_data",
"//pkg/storage/metamorphic:get_x_data",
"//pkg/storage/pebbleiter:get_x_data",
"//pkg/testutils:get_x_data",
"//pkg/testutils/buildutil:get_x_data",
"//pkg/testutils/colcontainerutils:get_x_data",
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/spanset/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ go_library(
"//pkg/keys",
"//pkg/roachpb",
"//pkg/storage",
"//pkg/storage/pebbleiter",
"//pkg/util/hlc",
"//pkg/util/log",
"//pkg/util/protoutil",
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/pebbleiter"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
Expand Down Expand Up @@ -401,7 +402,7 @@ func (i *EngineIterator) UnsafeRawEngineKey() []byte {
}

// GetRawIter is part of the storage.EngineIterator interface.
func (i *EngineIterator) GetRawIter() *pebble.Iterator {
func (i *EngineIterator) GetRawIter() pebbleiter.Iterator {
return i.i.GetRawIter()
}

Expand Down
1 change: 1 addition & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ go_library(
"//pkg/settings/cluster",
"//pkg/storage/enginepb",
"//pkg/storage/fs",
"//pkg/storage/pebbleiter",
"//pkg/util",
"//pkg/util/admission",
"//pkg/util/bufalloc",
Expand Down
3 changes: 2 additions & 1 deletion pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/storage/pebbleiter"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/iterutil"
Expand Down Expand Up @@ -356,7 +357,7 @@ type EngineIterator interface {
Value() ([]byte, error)
// GetRawIter is a low-level method only for use in the storage package,
// that returns the underlying pebble Iterator.
GetRawIter() *pebble.Iterator
GetRawIter() pebbleiter.Iterator
// SeekEngineKeyGEWithLimit is similar to SeekEngineKeyGE, but takes an
// additional exclusive upper limit parameter. The limit is semantically
// best-effort, and is an optimization to avoid O(n^2) iteration behavior in
Expand Down
5 changes: 3 additions & 2 deletions pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/storage/pebbleiter"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -1882,7 +1883,7 @@ type pebbleReadOnly struct {
prefixEngineIter pebbleIterator
normalEngineIter pebbleIterator

iter *pebble.Iterator
iter pebbleiter.Iterator
iterUsed bool // avoids cloning after PinEngineStateForIterators()
durability DurabilityRequirement
closed bool
Expand Down Expand Up @@ -2056,7 +2057,7 @@ func (p *pebbleReadOnly) PinEngineStateForIterators() error {
if p.durability == GuaranteedDurability {
o = &pebble.IterOptions{OnlyReadGuaranteedDurable: true}
}
p.iter = p.parent.db.NewIter(o)
p.iter = pebbleiter.MaybeWrap(p.parent.db.NewIter(o))
// NB: p.iterUsed == false avoids cloning this in NewMVCCIterator(), since
// we've just created it.
}
Expand Down
7 changes: 4 additions & 3 deletions pkg/storage/pebble_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/pebbleiter"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
Expand Down Expand Up @@ -45,7 +46,7 @@ type pebbleBatch struct {
prefixEngineIter pebbleIterator
normalEngineIter pebbleIterator

iter *pebble.Iterator
iter pebbleiter.Iterator
iterUsed bool // avoids cloning after PinEngineStateForIterators()
writeOnly bool
closed bool
Expand Down Expand Up @@ -243,9 +244,9 @@ func (p *pebbleBatch) SupportsRangeKeys() bool {
func (p *pebbleBatch) PinEngineStateForIterators() error {
if p.iter == nil {
if p.batch.Indexed() {
p.iter = p.batch.NewIter(nil)
p.iter = pebbleiter.MaybeWrap(p.batch.NewIter(nil))
} else {
p.iter = p.db.NewIter(nil)
p.iter = pebbleiter.MaybeWrap(p.db.NewIter(nil))
}
// NB: p.iterUsed == false avoids cloning this in NewMVCCIterator(). We've
// just created it, so cloning it would just be overhead.
Expand Down
26 changes: 17 additions & 9 deletions pkg/storage/pebble_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/pebbleiter"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
Expand All @@ -31,7 +32,7 @@ import (
// should only be used in one of the two modes.
type pebbleIterator struct {
// Underlying iterator for the DB.
iter *pebble.Iterator
iter pebbleiter.Iterator
options pebble.IterOptions
// Reusable buffer for MVCCKey or EngineKey encoding.
keyBuf []byte
Expand Down Expand Up @@ -91,14 +92,17 @@ func newPebbleIterator(
p := pebbleIterPool.Get().(*pebbleIterator)
p.reusable = false // defensive
p.init(nil, opts, durability, supportsRangeKeys)
p.iter = handle.NewIter(&p.options)
p.iter = pebbleiter.MaybeWrap(handle.NewIter(&p.options))
return p
}

// newPebbleIteratorByCloning creates a new Pebble iterator by cloning the given
// iterator and reconfiguring it.
func newPebbleIteratorByCloning(
iter *pebble.Iterator, opts IterOptions, durability DurabilityRequirement, supportsRangeKeys bool,
iter pebbleiter.Iterator,
opts IterOptions,
durability DurabilityRequirement,
supportsRangeKeys bool,
) *pebbleIterator {
var err error
p := pebbleIterPool.Get().(*pebbleIterator)
Expand Down Expand Up @@ -128,11 +132,12 @@ func newPebbleSSTIterator(
externalIterOpts = append(externalIterOpts, pebble.ExternalIterForwardOnly{})
}

var err error
if p.iter, err = pebble.NewExternalIter(DefaultPebbleOptions(), &p.options, files, externalIterOpts...); err != nil {
iter, err := pebble.NewExternalIter(DefaultPebbleOptions(), &p.options, files, externalIterOpts...)
if err != nil {
p.Close()
return nil, err
}
p.iter = pebbleiter.MaybeWrap(iter)
p.external = true
return p, nil
}
Expand All @@ -141,7 +146,10 @@ func newPebbleSSTIterator(
// reconfiguring the given iter. It is valid to pass a nil iter and then create
// p.iter using p.options, to avoid redundant reconfiguration via SetOptions().
func (p *pebbleIterator) init(
iter *pebble.Iterator, opts IterOptions, durability DurabilityRequirement, supportsRangeKeys bool,
iter pebbleiter.Iterator,
opts IterOptions,
durability DurabilityRequirement,
supportsRangeKeys bool,
) {
*p = pebbleIterator{
iter: iter,
Expand All @@ -164,7 +172,7 @@ func (p *pebbleIterator) init(
// 3. iter == nil: create a new iterator from handle.
func (p *pebbleIterator) initReuseOrCreate(
handle pebble.Reader,
iter *pebble.Iterator,
iter pebbleiter.Iterator,
clone bool,
opts IterOptions,
durability DurabilityRequirement,
Expand All @@ -177,7 +185,7 @@ func (p *pebbleIterator) initReuseOrCreate(

p.init(nil, opts, durability, supportsRangeKeys)
if iter == nil {
p.iter = handle.NewIter(&p.options)
p.iter = pebbleiter.MaybeWrap(handle.NewIter(&p.options))
} else if clone {
var err error
p.iter, err = iter.Clone(pebble.CloneOptions{
Expand Down Expand Up @@ -941,7 +949,7 @@ func (p *pebbleIterator) IsPrefix() bool {
}

// GetRawIter is part of the EngineIterator interface.
func (p *pebbleIterator) GetRawIter() *pebble.Iterator {
func (p *pebbleIterator) GetRawIter() pebbleiter.Iterator {
return p.iter
}

Expand Down
43 changes: 43 additions & 0 deletions pkg/storage/pebbleiter/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

# gazelle:exclude gen-crdb_test_off.go
# gazelle:exclude gen-crdb_test_on.go

# keep
go_library(
name = "pebbleiter",
srcs = select({
"//build/toolchains:crdb_test": [":gen-crdb-test-on"],
"//conditions:default": [":gen-crdb-test-off"],
}),
importpath = "github.com/cockroachdb/cockroach/pkg/storage/pebbleiter",
visibility = ["//visibility:public"],
deps = ["@com_github_cockroachdb_pebble//:pebble"],
)

REMOVE_GO_BUILD_CONSTRAINTS = "cat $< | grep -v '//go:build' | grep -v '// +build' > $@"

genrule(
name = "gen-crdb-test-on",
srcs = ["crdb_test_on.go"],
outs = ["gen-crdb_test_on.go"],
cmd = REMOVE_GO_BUILD_CONSTRAINTS,
)

genrule(
name = "gen-crdb-test-off",
srcs = ["crdb_test_off.go"],
outs = ["gen-crdb_test_off.go"],
cmd = REMOVE_GO_BUILD_CONSTRAINTS,
)

go_test(
name = "pebbleiter_test",
srcs = ["crdb_test_test.go"],
args = ["-test.timeout=295s"],
embed = [":pebbleiter"], # keep
deps = ["@com_github_stretchr_testify//require"],
)

get_x_data(name = "get_x_data")
24 changes: 24 additions & 0 deletions pkg/storage/pebbleiter/crdb_test_off.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

//go:build !crdb_test || crdb_test_off
// +build !crdb_test crdb_test_off

package pebbleiter

import "github.com/cockroachdb/pebble"

// Iterator redefines *pebble.Iterator.
type Iterator = *pebble.Iterator

// MaybeWrap returns the provided Iterator, unmodified.
func MaybeWrap(iter *pebble.Iterator) Iterator {
return iter
}
50 changes: 50 additions & 0 deletions pkg/storage/pebbleiter/crdb_test_on.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

//go:build crdb_test && !crdb_test_off
// +build crdb_test,!crdb_test_off

package pebbleiter

import "github.com/cockroachdb/pebble"

// Iterator wraps the *pebble.Iterator in crdb_test builds with an AssertionIter
// that detects when Close is called on the iterator twice. Double closes are
// problematic because they can result in an iterator being added to a sync pool
// twice, allowing concurrent use of the same iterator struct.
type Iterator = *AssertionIter

// MaybeWrap returns the provided Pebble iterator, wrapped with double close
// detection.
func MaybeWrap(iter *pebble.Iterator) Iterator {
return &AssertionIter{Iterator: iter}
}

// AssertionIter wraps a *pebble.Iterator with assertion checking.
type AssertionIter struct {
*pebble.Iterator
closed bool
}

func (i *AssertionIter) Clone(cloneOpts pebble.CloneOptions) (Iterator, error) {
iter, err := i.Iterator.Clone(cloneOpts)
if err != nil {
return nil, err
}
return MaybeWrap(iter), nil
}

func (i *AssertionIter) Close() error {
if i.closed {
panic("pebble.Iterator already closed")
}
i.closed = true
return i.Iterator.Close()
}
23 changes: 23 additions & 0 deletions pkg/storage/pebbleiter/crdb_test_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package pebbleiter

import (
"testing"

"github.com/stretchr/testify/require"
)

func TestPebbleIterWrapped(t *testing.T) {
// Sanity-check: make sure CrdbTestBuild is set. This should be true for
// any test.
require.NotNil(t, MaybeWrap(nil))
}
15 changes: 15 additions & 0 deletions pkg/storage/pebbleiter/pebbleiter.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

// Package pebbleiter exposes a type that selectively wraps a Pebble Iterator
// only in crdb_test builds. This wrapper type performs assertions in test-only
// builds. In non-test-only builds, pebbleiter exposees a direct type alias of
// the pebble.Iterator.
package pebbleiter

0 comments on commit 7e098bf

Please sign in to comment.