Skip to content

Commit

Permalink
storage: delete MVCCIterator.SeekIntentGE
Browse files Browse the repository at this point in the history
Now that the specialized method is no longer used in
`MVCCResolveWriteIntent`, we can delete it.

Epic: None
Release note: None
  • Loading branch information
nvanbenschoten committed Sep 21, 2023
1 parent 180c61b commit c01eeee
Show file tree
Hide file tree
Showing 10 changed files with 1 addition and 215 deletions.
1 change: 0 additions & 1 deletion pkg/kv/kvserver/spanset/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ go_library(
"//pkg/util/hlc",
"//pkg/util/log",
"//pkg/util/protoutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_pebble//:pebble",
"@com_github_cockroachdb_pebble//rangekey",
Expand Down
7 changes: 0 additions & 7 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/pebble"
"github.com/cockroachdb/pebble/rangekey"
)
Expand Down Expand Up @@ -85,12 +84,6 @@ func (i *MVCCIterator) SeekGE(key storage.MVCCKey) {
i.checkAllowed(roachpb.Span{Key: key.Key}, true)
}

// SeekIntentGE is part of the storage.MVCCIterator interface.
func (i *MVCCIterator) SeekIntentGE(key roachpb.Key, txnUUID uuid.UUID) {
i.i.SeekIntentGE(key, txnUUID)
i.checkAllowed(roachpb.Span{Key: key}, true)
}

