From 72cd8ce87ed47c58a12a9ad857e92ba303a3dc4c Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 18 Sep 2023 23:55:54 -0400 Subject: [PATCH] kv: ignore {exclusive,shared} locks in QueryResolvedTimestamp 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 --- .../batcheval/cmd_query_resolved_timestamp.go | 34 +++++++++++-------- .../cmd_query_resolved_timestamp_test.go | 11 ++++++ 2 files changed, 30 insertions(+), 15 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go index fbc56d39b33d..b4229065767a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go @@ -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" ) @@ -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 } @@ -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() { @@ -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 diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go index 0e283009b963..5ef00b4203f8 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go @@ -56,18 +56,26 @@ 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) @@ -75,9 +83,12 @@ func TestQueryResolvedTimestamp(t *testing.T) { 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