Skip to content

Commit

Permalink
storage: correctly update the stats for rolled back deletes
Browse files Browse the repository at this point in the history
Previously, a delete on a key rolled back by a savepoint did not
correctly update the MVCCStats when the intent for that key was
resolved. In particular, if the rollback revealed a non-deleted value
for the key, the stats considered that key/value non-live and included
other inaccuracies. This is an example from the output of kvnemesis,
which caught the bug:

```
engine stats:
{ContainsEstimates:0 LastUpdateNanos:1698322873881949000 LockAge:0
GCBytesAge:0 LiveBytes:771 LiveCount:13 KeyBytes:1350 KeyCount:34
ValBytes:714 ValCount:53 IntentBytes:8 IntentCount:0 LockBytes:0
LockCount:0 RangeKeyCount:0 RangeKeyBytes:0 RangeValCount:0
RangeValBytes:0 SysBytes:254 SysCount:6 AbortSpanBytes:0}

computed stats:
{ContainsEstimates:0 LastUpdateNanos:1698322873881949000 LockAge:0
GCBytesAge:0 LiveBytes:161 LiveCount:1 KeyBytes:0 KeyCount:0
ValBytes:8 ValCount:0 IntentBytes:8 IntentCount:0 LockBytes:0
LockCount:0 RangeKeyCount:0 RangeKeyBytes:0 RangeValCount:0
RangeValBytes:0 SysBytes:0 SysCount:0 AbortSpanBytes:0}
```

This patch fixes the logic responsible for computing stats upon intent
resolution and adds testing for rolled back deletes.

Informs: cockroachdb#97444

Release note (bug fix): Rolled back deletes no longer cause a
discrepancy between computed stats and the actual stored values.
  • Loading branch information
miraradeva committed Oct 31, 2023
1 parent b7fb29c commit 6c5489f
Show file tree
Hide file tree
Showing 3 changed files with 460 additions and 17 deletions.
34 changes: 17 additions & 17 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -644,13 +644,6 @@ func updateStatsOnResolve(
return ms
}

// An intent can't turn from deleted to non-deleted and vice versa while being
// resolved.
if orig.Deleted != meta.Deleted {
log.Fatalf(context.TODO(), "on resolve, original meta was deleted=%t, but new one is deleted=%t",
orig.Deleted, meta.Deleted)
}

