Skip to content

Commit

Permalink
kv: ignore {exclusive,shared} locks in QueryResolvedTimestamp
Browse files Browse the repository at this point in the history
Informs #100193.

The resolved timestamp is a function of the intents in a range, but not of
the other locks in a range. This commit updates the QueryResolvedTimestamp
evaluation method to use a LockTableIterator configured to ignore Exclusive
or Shared locks.

Release note: None
  • Loading branch information
nvanbenschoten committed Sep 19, 2023
1 parent baeffe5 commit 72cd8ce
Show file tree
Hide file tree
Showing 2 changed files with 30 additions and 15 deletions.
34 changes: 19 additions & 15 deletions pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,13 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -104,7 +104,13 @@ func computeMinIntentTimestamp(
) (hlc.Timestamp, []roachpb.Intent, error) {
ltStart, _ := keys.LockTableSingleKey(span.Key, nil)
ltEnd, _ := keys.LockTableSingleKey(span.EndKey, nil)
iter, err := reader.NewEngineIterator(storage.IterOptions{LowerBound: ltStart, UpperBound: ltEnd})
opts := storage.LockTableIteratorOptions{
LowerBound: ltStart,
UpperBound: ltEnd,
// Ignore Exclusive and Shared locks. We only care about intents.
MatchMinStr: lock.Intent,
}
iter, err := storage.NewLockTableIterator(reader, opts)
if err != nil {
return hlc.Timestamp{}, nil, err
}
Expand All @@ -124,22 +130,20 @@ func computeMinIntentTimestamp(
if err != nil {
return hlc.Timestamp{}, nil, err
}
lockedKey, err := keys.DecodeLockTableSingleKey(engineKey.Key)
ltKey, err := engineKey.ToLockTableKey()
if err != nil {
return hlc.Timestamp{}, nil, errors.Wrapf(err, "decoding LockTable key: %v", lockedKey)
return hlc.Timestamp{}, nil, errors.Wrapf(err, "decoding LockTable key: %v", ltKey)
}
// Unmarshal.
v, err := iter.UnsafeValue()
if err != nil {
return hlc.Timestamp{}, nil, err
if ltKey.Strength != lock.Intent {
return hlc.Timestamp{}, nil, errors.AssertionFailedf(
"unexpected strength for LockTableKey %s: %v", ltKey.Strength, ltKey)
}
if err := protoutil.Unmarshal(v, &meta); err != nil {
return hlc.Timestamp{}, nil, errors.Wrapf(err, "unmarshaling mvcc meta: %v", lockedKey)
// Unmarshal.
if err := iter.ValueProto(&meta); err != nil {
return hlc.Timestamp{}, nil, errors.Wrapf(err, "unmarshaling mvcc meta: %v", ltKey)
}
if meta.Txn == nil {
return hlc.Timestamp{}, nil,
errors.AssertionFailedf("nil transaction in LockTable. Key: %v,"+"mvcc meta: %v",
lockedKey, meta)
return hlc.Timestamp{}, nil, errors.AssertionFailedf("nil transaction in LockTable: %v", ltKey)
}

if minTS.IsEmpty() {
Expand All @@ -155,8 +159,8 @@ func computeMinIntentTimestamp(
intentFitsByCount := int64(len(encountered)) < maxEncounteredIntents
intentFitsByBytes := encounteredKeyBytes < maxEncounteredIntentKeyBytes
if oldEnough && intentFitsByCount && intentFitsByBytes {
encountered = append(encountered, roachpb.MakeIntent(meta.Txn, lockedKey))
encounteredKeyBytes += int64(len(lockedKey))
encountered = append(encountered, roachpb.MakeIntent(meta.Txn, ltKey.Key))
encounteredKeyBytes += int64(len(ltKey.Key))
}
}
return minTS, encountered, nil
Expand Down
11 changes: 11 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,28 +56,39 @@ func TestQueryResolvedTimestamp(t *testing.T) {
_, err := storage.MVCCDelete(ctx, db, roachpb.Key(k), hlc.Timestamp{}, storage.MVCCWriteOptions{})
require.NoError(t, err)
}
writeLock := func(k string, str lock.Strength) {
txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, 0, makeTS(1), 0, 1, 0)
err := storage.MVCCAcquireLock(ctx, db, &txn, str, roachpb.Key(k), nil, 0)
require.NoError(t, err)
}

// Setup: (with separated intents the actual key layout in the store is not what is listed below.)
//
// a: intent @ 5
// a: value @ 3
// b: inline value
// c: shared lock #1
// c: shared lock #2
// c: value @ 6
// c: value @ 4
// c: value @ 1
// d: intent @ 2
// e: intent @ 7
// f: inline value
// g: exclusive lock
//
// NB: must write each key in increasing timestamp order.
writeValue("a", 3)
writeIntent("a", 5)
writeInline("b")
writeValue("c", 1)
writeValue("c", 4)
writeLock("c", lock.Shared)
writeLock("c", lock.Shared)
writeIntent("d", 2)
writeIntent("e", 7)
writeInline("f")
writeLock("g", lock.Exclusive)

for _, cfg := range []struct {
name string
Expand Down

0 comments on commit 72cd8ce

Please sign in to comment.