From add67e39c0b0211c7a9326f67cce9c8ffe4592b0 Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Thu, 5 May 2022 14:36:41 -0400 Subject: [PATCH] storage: remove slow path for MVCCResolveWriteIntentRange We no longer have physically interleaved intents, that needed the slow path. Release note: None --- pkg/kv/kvserver/replica_evaluate_test.go | 18 ++- pkg/storage/bench_test.go | 4 +- pkg/storage/mvcc.go | 175 ++++++++--------------- 3 files changed, 75 insertions(+), 122 deletions(-) diff --git a/pkg/kv/kvserver/replica_evaluate_test.go b/pkg/kv/kvserver/replica_evaluate_test.go index dd94e385702f..17d07ee1d7d1 100644 --- a/pkg/kv/kvserver/replica_evaluate_test.go +++ b/pkg/kv/kvserver/replica_evaluate_test.go @@ -622,6 +622,22 @@ func TestEvaluateBatch(t *testing.T) { verifyResumeSpans(t, r, "", "", "") }, }, + { + // A batch limited to resolve only up to 2 keys should respect that + // limit. The limit is saturated by the first request in the batch. + name: "ranged intent resolution with MaxSpanRequestKeys=2", + setup: func(t *testing.T, d *data) { + writeABCDEFIntents(t, d, &txn) + d.ba.Add(resolveIntentRangeArgsString("a", "d", txn.TxnMeta, roachpb.COMMITTED)) + d.ba.Add(resolveIntentRangeArgsString("e", "f", txn.TxnMeta, roachpb.COMMITTED)) + d.ba.Add(resolveIntentRangeArgsString("h", "j", txn.TxnMeta, roachpb.COMMITTED)) + d.ba.MaxSpanRequestKeys = 2 + }, + check: func(t *testing.T, r resp) { + verifyNumKeys(t, r, 2, 0, 0) + verifyResumeSpans(t, r, "b\x00-d", "e-f", "h-j") + }, + }, { // A batch limited to resolve only up to 3 keys should respect that // limit. The limit is saturated by the first request in the batch. @@ -635,7 +651,7 @@ func TestEvaluateBatch(t *testing.T) { }, check: func(t *testing.T, r resp) { verifyNumKeys(t, r, 3, 0, 0) - verifyResumeSpans(t, r, "c\x00-d", "e-f", "h-j") + verifyResumeSpans(t, r, "", "e-f", "h-j") }, }, } diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index be4628e9c2a7..3cb03d2e64ca 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -494,8 +494,8 @@ func BenchmarkIntentResolution(b *testing.B) { } } -// BenchmarkIntentRangeResolution compares separated and interleaved intents, -// when doing ranged intent resolution. +// BenchmarkIntentRangeResolution benchmarks ranged intent resolution with +// various counts of mvcc versions and sparseness of intents. func BenchmarkIntentRangeResolution(b *testing.B) { skip.UnderShort(b, "setting up unflushed data takes too long") defer log.Scope(b).Close(b) diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 9f6f5bc3cabe..218a478578b0 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -3362,12 +3362,6 @@ type IterAndBuf struct { iter MVCCIterator } -// GetIterAndBuf returns an IterAndBuf for passing into various MVCC* methods -// that need to see intents. -func GetIterAndBuf(reader Reader, opts IterOptions) IterAndBuf { - return GetBufUsingIter(reader.NewMVCCIterator(MVCCKeyAndIntentsIterKind, opts)) -} - // GetBufUsingIter returns an IterAndBuf using the supplied iterator. func GetBufUsingIter(iter MVCCIterator) IterAndBuf { return IterAndBuf{ @@ -3397,134 +3391,77 @@ func MVCCResolveWriteIntentRange( resumeSpan := intent.Span // don't inline or `intent` would escape to heap return 0, &resumeSpan, nil } - - var putBuf *putBuffer - // Exactly one of sepIter and mvccIter is non-nil. sepIter is used when - // onlySeparatedIntents=true and rw provides consistent iterators, else - // mvccIter is initialized and used. The former allows for more efficient - // intent resolution. - var sepIter *separatedIntentAndVersionIter + ltStart, _ := keys.LockTableSingleKey(intent.Key, nil) + ltEnd, _ := keys.LockTableSingleKey(intent.EndKey, nil) + engineIter := rw.NewEngineIterator(IterOptions{LowerBound: ltStart, UpperBound: ltEnd}) var mvccIter MVCCIterator - // iter is set to either sepIter or mvccIter and used for individual intent - // resolution. - var iter iterForKeyVersions - - // We can find relevant intents quickly by - // iterating over the lock table. We additionally require - // ConsistentIterators() since we want the two iterators to be mutually - // consistent (and production code will have consistent iterators). - // - // TODO(sumeer): when removing the slow path, use - // newMVCCIteratorByCloningEngineIter for the inconsistent iterators case. + iterOpts := IterOptions{UpperBound: intent.EndKey} if rw.ConsistentIterators() { - ltStart, _ := keys.LockTableSingleKey(intent.Key, nil) - ltEnd, _ := keys.LockTableSingleKey(intent.EndKey, nil) - engineIter := rw.NewEngineIterator(IterOptions{LowerBound: ltStart, UpperBound: ltEnd}) - iterAndBuf := - GetBufUsingIter(rw.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: intent.EndKey})) - defer func() { - engineIter.Close() - iterAndBuf.Cleanup() - }() - putBuf = iterAndBuf.buf - sepIter = &separatedIntentAndVersionIter{ - engineIter: engineIter, - mvccIter: iterAndBuf.iter, - } - iter = sepIter - // Seek sepIter to position it for the loop below. The loop itself will - // only step the iterator and not seek. - sepIter.seekEngineKeyGE(EngineKey{Key: ltStart}) + // Production code should always have consistent iterators. + mvccIter = rw.NewMVCCIterator(MVCCKeyIterKind, iterOpts) } else { - iterAndBuf := GetIterAndBuf(rw, IterOptions{UpperBound: intent.EndKey}) - defer iterAndBuf.Cleanup() - putBuf = iterAndBuf.buf - mvccIter = iterAndBuf.iter - iter = mvccIter - } - nextKey := MakeMVCCMetadataKey(intent.Key) + // For correctness, we need mvccIter to be consistent with engineIter. + mvccIter = newMVCCIteratorByCloningEngineIter(engineIter, iterOpts) + } + iterAndBuf := GetBufUsingIter(mvccIter) + defer func() { + engineIter.Close() + iterAndBuf.Cleanup() + }() + putBuf := iterAndBuf.buf + sepIter := &separatedIntentAndVersionIter{ + engineIter: engineIter, + mvccIter: iterAndBuf.iter, + } + // Seek sepIter to position it for the loop below. The loop itself will + // only step the iterator and not seek. + sepIter.seekEngineKeyGE(EngineKey{Key: ltStart}) + intentEndKey := intent.EndKey intent.EndKey = nil - var keyBuf []byte + var lastResolvedKey roachpb.Key num := int64(0) for { + if valid, err := sepIter.Valid(); err != nil { + return 0, nil, err + } else if !valid { + // No more intents in the given range. + break + } if max > 0 && num == max { - return num, &roachpb.Span{Key: nextKey.Key, EndKey: intentEndKey}, nil - } - var key MVCCKey - if sepIter != nil { - // sepIter is already positioned since it is seeked prior to the loop - // and then stepped at the end of each iteration, to prepare for the - // next iteration. - if valid, err := sepIter.Valid(); err != nil { - return 0, nil, err - } else if !valid { - // No more intents in the given range. - break - } - // Parse the MVCCMetadata to see if it is a relevant intent. - meta := &putBuf.meta - if err := sepIter.ValueProto(meta); err != nil { - return 0, nil, err - } - if meta.Txn == nil { - return 0, nil, errors.Errorf("intent with no txn") - } - if intent.Txn.ID == meta.Txn.ID { - // Stash the parsed meta so don't need to parse it again in - // mvccResolveWriteIntent. This parsing can be ~10% of the - // resolution cost in some benchmarks. - sepIter.meta = meta - // Manually copy the underlying bytes of the unsafe key. This - // construction reuses keyBuf across iterations. - key = sepIter.UnsafeKey() - keyBuf = append(keyBuf[:0], key.Key...) - key.Key = keyBuf - } else { - sepIter.nextEngineKey() - continue - } - } else { - // mvccIter needs to be positioned at the start of each iteration. - mvccIter.SeekGE(nextKey) - if valid, err := mvccIter.Valid(); err != nil { - return 0, nil, err - } else if !valid { - // No more keys exists in the given range. - break - } - key = mvccIter.UnsafeKey() - // Manually copy the underlying bytes of the unsafe key. This - // construction reuses keyBuf across iterations. - keyBuf = append(keyBuf[:0], key.Key...) - key.Key = keyBuf + // We could also compute a tighter nextKey here if we wanted to. + return num, &roachpb.Span{Key: lastResolvedKey.Next(), EndKey: intentEndKey}, nil } - - var err error - var ok bool - if !key.IsValue() { - // NB: This if-condition is always true for the sepIter != nil path. - intent.Key = key.Key - ok, err = mvccResolveWriteIntent(ctx, rw, iter, ms, intent, putBuf) + // Parse the MVCCMetadata to see if it is a relevant intent. + meta := &putBuf.meta + if err := sepIter.ValueProto(meta); err != nil { + return 0, nil, err } - if err != nil { - log.Warningf(ctx, "failed to resolve intent for key %q: %+v", key.Key, err) - } else if ok { - num++ + if meta.Txn == nil { + return 0, nil, errors.Errorf("intent with no txn") } - - if sepIter != nil { + if intent.Txn.ID != meta.Txn.ID { + // Intent for a different txn, so ignore. sepIter.nextEngineKey() - // We could also compute a tighter nextKey here if we wanted to. + continue } - // nextKey is already a metadata key... - nextKey.Key = key.Key.Next() - if nextKey.Key.Compare(intentEndKey) >= 0 { - // ... but we don't want to Seek to a key outside of the range as we validate - // those span accesses (see TestSpanSetMVCCResolveWriteIntentRangeUsingIter). - break + // Stash the parsed meta so don't need to parse it again in + // mvccResolveWriteIntent. This parsing can be ~10% of the resolution cost + // in some benchmarks. + sepIter.meta = meta + // Copy the underlying bytes of the unsafe key. This is needed for + // stability of the key passed to mvccResolveWriteIntent, and for the + // subsequent iteration to construct a resume span. + lastResolvedKey = append(lastResolvedKey[:0], sepIter.UnsafeKey().Key...) + intent.Key = lastResolvedKey + ok, err := mvccResolveWriteIntent(ctx, rw, sepIter, ms, intent, putBuf) + if err != nil { + log.Warningf(ctx, "failed to resolve intent for key %q: %+v", lastResolvedKey, err) + } else if ok { + num++ } + sepIter.nextEngineKey() } return num, nil, nil }