Skip to content

Commit

Permalink
storage: ignore {exclusive,shared} locks in ScanConflictingIntentsFor…
Browse files Browse the repository at this point in the history
…DroppingLatchesEarly

ScanConflictingIntentsForDroppingLatchesEarly is called by non-locking
read requests to check for conflicts before evaluating. Non-locking
reads do not conflict with Exclusive or Shared locks -- so these can
be ignored when scanning the lock table. This patch does so, by making
use of an appropriately configured `LockTableIterator`.

Informs #100193

Release note: None
  • Loading branch information
arulajmani committed Sep 18, 2023
1 parent e25067a commit 0c9bbb4
Show file tree
Hide file tree
Showing 2 changed files with 98 additions and 11 deletions.
31 changes: 22 additions & 9 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -1971,7 +1972,9 @@ func assertMVCCIteratorInvariants(iter MVCCIterator) error {
// latches early. If found, conflicting intents are added to the supplied
// `intents` slice, which indicates to the caller that evaluation should not
// proceed until the intents are resolved. Intents that don't conflict with the
// transaction referenced by txnID[1] at the supplied `ts` are ignored.
// transaction referenced by txnID[1] at the supplied `ts` are ignored; so are
// {Shared,Exclusive} replicated locks, as they do not conflict with non-locking
// reads.
//
// The `needsIntentHistory` return value indicates whether the caller needs to
// consult intent history when performing a scan over the MVCC keyspace to
Expand All @@ -1998,14 +2001,24 @@ func ScanConflictingIntentsForDroppingLatchesEarly(
return true, errors.AssertionFailedf("start key must be less than end key")
}
ltStart, _ := keys.LockTableSingleKey(start, nil)
opts := IterOptions{LowerBound: ltStart}
opts := LockTableIteratorOptions{
LowerBound: ltStart,
// Ignore Exclusive and Shared locks; we only drop latches early for
// non-locking reads, which do not conflict with Shared or
// Exclusive[1] locks.
//
// [1] Specifically replicated Exclusive locks. Interaction with
// unreplicated locks is governed by the ExclusiveLocksBlockNonLockingReads
// cluster setting.
MatchMinStr: lock.Intent,
}
if upperBoundUnset {
opts.Prefix = true
} else {
ltEnd, _ := keys.LockTableSingleKey(end, nil)
opts.UpperBound = ltEnd
}
iter, err := reader.NewEngineIterator(opts)
iter, err := NewLockTableIterator(reader, opts)
if err != nil {
return false, err
}
Expand All @@ -2019,13 +2032,10 @@ func ScanConflictingIntentsForDroppingLatchesEarly(
// not needing intent history.
return true /* needsIntentHistory */, nil
}
v, err := iter.UnsafeValue()
err := iter.ValueProto(&meta)
if err != nil {
return false, err
}
if err = protoutil.Unmarshal(v, &meta); err != nil {
return false, err
}
if meta.Txn == nil {
return false, errors.Errorf("intent without transaction")
}
Expand Down Expand Up @@ -2070,11 +2080,14 @@ func ScanConflictingIntentsForDroppingLatchesEarly(
if err != nil {
return false, err
}
lockedKey, err := keys.DecodeLockTableSingleKey(key.Key)
ltKey, err := key.ToLockTableKey()
if err != nil {
return false, err
}
*intents = append(*intents, roachpb.MakeIntent(meta.Txn, lockedKey))
if ltKey.Strength != lock.Intent {
return false, errors.AssertionFailedf("unexpected strength for LockTableKey %s", ltKey.Strength)
}
*intents = append(*intents, roachpb.MakeIntent(meta.Txn, ltKey.Key))
}
if err != nil {
return false, err
Expand Down
78 changes: 76 additions & 2 deletions pkg/storage/engine_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
Expand Down Expand Up @@ -2086,6 +2087,7 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) {
keyA := roachpb.Key("a")
keyB := roachpb.Key("b")
keyC := roachpb.Key("c")
keyD := roachpb.Key("d")
val := roachpb.Value{RawBytes: []byte{'v'}}

testCases := []struct {
Expand Down Expand Up @@ -2120,7 +2122,7 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) {
},
{
name: "conflicting txn intent at lower timestamp",
setup: func(t *testing.T, rw ReadWriter, txn *roachpb.Transaction) {
setup: func(t *testing.T, rw ReadWriter, _ *roachpb.Transaction) {
conflictingTxn := newTxn(belowTxnTS) // test txn should see this intent
err := MVCCPut(
ctx, rw, keyA, conflictingTxn.WriteTimestamp, val, MVCCWriteOptions{Txn: conflictingTxn},
Expand All @@ -2134,7 +2136,7 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) {
},
{
name: "conflicting txn intent at higher timestamp",
setup: func(t *testing.T, rw ReadWriter, txn *roachpb.Transaction) {
setup: func(t *testing.T, rw ReadWriter, _ *roachpb.Transaction) {
conflictingTxn := newTxn(aboveTxnTS) // test txn shouldn't see this intent
err := MVCCPut(
ctx, rw, keyA, conflictingTxn.WriteTimestamp, val, MVCCWriteOptions{Txn: conflictingTxn},
Expand All @@ -2157,6 +2159,78 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) {
expNeedsIntentHistory: false,
expNumFoundIntents: 0,
},
{
name: "shared and exclusive locks should be ignored",
setup: func(t *testing.T, rw ReadWriter, _ *roachpb.Transaction) {
txnA := newTxn(belowTxnTS)
txnB := newTxn(belowTxnTS)
err := MVCCAcquireLock(ctx, rw, txnA, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnB, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnA, lock.Exclusive, keyB, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
},
start: keyA,
end: keyC,
expNeedsIntentHistory: false,
expNumFoundIntents: 0,
},
{
// Same thing as above, but no end key this time. This ends up using a
// prefix iterator.
name: "shared and exclusive locks should be ignored no end key",
setup: func(t *testing.T, rw ReadWriter, _ *roachpb.Transaction) {
txnA := newTxn(belowTxnTS)
err := MVCCAcquireLock(ctx, rw, txnA, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnA, lock.Exclusive, keyA, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
},
start: keyA,
end: nil,
expNeedsIntentHistory: false,
expNumFoundIntents: 0,
},
{
name: "{exclusive, shared} locks and intents",
setup: func(t *testing.T, rw ReadWriter, _ *roachpb.Transaction) {
txnA := newTxn(belowTxnTS)
txnB := newTxn(belowTxnTS)
err := MVCCAcquireLock(ctx, rw, txnA, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnB, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnA, lock.Exclusive, keyB, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
require.NoError(t, err)
err = MVCCPut(ctx, rw, keyC, txnA.WriteTimestamp, val, MVCCWriteOptions{Txn: txnA})
require.NoError(t, err)
},
start: keyA,
end: keyD,
expNeedsIntentHistory: false,
expNumFoundIntents: 1,
},
{
name: "{exclusive, shared} locks and own intents",
setup: func(t *testing.T, rw ReadWriter, txn *roachpb.Transaction) {
txnA := newTxn(belowTxnTS)
txnB := newTxn(belowTxnTS)
err := MVCCAcquireLock(ctx, rw, txnA, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnB, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnA, lock.Exclusive, keyB, nil /*ms*/, 0 /*maxConflicts*/)
require.NoError(t, err)
err = MVCCPut(ctx, rw, keyC, txn.WriteTimestamp, val, MVCCWriteOptions{Txn: txn})
require.NoError(t, err)
},
start: keyA,
end: keyD,
expNeedsIntentHistory: true,
expNumFoundIntents: 0,
},
}

for _, tc := range testCases {
Expand Down

0 comments on commit 0c9bbb4

Please sign in to comment.