Skip to content

Commit

Permalink
storage: disable simple MVCC value encoding scheme under metamorphic …
Browse files Browse the repository at this point in the history
…testing

This commit adds a metamorphic knob that randomly disables the simple MVCC value
encoding scheme. Doing so ensures that code which interacts with encoded MVCC
values does not mistake these values for encoded roachpb values. This could take
place in two different ways:
1. broken code could assign an encoded MVCC value directly to a roachpb.Value's
   `RawBytes` field. This typically caused the test to fail with an error.
2. broken code could assume that a non-zero-length value was not a tombstone.
   This caused tests to fail in more obscure ways.

The commit then fixes broken tests in one of three ways:
- it fixes incorrect assumptions about the MVCC value encoding being equivalent
  to the roachpb value encoding.
- it updates a few important tests (mostly related to MVCC stats) to work with
  and without the simple encoding.
- it skips a few unimportant tests when the simple encoding scheme is disabled.
  • Loading branch information
nvanbenschoten committed May 9, 2022
1 parent 52de4f9 commit 720c8cf
Show file tree
Hide file tree
Showing 17 changed files with 298 additions and 92 deletions.
6 changes: 5 additions & 1 deletion pkg/ccl/backupccl/restore_data_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,11 @@ func slurpSSTablesLatestKey(
} else if !ok || !it.UnsafeKey().Less(end) {
break
}
kvs = append(kvs, storage.MVCCKeyValue{Key: it.Key(), Value: it.Value()})
val, err := storage.DecodeMVCCValue(it.Value())
if err != nil {
t.Fatal(err)
}
kvs = append(kvs, storage.MVCCKeyValue{Key: it.Key(), Value: val.Value.RawBytes})
}
return kvs
}
Expand Down
12 changes: 9 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -772,8 +772,10 @@ func TestEvalAddSSTable(t *testing.T) {
ts := iter.Key().Timestamp.WallTime
var value []byte
if iter.Key().IsValue() {
if len(iter.Value()) > 0 {
value, err = roachpb.Value{RawBytes: iter.Value()}.GetBytes()
mvccVal, err := storage.DecodeMVCCValue(iter.Value())
require.NoError(t, err)
if !mvccVal.IsTombstone() {
value, err = mvccVal.Value.GetBytes()
require.NoError(t, err)
}
} else {
Expand Down Expand Up @@ -1091,6 +1093,7 @@ func runTestDBAddSSTable(
func TestAddSSTableMVCCStats(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
storage.SkipIfSimpleValueEncodingDisabled(t)

const max = 1 << 10
ctx := context.Background()
Expand Down Expand Up @@ -1209,6 +1212,7 @@ func TestAddSSTableMVCCStats(t *testing.T) {
func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
storage.SkipIfSimpleValueEncodingDisabled(t)

ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
Expand Down Expand Up @@ -1467,7 +1471,9 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsClosedTS(t *testing.T)
require.NoError(t, err)
require.Len(t, kvs, 1)
require.Equal(t, storage.MVCCKey{Key: roachpb.Key("key"), Timestamp: writeTS}, kvs[0].Key)
v, err := roachpb.Value{RawBytes: kvs[0].Value}.GetBytes()
mvccVal, err := storage.DecodeMVCCValue(kvs[0].Value)
require.NoError(t, err)
v, err := mvccVal.Value.GetBytes()
require.NoError(t, err)
require.Equal(t, "sst", string(v))
}
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/batcheval/cmd_export_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -674,6 +674,7 @@ func assertEqualKVs(

func TestRandomKeyAndTimestampExport(t *testing.T) {
defer leaktest.AfterTest(t)()
storage.SkipIfSimpleValueEncodingDisabled(t)

ctx := context.Background()

Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/consistency_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -234,6 +234,9 @@ func TestCheckConsistencyInconsistent(t *testing.T) {
// good to make sure we're overly redacting said diff.
defer log.TestingSetRedactable(true)()

// Test expects simple MVCC value encoding.
storage.SkipIfSimpleValueEncodingDisabled(t)

// Test uses sticky registry to have persistent pebble state that could
// be analyzed for existence of snapshots and to verify snapshot content
// after failures.
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/gc/gc_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
// engine and then validating the state of the iterator as it iterates that
// data.
func TestGCIterator(t *testing.T) {
storage.SkipIfSimpleValueEncodingDisabled(t)
// dataItem represents a version in the storage engine and optionally a
// corresponding transaction which will make the MVCCKeyValue an intent.
type dataItem struct {
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/mvcc_gc_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -356,6 +356,7 @@ func (cws *cachedWriteSimulator) shouldQueue(
func TestMVCCGCQueueMakeGCScoreRealistic(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
storage.SkipIfSimpleValueEncodingDisabled(t)

cws := newCachedWriteSimulator(t)

Expand Down Expand Up @@ -462,6 +463,7 @@ func TestMVCCGCQueueMakeGCScoreRealistic(t *testing.T) {
func TestMVCCGCQueueProcess(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
storage.SkipIfSimpleValueEncodingDisabled(t)
ctx := context.Background()
tc := testContext{}
stopper := stop.NewStopper()
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6247,6 +6247,7 @@ func verifyRangeStats(
func TestRangeStatsComputation(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
storage.SkipIfSimpleValueEncodingDisabled(t)
ctx := context.Background()
tc := testContext{}
stopper := stop.NewStopper()
Expand Down
12 changes: 8 additions & 4 deletions pkg/sql/row/fetcher_mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,10 +57,14 @@ func slurpUserDataKVs(t testing.TB, e storage.Engine) []roachpb.KeyValue {
if !it.UnsafeKey().IsValue() {
return errors.Errorf("found intent key %v", it.UnsafeKey())
}
kvs = append(kvs, roachpb.KeyValue{
Key: it.Key().Key,
Value: roachpb.Value{RawBytes: it.Value(), Timestamp: it.UnsafeKey().Timestamp},
})
mvccValue, err := storage.DecodeMVCCValue(it.Value())
if err != nil {
t.Fatal(err)
}
value := mvccValue.Value
value.Timestamp = it.UnsafeKey().Timestamp
kv := roachpb.KeyValue{Key: it.Key().Key, Value: value}
kvs = append(kvs, kv)
}
return nil
})
Expand Down
1 change: 1 addition & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ go_library(
"//pkg/settings/cluster",
"//pkg/storage/enginepb",
"//pkg/storage/fs",
"//pkg/testutils/skip",
"//pkg/util",
"//pkg/util/bufalloc",
"//pkg/util/encoding",
Expand Down
1 change: 1 addition & 0 deletions pkg/storage/mvcc_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,7 @@ import (
func TestMVCCHistories(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
SkipIfSimpleValueEncodingDisabled(t)

ctx := context.Background()

Expand Down
7 changes: 7 additions & 0 deletions pkg/storage/mvcc_incremental_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -497,6 +497,7 @@ func assertEqualKVs(
func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
SkipIfSimpleValueEncodingDisabled(t)
ctx := context.Background()

var (
Expand Down Expand Up @@ -634,6 +635,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) {
func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
SkipIfSimpleValueEncodingDisabled(t)
ctx := context.Background()

var (
Expand Down Expand Up @@ -764,6 +766,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) {
func TestMVCCIncrementalIteratorInlinePolicy(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
SkipIfSimpleValueEncodingDisabled(t)
ctx := context.Background()

var (
Expand Down Expand Up @@ -858,6 +861,7 @@ func TestMVCCIncrementalIteratorInlinePolicy(t *testing.T) {
func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
SkipIfSimpleValueEncodingDisabled(t)
ctx := context.Background()

var (
Expand Down Expand Up @@ -1038,6 +1042,7 @@ func expectIntent(t *testing.T, iter SimpleMVCCIterator, intent roachpb.Intent)
func TestMVCCIncrementalIterator(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
SkipIfSimpleValueEncodingDisabled(t)
ctx := context.Background()

var (
Expand Down Expand Up @@ -1318,6 +1323,7 @@ func TestMVCCIncrementalIteratorIntentRewrittenConcurrently(t *testing.T) {
func TestMVCCIncrementalIteratorIntentDeletion(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
SkipIfSimpleValueEncodingDisabled(t)

txn := func(key roachpb.Key, ts hlc.Timestamp) *roachpb.Transaction {
return &roachpb.Transaction{
Expand Down Expand Up @@ -1535,6 +1541,7 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) {
func TestMVCCIterateTimeBound(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
SkipIfSimpleValueEncodingDisabled(t)

dir, cleanupFn := testutils.TempDir(t)
defer cleanupFn()
Expand Down
Loading

0 comments on commit 720c8cf

Please sign in to comment.