Skip to content

Commit

Permalink
storage: implement intent resolution using LockTableIterator
Browse files Browse the repository at this point in the history
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 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: Nonet
  • Loading branch information
nvanbenschoten committed Sep 21, 2023
1 parent b4bd6d7 commit 180c61b
Show file tree
Hide file tree
Showing 4 changed files with 137 additions and 245 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
Loading

0 comments on commit 180c61b

Please sign in to comment.