Skip to content

Commit

Permalink
Merge #69902
Browse files Browse the repository at this point in the history
69902: storage: add randomized test to trigger intent single deletion bug r=sumeerbhola a=sumeerbhola

The test uses the case of savepoint rollbacks to trigger the bug.

Informs #69891

Release justification: non-production code change
Release note: None

Co-authored-by: sumeerbhola <[email protected]>
  • Loading branch information
craig[bot] and sumeerbhola committed Oct 6, 2021
2 parents 04d6d93 + e60f063 commit c52d1c3
Showing 1 changed file with 154 additions and 0 deletions.
154 changes: 154 additions & 0 deletions pkg/storage/mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4267,6 +4267,160 @@ func TestRandomizedMVCCResolveWriteIntentRange(t *testing.T) {
}
}

// TestRandomizedSavepointRollbackAndIntentResolution is a randomized test
// that tries to confirm that rolling back savepoints and then putting again
// does not cause incorrectness when doing intent resolution. This would fail
// under the bug documented in #69891.
func TestRandomizedSavepointRollbackAndIntentResolution(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

seed := *seedFlag
debug := true
if seed < 0 {
seed = rand.Int63()
debug = false
}
// Else, seed is being specified to debug a failure.

fmt.Printf("seed: %d\n", seed)
rng := rand.New(rand.NewSource(seed))
ctx := context.Background()
eng, err := Open(context.Background(), InMemory(), func(cfg *engineConfig) error {
cfg.Opts.LBaseMaxBytes = int64(100 + rng.Intn(16384))
log.Infof(ctx, "lbase: %d", cfg.Opts.LBaseMaxBytes)
return nil
})
require.NoError(t, err)
defer eng.Close()

var seq enginepb.TxnSeq
var puts []putState
timestamps := []hlc.Timestamp{{WallTime: 100}, {WallTime: 200}, {WallTime: 300}}
keys := make(map[string]struct{})
// 100 keys, each written to twice by the txn.
for i := 0; i < 100; i++ {
var key []byte
for {
key = generateBytes(rng, 10, 20)
if _, ok := keys[string(key)]; !ok {
break
}
}
put := putState{
key: key,
}
for j := 0; j < 2; j++ {
val := generateBytes(rng, 20, 30)
put.values = append(put.values, roachpb.Value{RawBytes: val})
put.seqs = append(put.seqs, seq)
seq++
put.writeTS = append(put.writeTS, timestamps[j])
}
puts = append(puts, put)
}
sort.Slice(puts, func(i, j int) bool {
return puts[i].key.Compare(puts[j].key) < 0
})
txn := *txn1
txn.ReadTimestamp = timestamps[0]
txn.MinTimestamp = txn.ReadTimestamp
writeToEngine(t, eng, puts, &txn, debug)
// The two SET calls for writing the intent are collapsed down to L6.
require.NoError(t, eng.Flush())
require.NoError(t, eng.Compact())

txn.WriteTimestamp = timestamps[1]
txn.Sequence = seq
ignoredSeqNums := []enginepb.IgnoredSeqNumRange{{Start: 0, End: seq - 1}}
lu := roachpb.LockUpdate{
Span: roachpb.Span{
Key: puts[0].key,
EndKey: roachpb.BytesNext(puts[len(puts)-1].key),
},
Txn: txn.TxnMeta,
Status: roachpb.PENDING,
IgnoredSeqNums: ignoredSeqNums,
}
if debug {
log.Infof(ctx, "LockUpdate: %s", lu.String())
}
// All the writes are ignored, so DEL is written for the intent. These
// should be buffered in the memtable.
_, _, err = MVCCResolveWriteIntentRange(ctx, eng, nil, lu, 0, true)
require.NoError(t, err)
{
iter := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind,
IterOptions{LowerBound: lu.Span.Key, UpperBound: lu.Span.EndKey})
defer iter.Close()
iter.SeekGE(MVCCKey{Key: lu.Span.Key})
valid, err := iter.Valid()
require.NoError(t, err)
require.False(t, valid)
}
// Do another put for all these keys. These will also be in the memtable.
for i := 0; i < 100; i++ {
puts[i].values = append(puts[i].values[:0],
roachpb.Value{RawBytes: generateBytes(rng, 2, 3)})
puts[i].seqs = append(puts[i].seqs[:0], seq)
seq++
puts[i].writeTS = append(puts[i].writeTS[:0], timestamps[2])
}
writeToEngine(t, eng, puts, &txn, debug)
// Flush of the memtable will collapse DEL=>SET into SETWITHDEL.
require.NoError(t, eng.Flush())

// Commit or abort the txn, so that we eventually get
// SET=>SETWITHDEL=>SINGLEDEL for the intents.
txn.WriteTimestamp = timestamps[2]
txn.Sequence = seq
lu.Txn = txn.TxnMeta
lu.Status = []roachpb.TransactionStatus{roachpb.COMMITTED, roachpb.ABORTED}[rng.Intn(2)]
if debug {
log.Infof(ctx, "LockUpdate: %s", lu.String())
}
_, _, err = MVCCResolveWriteIntentRange(ctx, eng, nil, lu, 0, false)
require.NoError(t, err)
// Compact the engine so that SINGLEDEL consumes the SETWITHDEL, becoming a
// DEL.
require.NoError(t, eng.Compact())
iter := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind,
IterOptions{LowerBound: lu.Span.Key, UpperBound: lu.Span.EndKey})
defer iter.Close()
iter.SeekGE(MVCCKey{Key: lu.Span.Key})
if lu.Status == roachpb.COMMITTED {
i := 0
for {
valid, err := iter.Valid()
require.NoError(t, err)
if !valid {
break
}
i++
// Expect only the committed values.
require.Equal(t, timestamps[2], iter.UnsafeKey().Timestamp)
iter.Next()
}
require.Equal(t, 100, i)
} else {
// ABORTED. Nothing to iterate over.
valid, err := iter.Valid()
require.NoError(t, err)
// The correct behavior is !valid. But if there is a bug, the
// intentInterleavingIter does not always expose its error immediately (in
// this case the error would an intent without a provisional value), so we
// step it forward once.
if valid {
iter.Next()
_, err = iter.Valid()
require.NoError(t, err)
// Should fail on previous statement, but this whole path is incorrect,
// so fail here.
t.Fatal(t, "iter is valid")
}
}
}

func TestValidSplitKeys(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down

0 comments on commit c52d1c3

Please sign in to comment.