From 1eccba3cd202ff7e656ef925b2ae0c68bc1cd15d Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Thu, 15 Nov 2018 11:16:05 -0500 Subject: [PATCH] storage/engine: extract an MVCCScanOptions struct The signatures of the functions in the MVCCScan family are extremely unwieldy. Extract an MVCCScanOptions struct for the fields that have reasonable default values. We can immediately realize the benefits of this change by removing MVCCReverseScan, MVCCReverseScanWithBytes, and MVCCIterateUsingIter, as the functionality provided by those functions can be requested with the appropriate option in the MVCCScanOptions struct. This paves the way towards introducing additional parameters, like minimum and maximum timestamp bounds. Updates #28358. Release note: None --- pkg/cli/debug.go | 6 +- pkg/kv/split_test.go | 4 +- pkg/server/node_test.go | 2 +- pkg/storage/abortspan/abortspan.go | 3 +- pkg/storage/addressing_test.go | 2 +- pkg/storage/batcheval/cmd_refresh_range.go | 42 +++---- pkg/storage/batcheval/cmd_reverse_scan.go | 18 ++- pkg/storage/batcheval/cmd_scan.go | 14 ++- pkg/storage/engine/bench_test.go | 8 +- pkg/storage/engine/engine.go | 12 +- pkg/storage/engine/mvcc.go | 105 +++++------------- pkg/storage/engine/mvcc_test.go | 90 +++++++-------- pkg/storage/engine/rocksdb.go | 24 ++-- pkg/storage/engine/rocksdb_iter_stats_test.go | 2 +- pkg/storage/gc_queue.go | 5 +- pkg/storage/gc_queue_test.go | 2 +- pkg/storage/replica_raftstorage.go | 5 +- pkg/storage/spanset/batch.go | 8 +- pkg/storage/store.go | 4 +- pkg/ts/db_test.go | 2 +- 20 files changed, 143 insertions(+), 215 deletions(-) diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index c637a0e9f354..70f9c34a1320 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -505,8 +505,7 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error { var descs []roachpb.RangeDescriptor if _, err := engine.MVCCIterate(context.Background(), db, start, end, hlc.MaxTimestamp, - false /* consistent */, false /* tombstones */, nil, /* txn */ - false /* reverse */, func(kv roachpb.KeyValue) (bool, error) { + engine.MVCCScanOptions{Inconsistent: true}, func(kv roachpb.KeyValue) (bool, error) { var desc roachpb.RangeDescriptor _, suffix, _, err := keys.DecodeRangeKey(kv.Key) if err != nil { @@ -652,8 +651,7 @@ func runDebugCheckStoreRaft(ctx context.Context, db *engine.RocksDB) error { var hasError bool if _, err := engine.MVCCIterate(ctx, db, start, end, hlc.MaxTimestamp, - false /* consistent */, false /* tombstones */, nil, /* txn */ - false /* reverse */, func(kv roachpb.KeyValue) (bool, error) { + engine.MVCCScanOptions{Inconsistent: true}, func(kv roachpb.KeyValue) (bool, error) { rangeID, _, suffix, detail, err := keys.DecodeRangeIDKey(kv.Key) if err != nil { return false, err diff --git a/pkg/kv/split_test.go b/pkg/kv/split_test.go index 9ec6a80ce25b..ccda392ea1f7 100644 --- a/pkg/kv/split_test.go +++ b/pkg/kv/split_test.go @@ -121,7 +121,7 @@ func TestRangeSplitMeta(t *testing.T) { } testutils.SucceedsSoon(t, func() error { - if _, _, _, err := engine.MVCCScan(ctx, s.Eng, keys.LocalMax, roachpb.KeyMax, math.MaxInt64, hlc.MaxTimestamp, true, nil); err != nil { + if _, _, _, err := engine.MVCCScan(ctx, s.Eng, keys.LocalMax, roachpb.KeyMax, math.MaxInt64, hlc.MaxTimestamp, engine.MVCCScanOptions{}); err != nil { return errors.Errorf("failed to verify no dangling intents: %s", err) } return nil @@ -229,7 +229,7 @@ func TestRangeSplitsWithWritePressure(t *testing.T) { // for timing of finishing the test writer and a possibly-ongoing // asynchronous split. testutils.SucceedsSoon(t, func() error { - if _, _, _, err := engine.MVCCScan(ctx, s.Eng, keys.LocalMax, roachpb.KeyMax, math.MaxInt64, hlc.MaxTimestamp, true, nil); err != nil { + if _, _, _, err := engine.MVCCScan(ctx, s.Eng, keys.LocalMax, roachpb.KeyMax, math.MaxInt64, hlc.MaxTimestamp, engine.MVCCScanOptions{}); err != nil { return errors.Errorf("failed to verify no dangling intents: %s", err) } return nil diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index 9ce658376f9a..19f914365d7d 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -211,7 +211,7 @@ func TestBootstrapCluster(t *testing.T) { } // Scan the complete contents of the local database directly from the engine. - rows, _, _, err := engine.MVCCScan(context.Background(), e, keys.LocalMax, roachpb.KeyMax, math.MaxInt64, hlc.MaxTimestamp, true, nil) + rows, _, _, err := engine.MVCCScan(context.Background(), e, keys.LocalMax, roachpb.KeyMax, math.MaxInt64, hlc.MaxTimestamp, engine.MVCCScanOptions{}) if err != nil { t.Fatal(err) } diff --git a/pkg/storage/abortspan/abortspan.go b/pkg/storage/abortspan/abortspan.go index 3c0557a6cb1b..f654fbc52161 100644 --- a/pkg/storage/abortspan/abortspan.go +++ b/pkg/storage/abortspan/abortspan.go @@ -109,8 +109,7 @@ func (sc *AbortSpan) Get( func (sc *AbortSpan) Iterate( ctx context.Context, e engine.Reader, f func(roachpb.Key, roachpb.AbortSpanEntry) error, ) error { - _, err := engine.MVCCIterate(ctx, e, sc.min(), sc.max(), hlc.Timestamp{}, - true /* consistent */, false /* tombstones */, nil /* txn */, false, /* reverse */ + _, err := engine.MVCCIterate(ctx, e, sc.min(), sc.max(), hlc.Timestamp{}, engine.MVCCScanOptions{}, func(kv roachpb.KeyValue) (bool, error) { var entry roachpb.AbortSpanEntry if _, err := keys.DecodeAbortSpanKey(kv.Key, nil); err != nil { diff --git a/pkg/storage/addressing_test.go b/pkg/storage/addressing_test.go index da5ffd227eb1..ced155351bb0 100644 --- a/pkg/storage/addressing_test.go +++ b/pkg/storage/addressing_test.go @@ -162,7 +162,7 @@ func TestUpdateRangeAddressing(t *testing.T) { t.Fatal(err) } // Scan meta keys directly from engine. - kvs, _, _, err := engine.MVCCScan(ctx, store.Engine(), keys.MetaMin, keys.MetaMax, math.MaxInt64, hlc.MaxTimestamp, true, nil) + kvs, _, _, err := engine.MVCCScan(ctx, store.Engine(), keys.MetaMin, keys.MetaMax, math.MaxInt64, hlc.MaxTimestamp, engine.MVCCScanOptions{}) if err != nil { t.Fatal(err) } diff --git a/pkg/storage/batcheval/cmd_refresh_range.go b/pkg/storage/batcheval/cmd_refresh_range.go index 7960e2e0a7a8..95b70f34d7fc 100644 --- a/pkg/storage/batcheval/cmd_refresh_range.go +++ b/pkg/storage/batcheval/cmd_refresh_range.go @@ -42,35 +42,21 @@ func RefreshRange( return result.Result{}, errors.Errorf("no transaction specified to %s", args.Method()) } - iter := batch.NewIterator(engine.IterOptions{ - UpperBound: args.EndKey, - }) - defer iter.Close() - // Iterate over values until we discover any value written at or - // after the original timestamp, but before or at the current - // timestamp. Note that we do not iterate using the txn and the - // iteration is done with consistent=false. This reads only - // committed values and returns all intents, including those from - // the txn itself. Note that we include tombstones, which must be - // considered as updates on refresh. + // Iterate over values until we discover any value written at or after the + // original timestamp, but before or at the current timestamp. Note that we + // iterate inconsistently without using the txn. This reads only committed + // values and returns all intents, including those from the txn itself. Note + // that we include tombstones, which must be considered as updates on refresh. log.VEventf(ctx, 2, "refresh %s @[%s-%s]", args.Span(), h.Txn.OrigTimestamp, h.Txn.Timestamp) - intents, err := engine.MVCCIterateUsingIter( - ctx, - iter, - args.Key, - args.EndKey, - h.Txn.Timestamp, - false, /* consistent */ - true, /* tombstones */ - nil, /* txn */ - false, /* reverse */ - func(kv roachpb.KeyValue) (bool, error) { - if ts := kv.Value.Timestamp; !ts.Less(h.Txn.OrigTimestamp) { - return true, errors.Errorf("encountered recently written key %s @%s", kv.Key, ts) - } - return false, nil - }, - ) + intents, err := engine.MVCCIterate(ctx, batch, args.Key, args.EndKey, h.Txn.Timestamp, engine.MVCCScanOptions{ + Inconsistent: true, + Tombstones: true, + }, func(kv roachpb.KeyValue) (bool, error) { + if ts := kv.Value.Timestamp; !ts.Less(h.Txn.OrigTimestamp) { + return true, errors.Errorf("encountered recently written key %s @%s", kv.Key, ts) + } + return false, nil + }) if err != nil { return result.Result{}, err } diff --git a/pkg/storage/batcheval/cmd_reverse_scan.go b/pkg/storage/batcheval/cmd_reverse_scan.go index 6edf696073d6..1a313a99a2ff 100644 --- a/pkg/storage/batcheval/cmd_reverse_scan.go +++ b/pkg/storage/batcheval/cmd_reverse_scan.go @@ -47,9 +47,13 @@ func ReverseScan( case roachpb.BATCH_RESPONSE: var kvData []byte var numKvs int64 - kvData, numKvs, resumeSpan, intents, err = engine.MVCCReverseScanToBytes( - ctx, batch, args.Key, args.EndKey, cArgs.MaxKeys, - h.Timestamp, h.ReadConsistency == roachpb.CONSISTENT, h.Txn) + kvData, numKvs, resumeSpan, intents, err = engine.MVCCScanToBytes( + ctx, batch, args.Key, args.EndKey, cArgs.MaxKeys, h.Timestamp, + engine.MVCCScanOptions{ + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Txn: h.Txn, + Reverse: true, + }) if err != nil { return result.Result{}, err } @@ -57,8 +61,12 @@ func ReverseScan( reply.BatchResponse = kvData case roachpb.KEY_VALUES: var rows []roachpb.KeyValue - rows, resumeSpan, intents, err = engine.MVCCReverseScan(ctx, batch, args.Key, args.EndKey, - cArgs.MaxKeys, h.Timestamp, h.ReadConsistency == roachpb.CONSISTENT, h.Txn) + rows, resumeSpan, intents, err = engine.MVCCScan( + ctx, batch, args.Key, args.EndKey, cArgs.MaxKeys, h.Timestamp, engine.MVCCScanOptions{ + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Txn: h.Txn, + Reverse: true, + }) if err != nil { return result.Result{}, err } diff --git a/pkg/storage/batcheval/cmd_scan.go b/pkg/storage/batcheval/cmd_scan.go index a373bb164471..7a181ea1886a 100644 --- a/pkg/storage/batcheval/cmd_scan.go +++ b/pkg/storage/batcheval/cmd_scan.go @@ -48,8 +48,11 @@ func Scan( var kvData []byte var numKvs int64 kvData, numKvs, resumeSpan, intents, err = engine.MVCCScanToBytes( - ctx, batch, args.Key, args.EndKey, cArgs.MaxKeys, - h.Timestamp, h.ReadConsistency == roachpb.CONSISTENT, h.Txn) + ctx, batch, args.Key, args.EndKey, cArgs.MaxKeys, h.Timestamp, + engine.MVCCScanOptions{ + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Txn: h.Txn, + }) if err != nil { return result.Result{}, err } @@ -57,8 +60,11 @@ func Scan( reply.BatchResponse = kvData case roachpb.KEY_VALUES: var rows []roachpb.KeyValue - rows, resumeSpan, intents, err = engine.MVCCScan(ctx, batch, args.Key, args.EndKey, - cArgs.MaxKeys, h.Timestamp, h.ReadConsistency == roachpb.CONSISTENT, h.Txn) + rows, resumeSpan, intents, err = engine.MVCCScan( + ctx, batch, args.Key, args.EndKey, cArgs.MaxKeys, h.Timestamp, engine.MVCCScanOptions{ + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Txn: h.Txn, + }) if err != nil { return result.Result{}, err } diff --git a/pkg/storage/engine/bench_test.go b/pkg/storage/engine/bench_test.go index 5054cd456a02..ee848e098b50 100644 --- a/pkg/storage/engine/bench_test.go +++ b/pkg/storage/engine/bench_test.go @@ -160,10 +160,6 @@ func runMVCCScan(emk engineMaker, numRows, numVersions, valueSize int, reverse b iter.Close() } - scan := MVCCScan - if reverse { - scan = MVCCReverseScan - } b.SetBytes(int64(numRows * valueSize)) b.ResetTimer() @@ -177,7 +173,9 @@ func runMVCCScan(emk engineMaker, numRows, numVersions, valueSize int, reverse b endKey = endKey.Next() walltime := int64(5 * (rand.Int31n(int32(numVersions)) + 1)) ts := hlc.Timestamp{WallTime: walltime} - kvs, _, _, err := scan(context.Background(), eng, startKey, endKey, int64(numRows), ts, true, nil) + kvs, _, _, err := MVCCScan(context.Background(), eng, startKey, endKey, int64(numRows), ts, MVCCScanOptions{ + Reverse: reverse, + }) if err != nil { b.Fatalf("failed scan: %s", err) } diff --git a/pkg/storage/engine/engine.go b/pkg/storage/engine/engine.go index 9af43c701e30..59e837b482ed 100644 --- a/pkg/storage/engine/engine.go +++ b/pkg/storage/engine/engine.go @@ -112,16 +112,14 @@ type Iterator interface { txn *roachpb.Transaction, consistent, tombstones bool, ) (*roachpb.Value, []roachpb.Intent, error) // MVCCScan is the internal implementation of the family of package-level - // MVCCScan functions. There are two notable differences. The first is that - // key/value pairs are returned raw, as a buffer of varint-prefixed slices, - // alternating from key to value, where numKVs specifies the number of pairs - // in the buffer. The second is that the tombstones parameter allows returning - // deleted values, where the value portion will be empty. + // MVCCScan functions. The notable difference is that key/value pairs are + // returned raw, as a buffer of varint-prefixed slices, alternating from key + // to value, where numKVs specifies the number of pairs in the buffer. // // There is little reason to use this function directly. Use the package-level // MVCCScan, or one of its variants, instead. - MVCCScan(start, end roachpb.Key, max int64, timestamp hlc.Timestamp, - txn *roachpb.Transaction, consistent, reverse, tombstone bool, + MVCCScan( + start, end roachpb.Key, max int64, timestamp hlc.Timestamp, opts MVCCScanOptions, ) (kvData []byte, numKVs int64, resumeSpan *roachpb.Span, intents []roachpb.Intent, err error) // SetUpperBound installs a new upper bound for this iterator. SetUpperBound(roachpb.Key) diff --git a/pkg/storage/engine/mvcc.go b/pkg/storage/engine/mvcc.go index 98f9b438fdb9..a44c39742977 100644 --- a/pkg/storage/engine/mvcc.go +++ b/pkg/storage/engine/mvcc.go @@ -1640,7 +1640,7 @@ func MVCCDeleteRange( // transaction with a newer timestamp, we need to use the max timestamp for // scan. kvs, resumeSpan, _, err := MVCCScan( - ctx, engine, key, endKey, max, hlc.MaxTimestamp, true /* consistent */, txn) + ctx, engine, key, endKey, max, hlc.MaxTimestamp, MVCCScanOptions{Txn: txn}) if err != nil { return nil, nil, 0, err } @@ -1675,17 +1675,12 @@ func MVCCDeleteRange( func mvccScanToKvs( ctx context.Context, iter Iterator, - key, - endKey roachpb.Key, + key, endKey roachpb.Key, max int64, timestamp hlc.Timestamp, - consistent bool, - tombstones bool, - txn *roachpb.Transaction, - reverse bool, + opts MVCCScanOptions, ) ([]roachpb.KeyValue, *roachpb.Span, []roachpb.Intent, error) { - kvData, numKVs, resumeSpan, intents, err := iter.MVCCScan( - key, endKey, max, timestamp, txn, consistent, reverse, tombstones) + kvData, numKVs, resumeSpan, intents, err := iter.MVCCScan(key, endKey, max, timestamp, opts) if err != nil { return nil, nil, nil, err } @@ -1737,6 +1732,20 @@ func buildScanIntents(data []byte) ([]roachpb.Intent, error) { return intents, nil } +// MVCCScanOptions bundles options for the MVCCScan family of functions. +type MVCCScanOptions struct { + // TODO(benesch): The max parameter should be moved into this struct. Its + // semantics make that tricky, though, as the zero value for this struct + // naturally represents an unbounded scan, but a max of zero currently means + // to return no results. + + // See the documentation for MVCCScan for information on these parameters. + Inconsistent bool + Tombstones bool + Reverse bool + Txn *roachpb.Transaction +} + // MVCCScan scans the key range [key, endKey) in the provided engine up to some // maximum number of results in ascending order. If it hits max, it returns a // "resume span" to be used in the next call to this function. If the limit is @@ -1766,67 +1775,28 @@ func buildScanIntents(data []byte) ([]roachpb.Intent, error) { func MVCCScan( ctx context.Context, engine Reader, - key, - endKey roachpb.Key, + key, endKey roachpb.Key, max int64, timestamp hlc.Timestamp, - consistent bool, - txn *roachpb.Transaction, + opts MVCCScanOptions, ) ([]roachpb.KeyValue, *roachpb.Span, []roachpb.Intent, error) { iter := engine.NewIterator(IterOptions{LowerBound: key, UpperBound: endKey}) defer iter.Close() - return mvccScanToKvs(ctx, iter, key, endKey, max, timestamp, - consistent, false /* tombstones */, txn, false /* reverse */) + return mvccScanToKvs(ctx, iter, key, endKey, max, timestamp, opts) } // MVCCScanToBytes is like MVCCScan, but it returns the results in a byte array. func MVCCScanToBytes( ctx context.Context, engine Reader, - key, - endKey roachpb.Key, - max int64, - timestamp hlc.Timestamp, - consistent bool, - txn *roachpb.Transaction, -) ([]byte, int64, *roachpb.Span, []roachpb.Intent, error) { - iter := engine.NewIterator(IterOptions{LowerBound: key, UpperBound: endKey}) - defer iter.Close() - return iter.MVCCScan(key, endKey, max, timestamp, txn, consistent, false /* reverse */, false /* tombstones */) -} - -// MVCCReverseScan is like MVCCScan, but it returns keys in descending order. -func MVCCReverseScan( - ctx context.Context, - engine Reader, - key, - endKey roachpb.Key, - max int64, - timestamp hlc.Timestamp, - consistent bool, - txn *roachpb.Transaction, -) ([]roachpb.KeyValue, *roachpb.Span, []roachpb.Intent, error) { - iter := engine.NewIterator(IterOptions{LowerBound: key, UpperBound: endKey}) - defer iter.Close() - return mvccScanToKvs(ctx, iter, key, endKey, max, timestamp, - consistent, false /* tombstones */, txn, true /* reverse */) -} - -// MVCCReverseScanToBytes is like MVCCReverseScan, but it returns the results -// in a byte array. -func MVCCReverseScanToBytes( - ctx context.Context, - engine Reader, - key, - endKey roachpb.Key, + key, endKey roachpb.Key, max int64, timestamp hlc.Timestamp, - consistent bool, - txn *roachpb.Transaction, + opts MVCCScanOptions, ) ([]byte, int64, *roachpb.Span, []roachpb.Intent, error) { iter := engine.NewIterator(IterOptions{LowerBound: key, UpperBound: endKey}) defer iter.Close() - return iter.MVCCScan(key, endKey, max, timestamp, txn, consistent, true /* reverse */, false /* tombstones */) + return iter.MVCCScan(key, endKey, max, timestamp, opts) } // MVCCIterate iterates over the key range [start,end). At each step of the @@ -1838,38 +1808,19 @@ func MVCCIterate( engine Reader, key, endKey roachpb.Key, timestamp hlc.Timestamp, - consistent bool, - tombstones bool, - txn *roachpb.Transaction, - reverse bool, + opts MVCCScanOptions, f func(roachpb.KeyValue) (bool, error), ) ([]roachpb.Intent, error) { iter := engine.NewIterator(IterOptions{LowerBound: key, UpperBound: endKey}) defer iter.Close() - return MVCCIterateUsingIter(ctx, iter, key, endKey, timestamp, consistent, - tombstones, txn, reverse, f) -} -// MVCCIterateUsingIter iterates over the key range [start,end) using the -// supplied iterator. See comments for MVCCIterate. -func MVCCIterateUsingIter( - ctx context.Context, - iter Iterator, - startKey, endKey roachpb.Key, - timestamp hlc.Timestamp, - consistent bool, - tombstones bool, - txn *roachpb.Transaction, - reverse bool, - f func(roachpb.KeyValue) (bool, error), -) ([]roachpb.Intent, error) { var intents []roachpb.Intent var wiErr error for { const maxKeysPerScan = 1000 kvs, resume, newIntents, err := mvccScanToKvs( - ctx, iter, startKey, endKey, maxKeysPerScan, timestamp, consistent, tombstones, txn, reverse) + ctx, iter, key, endKey, maxKeysPerScan, timestamp, opts) if err != nil { switch tErr := err.(type) { case *roachpb.WriteIntentError: @@ -1910,10 +1861,10 @@ func MVCCIterateUsingIter( if resume == nil { break } - if reverse { + if opts.Reverse { endKey = resume.EndKey } else { - startKey = resume.Key + key = resume.Key } } diff --git a/pkg/storage/engine/mvcc_test.go b/pkg/storage/engine/mvcc_test.go index 36d7543aae8c..454cf863415b 100644 --- a/pkg/storage/engine/mvcc_test.go +++ b/pkg/storage/engine/mvcc_test.go @@ -335,10 +335,10 @@ func TestMVCCEmptyKey(t *testing.T) { if err := MVCCPut(context.Background(), engine, nil, roachpb.Key{}, hlc.Timestamp{Logical: 1}, value1, nil); err == nil { t.Error("expected empty key error") } - if _, _, _, err := MVCCScan(context.Background(), engine, roachpb.Key{}, testKey1, math.MaxInt64, hlc.Timestamp{Logical: 1}, true, nil); err != nil { + if _, _, _, err := MVCCScan(context.Background(), engine, roachpb.Key{}, testKey1, math.MaxInt64, hlc.Timestamp{Logical: 1}, MVCCScanOptions{}); err != nil { t.Errorf("empty key allowed for start key in scan; got %s", err) } - if _, _, _, err := MVCCScan(context.Background(), engine, testKey1, roachpb.Key{}, math.MaxInt64, hlc.Timestamp{Logical: 1}, true, nil); err == nil { + if _, _, _, err := MVCCScan(context.Background(), engine, testKey1, roachpb.Key{}, math.MaxInt64, hlc.Timestamp{Logical: 1}, MVCCScanOptions{}); err == nil { t.Error("expected empty key error") } if err := MVCCResolveWriteIntent(context.Background(), engine, nil, roachpb.Intent{}); err == nil { @@ -751,7 +751,7 @@ func TestMVCCGetUncertainty(t *testing.T) { } else if _, ok := err.(*roachpb.ReadWithinUncertaintyIntervalError); !ok { t.Fatalf("wanted a ReadWithinUncertaintyIntervalError, got %+v", err) } - if _, _, _, err := MVCCScan(context.Background(), engine, testKey2, testKey2.PrefixEnd(), 10, hlc.Timestamp{WallTime: 7}, true, txn); err == nil { + if _, _, _, err := MVCCScan(context.Background(), engine, testKey2, testKey2.PrefixEnd(), 10, hlc.Timestamp{WallTime: 7}, MVCCScanOptions{Txn: txn}); err == nil { t.Fatal("wanted an error") } else if _, ok := err.(*roachpb.ReadWithinUncertaintyIntervalError); !ok { t.Fatalf("wanted a ReadWithinUncertaintyIntervalError, got %+v", err) @@ -761,7 +761,7 @@ func TestMVCCGetUncertainty(t *testing.T) { if _, _, err := mvccGet(context.Background(), engine, testKey2, hlc.Timestamp{WallTime: 7}, true, txn); err != nil { t.Fatal(err) } - if _, _, _, err := MVCCScan(context.Background(), engine, testKey2, testKey2.PrefixEnd(), 10, hlc.Timestamp{WallTime: 7}, true, txn); err != nil { + if _, _, _, err := MVCCScan(context.Background(), engine, testKey2, testKey2.PrefixEnd(), 10, hlc.Timestamp{WallTime: 7}, MVCCScanOptions{Txn: txn}); err != nil { t.Fatal(err) } @@ -776,7 +776,7 @@ func TestMVCCGetUncertainty(t *testing.T) { if err := MVCCPut(context.Background(), engine, nil, testKey3, hlc.Timestamp{WallTime: 99}, value2, nil); err != nil { t.Fatal(err) } - if _, _, _, err := MVCCScan(context.Background(), engine, testKey3, testKey3.PrefixEnd(), 10, hlc.Timestamp{WallTime: 7}, true, txn); err == nil { + if _, _, _, err := MVCCScan(context.Background(), engine, testKey3, testKey3.PrefixEnd(), 10, hlc.Timestamp{WallTime: 7}, MVCCScanOptions{Txn: txn}); err == nil { t.Fatal("wanted an error") } else if _, ok := err.(*roachpb.ReadWithinUncertaintyIntervalError); !ok { t.Fatalf("wanted a ReadWithinUncertaintyIntervalError, got %+v", err) @@ -1099,7 +1099,7 @@ func TestMVCCScanWriteIntentError(t *testing.T) { if scan.consistent { cStr = "consistent" } - kvs, _, intents, err := MVCCScan(context.Background(), engine, testKey1, testKey4.Next(), math.MaxInt64, hlc.Timestamp{WallTime: 1}, scan.consistent, scan.txn) + kvs, _, intents, err := MVCCScan(context.Background(), engine, testKey1, testKey4.Next(), math.MaxInt64, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Inconsistent: !scan.consistent, Txn: scan.txn}) wiErr, _ := err.(*roachpb.WriteIntentError) if (err == nil) != (wiErr == nil) { t.Errorf("%s(%d): unexpected error: %s", cStr, i, err) @@ -1325,7 +1325,7 @@ func TestMVCCInvalidateIterator(t *testing.T) { case "get": _, _, err = MVCCGet(ctx, batch, key, ts2, true, nil) case "scan": - _, _, _, err = MVCCScan(ctx, batch, key, roachpb.KeyMax, math.MaxInt64, ts2, true, nil) + _, _, _, err = MVCCScan(ctx, batch, key, roachpb.KeyMax, math.MaxInt64, ts2, MVCCScanOptions{}) case "findSplitKey": _, err = MVCCFindSplitKey(ctx, batch, roachpb.RKeyMin, roachpb.RKeyMax, 64<<20) case "computeStats": @@ -1377,7 +1377,7 @@ func TestMVCCScan(t *testing.T) { t.Fatal(err) } - kvs, resumeSpan, _, err := MVCCScan(context.Background(), engine, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, resumeSpan, _, err := MVCCScan(context.Background(), engine, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1392,7 +1392,7 @@ func TestMVCCScan(t *testing.T) { t.Fatalf("resumeSpan = %+v", resumeSpan) } - kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 4}, true, nil) + kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 4}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1407,7 +1407,7 @@ func TestMVCCScan(t *testing.T) { t.Fatalf("resumeSpan = %+v", resumeSpan) } - kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey4, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey4, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1423,7 +1423,7 @@ func TestMVCCScan(t *testing.T) { if _, _, err := MVCCGet(context.Background(), engine, testKey1, hlc.Timestamp{WallTime: 1}, true, txn2); err != nil { t.Fatal(err) } - kvs, _, _, err = MVCCScan(context.Background(), engine, keyMin, testKey2, math.MaxInt64, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, _, _, err = MVCCScan(context.Background(), engine, keyMin, testKey2, math.MaxInt64, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1455,7 +1455,7 @@ func TestMVCCScanMaxNum(t *testing.T) { t.Fatal(err) } - kvs, resumeSpan, _, err := MVCCScan(context.Background(), engine, testKey2, testKey4, 1, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, resumeSpan, _, err := MVCCScan(context.Background(), engine, testKey2, testKey4, 1, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1468,7 +1468,7 @@ func TestMVCCScanMaxNum(t *testing.T) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) } - kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey2, testKey4, 0, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey2, testKey4, 0, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1481,7 +1481,7 @@ func TestMVCCScanMaxNum(t *testing.T) { // Note: testKey6, though not scanned directly, is important in testing that // the computed resume span does not extend beyond the upper bound of a scan. - kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey4, testKey5, 1, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey4, testKey5, 1, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1492,7 +1492,7 @@ func TestMVCCScanMaxNum(t *testing.T) { t.Fatalf("resumeSpan = %+v", resumeSpan) } - kvs, resumeSpan, _, err = MVCCReverseScan(context.Background(), engine, testKey5, testKey6.Next(), 1, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey5, testKey6.Next(), 1, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) if err != nil { t.Fatal(err) } @@ -1536,7 +1536,7 @@ func TestMVCCScanWithKeyPrefix(t *testing.T) { t.Fatal(err) } - kvs, _, _, err := MVCCScan(context.Background(), engine, roachpb.Key("/a"), roachpb.Key("/b"), math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, err := MVCCScan(context.Background(), engine, roachpb.Key("/a"), roachpb.Key("/b"), math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1567,7 +1567,7 @@ func TestMVCCScanInTxn(t *testing.T) { t.Fatal(err) } - kvs, _, _, err := MVCCScan(context.Background(), engine, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, true, txn1) + kvs, _, _, err := MVCCScan(context.Background(), engine, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Txn: txn1}) if err != nil { t.Fatal(err) } @@ -1579,7 +1579,7 @@ func TestMVCCScanInTxn(t *testing.T) { t.Fatal("the value should not be empty") } - if _, _, _, err := MVCCScan(context.Background(), engine, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, true, nil); err == nil { + if _, _, _, err := MVCCScan(context.Background(), engine, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}); err == nil { t.Fatal("expected error on uncommitted write intent") } } @@ -1592,7 +1592,7 @@ func TestMVCCScanInconsistent(t *testing.T) { defer engine.Close() // A scan with consistent=false should fail in a txn. - if _, _, _, err := MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 1}, false, txn1); err == nil { + if _, _, _, err := MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Inconsistent: !false, Txn: txn1}); err == nil { t.Error("expected an error scanning with consistent=false in txn") } @@ -1625,7 +1625,7 @@ func TestMVCCScanInconsistent(t *testing.T) { {Span: roachpb.Span{Key: testKey1}, Txn: txn1.TxnMeta}, {Span: roachpb.Span{Key: testKey3}, Txn: txn2.TxnMeta}, } - kvs, _, intents, err := MVCCScan(context.Background(), engine, testKey1, testKey4.Next(), math.MaxInt64, hlc.Timestamp{WallTime: 7}, false, nil) + kvs, _, intents, err := MVCCScan(context.Background(), engine, testKey1, testKey4.Next(), math.MaxInt64, hlc.Timestamp{WallTime: 7}, MVCCScanOptions{Inconsistent: !false}) if err != nil { t.Fatal(err) } @@ -1649,7 +1649,7 @@ func TestMVCCScanInconsistent(t *testing.T) { // Now try a scan at a historical timestamp. expIntents = expIntents[:1] - kvs, _, intents, err = MVCCScan(context.Background(), engine, testKey1, testKey4.Next(), math.MaxInt64, hlc.Timestamp{WallTime: 3}, false, nil) + kvs, _, intents, err = MVCCScan(context.Background(), engine, testKey1, testKey4.Next(), math.MaxInt64, hlc.Timestamp{WallTime: 3}, MVCCScanOptions{Inconsistent: !false}) if !reflect.DeepEqual(intents, expIntents) { t.Fatal(err) } @@ -1702,7 +1702,7 @@ func TestMVCCDeleteRange(t *testing.T) { if expected := (roachpb.Span{Key: testKey4, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) } - kvs, _, _, _ := MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, _ := MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) if len(kvs) != 4 || !bytes.Equal(kvs[0].Key, testKey1) || !bytes.Equal(kvs[1].Key, testKey4) || @@ -1719,7 +1719,7 @@ func TestMVCCDeleteRange(t *testing.T) { kvs = []roachpb.KeyValue{} if _, err = MVCCIterate( context.Background(), engine, keyMin, keyMax, hlc.Timestamp{WallTime: 2}, - true, true /* tombstones */, nil, false, func(kv roachpb.KeyValue) (bool, error) { + MVCCScanOptions{Tombstones: true}, func(kv roachpb.KeyValue) (bool, error) { kvs = append(kvs, kv) return false, nil }, @@ -1758,7 +1758,7 @@ func TestMVCCDeleteRange(t *testing.T) { if expected := (roachpb.Span{Key: testKey2, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) } - kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) if len(kvs) != 4 || !bytes.Equal(kvs[0].Key, testKey1) || !bytes.Equal(kvs[1].Key, testKey4) || @@ -1786,7 +1786,7 @@ func TestMVCCDeleteRange(t *testing.T) { if resumeSpan != nil { t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) } - kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) if len(kvs) != 1 || !bytes.Equal(kvs[0].Key, testKey1) || !bytes.Equal(kvs[0].Value.RawBytes, value1.RawBytes) { @@ -1808,7 +1808,7 @@ func TestMVCCDeleteRange(t *testing.T) { if resumeSpan != nil { t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) } - kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) if len(kvs) != 0 { t.Fatal("the value should be empty") } @@ -1860,7 +1860,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { if expected := (roachpb.Span{Key: testKey4, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) } - kvs, _, _, _ := MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, _ := MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) if len(kvs) != 4 || !bytes.Equal(kvs[0].Key, testKey1) || !bytes.Equal(kvs[1].Key, testKey4) || @@ -1889,7 +1889,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { if expected := (roachpb.Span{Key: testKey2, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) } - kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) if len(kvs) != 4 || !bytes.Equal(kvs[0].Key, testKey1) || !bytes.Equal(kvs[1].Key, testKey4) || @@ -1926,7 +1926,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { if resumeSpan != nil { t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) } - kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) if len(kvs) != 1 || !bytes.Equal(kvs[0].Key, testKey1) || !bytes.Equal(kvs[0].Value.RawBytes, value1.RawBytes) { @@ -1951,7 +1951,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { if resumeSpan != nil { t.Fatalf("wrong resume key: %v", resumeSpan) } - kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, _ = MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) if len(kvs) != 0 { t.Fatal("the value should be empty") } @@ -2048,7 +2048,7 @@ func TestMVCCUncommittedDeleteRangeVisible(t *testing.T) { } txn.Epoch++ - kvs, _, _, _ := MVCCScan(context.Background(), engine, testKey1, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 3}, true, &txn) + kvs, _, _, _ := MVCCScan(context.Background(), engine, testKey1, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 3}, MVCCScanOptions{Txn: &txn}) if e := 2; len(kvs) != e { t.Fatalf("e = %d, got %d", e, len(kvs)) } @@ -2139,7 +2139,7 @@ func TestMVCCDeleteRangeInline(t *testing.T) { Value: value6, }, } - kvs, _, _, err := MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, err := MVCCScan(context.Background(), engine, keyMin, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -2547,8 +2547,8 @@ func TestMVCCReverseScan(t *testing.T) { t.Fatal(err) } - kvs, resumeSpan, _, err := MVCCReverseScan(context.Background(), engine, - testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, resumeSpan, _, err := MVCCScan(context.Background(), engine, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) + if err != nil { t.Fatal(err) } @@ -2563,8 +2563,8 @@ func TestMVCCReverseScan(t *testing.T) { t.Fatalf("resumeSpan = %+v", resumeSpan) } - kvs, resumeSpan, _, err = MVCCReverseScan(context.Background(), engine, - testKey2, testKey4, 1, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey2, testKey4, 1, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) + if err != nil { t.Fatal(err) } @@ -2577,8 +2577,8 @@ func TestMVCCReverseScan(t *testing.T) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) } - kvs, resumeSpan, _, err = MVCCReverseScan(context.Background(), engine, - testKey2, testKey4, 0, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, resumeSpan, _, err = MVCCScan(context.Background(), engine, testKey2, testKey4, 0, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) + if err != nil { t.Fatal(err) } @@ -2591,8 +2591,8 @@ func TestMVCCReverseScan(t *testing.T) { // The first key we encounter has multiple versions and we need to read the // latest. - kvs, _, _, err = MVCCReverseScan(context.Background(), engine, - testKey2, testKey3, 1, hlc.Timestamp{WallTime: 4}, true, nil) + kvs, _, _, err = MVCCScan(context.Background(), engine, testKey2, testKey3, 1, hlc.Timestamp{WallTime: 4}, MVCCScanOptions{Reverse: true}) + if err != nil { t.Fatal(err) } @@ -2604,8 +2604,8 @@ func TestMVCCReverseScan(t *testing.T) { // The first key we encounter is newer than our read timestamp and we need to // back up to the previous key. - kvs, _, _, err = MVCCReverseScan(context.Background(), engine, - testKey4, testKey6, 1, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, _, _, err = MVCCScan(context.Background(), engine, testKey4, testKey6, 1, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) + if err != nil { t.Fatal(err) } @@ -2616,8 +2616,8 @@ func TestMVCCReverseScan(t *testing.T) { } // Scan only the first key in the key space. - kvs, _, _, err = MVCCReverseScan(context.Background(), engine, - testKey1, testKey1.Next(), 1, hlc.Timestamp{WallTime: 1}, true, nil) + kvs, _, _, err = MVCCScan(context.Background(), engine, testKey1, testKey1.Next(), 1, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) + if err != nil { t.Fatal(err) } @@ -2652,7 +2652,7 @@ func TestMVCCReverseScanFirstKeyInFuture(t *testing.T) { t.Fatal(err) } - kvs, _, _, err := MVCCReverseScan(context.Background(), engine, testKey1, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 2}, true, nil) + kvs, _, _, err := MVCCScan(context.Background(), engine, testKey1, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{Reverse: true}) if err != nil { t.Fatal(err) } diff --git a/pkg/storage/engine/rocksdb.go b/pkg/storage/engine/rocksdb.go index 0ae265b3985b..51c02b1c14f9 100644 --- a/pkg/storage/engine/rocksdb.go +++ b/pkg/storage/engine/rocksdb.go @@ -1465,14 +1465,10 @@ func (r *batchIterator) MVCCGet( } func (r *batchIterator) MVCCScan( - start, end roachpb.Key, - max int64, - timestamp hlc.Timestamp, - txn *roachpb.Transaction, - consistent, reverse, tombstones bool, + start, end roachpb.Key, max int64, timestamp hlc.Timestamp, opts MVCCScanOptions, ) (kvData []byte, numKVs int64, resumeSpan *roachpb.Span, intents []roachpb.Intent, err error) { r.batch.flushMutations() - return r.iter.MVCCScan(start, end, max, timestamp, txn, consistent, reverse, tombstones) + return r.iter.MVCCScan(start, end, max, timestamp, opts) } func (r *batchIterator) SetUpperBound(key roachpb.Key) { @@ -2292,13 +2288,9 @@ func (r *rocksDBIterator) MVCCGet( } func (r *rocksDBIterator) MVCCScan( - start, end roachpb.Key, - max int64, - timestamp hlc.Timestamp, - txn *roachpb.Transaction, - consistent, reverse, tombstones bool, + start, end roachpb.Key, max int64, timestamp hlc.Timestamp, opts MVCCScanOptions, ) (kvData []byte, numKVs int64, resumeSpan *roachpb.Span, intents []roachpb.Intent, err error) { - if !consistent && txn != nil { + if opts.Inconsistent && opts.Txn != nil { return nil, 0, nil, nil, errors.Errorf("cannot allow inconsistent reads within a transaction") } if len(end) == 0 { @@ -2313,13 +2305,13 @@ func (r *rocksDBIterator) MVCCScan( state := C.MVCCScan( r.iter, goToCSlice(start), goToCSlice(end), goToCTimestamp(timestamp), C.int64_t(max), - goToCTxn(txn), C.bool(consistent), C.bool(reverse), C.bool(tombstones), + goToCTxn(opts.Txn), C.bool(!opts.Inconsistent), C.bool(opts.Reverse), C.bool(opts.Tombstones), ) if err := statusToError(state.status); err != nil { return nil, 0, nil, nil, err } - if err := uncertaintyToError(timestamp, state.uncertainty_timestamp, txn); err != nil { + if err := uncertaintyToError(timestamp, state.uncertainty_timestamp, opts.Txn); err != nil { return nil, 0, nil, nil, err } @@ -2327,7 +2319,7 @@ func (r *rocksDBIterator) MVCCScan( numKVs = int64(state.data.count) if resumeKey := cSliceToGoBytes(state.resume_key); resumeKey != nil { - if reverse { + if opts.Reverse { resumeSpan = &roachpb.Span{Key: start, EndKey: roachpb.Key(resumeKey).Next()} } else { resumeSpan = &roachpb.Span{Key: resumeKey, EndKey: end} @@ -2338,7 +2330,7 @@ func (r *rocksDBIterator) MVCCScan( if err != nil { return nil, 0, nil, nil, err } - if consistent && len(intents) > 0 { + if !opts.Inconsistent && len(intents) > 0 { // When encountering intents during a consistent scan we still need to // return the resume key. return nil, 0, resumeSpan, nil, &roachpb.WriteIntentError{Intents: intents} diff --git a/pkg/storage/engine/rocksdb_iter_stats_test.go b/pkg/storage/engine/rocksdb_iter_stats_test.go index f96d9fa408cb..c6f894fd88d8 100644 --- a/pkg/storage/engine/rocksdb_iter_stats_test.go +++ b/pkg/storage/engine/rocksdb_iter_stats_test.go @@ -66,7 +66,7 @@ func TestIterStats(t *testing.T) { // Scanning a key range containing the tombstone sees it. for i := 0; i < 10; i++ { if _, _, _, _, err := iter.MVCCScan( - roachpb.KeyMin, roachpb.KeyMax, math.MaxInt64, hlc.Timestamp{}, nil, true, false, false, + roachpb.KeyMin, roachpb.KeyMax, math.MaxInt64, hlc.Timestamp{}, MVCCScanOptions{}, ); err != nil { t.Fatal(err) } diff --git a/pkg/storage/gc_queue.go b/pkg/storage/gc_queue.go index 7b2671ad5d14..eaf55de3c3c4 100644 --- a/pkg/storage/gc_queue.go +++ b/pkg/storage/gc_queue.go @@ -425,9 +425,8 @@ func processLocalKeyRange( startKey := keys.MakeRangeKeyPrefix(desc.StartKey) endKey := keys.MakeRangeKeyPrefix(desc.EndKey) - _, err := engine.MVCCIterate(ctx, snap, startKey, endKey, - hlc.Timestamp{}, true /* consistent */, false /* tombstones */, nil, /* txn */ - false /* reverse */, func(kv roachpb.KeyValue) (bool, error) { + _, err := engine.MVCCIterate(ctx, snap, startKey, endKey, hlc.Timestamp{}, engine.MVCCScanOptions{}, + func(kv roachpb.KeyValue) (bool, error) { return false, handleOne(kv) }) return gcKeys, err diff --git a/pkg/storage/gc_queue_test.go b/pkg/storage/gc_queue_test.go index f37460a6d3c3..a98b2ed6df44 100644 --- a/pkg/storage/gc_queue_test.go +++ b/pkg/storage/gc_queue_test.go @@ -786,7 +786,7 @@ func TestGCQueueTransactionTable(t *testing.T) { outsideTxnPrefixEnd := keys.TransactionKey(outsideKey.Next(), uuid.UUID{}) var count int if _, err := engine.MVCCIterate(context.Background(), tc.store.Engine(), outsideTxnPrefix, outsideTxnPrefixEnd, hlc.Timestamp{}, - true, false, nil, false, func(roachpb.KeyValue) (bool, error) { + engine.MVCCScanOptions{}, func(roachpb.KeyValue) (bool, error) { count++ return false, nil }); err != nil { diff --git a/pkg/storage/replica_raftstorage.go b/pkg/storage/replica_raftstorage.go index 37035c4675b5..93986ed3e3c5 100644 --- a/pkg/storage/replica_raftstorage.go +++ b/pkg/storage/replica_raftstorage.go @@ -237,10 +237,7 @@ func iterateEntries( keys.RaftLogKey(rangeID, lo), keys.RaftLogKey(rangeID, hi), hlc.Timestamp{}, - true, /* consistent */ - false, /* tombstones */ - nil, /* txn */ - false, /* reverse */ + engine.MVCCScanOptions{}, scanFunc, ) return err diff --git a/pkg/storage/spanset/batch.go b/pkg/storage/spanset/batch.go index 1d91c77db812..2d42f7cb89f0 100644 --- a/pkg/storage/spanset/batch.go +++ b/pkg/storage/spanset/batch.go @@ -181,16 +181,12 @@ func (s *Iterator) MVCCGet( // MVCCScan is part of the engine.Iterator interface. func (s *Iterator) MVCCScan( - start, end roachpb.Key, - max int64, - timestamp hlc.Timestamp, - txn *roachpb.Transaction, - consistent, reverse, tombstones bool, + start, end roachpb.Key, max int64, timestamp hlc.Timestamp, opts engine.MVCCScanOptions, ) (kvData []byte, numKVs int64, resumeSpan *roachpb.Span, intents []roachpb.Intent, err error) { if err := s.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil { return nil, 0, nil, nil, err } - return s.i.MVCCScan(start, end, max, timestamp, txn, consistent, reverse, tombstones) + return s.i.MVCCScan(start, end, max, timestamp, opts) } // SetUpperBound is part of the engine.Iterator interface. diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 51a6c6733fc5..21f6ac735cfa 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -1177,8 +1177,8 @@ func IterateRangeDescriptors( return fn(desc) } - _, err := engine.MVCCIterate(ctx, eng, start, end, hlc.MaxTimestamp, false /* consistent */, false, /* tombstones */ - nil /* txn */, false /* reverse */, kvToDesc) + _, err := engine.MVCCIterate(ctx, eng, start, end, hlc.MaxTimestamp, + engine.MVCCScanOptions{Inconsistent: true}, kvToDesc) log.Eventf(ctx, "iterated over %d keys to find %d range descriptors (by suffix: %v)", allCount, matchCount, bySuffix) return err diff --git a/pkg/ts/db_test.go b/pkg/ts/db_test.go index f051da4e2d69..31b1f032f60a 100644 --- a/pkg/ts/db_test.go +++ b/pkg/ts/db_test.go @@ -128,7 +128,7 @@ func (tm *testModelRunner) getActualData() map[string]roachpb.Value { // Scan over all TS Keys stored in the engine startKey := keys.TimeseriesPrefix endKey := startKey.PrefixEnd() - keyValues, _, _, err := engine.MVCCScan(context.Background(), tm.Eng, startKey, endKey, math.MaxInt64, tm.Clock.Now(), true, nil) + keyValues, _, _, err := engine.MVCCScan(context.Background(), tm.Eng, startKey, endKey, math.MaxInt64, tm.Clock.Now(), engine.MVCCScanOptions{}) if err != nil { tm.t.Fatalf("error scanning TS data from engine: %s", err) }