Skip to content

Commit

Permalink
Merge #110324
Browse files Browse the repository at this point in the history
110324: storage: implement intent resolution using LockTableIterator r=nvanbenschoten a=nvanbenschoten

Informs #109648.

This commit implements intent resolution (point and ranged) using a `LockTableIterator`, configured to return all locks for the transaction being resolved and no locks from other transactions. This is the first step towards releasing replicated locks during intent resolution.

While switching to a LockTableIterator, the commit is also able to remove separatedIntentAndVersionIter, iterForKeyVersions and mvccGetIntent, which were all used to avoid handing an MVCCMetadata directly to mvccResolveWriteIntent. Instead of continuing to treat intents as interleaved, we switch to handling intents entirely separately from their provisional value during intent resolution, which avoids jumping through these hoops and makes the code simpler.

The change to `TestMVCCResolveTxnRangeResumeWithManyVersions` is immaterial and has to do with the transaction ID filter being applied before the key limit (inside LockTableIterator), instead of after. The new behavior is actually better.

----

One concern I have about this change is that it removes the call to `SeekIntentGE` in `MVCCResolveWriteIntent`, which was added in d1c91e0 to guard against the case where many pebble tombstones from prior intents from different txns on a key surround the intent being resolved. Conceptually, we'd like to push optimizations that avoid scanning over these tombstones into the `LockTableIterator` like we plan to do for skipping over non-conflicting locks. Doing so would benefit all lock strengths. It would also benefit the case where an intent is not found and the seek hits tombstones from prior intents on later versions.

However, it's not clear how to do this with the current Pebble API. Pebble exposes a `SeekGEWithLimit` method, but this "limit" value is expressed as a key and not as a number of steps. How would we construct a limit key to bound the number of tombstones a seek observes before seeking directly to a specific (txn_id, lock_strength) version?

One option would be to seek to specific versions in the `LockTableIterator` when advancing the iterator in cases where the iterator is configured to match a specific txn ID. For example, performing the following translations:
```
SeekGE({Key: k}) -> SeekGE({Key: k, Strength: Intent, TxnID: <txn_id>})
Next()           -> SeekGE({Key: k, Strength: Exclusive, TxnID: <txn_id>})
Next()           -> SeekGE({Key: k, Strength: Shared, TxnID: <txn_id>})
```
Of course, this gets more complicated when some of these locks are not found and the iterator advances past them while seeking. In such cases, we're back to paying the cost of scanning over the tombstones.

If we knew which lock strengths we had acquired on a key, we could avoid some of this cost, but that would require API changes and client buy-in to track lock spans on a per-strength basis.

I'll capture the impact of this change on the following benchmarks and evaluate:
* BenchmarkIntentResolution
* BenchmarkIntentRangeResolution
* BenchmarkIntentScan

Release note: None

Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
craig[bot] and nvanbenschoten committed Sep 23, 2023
2 parents b1a641f + 4e46443 commit 43326b2
Show file tree
Hide file tree
Showing 14 changed files with 138 additions and 460 deletions.
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batch_spanset_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -581,7 +582,7 @@ func TestSpanSetMVCCResolveWriteIntentRange(t *testing.T) {
defer batch.Close()
intent := roachpb.LockUpdate{
Span: roachpb.Span{Key: roachpb.Key("a"), EndKey: roachpb.Key("b\x00")},
Txn: enginepb.TxnMeta{}, // unused
Txn: enginepb.TxnMeta{ID: uuid.MakeV4()}, // unused
Status: roachpb.PENDING,
}
if _, _, _, _, err := storage.MVCCResolveWriteIntentRange(
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4722,7 +4722,7 @@ func TestEndTxnRollbackAbortedTransaction(t *testing.T) {

if pErr := tc.store.intentResolver.ResolveIntents(ctx,
[]roachpb.LockUpdate{
roachpb.MakeLockUpdate(&txnRecord, roachpb.Span{Key: key}),
roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}),
}, intentresolver.ResolveOptions{Poison: true}); pErr != nil {
t.Fatal(pErr)
}
Expand Down
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
Loading

0 comments on commit 43326b2

Please sign in to comment.