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