// SeekLT is part of the storage.MVCCIterator interface.
func (i *MVCCIterator) SeekLT(key storage.MVCCKey) {
i.i.SeekLT(key)
Expand Down
7 changes: 0 additions & 7 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -257,13 +257,6 @@ type MVCCIterator interface {
// the first key.
Prev()

// SeekIntentGE is a specialized version of SeekGE(MVCCKey{Key: key}), when
// the caller expects to find an intent, and additionally has the txnUUID
// for the intent it is looking for. When running with separated intents,
// this can optimize the behavior of the underlying Engine for write heavy
// keys by avoiding the need to iterate over many deleted intents.
SeekIntentGE(key roachpb.Key, txnUUID uuid.UUID)

// UnsafeRawKey returns the current raw key which could be an encoded
// MVCCKey, or the more general EngineKey (for a lock table key).
// This is a low-level and dangerous method since it will expose the
Expand Down
39 changes: 0 additions & 39 deletions pkg/storage/intent_interleaving_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
)
Expand Down Expand Up @@ -588,44 +587,6 @@ func (i *intentInterleavingIter) SeekGE(key MVCCKey) {
i.computePos()
}

func (i *intentInterleavingIter) SeekIntentGE(key roachpb.Key, txnUUID uuid.UUID) {
adjustRangeKeyChanged := i.shouldAdjustSeekRangeKeyChanged()

i.dir = +1
i.valid = true
i.err = nil

if i.constraint != notConstrained {
i.checkConstraint(key, false)
}
i.iter.SeekGE(MVCCKey{Key: key})
if err := i.tryDecodeKey(); err != nil {
return
}
i.rangeKeyChanged = i.iter.RangeKeyChanged()
if adjustRangeKeyChanged {
i.adjustSeekRangeKeyChanged()
}
var engineKey EngineKey
engineKey, i.intentKeyBuf = LockTableKey{
Key: key,
Strength: lock.Intent,
TxnUUID: txnUUID,
}.ToEngineKey(i.intentKeyBuf)
var limitKey roachpb.Key
if i.iterValid && !i.prefix {
limitKey = i.makeUpperLimitKey()
}
iterState, err := i.intentIter.SeekEngineKeyGEWithLimit(engineKey, limitKey)
if err = i.tryDecodeLockKey(iterState, err); err != nil {
return
}
if err := i.maybeSkipIntentRangeKey(); err != nil {
return
}
i.computePos()
}

func (i *intentInterleavingIter) checkConstraint(k roachpb.Key, isExclusiveUpper bool) {
kConstraint := constrainedToGlobal
if isLocal(k) {
Expand Down
11 changes: 0 additions & 11 deletions pkg/storage/mvcc_history_metamorphic_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"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/cockroachdb/pebble"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -146,11 +145,6 @@ func (m *metamorphicIterator) moveAround() {
actions = append(actions, action{
"SeekLT(cur)",
func() { mvccIt.SeekLT(cur) },
}, action{
"SeekIntentGE(cur, 00000)",
func() {
mvccIt.SeekIntentGE(cur.Key, uuid.Nil)
},
}, action{
"SeekLT(Max)",
func() { mvccIt.SeekLT(storage.MVCCKeyMax) },
Expand Down Expand Up @@ -369,11 +363,6 @@ func (m *metamorphicMVCCIterator) UnsafeLazyValue() pebble.LazyValue {
return m.it.(storage.MVCCIterator).UnsafeLazyValue()
}

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

func (m *metamorphicMVCCIterator) UnsafeRawKey() []byte {
return m.it.(storage.MVCCIterator).UnsafeRawKey()
}
Expand Down
12 changes: 0 additions & 12 deletions pkg/storage/mvcc_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,6 @@ var (
// iter_new_incremental [k=<key>] [end=<key>] [startTs=<int>[,<int>]] [endTs=<int>[,<int>]] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly] [maskBelow=<int>[,<int>]] [intents=error|aggregate|emit]
// iter_seek_ge k=<key> [ts=<int>[,<int>]]
// iter_seek_lt k=<key> [ts=<int>[,<int>]]
// iter_seek_intent_ge k=<key> txn=<name>
// iter_next
// iter_next_ignoring_time
// iter_next_key_ignoring_time
Expand Down Expand Up @@ -805,7 +804,6 @@ var commands = map[string]cmd{
"iter_new_read_as_of": {typReadOnly, cmdIterNewReadAsOf}, // readAsOfIterator
"iter_seek_ge": {typReadOnly, cmdIterSeekGE},
"iter_seek_lt": {typReadOnly, cmdIterSeekLT},
"iter_seek_intent_ge": {typReadOnly, cmdIterSeekIntentGE},
"iter_next": {typReadOnly, cmdIterNext},
"iter_next_ignoring_time": {typReadOnly, cmdIterNextIgnoringTime}, // MVCCIncrementalIterator
"iter_next_key_ignoring_time": {typReadOnly, cmdIterNextKeyIgnoringTime}, // MVCCIncrementalIterator
Expand Down Expand Up @@ -1925,16 +1923,6 @@ func cmdIterSeekGE(e *evalCtx) error {
return nil
}

func cmdIterSeekIntentGE(e *evalCtx) error {
key := e.getKey()
var txnName string
e.scanArg("txn", &txnName)
txn := e.txns[txnName]
e.mvccIter().SeekIntentGE(key, txn.ID)
printIter(e)
return nil
}

func cmdIterSeekLT(e *evalCtx) error {
key := e.getKey()
ts := e.getTs(nil)
Expand Down
6 changes: 0 additions & 6 deletions pkg/storage/pebble_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/pebbleiter"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
"github.com/cockroachdb/pebble/sstable"
Expand Down Expand Up @@ -337,11 +336,6 @@ func (p *pebbleIterator) SeekGE(key MVCCKey) {
}
}

// SeekIntentGE implements the MVCCIterator interface.
func (p *pebbleIterator) SeekIntentGE(key roachpb.Key, _ uuid.UUID) {
p.SeekGE(MVCCKey{Key: key})
}

// SeekEngineKeyGE implements the EngineIterator interface.
func (p *pebbleIterator) SeekEngineKeyGE(key EngineKey) (valid bool, err error) {
p.keyBuf = key.EncodeToBuf(p.keyBuf[:0])
Expand Down
57 changes: 0 additions & 57 deletions pkg/storage/testdata/mvcc_histories/range_tombstone_iter
Original file line number Diff line number Diff line change
Expand Up @@ -998,63 +998,6 @@ iter_next_key: {h-k}/[1.000000000,0=/<empty>] !
iter_seek_ge: "d"/0,0=txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true {d-f}/[5.000000000,0=/<empty> 1.000000000,0=/<empty>] !
iter_next_key: "e"/3.000000000,0=/BYTES/e3 {d-f}/[5.000000000,0=/<empty> 1.000000000,0=/<empty>]

# Test SeekIntentGE both with and without intents and range keys.
run ok
iter_new types=pointsAndRanges
iter_seek_intent_ge k=b txn=A
iter_seek_intent_ge k=d txn=A
iter_seek_intent_ge k=i txn=A
iter_seek_intent_ge k=j txn=A
iter_seek_intent_ge k=k txn=A
----
iter_seek_intent_ge: {b-c}/[3.000000000,0=/<empty> 1.000000000,0=/<empty>] !
iter_seek_intent_ge: "d"/0,0=txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true {d-f}/[5.000000000,0=/<empty> 1.000000000,0=/<empty>] !
iter_seek_intent_ge: {h-k}/[1.000000000,0=/<empty>] !
iter_seek_intent_ge: "j"/0,0=txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true {h-k}/[1.000000000,0=/<empty>]
iter_seek_intent_ge: "k"/5.000000000,0=/BYTES/k5 !

run ok
iter_new kind=keys types=pointsAndRanges
iter_seek_intent_ge k=b txn=A
iter_seek_intent_ge k=d txn=A
iter_seek_intent_ge k=i txn=A
iter_seek_intent_ge k=j txn=A
iter_seek_intent_ge k=k txn=A
----
iter_seek_intent_ge: {b-c}/[3.000000000,0=/<empty> 1.000000000,0=/<empty>] !
iter_seek_intent_ge: {d-f}/[5.000000000,0=/<empty> 1.000000000,0=/<empty>] !
iter_seek_intent_ge: {h-k}/[1.000000000,0=/<empty>] !
iter_seek_intent_ge: {h-k}/[1.000000000,0=/<empty>]
iter_seek_intent_ge: "k"/5.000000000,0=/BYTES/k5 !

run ok
iter_new types=pointsOnly
iter_seek_intent_ge k=b txn=A
iter_seek_intent_ge k=d txn=A
iter_seek_intent_ge k=i txn=A
iter_seek_intent_ge k=j txn=A
iter_seek_intent_ge k=k txn=A
----
iter_seek_intent_ge: "b"/4.000000000,0=/<empty>
iter_seek_intent_ge: "d"/0,0=txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_intent_ge: "j"/0,0=txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_intent_ge: "j"/0,0=txn={id=00000001 key=/Min iso=Serializable pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_intent_ge: "k"/5.000000000,0=/BYTES/k5

run ok
iter_new types=rangesOnly
iter_seek_intent_ge k=b txn=A
iter_seek_intent_ge k=d txn=A
iter_seek_intent_ge k=i txn=A
iter_seek_intent_ge k=j txn=A
iter_seek_intent_ge k=k txn=A
----
iter_seek_intent_ge: {b-c}/[3.000000000,0=/<empty> 1.000000000,0=/<empty>] !
iter_seek_intent_ge: {d-f}/[5.000000000,0=/<empty> 1.000000000,0=/<empty>] !
iter_seek_intent_ge: {h-k}/[1.000000000,0=/<empty>] !
iter_seek_intent_ge: {h-k}/[1.000000000,0=/<empty>]
iter_seek_intent_ge: {m-n}/[3.000000000,0={localTs=2.000000000,0}/<empty>] !

# Try some masked scans at increasing timestamps.
run ok
iter_new types=pointsAndRanges maskBelow=1
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,27 +58,6 @@ iter_seek_lt: "b"/3.000000000,0=/BYTES/b3
iter_prev: "b"/0,0=txn={id=00000001 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_ge: {a-b}/[1.000000000,0=/<empty>] !

# Test the same for SeekIntentGE.
run ok
iter_new types=pointsAndRanges
iter_seek_lt k=b+
iter_prev
iter_seek_ge txn=A k=b
----
iter_seek_lt: "b"/3.000000000,0=/BYTES/b3
iter_prev: "b"/0,0=txn={id=00000001 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_ge: "b"/0,0=txn={id=00000001 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true

run ok
iter_new types=pointsAndRanges
iter_seek_lt k=b+
iter_prev
iter_seek_intent_ge txn=A k=a
----
iter_seek_lt: "b"/3.000000000,0=/BYTES/b3
iter_prev: "b"/0,0=txn={id=00000001 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_intent_ge: {a-b}/[1.000000000,0=/<empty>] !

# Test the same for SeekLT.
run ok
iter_new types=pointsAndRanges
Expand Down Expand Up @@ -113,18 +92,6 @@ iter_prev: "b"/0,0=txn={id=00000001 key="b" iso=Serializable pri=0.00000000 epo=
iter_prev: "a"/2.000000000,0=/BYTES/a2 {a-b}/[1.000000000,0=/<empty>] !
iter_seek_ge: "b"/0,0=txn={id=00000001 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true !

run ok
iter_new types=pointsAndRanges
iter_seek_lt k=b+
iter_prev
iter_prev
iter_seek_intent_ge txn=A k=b
----
iter_seek_lt: "b"/3.000000000,0=/BYTES/b3
iter_prev: "b"/0,0=txn={id=00000001 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_prev: "a"/2.000000000,0=/BYTES/a2 {a-b}/[1.000000000,0=/<empty>] !
iter_seek_intent_ge: "b"/0,0=txn={id=00000001 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true !

run ok
iter_new types=pointsAndRanges
iter_seek_lt k=b+
Expand Down Expand Up @@ -159,16 +126,6 @@ iter_seek_lt: "b"/3.000000000,0=/BYTES/b3
iter_prev: "b"/0,0=txn={id=00000001 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_ge: {c-d}/[1.000000000,0=/<empty>] !

run ok
iter_new types=pointsAndRanges
iter_seek_lt k=b+
iter_prev
iter_seek_intent_ge txn=A k=c
----
iter_seek_lt: "b"/3.000000000,0=/BYTES/b3
iter_prev: "b"/0,0=txn={id=00000001 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_intent_ge: {c-d}/[1.000000000,0=/<empty>] !

run ok
iter_new types=pointsAndRanges
iter_seek_lt k=b+
Expand Down Expand Up @@ -225,27 +182,6 @@ iter_seek_lt: "b"/3.000000000,0=/BYTES/b3
iter_prev: "b"/0,0=txn={id=00000002 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_ge: {a-b}/[2.000000000,0=/<empty> 1.000000000,0=/<empty>] !

# Test the same for SeekIntentGE.
run ok
iter_new types=pointsAndRanges
iter_seek_lt k=b+
iter_prev
iter_seek_ge txn=A k=b
----
iter_seek_lt: "b"/3.000000000,0=/BYTES/b3
iter_prev: "b"/0,0=txn={id=00000002 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_ge: "b"/0,0=txn={id=00000002 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true

run ok
iter_new types=pointsAndRanges
iter_seek_lt k=b+
iter_prev
iter_seek_intent_ge txn=A k=a
----
iter_seek_lt: "b"/3.000000000,0=/BYTES/b3
iter_prev: "b"/0,0=txn={id=00000002 key="b" iso=Serializable pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} ts=3.000000000,0 del=false klen=12 vlen=7 mergeTs=<nil> txnDidNotUpdateMeta=true
iter_seek_intent_ge: {a-b}/[2.000000000,0=/<empty> 1.000000000,0=/<empty>] !

# Test the same for SeekLT.
run ok
iter_new types=pointsAndRanges
Expand Down
12 changes: 1 addition & 11 deletions pkg/storage/verifying_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,11 +10,7 @@

package storage

import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/pebble"
)
import "github.com/cockroachdb/pebble"

// verifyingMVCCIterator is an MVCC iterator that wraps a pebbleIterator and
// verifies roachpb.Value checksums for encountered values.
Expand Down Expand Up @@ -85,12 +81,6 @@ func (i *verifyingMVCCIterator) SeekGE(key MVCCKey) {
i.saveAndVerify()
}

// SeekIntentGE implements MVCCIterator.
func (i *verifyingMVCCIterator) SeekIntentGE(key roachpb.Key, txnUUID uuid.UUID) {
i.pebbleIterator.SeekIntentGE(key, txnUUID)
i.saveAndVerify()
}

// SeekLT implements MVCCIterator.
func (i *verifyingMVCCIterator) SeekLT(key MVCCKey) {
i.pebbleIterator.SeekLT(key)
Expand Down

0 comments on commit c01eeee

Please sign in to comment.