// In the main case, we had an old intent at orig.Timestamp, and a new intent
// or value at meta.Timestamp. We'll walk through the contributions below,
// taking special care for LockAge and GCBytesAge.
Expand All @@ -669,14 +662,16 @@ func updateStatsOnResolve(
ms.KeyBytes -= origMetaKeySize + orig.KeyBytes
ms.ValBytes -= origMetaValSize + orig.ValBytes

// If the old intent is a deletion, then the key already isn't tracked
// in LiveBytes any more (and the new intent/value is also a deletion).
// If we're looking at a non-deletion intent/value, update the live
// bytes to account for the difference between the previous intent and
// the new intent/value.
if !meta.Deleted {
// Next, we adjust LiveBytes based on meta.Deleted and orig.Deleted.
// Note that LiveBytes here corresponds to ts = orig.Timestamp.WallTime.
// LiveBytes at ts = meta.Timestamp.WallTime is adjusted below.
// If the original value was deleted, there is no need to adjust the
// contribution of the original key and value to LiveBytes. Otherwise, we need
// to subtract the original key and value's contribution from LiveBytes.
if !orig.Deleted {
ms.LiveBytes -= origMetaKeySize + origMetaValSize
ms.LiveBytes -= orig.KeyBytes + orig.ValBytes
ms.LiveCount--
}

// LockAge is always accrued from the intent's own timestamp on.
Expand Down Expand Up @@ -713,6 +708,7 @@ func updateStatsOnResolve(
// The new meta key appears.
if !meta.Deleted {
ms.LiveBytes += (metaKeySize + metaValSize) + (meta.KeyBytes + meta.ValBytes)
ms.LiveCount++
}

if !commit {
Expand Down Expand Up @@ -5077,13 +5073,20 @@ func mvccResolveWriteIntent(
// remains, the rolledBackVal is set to a non-nil value.
var rolledBackVal *MVCCValue
var err error
buf.newMeta = *meta
newMeta := &buf.newMeta
if len(intent.IgnoredSeqNums) > 0 {
// NOTE: mvccMaybeRewriteIntentHistory mutates its meta argument.
// TODO(nvanbenschoten): this is an awkward interface. We shouldn't
// be mutating meta and we shouldn't be restoring the previous value
// here. Instead, this should all be handled down below.
var removeIntent bool
removeIntent, rolledBackVal, err = mvccMaybeRewriteIntentHistory(ctx, writer, intent.IgnoredSeqNums, meta, latestKey)
// Instead of modifying meta, pass a copy of it (newMeta), which will be the
// starting point for the updated metadata. It's important to keep meta
// intact and corresponding to the stats in ms to ensure that later on (in
// updateStatsOnResolve) the stats will be updated correctly based on the
// old meta (meta) and the new meta (newMeta).
removeIntent, rolledBackVal, err = mvccMaybeRewriteIntentHistory(ctx, writer, intent.IgnoredSeqNums, newMeta, latestKey)
if err != nil {
return false, err
}
Expand Down Expand Up @@ -5125,9 +5128,6 @@ func mvccResolveWriteIntent(
// is because removeIntent implies rolledBackVal == nil, pushed == false, and
// commit == false.
if commit || pushed || rolledBackVal != nil {
buf.newMeta = *meta
newMeta := &buf.newMeta

// The intent might be committing at a higher timestamp, or it might be
// getting pushed.
newTimestamp := intent.Txn.WriteTimestamp
Expand Down
270 changes: 270 additions & 0 deletions pkg/storage/mvcc_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1473,6 +1473,261 @@ func TestMVCCStatsSysPutPut(t *testing.T) {
assertEqLocal(t, engine, "after second put", aggMS, &expMS)
}

// TestMVCCStatsPutRollbackDelete exercises the case in which an intent is
// written, then re-written by a deletion tombstone, which is rolled back. We
// expect the stats to be adjusted correctly for the rolled back delete, and
// crucially, for the LiveBytes and LiveCount to be non-zero and corresponding
// to the original value.
func TestMVCCStatsPutRollbackDelete(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
engine := NewDefaultInMemForTesting()
defer engine.Close()

ctx := context.Background()
aggMS := &enginepb.MVCCStats{}
assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{})

key := roachpb.Key("a")
value := roachpb.MakeValueFromString("value")
ts := hlc.Timestamp{WallTime: 1e9}
txn := &roachpb.Transaction{
TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts},
ReadTimestamp: ts,
}

// Put a value.
if err := MVCCPut(ctx, engine, key, txn.ReadTimestamp, value, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil {
t.Fatal(err)
}

mKeySize := int64(mvccKey(key).EncodedSize()) // 2
mValSize := int64((&enginepb.MVCCMetadata{ // 46
Timestamp: ts.ToLegacyTimestamp(),
Deleted: false,
Txn: &txn.TxnMeta,
}).Size())
mValSize += 2
vKeySize := MVCCVersionTimestampSize // 12
vValSize := int64(len(value.RawBytes)) // 10
if disableSimpleValueEncoding {
vValSize += emptyMVCCValueHeaderSize // 17
}

expMS := enginepb.MVCCStats{
LastUpdateNanos: 1e9,
LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 68[+2][+7]
LiveCount: 1,
KeyBytes: mKeySize + vKeySize, // 2+12 =14
KeyCount: 1,
ValBytes: mValSize + vValSize, // (46[+2])+(10[+7]) = 54[+2][+7]
ValCount: 1,
IntentCount: 1,
LockCount: 1,
IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7]
GCBytesAge: 0,
}
assertEq(t, engine, "after put", aggMS, &expMS)

txn.Sequence++

// Annoyingly, the new meta value is actually a little larger thanks to the
// sequence number. Also since there was a write previously on the same
// transaction, the IntentHistory will add a few bytes to the metadata.
encValue, err := EncodeMVCCValue(MVCCValue{Value: value})
require.NoError(t, err)
m2ValSize := int64((&enginepb.MVCCMetadata{
Timestamp: ts.ToLegacyTimestamp(),
Deleted: true,
Txn: &txn.TxnMeta,
IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{
{Sequence: 0, Value: encValue},
},
}).Size())
expM2ValSize := 64
if disableSimpleValueEncoding {
expM2ValSize += int(emptyMVCCValueHeaderSize)
}
require.EqualValues(t, expM2ValSize, m2ValSize)

// Delete the value.
if _, err := MVCCDelete(ctx, engine, key, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil {
t.Fatal(err)
}

v2ValSize := int64(0) // tombstone
if disableSimpleValueEncoding {
v2ValSize += emptyMVCCValueHeaderSize // 7
}

expAggMS := enginepb.MVCCStats{
LastUpdateNanos: 1e9,
LiveBytes: 0,
LiveCount: 0,
KeyCount: 1,
ValCount: 1,
KeyBytes: mKeySize + vKeySize,
ValBytes: m2ValSize + v2ValSize,
LockAge: 0,
IntentCount: 1,
LockCount: 1,
IntentBytes: vKeySize + v2ValSize,
GCBytesAge: 0,
}

assertEq(t, engine, "after deleting", aggMS, &expAggMS)

// Now commit the value and roll back the delete.
txn.Status = roachpb.COMMITTED
txn.AddIgnoredSeqNumRange(enginepb.IgnoredSeqNumRange{Start: 1, End: 1})
if _, _, _, _, err := MVCCResolveWriteIntent(ctx, engine, aggMS,
roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}),
MVCCResolveWriteIntentOptions{},
); err != nil {
t.Fatal(err)
}

expAggMS = enginepb.MVCCStats{
LastUpdateNanos: 1e9,
LiveBytes: mKeySize + vKeySize + vValSize,
LiveCount: 1, // the key is live after the rollback
KeyCount: 1,
ValCount: 1,
KeyBytes: mKeySize + vKeySize,
ValBytes: vValSize,
GCBytesAge: 0,
}

assertEq(t, engine, "after committing", aggMS, &expAggMS)
}

// TestMVCCStatsDeleteRollbackPut exercises the case in which a deletion
// tombstone is written, then re-written by an intent, which is rolled back. We
// expect the stats to be adjusted correctly for the rolled back intent, and
// crucially, for the LiveBytes and LiveCount to be zero.
func TestMVCCStatsDeleteRollbackPut(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
engine := NewDefaultInMemForTesting()
defer engine.Close()

ctx := context.Background()
aggMS := &enginepb.MVCCStats{}
assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{})

key := roachpb.Key("a")
value := roachpb.MakeValueFromString("value")
ts := hlc.Timestamp{WallTime: 1e9}
txn := &roachpb.Transaction{
TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts},
ReadTimestamp: ts,
}

// Delete the value.
if _, err := MVCCDelete(ctx, engine, key, txn.ReadTimestamp, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil {
t.Fatal(err)
}

mKeySize := int64(mvccKey(key).EncodedSize()) // 2
mValSize := int64((&enginepb.MVCCMetadata{ // 46
Timestamp: ts.ToLegacyTimestamp(),
Deleted: true,
Txn: &txn.TxnMeta,
}).Size())
mValSize += 2
vKeySize := MVCCVersionTimestampSize // 12
vValSize := int64(0) // tombstone
if disableSimpleValueEncoding {
vValSize += emptyMVCCValueHeaderSize // 7
}

expMS := enginepb.MVCCStats{
LastUpdateNanos: 1e9,
LiveBytes: 0,
LiveCount: 0,
KeyBytes: mKeySize + vKeySize,
KeyCount: 1,
ValBytes: mValSize + vValSize,
ValCount: 1,
IntentCount: 1,
LockCount: 1,
IntentBytes: vKeySize + vValSize,
GCBytesAge: 0,
}
assertEq(t, engine, "after delete", aggMS, &expMS)

txn.Sequence++

// Annoyingly, the new meta value is actually a little larger thanks to the
// sequence number. Also since there was a write previously on the same
// transaction, the IntentHistory will add a few bytes to the metadata.
encValue, err := EncodeMVCCValue(MVCCValue{Value: roachpb.Value{RawBytes: []byte{}}})
require.NoError(t, err)
m2ValSize := int64((&enginepb.MVCCMetadata{
Timestamp: ts.ToLegacyTimestamp(),
Txn: &txn.TxnMeta,
Deleted: false,
IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{
{Sequence: 0, Value: encValue},
},
}).Size())
expM2ValSize := 54
if disableSimpleValueEncoding {
expM2ValSize += int(emptyMVCCValueHeaderSize)
}
require.EqualValues(t, expM2ValSize, m2ValSize)

// Put the value.
if err := MVCCPut(ctx, engine, key, ts, value, MVCCWriteOptions{Txn: txn, Stats: aggMS}); err != nil {
t.Fatal(err)
}

v2ValSize := int64(len(value.RawBytes))
if disableSimpleValueEncoding {
v2ValSize += emptyMVCCValueHeaderSize // 17
}

expAggMS := enginepb.MVCCStats{
LastUpdateNanos: 1e9,
LiveBytes: mKeySize + m2ValSize + vKeySize + v2ValSize,
LiveCount: 1,
KeyCount: 1,
ValCount: 1,
KeyBytes: mKeySize + vKeySize,
ValBytes: m2ValSize + v2ValSize,
LockAge: 0,
IntentCount: 1,
LockCount: 1,
IntentBytes: vKeySize + v2ValSize,
GCBytesAge: 0,
}

assertEq(t, engine, "after put", aggMS, &expAggMS)

// Now commit the value and roll back the put.
txn.Status = roachpb.COMMITTED
txn.AddIgnoredSeqNumRange(enginepb.IgnoredSeqNumRange{Start: 1, End: 1})
if _, _, _, _, err := MVCCResolveWriteIntent(ctx, engine, aggMS,
roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}),
MVCCResolveWriteIntentOptions{},
); err != nil {
t.Fatal(err)
}

expAggMS = enginepb.MVCCStats{
LastUpdateNanos: 1e9,
LiveBytes: 0,
LiveCount: 0, // the key is not live after the rollback
KeyCount: 1,
ValCount: 1,
KeyBytes: mKeySize + vKeySize,
ValBytes: vValSize,
GCBytesAge: 0,
}

assertEq(t, engine, "after committing", aggMS, &expAggMS)
}

var mvccStatsTests = []struct {
name string
fn func(Reader, roachpb.Key, roachpb.Key, int64) (enginepb.MVCCStats, error)
Expand Down Expand Up @@ -1587,6 +1842,9 @@ func (s *randomTest) step(t *testing.T) {
actName := s.actionNames[s.rng.Intn(len(s.actionNames))]

preTxn := s.Txn
if s.Txn != nil {
s.Txn.Sequence++
}
s.batch = s.internal.eng.NewBatch()
*s.MSDelta = enginepb.MVCCStats{}
ok, info := s.actions[actName](&s.state)
Expand Down Expand Up @@ -1837,6 +2095,18 @@ func TestMVCCStatsRandomized(t *testing.T) {
actions["Commit"] = func(s *state) (bool, string) {
return resolve(s, roachpb.COMMITTED)
}
actions["Rollback"] = func(s *state) (bool, string) {
if s.Txn != nil {
for i := 0; i < int(s.Txn.Sequence); i++ {
if s.rng.Intn(2) == 0 {
s.Txn.AddIgnoredSeqNumRange(enginepb.IgnoredSeqNumRange{Start: enginepb.TxnSeq(i), End: enginepb.TxnSeq(i)})
}
}
desc := fmt.Sprintf("ignored=%v", s.Txn.IgnoredSeqNums)
return true, desc
}
return false, ""
}
actions["Push"] = func(s *state) (bool, string) {
return resolve(s, roachpb.PENDING)
}
Expand Down
Loading

0 comments on commit 6c5489f

Please sign in to comment.