diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 61780523cec9..7b86246eaca2 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -76,14 +76,7 @@ type SimpleMVCCIterator interface { // IteratorStats is returned from {MVCCIterator,EngineIterator}.Stats. type IteratorStats struct { - // TODO(sumeer): populate this stat that was inherited from RocksDB, which - // counts the number of deletes or single deletes skipped over during - // iteration. It may be better to replace this with the number of Pebble - // key-values encountered, which would encompass Pebble versions that were - // explicitly deleted and those that were obsoleted due to newer Pebble - // versions. - InternalDeleteSkippedCount int - TimeBoundNumSSTs int + TimeBoundNumSSTs int // Iteration stats. We directly expose pebble.IteratorStats. Callers // may want to aggregate and interpret these in the following manner: diff --git a/pkg/storage/engine_key.go b/pkg/storage/engine_key.go index fcf3b89dbfc2..0ad9d0cd2112 100644 --- a/pkg/storage/engine_key.go +++ b/pkg/storage/engine_key.go @@ -196,9 +196,9 @@ func (k EngineKey) Validate() error { return nil } -// DecodeEngineKey decodes the given bytes as an EngineKey. This function is -// similar to enginepb.SplitMVCCKey. -// TODO(sumeer): consider removing SplitMVCCKey. +// DecodeEngineKey decodes the given bytes as an EngineKey. If the caller +// already knows that the key is an MVCCKey, the Version returned is the +// encoded timestamp. func DecodeEngineKey(b []byte) (key EngineKey, ok bool) { if len(b) == 0 { return EngineKey{}, false diff --git a/pkg/storage/engine_key_test.go b/pkg/storage/engine_key_test.go index 80703857e117..041fb03b10e2 100644 --- a/pkg/storage/engine_key_test.go +++ b/pkg/storage/engine_key_test.go @@ -122,6 +122,10 @@ func TestMVCCAndEngineKeyEncodeDecode(t *testing.T) { require.True(t, ok) require.Equal(t, k3, []byte(test.key.Key)) require.Equal(t, ts, encodedTS) + k, ok := DecodeEngineKey(b3) + require.True(t, ok) + require.Equal(t, k.Key, test.key.Key) + require.Equal(t, k.Version, encodedTS) }) } } diff --git a/pkg/storage/enginepb/decode.go b/pkg/storage/enginepb/decode.go index 8b876dd79587..bc2d7235a3c6 100644 --- a/pkg/storage/enginepb/decode.go +++ b/pkg/storage/enginepb/decode.go @@ -25,6 +25,9 @@ import ( // code out from abstract interfaces -- See #30114 and #30001. // SplitMVCCKey returns the key and timestamp components of an encoded MVCC key. +// This function is similar to storage.DecodeEngineKey. +// TODO(sumeer): remove SplitMVCCKey. It requires moving EngineKey into +// enginepb. func SplitMVCCKey(mvccKey []byte) (key []byte, ts []byte, ok bool) { if len(mvccKey) == 0 { return nil, nil, false diff --git a/pkg/storage/intent_interleaving_iter.go b/pkg/storage/intent_interleaving_iter.go index 07ec2b1d4e6e..42757860faa5 100644 --- a/pkg/storage/intent_interleaving_iter.go +++ b/pkg/storage/intent_interleaving_iter.go @@ -968,7 +968,6 @@ func (i *intentInterleavingIter) SetUpperBound(key roachpb.Key) { func (i *intentInterleavingIter) Stats() IteratorStats { stats := i.iter.Stats() intentStats := i.intentIter.Stats() - stats.InternalDeleteSkippedCount += intentStats.InternalDeleteSkippedCount stats.TimeBoundNumSSTs += intentStats.TimeBoundNumSSTs for i := pebble.IteratorStatsKind(0); i < pebble.NumStatsKind; i++ { stats.Stats.ForwardSeekCount[i] += intentStats.Stats.ForwardSeekCount[i] diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index 494dc2d30066..e5f971b07c2a 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -1443,7 +1443,7 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - // Create a DB containing 2 keys, a and b, where b has an intent. We use the + // Create a DB containing 2 keys, a and b, where a has an intent. We use the // regular MVCCPut operation to generate these keys, which we'll later be // copying into manually created sstables. ctx := context.Background() @@ -1460,8 +1460,7 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { } } - put("a", "a value", 1, nil) - put("b", "b value", 2, &roachpb.Transaction{ + put("a", "a value", 2, &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ Key: roachpb.Key("b"), ID: uuid.MakeV4(), @@ -1470,45 +1469,40 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { }, ReadTimestamp: hlc.Timestamp{WallTime: 2}, }) + put("b", "b value", 1, nil) - // Create a second DB in which we'll create a specific SSTable structure: the - // first SSTable contains 2 KVs where the first is a regular versioned key - // and the second is the MVCC metadata entry (i.e. an intent). The next - // SSTable contains the provisional value for the intent. The effect is that - // the metadata entry is separated from the entry it is metadata for. + // Create a second DB in which we'll create a specific SSTable structure: + // the first SSTable contains the intent, and the second contains the two + // versioned keys. The effect is that the intent is in a separate sstable + // from the entry it is the intent for. // // SSTable 1: - // a@1 - // b@ + // a@ (in the lock table) // // SSTable 2: - // b@2 + // a@2 + // b@1 db2, err := Open(ctx, InMemory(), ForTesting) require.NoError(t, err) defer db2.Close() - // NB: If the original intent was separated, iterating using an interleaving - // iterator, as done below, and writing to an sst, transforms the separated - // intent to an interleaved intent. This is ok for now since both kinds of - // intents are supported. - // TODO(sumeer): change this test before interleaved intents are disallowed. - ingest := func(it MVCCIterator, count int) { + ingest := func(it EngineIterator, valid bool, err error, count int) { memFile := &MemFile{} sst := MakeIngestionSSTWriter(ctx, db2.settings, memFile) defer sst.Close() for i := 0; i < count; i++ { - ok, err := it.Valid() - if err != nil { - t.Fatal(err) - } - if !ok { - t.Fatal("expected key") - } - if err := sst.Put(it.Key(), it.Value()); err != nil { + require.NoError(t, err) + require.True(t, valid) + ek, err := it.EngineKey() + require.NoError(t, err) + require.NoError(t, err) + if err := sst.PutEngineKey(ek, it.Value()); err != nil { t.Fatal(err) } - it.Next() + valid, err = it.NextEngineKey() + // Make linter happy. + require.NoError(t, err) } if err := sst.Finish(); err != nil { t.Fatal(err) @@ -1524,13 +1518,12 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { { // Iterate over the entries in the first DB, ingesting them into SSTables // in the second DB. - it := db1.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ - UpperBound: keys.MaxKey, - }) + it := db1.NewEngineIterator(IterOptions{UpperBound: keys.MaxKey}) defer it.Close() - it.SeekGE(MVCCKey{Key: keys.LocalMax}) - ingest(it, 2) - ingest(it, 1) + valid, err := it.SeekEngineKeyGE(EngineKey{Key: keys.LocalRangeLockTablePrefix}) + ingest(it, valid, err, 1) + valid, err = it.SeekEngineKeyGE(EngineKey{Key: keys.LocalMax}) + ingest(it, valid, err, 2) } { diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index b5a4ecb12e09..d82b436b7d7b 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -274,11 +274,11 @@ func getRowPrefix(key roachpb.Key) []byte { // KV representation, otherwise this may panic. func extractResultKey(repr []byte) roachpb.Key { keyLen := binary.LittleEndian.Uint32(repr[4:8]) - key, _, ok := enginepb.SplitMVCCKey(repr[8 : 8+keyLen]) + key, ok := DecodeEngineKey(repr[8 : 8+keyLen]) if !ok { return nil } - return key + return key.Key } // Go port of mvccScanner in libroach/mvcc.h. Stores all variables relating to