Skip to content

Commit

Permalink
Merge #89004 #89116
Browse files Browse the repository at this point in the history
89004: storage: add `TestMVCCHistories` cases r=erikgrinaker a=erikgrinaker

**storage: add `TestMVCCHistories` metamorphic param for peek bounds**

This patch adds a metamorphic test parameter that enables peek bounds
for MVCC range key-related commands. These peek bounds will always
result in identical, correct MVCC stats.

Unfortunately, we can't assert that the peek bounds are enforced,
because importing the `spanset` package causes a cyclic dependency.

Release note: None
  
**storage: use `del_range_ts` in `TestMVCCHistories`**

Many of the early `TestMVCCHistories` tests for MVCC range tombstones
used the direct engine method `put_rangekey` rather than the MVCC range
tombstone write `del_range_ts` (which does conflict checks, MVCC stats
adjustments, etc), because the latter did not exist yet.

This patch migrates most tests to `del_range_ts`, and also renames some
of the test files to refer to range tombstone rather than range key.
Stats assertions have also been enabled for some tests.

There are no significant changes to the tests themselves.

Release note: None
  
**storage: add `TestMVCCHistories` cases**

This patch adds additional test cases for `TestMVCCHistories`. These
were primarily designed by deliberately introducing bugs in MVCC code
that did not cause existing test cases to fail.

Resolves #86655.

Release note: None

89116: kvserver: skip TestLeasePreferencesDuringOutage r=mgartner a=mgartner

See #88769.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
  • Loading branch information
3 people committed Sep 30, 2022
3 parents 8e6b81b + aaac54c + 0eafb65 commit e786cba
Show file tree
Hide file tree
Showing 31 changed files with 2,930 additions and 1,421 deletions.
1 change: 1 addition & 0 deletions pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -780,6 +780,7 @@ func gossipLiveness(t *testing.T, tc *testcluster.TestCluster) {
// lease in a single cycle of the replicate_queue.
func TestLeasePreferencesDuringOutage(t *testing.T) {
defer leaktest.AfterTest(t)()
skip.WithIssue(t, 88769, "flaky test")
defer log.Scope(t).Close(t)

stickyRegistry := server.NewStickyInMemEnginesRegistry()
Expand Down
1 change: 1 addition & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,7 @@ go_test(
"//pkg/keys",
"//pkg/kv/kvserver/concurrency/lock",
"//pkg/kv/kvserver/diskmap",
"//pkg/kv/kvserver/spanset",
"//pkg/kv/kvserver/uncertainty",
"//pkg/roachpb",
"//pkg/settings/cluster",
Expand Down
85 changes: 44 additions & 41 deletions pkg/storage/mvcc_history_metamorphic_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@
// licenses/APL.txt.
//

package storage
package storage_test

import (
"bytes"
Expand All @@ -19,6 +19,7 @@ import (
"testing"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/stretchr/testify/require"
Expand All @@ -32,7 +33,7 @@ type metamorphicIterator struct {
seed int64
t *testing.T
r *rand.Rand
it SimpleMVCCIterator
it storage.SimpleMVCCIterator
// isForward is true if the wrapped iterator is in forward mode at the
// beginning of moveAround. We then need to leave the iterator in forward mode
// because the caller might subsequently invoke NextKey which is illegal on an
Expand All @@ -47,12 +48,14 @@ type metamorphicIterator struct {
// - a metamorphicMVCCIterator, if `it` is an MVCCIterator
// - a metamorphicMVCCIncrementalIterator, if `it` is an MVCCIncrementalIterator
// a metamorphicIterator otherwise.
func newMetamorphicIterator(t *testing.T, seed int64, it SimpleMVCCIterator) SimpleMVCCIterator {
func newMetamorphicIterator(
t *testing.T, seed int64, it storage.SimpleMVCCIterator,
) storage.SimpleMVCCIterator {
iter := &metamorphicIterator{t: t, seed: seed, r: rand.New(rand.NewSource(seed)), it: it}
if _, isMVCC := it.(MVCCIterator); isMVCC {
if _, isMVCC := it.(storage.MVCCIterator); isMVCC {
return &metamorphicMVCCIterator{metamorphicIterator: iter}
}
if _, isIncremental := it.(*MVCCIncrementalIterator); isIncremental {
if _, isIncremental := it.(*storage.MVCCIncrementalIterator); isIncremental {
return &metamorphicMVCCIncrementalIterator{metamorphicIterator: iter}
}
return iter
Expand Down Expand Up @@ -100,8 +103,8 @@ func (m *metamorphicIterator) moveAround() {
}

cur := m.it.UnsafeKey().Clone()
mvccIt, _ := m.it.(MVCCIterator)
iit, _ := m.it.(*MVCCIncrementalIterator)
mvccIt, _ := m.it.(storage.MVCCIterator)
iit, _ := m.it.(*storage.MVCCIncrementalIterator)
var resetActions []action

actions := []action{
Expand All @@ -124,7 +127,7 @@ func (m *metamorphicIterator) moveAround() {
},
{
"SeekGE(Max)",
func() { m.it.SeekGE(MVCCKeyMax) },
func() { m.it.SeekGE(storage.MVCCKeyMax) },
},
}

Expand All @@ -149,7 +152,7 @@ func (m *metamorphicIterator) moveAround() {
},
}, action{
"SeekLT(Max)",
func() { mvccIt.SeekLT(MVCCKeyMax) },
func() { mvccIt.SeekLT(storage.MVCCKeyMax) },
})
// Can only leave iterator in reverse mode if it's in reverse
// initially, otherwise caller wouldn't be allowed to invoke NextKey
Expand All @@ -170,7 +173,7 @@ func (m *metamorphicIterator) moveAround() {

hasPoint, _ := m.it.HasPointAndRange()
rangeKeys := m.it.RangeKeys().Clone()
var rangeKeysIgnoringTime MVCCRangeKeyStack
var rangeKeysIgnoringTime storage.MVCCRangeKeyStack
if iit != nil {
rangeKeysIgnoringTime = iit.RangeKeysIgnoringTime()
}
Expand All @@ -189,9 +192,9 @@ func (m *metamorphicIterator) moveAround() {
choice := actions[m.r.Intn(len(actions))]
printfln("action: %s", choice)

// NB: if this is an incr iter that ignores time, we can't expect SeekGE(cur) to
// NB: if this is an incr iter it may be ignoring time, so we can't expect SeekGE(cur) to
// be able to retrieve the current key, as SeekGE always respects the time bound.
if iit == nil || !iit.ignoringTime {
if iit == nil || !iit.IgnoringTime() {
resetActions = append(resetActions, action{
"SeekGE(cur)",
func() {
Expand All @@ -209,10 +212,10 @@ func (m *metamorphicIterator) moveAround() {
if bytes.Compare(cur.Key, roachpb.LocalMax) >= 0 {
// Make sure we don't put a global-only iter into local keyspace.
printfln("seeking to LocalMax")
m.it.SeekGE(MakeMVCCMetadataKey(roachpb.LocalMax))
m.it.SeekGE(storage.MakeMVCCMetadataKey(roachpb.LocalMax))
} else {
printfln("seeking to KeyMin")
m.it.SeekGE(NilKey)
m.it.SeekGE(storage.NilKey)
}
for {
valid, err := m.it.Valid()
Expand All @@ -239,7 +242,7 @@ func (m *metamorphicIterator) moveAround() {
resetActions = append(resetActions, action{
"SeekLT(max) && RevIterate",
func() {
mvccIt.SeekLT(MVCCKeyMax) // NB: incompatible with IsPrefix, so we excluded that above
mvccIt.SeekLT(storage.MVCCKeyMax) // NB: incompatible with IsPrefix, so we excluded that above
for {
valid, err := m.it.Valid()
require.Nil(m.t, err)
Expand All @@ -260,7 +263,7 @@ func (m *metamorphicIterator) moveAround() {
resetAction.do()
{
hasPoint2, _ := m.it.HasPointAndRange() // circumvent hated shadowing lint
var rangeKeysIgnoringTime2 MVCCRangeKeyStack
var rangeKeysIgnoringTime2 storage.MVCCRangeKeyStack
if iit != nil {
rangeKeysIgnoringTime2 = iit.RangeKeysIgnoringTime()
}
Expand All @@ -287,7 +290,7 @@ func (m *metamorphicIterator) Close() {
m.it.Close()
}

func (m *metamorphicIterator) SeekGE(key MVCCKey) {
func (m *metamorphicIterator) SeekGE(key storage.MVCCKey) {
m.isForward = true
m.it.SeekGE(key)
m.moveAround()
Expand All @@ -309,7 +312,7 @@ func (m *metamorphicIterator) NextKey() {
m.moveAround()
}

func (m *metamorphicIterator) UnsafeKey() MVCCKey {
func (m *metamorphicIterator) UnsafeKey() storage.MVCCKey {
return m.it.UnsafeKey()
}

Expand All @@ -325,7 +328,7 @@ func (m *metamorphicIterator) RangeBounds() roachpb.Span {
return m.it.RangeBounds()
}

func (m *metamorphicIterator) RangeKeys() MVCCRangeKeyStack {
func (m *metamorphicIterator) RangeKeys() storage.MVCCRangeKeyStack {
return m.it.RangeKeys()
}

Expand All @@ -340,56 +343,56 @@ type metamorphicMVCCIterator struct {
*metamorphicIterator
}

var _ MVCCIterator = (*metamorphicMVCCIterator)(nil)
var _ storage.MVCCIterator = (*metamorphicMVCCIterator)(nil)

func (m *metamorphicMVCCIterator) SeekLT(key MVCCKey) {
m.it.(MVCCIterator).SeekLT(key)
func (m *metamorphicMVCCIterator) SeekLT(key storage.MVCCKey) {
m.it.(storage.MVCCIterator).SeekLT(key)
m.moveAround()
}

func (m *metamorphicMVCCIterator) Prev() {
m.it.(MVCCIterator).Prev()
m.it.(storage.MVCCIterator).Prev()
m.moveAround()
}

func (m *metamorphicMVCCIterator) SeekIntentGE(key roachpb.Key, txnUUID uuid.UUID) {
m.it.(MVCCIterator).SeekIntentGE(key, txnUUID)
m.it.(storage.MVCCIterator).SeekIntentGE(key, txnUUID)
m.moveAround()
}

func (m *metamorphicMVCCIterator) Key() MVCCKey {
return m.it.(MVCCIterator).Key()
func (m *metamorphicMVCCIterator) Key() storage.MVCCKey {
return m.it.(storage.MVCCIterator).Key()
}

func (m *metamorphicMVCCIterator) UnsafeRawKey() []byte {
return m.it.(MVCCIterator).UnsafeRawKey()
return m.it.(storage.MVCCIterator).UnsafeRawKey()
}

func (m *metamorphicMVCCIterator) UnsafeRawMVCCKey() []byte {
return m.it.(MVCCIterator).UnsafeRawMVCCKey()
return m.it.(storage.MVCCIterator).UnsafeRawMVCCKey()
}

func (m *metamorphicMVCCIterator) Value() []byte {
return m.it.(MVCCIterator).Value()
return m.it.(storage.MVCCIterator).Value()
}

func (m *metamorphicMVCCIterator) ValueProto(msg protoutil.Message) error {
return m.it.(MVCCIterator).ValueProto(msg)
return m.it.(storage.MVCCIterator).ValueProto(msg)
}

func (m *metamorphicMVCCIterator) FindSplitKey(
start, end, minSplitKey roachpb.Key, targetSize int64,
) (MVCCKey, error) {
return m.it.(MVCCIterator).FindSplitKey(start, end, minSplitKey, targetSize)
) (storage.MVCCKey, error) {
return m.it.(storage.MVCCIterator).FindSplitKey(start, end, minSplitKey, targetSize)
}

func (m *metamorphicMVCCIterator) Stats() IteratorStats {
func (m *metamorphicMVCCIterator) Stats() storage.IteratorStats {
// TODO(tbg): these will be wrong since we do extra movement.
return m.it.(MVCCIterator).Stats()
return m.it.(storage.MVCCIterator).Stats()
}

func (m *metamorphicMVCCIterator) IsPrefix() bool {
return m.it.(MVCCIterator).IsPrefix()
return m.it.(storage.MVCCIterator).IsPrefix()
}

type metamorphicMVCCIncrementalIterator struct {
Expand All @@ -398,29 +401,29 @@ type metamorphicMVCCIncrementalIterator struct {

var _ mvccIncrementalIteratorI = (*metamorphicMVCCIncrementalIterator)(nil)

func (m *metamorphicMVCCIncrementalIterator) RangeKeysIgnoringTime() MVCCRangeKeyStack {
return m.it.(*MVCCIncrementalIterator).RangeKeysIgnoringTime()
func (m *metamorphicMVCCIncrementalIterator) RangeKeysIgnoringTime() storage.MVCCRangeKeyStack {
return m.it.(*storage.MVCCIncrementalIterator).RangeKeysIgnoringTime()
}

func (m *metamorphicMVCCIncrementalIterator) RangeKeyChangedIgnoringTime() bool {
if m.seed != 0 {
return m.rangeKeyChangedIgnoringTime
}
return m.it.(*MVCCIncrementalIterator).RangeKeyChangedIgnoringTime()
return m.it.(*storage.MVCCIncrementalIterator).RangeKeyChangedIgnoringTime()
}

func (m *metamorphicMVCCIncrementalIterator) NextIgnoringTime() {
m.it.(*MVCCIncrementalIterator).NextIgnoringTime()
m.it.(*storage.MVCCIncrementalIterator).NextIgnoringTime()
m.isForward = true
m.moveAround()
}

func (m *metamorphicMVCCIncrementalIterator) NextKeyIgnoringTime() {
m.it.(*MVCCIncrementalIterator).NextKeyIgnoringTime()
m.it.(*storage.MVCCIncrementalIterator).NextKeyIgnoringTime()
m.isForward = true
m.moveAround()
}

func (m *metamorphicMVCCIncrementalIterator) TryGetIntentError() error {
return m.it.(*MVCCIncrementalIterator).TryGetIntentError()
return m.it.(*storage.MVCCIncrementalIterator).TryGetIntentError()
}
Loading

0 comments on commit e786cba

Please sign in to comment.