diff --git a/build/bazelutil/check.sh b/build/bazelutil/check.sh index 82125e46e018..d00f013520b9 100755 --- a/build/bazelutil/check.sh +++ b/build/bazelutil/check.sh @@ -57,6 +57,8 @@ pkg/cmd/prereqs/BUILD.bazel EXISTING_CRDB_TEST_BUILD_CONSTRAINTS=" pkg/util/buildutil/crdb_test_off.go://go:build !crdb_test || crdb_test_off pkg/util/buildutil/crdb_test_on.go://go:build crdb_test && !crdb_test_off +pkg/kv/kvnemesis/kvnemesisutil/crdb_test_off.go://go:build !crdb_test || crdb_test_off +pkg/kv/kvnemesis/kvnemesisutil/crdb_test_on.go://go:build crdb_test && !crdb_test_off " if [ -z "${COCKROACH_BAZEL_CHECK_FAST:-}" ]; then diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index b17df508d36c..5d2f2f0545ab 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1118,6 +1118,7 @@ GO_TARGETS = [ "//pkg/kv/kvclient/rangefeed:rangefeed_test", "//pkg/kv/kvclient/rangestats:rangestats", "//pkg/kv/kvclient:kvclient", + "//pkg/kv/kvnemesis/kvnemesisutil:kvnemesisutil", "//pkg/kv/kvnemesis:kvnemesis", "//pkg/kv/kvnemesis:kvnemesis_test", "//pkg/kv/kvprober:kvprober", @@ -2477,6 +2478,7 @@ GET_X_DATA_TARGETS = [ "//pkg/kv/kvclient/rangefeed/rangefeedcache:get_x_data", "//pkg/kv/kvclient/rangestats:get_x_data", "//pkg/kv/kvnemesis:get_x_data", + "//pkg/kv/kvnemesis/kvnemesisutil:get_x_data", "//pkg/kv/kvprober:get_x_data", "//pkg/kv/kvserver:get_x_data", "//pkg/kv/kvserver/abortspan:get_x_data", diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index d6ce4bbdcef0..0bd8df8c6aa9 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -78,6 +78,11 @@ func (b *Batch) ApproximateMutationBytes() int { return b.approxMutationReqBytes } +// Requests exposes the requests stashed in the batch thus far. +func (b *Batch) Requests() []roachpb.RequestUnion { + return b.reqs +} + // RawResponse returns the BatchResponse which was the result of a successful // execution of the batch, and nil otherwise. func (b *Batch) RawResponse() *roachpb.BatchResponse { diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index fd4fd9c8b61d..b72ac9b30a60 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -346,6 +346,8 @@ type DistSender struct { // LatencyFunc is used to estimate the latency to other nodes. latencyFunc LatencyFunc + onRangeSpanningNonTxnalBatch func(ba *roachpb.BatchRequest) *roachpb.Error + // locality is the description of the topography of the server on which the // DistSender is running. It is used to estimate the latency to other nodes // in the absence of a latency function. @@ -499,6 +501,11 @@ func NewDistSender(cfg DistSenderConfig) *DistSender { } else { ds.latencyFunc = ds.rpcContext.RemoteClocks.Latency } + + if cfg.TestingKnobs.OnRangeSpanningNonTxnalBatch != nil { + ds.onRangeSpanningNonTxnalBatch = cfg.TestingKnobs.OnRangeSpanningNonTxnalBatch + } + return ds } @@ -1256,8 +1263,23 @@ func (ds *DistSender) divideAndSendBatchToRanges( // If there's no transaction and ba spans ranges, possibly re-run as part of // a transaction for consistency. The case where we don't need to re-run is // if the read consistency is not required. - if ba.Txn == nil && ba.IsTransactional() && ba.ReadConsistency == roachpb.CONSISTENT { - return nil, roachpb.NewError(&roachpb.OpRequiresTxnError{}) + if ba.Txn == nil { + if ba.IsTransactional() && ba.ReadConsistency == roachpb.CONSISTENT { + // NB: this check isn't quite right. We enter this if there's *any* transactional + // request here, but there could be a mix (for example a DeleteRangeUsingTombstone + // and a Put). DeleteRangeUsingTombstone gets split non-transactionally across + // batches, so that is probably what we would want for the mixed batch as well. + // + // Revisit if this ever becomes something we actually want to do, for now such + // batches will fail (re-wrapped in txn and then fail because some requests + // don't support txns). + return nil, roachpb.NewError(&roachpb.OpRequiresTxnError{}) + } + if fn := ds.onRangeSpanningNonTxnalBatch; fn != nil { + if pErr := fn(ba); pErr != nil { + return nil, pErr + } + } } // If the batch contains a non-parallel commit EndTxn and spans ranges then // we want the caller to come again with the EndTxn in a separate diff --git a/pkg/kv/kvclient/kvcoord/testing_knobs.go b/pkg/kv/kvclient/kvcoord/testing_knobs.go index 9674aa6df967..1da19d7a75ed 100644 --- a/pkg/kv/kvclient/kvcoord/testing_knobs.go +++ b/pkg/kv/kvclient/kvcoord/testing_knobs.go @@ -10,7 +10,10 @@ package kvcoord -import "github.com/cockroachdb/cockroach/pkg/base" +import ( + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) // ClientTestingKnobs contains testing options that dictate the behavior // of the key-value client. @@ -52,6 +55,12 @@ type ClientTestingKnobs struct { // CommitWaitFilter allows tests to instrument the beginning of a transaction // commit wait sleep. CommitWaitFilter func() + + // OnRangeSpanningNonTxnalBatch is invoked whenever DistSender attempts to split + // a non-transactional batch across a range boundary. The method may inject an + // error which, if non-nil, becomes the result of the batch. Otherwise, execution + // continues. + OnRangeSpanningNonTxnalBatch func(ba *roachpb.BatchRequest) *roachpb.Error } var _ base.ModuleTestingKnobs = &ClientTestingKnobs{} diff --git a/pkg/kv/kvnemesis/BUILD.bazel b/pkg/kv/kvnemesis/BUILD.bazel index a094482ca25e..b71bc8c6a6d4 100644 --- a/pkg/kv/kvnemesis/BUILD.bazel +++ b/pkg/kv/kvnemesis/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "generator.go", "kvnemesis.go", "operations.go", + "seq_tracker.go", "validator.go", "watcher.go", ], @@ -24,11 +25,13 @@ go_library( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvnemesis/kvnemesisutil", "//pkg/kv/kvserver", "//pkg/kv/kvserver/liveness", "//pkg/roachpb", "//pkg/sql/catalog/bootstrap", "//pkg/storage", + "//pkg/storage/enginepb", "//pkg/util/bufalloc", "//pkg/util/ctxgroup", "//pkg/util/encoding", @@ -41,7 +44,6 @@ go_library( "//pkg/util/timeutil", "//pkg/util/tracing", "//pkg/util/tracing/tracingpb", - "//pkg/util/uuid", "@com_github_cockroachdb_cockroach_go_v2//crdb", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_pebble//:pebble", @@ -63,21 +65,28 @@ go_test( "validator_test.go", ], args = ["-test.timeout=55s"], + data = glob(["testdata/**"]), embed = [":kvnemesis"], deps = [ "//pkg/base", "//pkg/config/zonepb", "//pkg/kv", + "//pkg/kv/kvclient/kvcoord", + "//pkg/kv/kvnemesis/kvnemesisutil", "//pkg/kv/kvserver", "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", "//pkg/storage", + "//pkg/storage/enginepb", + "//pkg/testutils", + "//pkg/testutils/echotest", "//pkg/testutils/serverutils", "//pkg/testutils/skip", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", + "//pkg/util/buildutil", "//pkg/util/envutil", "//pkg/util/hlc", "//pkg/util/leaktest", diff --git a/pkg/kv/kvnemesis/applier.go b/pkg/kv/kvnemesis/applier.go index a6aae4df7642..db0e3717e79d 100644 --- a/pkg/kv/kvnemesis/applier.go +++ b/pkg/kv/kvnemesis/applier.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -76,6 +77,37 @@ func (a *Applier) getNextDBRoundRobin() (*kv.DB, int32) { return a.dbs[dbIdx], int32(dbIdx) } +// Sentinel errors. +var ( + errOmitted = errors.New("omitted") + errClosureTxnRollback = errors.New("rollback") + errDelRangeUsingTombstoneStraddlesRangeBoundary = errors.New("DeleteRangeUsingTombstone can not straddle range boundary") +) + +func exceptOmitted(err error) bool { // true if errOmitted + return errors.Is(err, errOmitted) +} + +func exceptRollback(err error) bool { // true if intentional txn rollback + return errors.Is(err, errClosureTxnRollback) +} + +func exceptRetry(err error) bool { // true if retry error + return errors.HasInterface(err, (*roachpb.ClientVisibleRetryError)(nil)) +} + +func exceptUnhandledRetry(err error) bool { + return errors.HasType(err, (*roachpb.UnhandledRetryableError)(nil)) +} + +func exceptAmbiguous(err error) bool { // true if ambiguous result + return errors.HasInterface(err, (*roachpb.ClientVisibleAmbiguousError)(nil)) +} + +func exceptDelRangeUsingTombstoneStraddlesRangeBoundary(err error) bool { + return errors.Is(err, errDelRangeUsingTombstoneStraddlesRangeBoundary) +} + func applyOp(ctx context.Context, env *Env, db *kv.DB, op *Operation) { switch o := op.GetValue().(type) { case *GetOperation, @@ -83,8 +115,9 @@ func applyOp(ctx context.Context, env *Env, db *kv.DB, op *Operation) { *ScanOperation, *BatchOperation, *DeleteOperation, - *DeleteRangeOperation: - applyClientOp(ctx, db, op, false /* inTxn */) + *DeleteRangeOperation, + *DeleteRangeUsingTombstoneOperation: + applyClientOp(ctx, db, op, false) case *SplitOperation: err := db.AdminSplit(ctx, o.Key, hlc.MaxTimestamp, roachpb.AdminSplitRequest_INGESTION) o.Result = resultInit(ctx, err) @@ -94,7 +127,6 @@ func applyOp(ctx context.Context, env *Env, db *kv.DB, op *Operation) { case *ChangeReplicasOperation: desc := getRangeDesc(ctx, o.Key, db) _, err := db.AdminChangeReplicas(ctx, o.Key, desc, o.Changes) - // TODO(dan): Save returned desc? o.Result = resultInit(ctx, err) case *TransferLeaseOperation: err := db.AdminTransferLease(ctx, o.Key, o.Target) @@ -117,18 +149,32 @@ func applyOp(ctx context.Context, env *Env, db *kv.DB, op *Operation) { retryOnAbort.Next() } savedTxn = txn - for i := range o.Ops { - op := &o.Ops[i] - op.Result().Reset() // in case we're a retry - applyClientOp(ctx, txn, op, true /* inTxn */) - // The KV api disallows use of a txn after an operation on it errors. - if r := op.Result(); r.Type == ResultType_Error { - return errors.DecodeError(ctx, *r.Err) + { + var err error + for i := range o.Ops { + op := &o.Ops[i] + op.Result().Reset() // in case we're a retry + if err != nil { + // If a previous op failed, mark this op as never invoked. We need + // to do this because we want, as an invariant, to have marked all + // operations as either failed or succeeded. + *op.Result() = resultInit(ctx, errOmitted) + continue + } + + applyClientOp(ctx, txn, op, true) + // The KV api disallows use of a txn after an operation on it errors. + if r := op.Result(); r.Type == ResultType_Error { + err = errors.DecodeError(ctx, *r.Err) + } + } + if err != nil { + return err } } if o.CommitInBatch != nil { b := txn.NewBatch() - applyBatchOp(ctx, b, txn.CommitInBatch, o.CommitInBatch, true) + applyBatchOp(ctx, b, txn.CommitInBatch, o.CommitInBatch) // The KV api disallows use of a txn after an operation on it errors. if r := o.CommitInBatch.Result; r.Type == ResultType_Error { return errors.DecodeError(ctx, *r.Err) @@ -138,7 +184,7 @@ func applyOp(ctx context.Context, env *Env, db *kv.DB, op *Operation) { case ClosureTxnType_Commit: return nil case ClosureTxnType_Rollback: - return errors.New("rollback") + return errClosureTxnRollback default: panic(errors.AssertionFailedf(`unknown closure txn type: %s`, o.Type)) } @@ -221,7 +267,8 @@ func applyClientOp(ctx context.Context, db clientI, op *Operation, inTxn bool) { } case *PutOperation: _, ts, err := dbRunWithResultAndTimestamp(ctx, db, func(b *kv.Batch) { - b.Put(o.Key, o.Value) + b.Put(o.Key, o.Value()) + setLastReqSeq(b, o.Seq) }) o.Result = resultInit(ctx, err) if err != nil { @@ -250,13 +297,14 @@ func applyClientOp(ctx context.Context, db clientI, op *Operation, inTxn bool) { o.Result.Values = make([]KeyValue, len(kvs)) for i, kv := range kvs { o.Result.Values[i] = KeyValue{ - Key: []byte(kv.Key), + Key: kv.Key, Value: kv.Value.RawBytes, } } case *DeleteOperation: res, ts, err := dbRunWithResultAndTimestamp(ctx, db, func(b *kv.Batch) { b.Del(o.Key) + setLastReqSeq(b, o.Seq) }) o.Result = resultInit(ctx, err) if err != nil { @@ -270,11 +318,9 @@ func applyClientOp(ctx context.Context, db clientI, op *Operation, inTxn bool) { o.Result.Keys[i] = deletedKey } case *DeleteRangeOperation: - if !inTxn { - panic(errors.AssertionFailedf(`non-transactional DelRange operations currently unsupported`)) - } res, ts, err := dbRunWithResultAndTimestamp(ctx, db, func(b *kv.Batch) { b.DelRange(o.Key, o.EndKey, true /* returnKeys */) + setLastReqSeq(b, o.Seq) }) o.Result = resultInit(ctx, err) if err != nil { @@ -287,20 +333,34 @@ func applyClientOp(ctx context.Context, db clientI, op *Operation, inTxn bool) { for i, deletedKey := range deletedKeys { o.Result.Keys[i] = deletedKey } + case *DeleteRangeUsingTombstoneOperation: + _, ts, err := dbRunWithResultAndTimestamp(ctx, db, func(b *kv.Batch) { + b.DelRangeUsingTombstone(o.Key, o.EndKey) + setLastReqSeq(b, o.Seq) + }) + o.Result = resultInit(ctx, err) + if err != nil { + return + } + o.Result.OptionalTimestamp = ts case *BatchOperation: b := &kv.Batch{} - applyBatchOp(ctx, b, db.Run, o, inTxn) + applyBatchOp(ctx, b, db.Run, o) default: panic(errors.AssertionFailedf(`unknown batch operation type: %T %v`, o, o)) } } +func setLastReqSeq(b *kv.Batch, seq kvnemesisutil.Seq) { + sl := b.Requests() + req := sl[len(sl)-1].GetInner() + h := req.Header() + h.KVNemesisSeq.Set(seq) + req.SetHeader(h) +} + func applyBatchOp( - ctx context.Context, - b *kv.Batch, - run func(context.Context, *kv.Batch) error, - o *BatchOperation, - inTxn bool, + ctx context.Context, b *kv.Batch, run func(context.Context, *kv.Batch) error, o *BatchOperation, ) { for i := range o.Ops { switch subO := o.Ops[i].GetValue().(type) { @@ -311,7 +371,8 @@ func applyBatchOp( b.Get(subO.Key) } case *PutOperation: - b.Put(subO.Key, subO.Value) + b.Put(subO.Key, subO.Value()) + setLastReqSeq(b, subO.Seq) case *ScanOperation: if subO.Reverse && subO.ForUpdate { b.ReverseScanForUpdate(subO.Key, subO.EndKey) @@ -324,11 +385,13 @@ func applyBatchOp( } case *DeleteOperation: b.Del(subO.Key) + setLastReqSeq(b, subO.Seq) case *DeleteRangeOperation: - if !inTxn { - panic(errors.AssertionFailedf(`non-transactional batch DelRange operations currently unsupported`)) - } b.DelRange(subO.Key, subO.EndKey, true /* returnKeys */) + setLastReqSeq(b, subO.Seq) + case *DeleteRangeUsingTombstoneOperation: + b.DelRangeUsingTombstone(subO.Key, subO.EndKey) + setLastReqSeq(b, subO.Seq) default: panic(errors.AssertionFailedf(`unknown batch operation type: %T %v`, subO, subO)) } @@ -384,6 +447,8 @@ func applyBatchOp( subO.Result.Keys[j] = key } } + case *DeleteRangeUsingTombstoneOperation: + subO.Result = resultInit(ctx, err) default: panic(errors.AssertionFailedf(`unknown batch operation type: %T %v`, subO, subO)) } diff --git a/pkg/kv/kvnemesis/applier_test.go b/pkg/kv/kvnemesis/applier_test.go index 6b68eb12ff10..20cad70642de 100644 --- a/pkg/kv/kvnemesis/applier_test.go +++ b/pkg/kv/kvnemesis/applier_test.go @@ -13,17 +13,17 @@ package kvnemesis import ( "context" gosql "database/sql" - "fmt" "regexp" "strings" "testing" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "google.golang.org/protobuf/proto" ) @@ -37,175 +37,156 @@ func TestApplier(t *testing.T) { defer tc.Stopper().Stop(ctx) db := tc.Server(0).DB() sqlDB := tc.ServerConn(0) - env := &Env{sqlDBs: []*gosql.DB{sqlDB}} + env := &Env{SQLDBs: []*gosql.DB{sqlDB}} - a := MakeApplier(env, db, db) - check := func(t *testing.T, s Step, expected string) { - t.Helper() - _ /* trace */, err := a.Apply(ctx, &s) - require.NoError(t, err) - actual := s.String() - // Trim out the txn stuff. It has things like timestamps in it that are not - // stable from run to run. - actual = regexp.MustCompile(` // nil txnpb:\(.*\)`).ReplaceAllString(actual, ` // nil txnpb:(...)`) - assert.Equal(t, strings.TrimSpace(expected), strings.TrimSpace(actual)) - } - checkErr := func(t *testing.T, s Step, expected string) { - t.Helper() - cancelledCtx, cancel := context.WithCancel(context.Background()) - cancel() - _ /* trace */, err := a.Apply(cancelledCtx, &s) - require.NoError(t, err) - actual := s.String() - // Trim out context canceled location, which can be non-deterministic. - // The wrapped string around the context canceled error depends on where - // the context cancellation was noticed. - actual = regexp.MustCompile(` aborted .*: context canceled`).ReplaceAllString(actual, ` context canceled`) - assert.Equal(t, strings.TrimSpace(expected), strings.TrimSpace(actual)) + type testCase struct { + name string + step Step } - checkPanics := func(t *testing.T, s Step, expectedPanic string) { - t.Helper() - _ /* trace */, err := a.Apply(ctx, &s) - require.EqualError(t, err, fmt.Sprintf("panic applying step %s: %v", s, expectedPanic)) + a := MakeApplier(env, db, db) + + tests := []testCase{ + { + "get", step(get(k1)), + }, + { + "scan", step(scan(k1, k3)), + }, + { + "put", step(put(k1, 1)), + }, + { + "get-for-update", step(getForUpdate(k1)), + }, + { + "scan-for-update", step(scanForUpdate(k1, k3)), + }, + { + "batch", step(batch(put(k1, 21), delRange(k2, k3, 22))), + }, + { + + "rscan", step(reverseScan(k1, k3)), + }, + { + "rscan-for-update", step(reverseScanForUpdate(k1, k2)), + }, + { + "del", step(del(k2, 1)), + }, + { + "delrange", step(delRange(k1, k3, 6)), + }, + { + "txn-delrange", step(closureTxn(ClosureTxnType_Commit, delRange(k2, k4, 1))), + }, + { + "get-err", step(get(k1)), + }, + { + "put-err", step(put(k1, 1)), + }, + { + "scan-for-update-err", step(scanForUpdate(k1, k3)), + }, + { + "rscan-err", step(reverseScan(k1, k3)), + }, + { + "rscan-for-update-err", step(reverseScanForUpdate(k1, k3)), + }, + { + "del-err", step(del(k2, 1)), + }, + { + "delrange-err", step(delRange(k2, k3, 12)), + }, + { + "txn-err", step(closureTxn(ClosureTxnType_Commit, delRange(k2, k4, 1))), + }, + { + "batch-mixed", step(batch(put(k2, 2), get(k1), del(k2, 1), del(k3, 1), scan(k1, k3), reverseScanForUpdate(k1, k5))), + }, + { + "batch-mixed-err", step(batch(put(k2, 2), getForUpdate(k1), scanForUpdate(k1, k3), reverseScan(k1, k3))), + }, + { + "txn-commit-mixed", step(closureTxn(ClosureTxnType_Commit, put(k5, 5), batch(put(k6, 6), delRange(k3, k5, 1)))), + }, + { + "txn-commit-batch", step(closureTxnCommitInBatch(opSlice(get(k1), put(k6, 6)), put(k5, 5))), + }, + { + "txn-rollback", step(closureTxn(ClosureTxnType_Rollback, put(k5, 5))), + }, + { + "txn-error", step(closureTxn(ClosureTxnType_Rollback, put(k5, 5))), + }, + { + "split", step(split(k2)), + }, + { + "merge", step(merge(k1)), // NB: this undoes the split at k2 + }, + { + "split-again", step(split(k2)), + }, + { + "merge-again", step(merge(k1)), // ditto + }, + { + "transfer", step(transferLease(k6, 1)), + }, + { + "transfer-again", step(transferLease(k6, 1)), + }, + { + "zcfg", step(changeZone(ChangeZoneType_ToggleGlobalReads)), + }, + { + "zcfg-again", step(changeZone(ChangeZoneType_ToggleGlobalReads)), + }, } - // Basic operations - check(t, step(get(`a`)), `db0.Get(ctx, "a") // (nil, nil)`) - check(t, step(scan(`a`, `c`)), `db1.Scan(ctx, "a", "c", 0) // ([], nil)`) - - check(t, step(put(`a`, `1`)), `db0.Put(ctx, "a", 1) // nil`) - check(t, step(getForUpdate(`a`)), `db1.GetForUpdate(ctx, "a") // ("1", nil)`) - check(t, step(scanForUpdate(`a`, `c`)), `db0.ScanForUpdate(ctx, "a", "c", 0) // (["a":"1"], nil)`) - - check(t, step(put(`b`, `2`)), `db1.Put(ctx, "b", 2) // nil`) - check(t, step(get(`b`)), `db0.Get(ctx, "b") // ("2", nil)`) - check(t, step(scan(`a`, `c`)), `db1.Scan(ctx, "a", "c", 0) // (["a":"1", "b":"2"], nil)`) - - check(t, step(reverseScan(`a`, `c`)), `db0.ReverseScan(ctx, "a", "c", 0) // (["b":"2", "a":"1"], nil)`) - check(t, step(reverseScanForUpdate(`a`, `b`)), `db1.ReverseScanForUpdate(ctx, "a", "b", 0) // (["a":"1"], nil)`) - - check(t, step(del(`b`)), `db0.Del(ctx, "b")`) - check(t, step(get(`b`)), `db1.Get(ctx, "b") // (nil, nil)`) - - check(t, step(put(`c`, `3`)), `db0.Put(ctx, "c", 3) // nil`) - check(t, step(put(`d`, `4`)), `db1.Put(ctx, "d", 4) // nil`) - - check(t, step(del(`c`)), `db0.Del(ctx, "c")`) - check(t, step(scan(`a`, `e`)), `db1.Scan(ctx, "a", "e", 0) // (["a":"1", "d":"4"], nil)`) - - check(t, step(put(`c`, `5`)), `db0.Put(ctx, "c", 5) // nil`) - check(t, step(closureTxn(ClosureTxnType_Commit, delRange(`b`, `d`))), ` -db1.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - txn.DelRange(ctx, "b", "d", true) // (["c"], nil) - return nil -}) // nil txnpb:(...) - `) - - checkErr(t, step(get(`a`)), `db0.Get(ctx, "a") // (nil, context canceled)`) - checkErr(t, step(put(`a`, `1`)), `db1.Put(ctx, "a", 1) // context canceled`) - - checkErr(t, step(scanForUpdate(`a`, `c`)), `db0.ScanForUpdate(ctx, "a", "c", 0) // (nil, context canceled)`) - checkErr(t, step(reverseScan(`a`, `c`)), `db1.ReverseScan(ctx, "a", "c", 0) // (nil, context canceled)`) - - checkErr(t, step(reverseScanForUpdate(`a`, `c`)), `db0.ReverseScanForUpdate(ctx, "a", "c", 0) // (nil, context canceled)`) - checkErr(t, step(del(`b`)), `db1.Del(ctx, "b") // context canceled`) - - checkErr(t, step(closureTxn(ClosureTxnType_Commit, delRange(`b`, `d`))), ` -db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - txn.DelRange(ctx, "b", "d", true) - return nil -}) // context canceled - `) - - checkPanics(t, step(delRange(`b`, `d`)), `non-transactional DelRange operations currently unsupported`) - checkPanics(t, step(batch(delRange(`b`, `d`))), `non-transactional batch DelRange operations currently unsupported`) - - // Batch - check(t, step(batch(put(`b`, `2`), get(`a`), del(`b`), del(`c`), scan(`a`, `c`), reverseScanForUpdate(`a`, `e`))), ` -{ - b := &Batch{} - b.Put(ctx, "b", 2) // nil - b.Get(ctx, "a") // ("1", nil) - b.Del(ctx, "b") // nil - b.Del(ctx, "c") // nil - b.Scan(ctx, "a", "c") // (["a":"1"], nil) - b.ReverseScanForUpdate(ctx, "a", "e") // (["d":"4", "a":"1"], nil) - db1.Run(ctx, b) // nil -} -`) - checkErr(t, step(batch(put(`b`, `2`), getForUpdate(`a`), scanForUpdate(`a`, `c`), reverseScan(`a`, `c`))), ` -{ - b := &Batch{} - b.Put(ctx, "b", 2) // context canceled - b.GetForUpdate(ctx, "a") // (nil, context canceled) - b.ScanForUpdate(ctx, "a", "c") // (nil, context canceled) - b.ReverseScan(ctx, "a", "c") // (nil, context canceled) - db0.Run(ctx, b) // context canceled -} -`) - - // Txn commit - check(t, step(closureTxn(ClosureTxnType_Commit, put(`e`, `5`), batch(put(`f`, `6`), delRange(`c`, `e`)))), ` -db1.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - txn.Put(ctx, "e", 5) // nil - { - b := &Batch{} - b.Put(ctx, "f", 6) // nil - b.DelRange(ctx, "c", "e", true) // (["d"], nil) - txn.Run(ctx, b) // nil - } - return nil -}) // nil txnpb:(...) - `) - - // Txn commit in batch - check(t, step(closureTxnCommitInBatch(opSlice(get(`a`), put(`f`, `6`)), put(`e`, `5`))), ` -db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - txn.Put(ctx, "e", 5) // nil - b := &Batch{} - b.Get(ctx, "a") // ("1", nil) - b.Put(ctx, "f", 6) // nil - txn.CommitInBatch(ctx, b) // nil - return nil -}) // nil txnpb:(...) - `) - - // Txn rollback - check(t, step(closureTxn(ClosureTxnType_Rollback, put(`e`, `5`))), ` -db1.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - txn.Put(ctx, "e", 5) // nil - return errors.New("rollback") -}) // rollback - `) - - // Txn error - checkErr(t, step(closureTxn(ClosureTxnType_Rollback, put(`e`, `5`))), ` -db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - txn.Put(ctx, "e", 5) - return errors.New("rollback") -}) // context canceled - `) - - // Splits and merges - check(t, step(split(`foo`)), `db1.AdminSplit(ctx, "foo") // nil`) - check(t, step(merge(`foo`)), `db0.AdminMerge(ctx, "foo") // nil`) - checkErr(t, step(split(`foo`)), - `db1.AdminSplit(ctx, "foo") // context canceled`) - checkErr(t, step(merge(`foo`)), - `db0.AdminMerge(ctx, "foo") // context canceled`) - - // Lease transfers - check(t, step(transferLease(`foo`, 1)), - `db1.TransferLeaseOperation(ctx, "foo", 1) // nil`) - checkErr(t, step(transferLease(`foo`, 1)), - `db0.TransferLeaseOperation(ctx, "foo", 1) // context canceled`) - - // Zone config changes - check(t, step(changeZone(ChangeZoneType_ToggleGlobalReads)), - `env.UpdateZoneConfig(ctx, ToggleGlobalReads) // nil`) - checkErr(t, step(changeZone(ChangeZoneType_ToggleGlobalReads)), - `env.UpdateZoneConfig(ctx, ToggleGlobalReads) // context canceled`) + w := echotest.NewWalker(t, testutils.TestDataPath(t, t.Name())) + defer w.Check(t) + for _, test := range tests { + s := test.step + t.Run(test.name, w.Run(t, test.name, func(t *testing.T) string { + isErr := strings.HasSuffix(test.name, "-err") || strings.HasSuffix(test.name, "-again") + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + if isErr { + cancel() + } + + var buf strings.Builder + trace, err := a.Apply(ctx, &s) + require.NoError(t, err) + + actual := strings.TrimLeft(s.String(), "\n") + + if isErr { + // Trim out context canceled location, which can be non-deterministic. + // The wrapped string around the context canceled error depends on where + // the context cancellation was noticed. + actual = regexp.MustCompile(` aborted .*: context canceled`).ReplaceAllString(actual, ` context canceled`) + } else { + // Trim out the txn to avoid nondeterminism. + actual = regexp.MustCompile(` txnpb:\(.*\)`).ReplaceAllLiteralString(actual, ` txnpb:`) + // Replace timestamps. + actual = regexp.MustCompile(`[0-9]+\.[0-9]+,[0-9]+`).ReplaceAllLiteralString(actual, ``) + } + buf.WriteString(actual) + + t.Log(buf.String()) + t.Log(trace) + + return buf.String() + })) + } } func TestUpdateZoneConfig(t *testing.T) { diff --git a/pkg/kv/kvnemesis/doc.go b/pkg/kv/kvnemesis/doc.go index 6ed0e4fdbc42..6685f6ffd870 100644 --- a/pkg/kv/kvnemesis/doc.go +++ b/pkg/kv/kvnemesis/doc.go @@ -8,34 +8,106 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -// Package kvnemesis exercises the KV api with random traffic and then validates -// that the observed behaviors are consistent with our guarantees. -// -// A set of Operations are generated which represent usage of the public KV api. -// These include both "workload" operations like Gets and Puts as well as -// "admin" operations like rebalances. These Operations can be handed to an -// Applier, which runs them against the KV api and records the results. -// -// Operations do allow for concurrency (this testing is much less interesting -// otherwise), which means that the state of the KV map is not recoverable from -// _only_ the input. TODO(dan): We can use RangeFeed to recover the exact KV -// history. This plus some Kyle magic can be used to check our transactional -// guarantees. -// -// TODO -// - CPut/InitPut/Increment -// - ClearRange/RevertRange -// - AdminRelocateRange -// - AdminUnsplit -// - AdminScatter -// - CheckConsistency -// - ExportRequest -// - AddSSTable -// - Root and leaf transactions -// - GCRequest -// - Protected timestamps -// - Transactions being abandoned by their coordinator -// - Continuing txns after CPut and WriteIntent errors (generally continuing -// after errors is not allowed, but it is allowed after ConditionFailedError and -// WriteIntentError as a special case) +// Package kvnemesis exercises the KV API with random concurrent traffic (as +// well as splits, merges, etc) and then validates that the observed behaviors +// are serializable. +// +// It does so in polynomial time based on the techniques used by [Elle] (see in +// particular section 4.2.3), using the after-the-fact MVCC history as a record +// of truth. It ensures that all write operations embed a unique identifier that +// is stored in MVCC history, and can thus identify which of its operations' +// mutations are reflected in the database ("recoverability" in Elle parlance). +// +// A run of kvnemesis proceeds as follows. +// +// First, a Generator is instantiated. It can create, upon request, a sequence +// in which each element is a random Operation. Operations that are mutations +// (i.e. not pure reads) are assigned a unique kvnemesisutil.Seq which will be +// stored alongside the MVCC data written by the Operation. +// +// A pool of worker threads concurrently generates Operations and executes them +// against a CockroachDB cluster. Some of these Operations may +// succeed, some may fail, and for some of them an ambiguous result may be +// encountered. +// Alongside this random traffic, kvnemesis maintains a RangeFeed that ingests +// the MVCC history. This creates a "carbon copy" of the MVCC history. +// +// All of these workers wind down once they have in aggregate completed a +// configured number of steps. +// +// Next, kvnemesis validates that the Operations that were executed and the +// results they saw match the MVCC history, i.e. checks for Serializability. In +// general, this is an NP-hard problem[^1], but the use of unique sequence +// numbers (kvnemesisutil.Seq) makes it tractable, as each mutation in the MVCC +// keyspace uniquely identifies the Operation that must have written the value. +// +// We make use of this property as follows. First, the Validate method iterates +// through all Operations performed and, for each, inspects +// +// - the type of the Operation (i.e. Put, Delete, Get, ...), +// - the (point or range) key of the operation, and +// - its results (i.e. whether there was an error or which key-value pairs were returned). +// +// Each atomic unit (i.e. individual non-transactional operation, or batch of +// non-transactional operations, or transaction) results in a slice (in order +// in which the Operations within executed[^2]) of observations, where each +// element is either an observed read or an observed write. +// +// For example, a Batch that first writes `v1` to `k1`, then scans `[k1-k3)` +// (reading its own write as well as some other key k2 with value v2) and then +// deletes `k3` would generate the following slice: +// +// [ +// observedWrite(k1->v1), +// observedScan(k1->v1, k2->v2), +// observedWrite(k3->v3), +// ] +// +// Each such slice (i.e. atomic unit) will then be compared to the MVCC history. +// For all units that succeeded, their writes must match up with versions in +// the MVCC history, and this matching is trivial thanks to unique values +// (including for deletions, since we embed the kvnemesisutil.Seq in the value), +// and in particular a single write will entirely fix the MVCC timestamp at +// which the atomic unit must have executed. +// +// For each read (i.e. get or scan), we compute at which time intervals each +// read would have been valid. For example, if the MVCC history for a key `k1` +// is as follows: +// +// k1 +// +// ----------------- +// t5 v2 +// t4 +// t3 +// t2 +// t1 v1 +// +// then +// +// - observedGet(k1->v1) is valid for [t1,t2), +// - observedGet(k1->nil) is valid for [0,t1) and [t2,t5), and +// - observedGet(k1->v2) is valid for [t5,inf). +// +// By intersecting the time intervals for each Operation in an atomic unit, we +// then get the MVCC timestamps at which this Operation would have observed what +// it ended up observing. If this intersection is empty, serializability must have +// been violated. +// +// In the error case, kvnemesis verifies that no part of the Operation became visible. +// For ambiguous results, kvnemesis requires that either no Operation become visible, +// or otherwise treats the atomic unit as committed. +// +// The KV API also has the capability to return the actual execution timestamp directly +// with responses. At the time of writing, kvnemesis does verify that it does do this +// reliably, but it does not verify that the execution timestamp is contained in the +// intersection of time intervals obtained from inspecting MVCC history[^3]. +// +// [Elle]: https://arxiv.org/pdf/2003.10554.pdf +// [^1]: https://dl.acm.org/doi/10.1145/322154.322158 +// [^2]: there is currently concurrency within the atomic unit in kvnemesis. It +// could in theory carry out multiple reads concurrently while not also writing, +// such as DistSQL does, but this is not implemented today. See: +// https://github.com/cockroachdb/cockroach/issues/64825 +// [^3]: tracked in https://github.com/cockroachdb/cockroach/issues/92898. package kvnemesis diff --git a/pkg/kv/kvnemesis/engine.go b/pkg/kv/kvnemesis/engine.go index 257acba4b4e3..4ddc045e58f4 100644 --- a/pkg/kv/kvnemesis/engine.go +++ b/pkg/kv/kvnemesis/engine.go @@ -34,6 +34,7 @@ type Engine struct { // MakeEngine returns a new Engine. func MakeEngine() (*Engine, error) { opts := storage.DefaultPebbleOptions() + opts.FormatMajorVersion = pebble.FormatNewest // for range key deletions opts.FS = vfs.NewMem() kvs, err := pebble.Open(`kvnemesis`, opts) if err != nil { @@ -52,12 +53,32 @@ func (e *Engine) Close() { // Get returns the value for this key with the highest timestamp <= ts. If no // such value exists, the returned value's RawBytes is nil. func (e *Engine) Get(key roachpb.Key, ts hlc.Timestamp) roachpb.Value { - iter := e.kvs.NewIter(nil) + opts := pebble.IterOptions{ + KeyTypes: pebble.IterKeyTypePointsAndRanges, + // Make MVCC range deletions actually appear to delete points in + // this low-level iterator, so we don't have to implement it manually + // a second time. + RangeKeyMasking: pebble.RangeKeyMasking{ + Suffix: storage.EncodeMVCCTimestampSuffix(ts), + }, + } + iter := e.kvs.NewIter(&opts) defer func() { _ = iter.Close() }() iter.SeekGE(storage.EncodeMVCCKey(storage.MVCCKey{Key: key, Timestamp: ts})) + for iter.Valid() { + hasPoint, _ := iter.HasPointAndRange() + if !hasPoint { + iter.Next() + } else { + break + } + } if !iter.Valid() { return roachpb.Value{} } + + // We're on the first point the iter is seeing. + // This use of iter.Key() is safe because it comes entirely before the // deferred iter.Close. mvccKey, err := storage.DecodeMVCCKey(iter.Key()) @@ -89,33 +110,52 @@ func (e *Engine) Put(key storage.MVCCKey, value []byte) { } } -// Delete writes a tombstone value for a given key/timestamp. This is -// equivalent to a Put with an empty value. -func (e *Engine) Delete(key storage.MVCCKey) { - if err := e.kvs.Set(storage.EncodeMVCCKey(key), nil, nil); err != nil { +func (e *Engine) DeleteRange(from, to roachpb.Key, ts hlc.Timestamp, val []byte) { + suffix := storage.EncodeMVCCTimestampSuffix(ts) + if err := e.kvs.RangeKeySet(from, to, suffix, val, nil); err != nil { panic(err) } } // Iterate calls the given closure with every KV in the Engine, in ascending // order. -func (e *Engine) Iterate(fn func(key storage.MVCCKey, value []byte, err error)) { - iter := e.kvs.NewIter(nil) +func (e *Engine) Iterate( + fn func(key, endKey roachpb.Key, ts hlc.Timestamp, value []byte, err error), +) { + iter := e.kvs.NewIter(&pebble.IterOptions{KeyTypes: pebble.IterKeyTypePointsAndRanges}) defer func() { _ = iter.Close() }() for iter.First(); iter.Valid(); iter.Next() { - if err := iter.Error(); err != nil { - fn(storage.MVCCKey{}, nil, err) - continue - } + hasPoint, _ := iter.HasPointAndRange() var keyCopy, valCopy []byte e.b, keyCopy = e.b.Copy(iter.Key(), 0 /* extraCap */) e.b, valCopy = e.b.Copy(iter.Value(), 0 /* extraCap */) - key, err := storage.DecodeMVCCKey(keyCopy) - if err != nil { - fn(storage.MVCCKey{}, nil, err) - continue + if hasPoint { + key, err := storage.DecodeMVCCKey(keyCopy) + if err != nil { + fn(nil, nil, hlc.Timestamp{}, nil, err) + } else { + fn(key.Key, nil, key.Timestamp, valCopy, nil) + } + } + if iter.RangeKeyChanged() { + key, endKey := iter.RangeBounds() + e.b, key = e.b.Copy(key, 0 /* extraCap */) + e.b, endKey = e.b.Copy(endKey, 0 /* extraCap */) + for _, rk := range iter.RangeKeys() { + ts, err := storage.DecodeMVCCTimestampSuffix(rk.Suffix) + if err != nil { + fn(nil, nil, hlc.Timestamp{}, nil, err) + continue + } + + e.b, rk.Value = e.b.Copy(rk.Value, 0) + fn(key, endKey, ts, rk.Value, nil) + } } - fn(key, valCopy, nil) + } + + if err := iter.Error(); err != nil { + fn(nil, nil, hlc.Timestamp{}, nil, err) } } @@ -123,7 +163,7 @@ func (e *Engine) Iterate(fn func(key storage.MVCCKey, value []byte, err error)) // debugging. func (e *Engine) DebugPrint(indent string) string { var buf strings.Builder - e.Iterate(func(key storage.MVCCKey, value []byte, err error) { + e.Iterate(func(key, endKey roachpb.Key, ts hlc.Timestamp, value []byte, err error) { if buf.Len() > 0 { buf.WriteString("\n") } @@ -133,9 +173,14 @@ func (e *Engine) DebugPrint(indent string) string { v, err := storage.DecodeMVCCValue(value) if err != nil { fmt.Fprintf(&buf, "(err:%s)", err) - } else { + return + } + if len(endKey) == 0 { fmt.Fprintf(&buf, "%s%s %s -> %s", - indent, key.Key, key.Timestamp, v.Value.PrettyPrint()) + indent, key, ts, v.Value.PrettyPrint()) + } else { + fmt.Fprintf(&buf, "%s%s-%s %s -> %s", + indent, key, endKey, ts, v.Value.PrettyPrint()) } } }) diff --git a/pkg/kv/kvnemesis/engine_test.go b/pkg/kv/kvnemesis/engine_test.go index 8b73b089e1b1..44fb2c383b76 100644 --- a/pkg/kv/kvnemesis/engine_test.go +++ b/pkg/kv/kvnemesis/engine_test.go @@ -11,11 +11,12 @@ package kvnemesis import ( - "strings" "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -47,11 +48,12 @@ func TestEngine(t *testing.T) { e.Put(k(`a`, ts(1)), roachpb.MakeValueFromString(`a-1`).RawBytes) e.Put(k(`a`, ts(2)), roachpb.MakeValueFromString(`a-2`).RawBytes) e.Put(k(`b`, ts(2)), roachpb.MakeValueFromString(`b-2`).RawBytes) - e.Delete(k(`b`, ts(3))) - e.Delete(k(`c`, ts(4))) + e.Put(k(`b`, ts(3)), nil) + e.Put(k(`c`, ts(4)), nil) e.Put(k(`d`, ts(4)), roachpb.MakeValueFromString(`d-4`).RawBytes) e.Put(k(`e`, ts(4)), roachpb.MakeValueFromString(`e-4`).RawBytes) - e.Delete(k(`d`, ts(5))) + e.Put(k(`d`, ts(5)), nil) + e.DeleteRange(roachpb.Key("f"), roachpb.Key("g"), ts(7), nil) assert.Equal(t, v(`a-2`, ts(2)), e.Get(roachpb.Key(`a`), ts(3))) assert.Equal(t, v(`a-2`, ts(2)), e.Get(roachpb.Key(`a`), ts(2))) assert.Equal(t, v(`a-1`, ts(1)), e.Get(roachpb.Key(`a`), ts(2).Prev())) @@ -65,14 +67,6 @@ func TestEngine(t *testing.T) { assert.Equal(t, missing, e.Get(roachpb.Key(`d`), ts(5))) assert.Equal(t, v(`e-4`, ts(4)), e.Get(roachpb.Key(`e`), ts(5))) - assert.Equal(t, strings.TrimSpace(` -"a" 0.000000002,0 -> /BYTES/a-2 -"a" 0.000000001,0 -> /BYTES/a-1 -"b" 0.000000003,0 -> / -"b" 0.000000002,0 -> /BYTES/b-2 -"c" 0.000000004,0 -> / -"d" 0.000000005,0 -> / -"d" 0.000000004,0 -> /BYTES/d-4 -"e" 0.000000004,0 -> /BYTES/e-4 - `), e.DebugPrint("")) + _, _ = k, v + echotest.Require(t, e.DebugPrint(""), testutils.TestDataPath(t, t.Name(), "output.txt")) } diff --git a/pkg/kv/kvnemesis/env.go b/pkg/kv/kvnemesis/env.go index c87e65c48d4b..3cbfc12e3111 100644 --- a/pkg/kv/kvnemesis/env.go +++ b/pkg/kv/kvnemesis/env.go @@ -23,16 +23,25 @@ import ( "github.com/cockroachdb/errors" ) +// Logger is the log sink used by kvnemesis. +type Logger interface { + Helper() + Logf(string, ...interface{}) + WriteFile(basename string, contents string) string +} + // Env manipulates the environment (cluster settings, zone configurations) that // the Applier operates in. type Env struct { - sqlDBs []*gosql.DB + SQLDBs []*gosql.DB + Tracker *SeqTracker + L Logger } func (e *Env) anyNode() *gosql.DB { // NOTE: There is currently no need to round-robin through the sql gateways, // so we always just return the first DB. - return e.sqlDBs[0] + return e.SQLDBs[0] } // CheckConsistency runs a consistency check on all ranges in the given span, @@ -59,8 +68,12 @@ func (e *Env) CheckConsistency(ctx context.Context, span roachpb.Span) []error { return []error{err} } switch status { + case roachpb.CheckConsistencyResponse_RANGE_INDETERMINATE.String(): + // Can't do anything, so let it slide. case roachpb.CheckConsistencyResponse_RANGE_CONSISTENT.String(): + // Good. case roachpb.CheckConsistencyResponse_RANGE_CONSISTENT_STATS_ESTIMATED.String(): + // Ok. default: failures = append(failures, errors.Errorf("range %d (%s) %s: %s", rangeID, key, status, detail)) } diff --git a/pkg/kv/kvnemesis/generator.go b/pkg/kv/kvnemesis/generator.go index 2289a37e1e72..f3e1f7555bf9 100644 --- a/pkg/kv/kvnemesis/generator.go +++ b/pkg/kv/kvnemesis/generator.go @@ -11,15 +11,19 @@ package kvnemesis import ( + "encoding/binary" + "encoding/hex" + "fmt" + "math" "math/rand" - "strconv" + "sort" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/syncutil" - "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" ) @@ -104,6 +108,8 @@ type ClientOperationConfig struct { DeleteExisting int // DeleteRange is an operation that Deletes a key range that may contain values. DeleteRange int + // DeleteRange is an operation that invokes DeleteRangeUsingTombstone. + DeleteRangeUsingTombstone int } // BatchOperationConfig configures the relative probability of generating a @@ -170,19 +176,20 @@ type ChangeZoneConfig struct { // yet pass (for example, if the new operation finds a kv bug or edge case). func newAllOperationsConfig() GeneratorConfig { clientOpConfig := ClientOperationConfig{ - GetMissing: 1, - GetMissingForUpdate: 1, - GetExisting: 1, - GetExistingForUpdate: 1, - PutMissing: 1, - PutExisting: 1, - Scan: 1, - ScanForUpdate: 1, - ReverseScan: 1, - ReverseScanForUpdate: 1, - DeleteMissing: 1, - DeleteExisting: 1, - DeleteRange: 1, + GetMissing: 1, + GetMissingForUpdate: 1, + GetExisting: 1, + GetExistingForUpdate: 1, + PutMissing: 1, + PutExisting: 1, + Scan: 1, + ScanForUpdate: 1, + ReverseScan: 1, + ReverseScanForUpdate: 1, + DeleteMissing: 1, + DeleteExisting: 1, + DeleteRange: 1, + DeleteRangeUsingTombstone: 1, } batchOpConfig := BatchOperationConfig{ Batch: 4, @@ -227,9 +234,19 @@ func newAllOperationsConfig() GeneratorConfig { // operations/make some operations more likely. func NewDefaultConfig() GeneratorConfig { config := newAllOperationsConfig() - // TODO(sarkesian): Enable non-transactional DelRange once #69642 is fixed. - config.Ops.DB.DeleteRange = 0 - config.Ops.Batch.Ops.DeleteRange = 0 + // DeleteRangeUsingTombstone does not support transactions. + config.Ops.ClosureTxn.TxnClientOps.DeleteRangeUsingTombstone = 0 + config.Ops.ClosureTxn.TxnBatchOps.Ops.DeleteRangeUsingTombstone = 0 + config.Ops.ClosureTxn.CommitBatchOps.DeleteRangeUsingTombstone = 0 + // DeleteRangeUsingTombstone does in principle support batches, but + // in kvnemesis we don't let it span ranges non-atomically (as it + // is allowed to do in CRDB). The generator already tries to avoid + // crossing range boundaries quite a fair bit, so we could enable this + // after some investigation to ensure that significant enough coverage + // remains on the batch path. + // Note also that at the time of writing `config.Ops.Batch` is cleared in its + // entirety below, so changing this line alonewon't have an effect. + config.Ops.Batch.Ops.DeleteRangeUsingTombstone = 0 // TODO(sarkesian): Enable DeleteRange in comingled batches once #71236 is fixed. config.Ops.ClosureTxn.CommitBatchOps.DeleteRange = 0 config.Ops.ClosureTxn.TxnBatchOps.Ops.DeleteRange = 0 @@ -241,10 +258,13 @@ func NewDefaultConfig() GeneratorConfig { // (see CrossRangeTxnWrapperSender) if they are. roachpb.SpanGroup can be used // to efficiently check this. // - // TODO(dan): Make this `config.Ops.Batch.Ops.PutExisting = 0` once #46081 is - // fixed. + // TODO(tbg): could make this `config.Ops.Batch.Ops.PutExisting = 0` (and + // DeleteRange, etc, all ops that can overwrite existing keys basically), as + // #46081 has long been fixed. Then file an issue about generating + // non-self-overlapping operations for batches. config.Ops.Batch = BatchOperationConfig{} - // TODO(dan): Remove when #45586 is addressed. + // TODO(tbg): should be able to remove the two lines below, since + // #45586 has already been addressed. config.Ops.ClosureTxn.CommitBatchOps.GetExisting = 0 config.Ops.ClosureTxn.CommitBatchOps.GetMissing = 0 return config @@ -331,7 +351,7 @@ type generator struct { Config GeneratorConfig replicasFn GetReplicasFn - nextValue int + seqGen kvnemesisutil.Seq // keys is the set of every key that has been written to, including those // deleted or in rolled back transactions. @@ -390,6 +410,11 @@ func (g *generator) RandStep(rng *rand.Rand) Step { return step(g.selectOp(rng, allowed)) } +func (g *generator) nextSeq() kvnemesisutil.Seq { + g.seqGen++ + return g.seqGen +} + type opGenFunc func(*generator, *rand.Rand) Operation type opGen struct { @@ -433,6 +458,7 @@ func (g *generator) registerClientOps(allowed *[]opGen, c *ClientOperationConfig addOpGen(allowed, randReverseScan, c.ReverseScan) addOpGen(allowed, randReverseScanForUpdate, c.ReverseScanForUpdate) addOpGen(allowed, randDelRange, c.DeleteRange) + addOpGen(allowed, randDelRangeUsingTombstone, c.DeleteRangeUsingTombstone) } func (g *generator) registerBatchOps(allowed *[]opGen, c *BatchOperationConfig) { @@ -462,16 +488,16 @@ func randGetExistingForUpdate(g *generator, rng *rand.Rand) Operation { } func randPutMissing(g *generator, rng *rand.Rand) Operation { - value := g.getNextValue() + seq := g.nextSeq() key := randKey(rng) g.keys[key] = struct{}{} - return put(key, value) + return put(key, seq) } func randPutExisting(g *generator, rng *rand.Rand) Operation { - value := g.getNextValue() + seq := g.nextSeq() key := randMapKey(rng, g.keys) - return put(key, value) + return put(key, seq) } func randScan(g *generator, rng *rand.Rand) Operation { @@ -500,19 +526,78 @@ func randReverseScanForUpdate(g *generator, rng *rand.Rand) Operation { func randDelMissing(g *generator, rng *rand.Rand) Operation { key := randKey(rng) g.keys[key] = struct{}{} - return del(key) + seq := g.nextSeq() + return del(key, seq) } func randDelExisting(g *generator, rng *rand.Rand) Operation { key := randMapKey(rng, g.keys) - return del(key) + seq := g.nextSeq() + return del(key, seq) } func randDelRange(g *generator, rng *rand.Rand) Operation { // We don't write any new keys to `g.keys` on a DeleteRange operation, // because DelRange(..) only deletes existing keys. key, endKey := randSpan(rng) - return delRange(key, endKey) + seq := g.nextSeq() + return delRange(key, endKey, seq) +} + +func randDelRangeUsingTombstone(g *generator, rng *rand.Rand) Operation { + return randDelRangeUsingTombstoneImpl(g.currentSplits, g.keys, g.nextSeq, rng) +} + +func randDelRangeUsingTombstoneImpl( + currentSplits, keys map[string]struct{}, nextSeq func() kvnemesisutil.Seq, rng *rand.Rand, +) Operation { + yn := func(probY float64) bool { + return rng.Float64() <= probY + } + + var k, ek string + if yn(0.90) { + // 90% chance of picking an entire existing range. + // + // In kvnemesis, DeleteRangeUsingTombstone is prevented from spanning ranges since + // CRDB executes such requests non-atomically and so we can't verify them + // well. Thus, pick spans that are likely single-range most of the time. + // + // 75% (of the 90%) of the time we'll also modify the bounds. + k, ek = randRangeSpan(rng, currentSplits) + if yn(0.5) { + // In 50% of cases, move startKey forward. + k = randKeyBetween(rng, k, ek) + } + if yn(0.5) { + // In 50% of cases, move endKey backward. + nk := fk(k) + 1 + nek := fk(ek) + if nek < math.MaxUint64 { + nek++ + } + ek = randKeyBetween(rng, tk(nk), tk(nek)) + } + } else if yn(0.5) { + // (100%-90%)*50% = 5% chance of turning the span we have now into a + // point write. Half the time random key, otherwise prefer existing key. + if yn(0.5) || len(keys) == 0 { + k = randKey(rng) + } else { + k = randMapKey(rng, keys) + } + ek = tk(fk(k) + 1) + } else { + // 5% chance of picking a completely random span. This will often span range + // boundaries and be rejected, so these are essentially doomed to fail. + k, ek = randKey(rng), randKey(rng) + if ek < k { + // NB: if they're equal, that's just tough luck; we'll have an empty range. + k, ek = ek, k + } + } + + return delRangeUsingTombstone(k, ek, nextSeq()) } func randSplitNew(g *generator, rng *rand.Rand) Operation { @@ -595,7 +680,6 @@ func makeRandBatch(c *ClientOperationConfig) opGenFunc { return func(g *generator, rng *rand.Rand) Operation { var allowed []opGen g.registerClientOps(&allowed, c) - numOps := rng.Intn(4) ops := make([]Operation, numOps) for i := range ops { @@ -640,31 +724,96 @@ func makeClosureTxn( } } -func (g *generator) getNextValue() string { - value := `v-` + strconv.Itoa(g.nextValue) - g.nextValue++ - return value -} - -func randKey(rng *rand.Rand) string { - u, err := uuid.NewGenWithReader(rng).NewV4() +// fk stands for "from key", i.e. decode the uint64 the key represents. +// Panics on error. +func fk(k string) uint64 { + k = k[len(GeneratorDataSpan().Key):] + _, s, err := encoding.DecodeUnsafeStringAscendingDeepCopy([]byte(k), nil) if err != nil { panic(err) } + sl, err := hex.DecodeString(s) + if err != nil { + panic(err) + } + return binary.BigEndian.Uint64(sl) +} + +// tk stands for toKey, i.e. encode the uint64 into its key representation. +func tk(n uint64) string { + var sl [8]byte + binary.BigEndian.PutUint64(sl[:8], n) + s := hex.EncodeToString(sl[:8]) key := GeneratorDataSpan().Key - key = encoding.EncodeStringAscending(key, u.Short()) + key = encoding.EncodeStringAscending(key, s) return string(key) } -func randMapKey(rng *rand.Rand, m map[string]struct{}) string { - keys := make([]string, 0, len(m)) - for key := range m { +func randKey(rng *rand.Rand) string { + // Avoid the endpoints because having point writes at the + // endpoints complicates randRangeSpan. + n := rng.Uint64() + if n == 0 { + n++ + } + if n == math.MaxUint64 { + n-- + } + return tk(n) +} + +// Interprets the provided map as the split points of the key space and returns +// the boundaries of a random range. +func randRangeSpan(rng *rand.Rand, curOrHistSplits map[string]struct{}) (string, string) { + keys := make([]string, 0, len(curOrHistSplits)) + for key := range curOrHistSplits { keys = append(keys, key) } + sort.Strings(keys) if len(keys) == 0 { + // No splits. + return tk(0), tk(math.MaxUint64) + } + idx := rng.Intn(len(keys) + 1) + if idx == len(keys) { + // Last range. + return keys[idx-1], tk(math.MaxUint64) + } + if idx == 0 { + // First range. We avoid having splits at 0 so this will be a well-formed + // range. (If it isn't, we'll likely catch an error because we'll send an + // ill-formed request and kvserver will error it out). + return tk(0), keys[0] + } + return keys[idx-1], keys[idx] +} + +func randMapKey(rng *rand.Rand, m map[string]struct{}) string { + if len(m) == 0 { return randKey(rng) } - return keys[rng.Intn(len(keys))] + k, ek := randRangeSpan(rng, m) + // If there is only one key in the map we will get [0,x) or [x,max) + // back and want to return `x` to avoid the endpoints, which are + // reserved. + if fk(k) == 0 { + return ek + } + return k +} + +// Returns a key that falls into `[k,ek)`. +func randKeyBetween(rng *rand.Rand, k, ek string) string { + a, b := fk(k), fk(ek) + if b <= a { + b = a + 1 // we will return `a` + } + defer func() { + if r := recover(); r != nil { + panic(fmt.Sprintf("a=%d b=%d b-a=%d: %v", a, b, int64(b-a), r)) + } + }() + return tk(a + (rng.Uint64() % (b - a))) } func randSpan(rng *rand.Rand) (string, string) { @@ -682,7 +831,9 @@ func step(op Operation) Step { } func batch(ops ...Operation) Operation { - return Operation{Batch: &BatchOperation{Ops: ops}} + return Operation{Batch: &BatchOperation{ + Ops: ops, + }} } func opSlice(ops ...Operation) []Operation { @@ -709,8 +860,8 @@ func getForUpdate(key string) Operation { return Operation{Get: &GetOperation{Key: []byte(key), ForUpdate: true}} } -func put(key, value string) Operation { - return Operation{Put: &PutOperation{Key: []byte(key), Value: []byte(value)}} +func put(key string, seq kvnemesisutil.Seq) Operation { + return Operation{Put: &PutOperation{Key: []byte(key), Seq: seq}} } func scan(key, endKey string) Operation { @@ -729,12 +880,19 @@ func reverseScanForUpdate(key, endKey string) Operation { return Operation{Scan: &ScanOperation{Key: []byte(key), EndKey: []byte(endKey), Reverse: true, ForUpdate: true}} } -func del(key string) Operation { - return Operation{Delete: &DeleteOperation{Key: []byte(key)}} +func del(key string, seq kvnemesisutil.Seq) Operation { + return Operation{Delete: &DeleteOperation{ + Key: []byte(key), + Seq: seq, + }} +} + +func delRange(key, endKey string, seq kvnemesisutil.Seq) Operation { + return Operation{DeleteRange: &DeleteRangeOperation{Key: []byte(key), EndKey: []byte(endKey), Seq: seq}} } -func delRange(key, endKey string) Operation { - return Operation{DeleteRange: &DeleteRangeOperation{Key: []byte(key), EndKey: []byte(endKey)}} +func delRangeUsingTombstone(key, endKey string, seq kvnemesisutil.Seq) Operation { + return Operation{DeleteRangeUsingTombstone: &DeleteRangeUsingTombstoneOperation{Key: []byte(key), EndKey: []byte(endKey), Seq: seq}} } func split(key string) Operation { diff --git a/pkg/kv/kvnemesis/generator_test.go b/pkg/kv/kvnemesis/generator_test.go index de66d2899f50..6832829e9fb3 100644 --- a/pkg/kv/kvnemesis/generator_test.go +++ b/pkg/kv/kvnemesis/generator_test.go @@ -12,10 +12,17 @@ package kvnemesis import ( "context" + "fmt" + "math" + "math/rand" "reflect" + "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -136,9 +143,13 @@ func TestRandStep(t *testing.T) { } case *DeleteRangeOperation: client.DeleteRange++ + case *DeleteRangeUsingTombstoneOperation: + client.DeleteRangeUsingTombstone++ case *BatchOperation: batch.Batch++ countClientOps(&batch.Ops, nil, o.Ops...) + default: + t.Fatalf("%T", o) } } } @@ -152,7 +163,8 @@ func TestRandStep(t *testing.T) { *ScanOperation, *BatchOperation, *DeleteOperation, - *DeleteRangeOperation: + *DeleteRangeOperation, + *DeleteRangeUsingTombstoneOperation: countClientOps(&counts.DB, &counts.Batch, step.Op) case *ClosureTxnOperation: countClientOps(&counts.ClosureTxn.TxnClientOps, &counts.ClosureTxn.TxnBatchOps, o.Ops...) @@ -201,6 +213,8 @@ func TestRandStep(t *testing.T) { case ChangeZoneType_ToggleGlobalReads: counts.ChangeZone.ToggleGlobalReads++ } + default: + t.Fatalf("%T", o) } updateKeys(step.Op) @@ -211,3 +225,127 @@ func TestRandStep(t *testing.T) { } } } + +func TestRandKeyDecode(t *testing.T) { + defer leaktest.AfterTest(t)() + + for i := 0; i < 10; i++ { + rng := rand.New(rand.NewSource(int64(i))) + k := randKey(rng) + n := fk(k) + require.Equal(t, k, tk(n)) + } +} + +func TestRandDelRangeUsingTombstone(t *testing.T) { + defer leaktest.AfterTest(t)() + + var seq kvnemesisutil.Seq + nextSeq := func() kvnemesisutil.Seq { + seq++ + return seq + } + + // We'll add temporary elements below. The linters prevent shadowing so we + // need to pick a name that makes it clear which is which. + // + // Keep this sorted. + goldenSplitKeys := []uint64{ + 5000, 10000, 15000, 20000, 25000, 30000, math.MaxUint64 / 2, + } + + splitPointMap := map[string]struct{}{} + splitPointCountMap := map[string]int{} + var splitSpans []roachpb.Span + { + // Temporarily put 0 and MaxUint64 into the slice to + // help generate the leftmost and rightmost range. + splitKeys := append([]uint64{0}, goldenSplitKeys...) + splitKeys = append(splitKeys, math.MaxUint64) + for i := range splitKeys { + if i == 0 { + continue + } + k := tk(splitKeys[i-1]) + ek := tk(splitKeys[i]) + splitSpans = append(splitSpans, roachpb.Span{ + Key: roachpb.Key(k), + EndKey: roachpb.Key(ek), + }) + splitPointCountMap[ek] = 0 + if splitKeys[i] == math.MaxUint64 { + // Don't put MaxUint64 into splitPointMap to make sure we're always + // generating useful ranges. There's no room to the right of this split + // point. + continue + } + splitPointMap[ek] = struct{}{} + } + } + + rng := rand.New(rand.NewSource(1)) // deterministic + const num = 1000 + + // keysMap plays no role in this test but we need to pass one. + // We could also check that we're hitting the keys in this map + // randomly, etc, but don't currently. + keysMap := map[string]struct{}{ + tk(5): {}, + } + + var numSingleRange, numCrossRange, numPoint int + for i := 0; i < num; i++ { + dr := randDelRangeUsingTombstoneImpl(splitPointMap, keysMap, nextSeq, rng).DeleteRangeUsingTombstone + sp := roachpb.Span{Key: dr.Key, EndKey: dr.EndKey} + nk, nek := fk(string(dr.Key)), fk(string(dr.EndKey)) + s := fmt.Sprintf("[%d,%d)", nk, nek) + if fk(string(dr.Key))+1 == fk(string(dr.EndKey)) { + if numPoint == 0 { + t.Logf("first point request: %s", s) + } + numPoint++ + continue + } + var contained bool + for _, splitSp := range splitSpans { + if splitSp.Contains(sp) { + // `sp` does not contain a split point, i.e. this would likely end up + // being a single-range request. + if numSingleRange == 0 { + t.Logf("first single-range request: %s", s) + } + numSingleRange++ + contained = true + splitPointCountMap[string(splitSp.EndKey)]++ + break + } + } + if !contained { + if numCrossRange == 0 { + t.Logf("first cross-range request: %s", s) + } + numCrossRange++ + } + } + + fracSingleRange := float64(numSingleRange) / float64(num) + fracCrossRange := float64(numCrossRange) / float64(num) + fracPoint := float64(numPoint) / float64(num) + + var buf strings.Builder + + fmt.Fprintf(&buf, "point: %.3f n=%d\n", fracPoint, numPoint) + fmt.Fprintf(&buf, "cross-range: %.3f n=%d\n", fracCrossRange, numCrossRange) + + fmt.Fprintf(&buf, "single-range: %.3f n=%d\n", fracSingleRange, numSingleRange) + + for _, splitSp := range splitSpans { + frac := float64(splitPointCountMap[string(splitSp.EndKey)]) / float64(numSingleRange) + fmt.Fprintf(&buf, " ^---- %.3f [%d,%d)\n", + frac, fk(string(splitSp.Key)), fk(string(splitSp.EndKey))) + } + + fmt.Fprintf(&buf, "------------------\ntotal %.3f", fracSingleRange+fracPoint+fracCrossRange) + + echotest.Require(t, buf.String(), testutils.TestDataPath(t, t.Name()+".txt")) +} diff --git a/pkg/kv/kvnemesis/kvnemesis.go b/pkg/kv/kvnemesis/kvnemesis.go index 5ad20efdefae..62d29df2b615 100644 --- a/pkg/kv/kvnemesis/kvnemesis.go +++ b/pkg/kv/kvnemesis/kvnemesis.go @@ -13,7 +13,11 @@ package kvnemesis import ( "context" "fmt" + "io" "math/rand" + "os" + "path/filepath" + "reflect" "strings" "sync/atomic" @@ -22,6 +26,47 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" ) +type loggerKey struct{} + +type logLogger struct { + dir string +} + +func (l *logLogger) WriteFile(basename string, contents string) string { + f, err := os.Create(filepath.Join(l.dir, basename)) + if err != nil { + return err.Error() + } + defer f.Close() + _, err = io.WriteString(f, contents) + if err != nil { + return err.Error() + } + return f.Name() +} + +func (l *logLogger) Helper() { /* no-op */ } + +func (l *logLogger) Logf(format string, args ...interface{}) { + log.InfofDepth(context.Background(), 2, format, args...) +} + +func l(ctx context.Context, basename string, format string, args ...interface{}) (optFile string) { + var logger Logger + logger, _ = ctx.Value(loggerKey{}).(Logger) + if logger == nil { + logger = &logLogger{dir: os.TempDir()} + } + logger.Helper() + + if basename != "" { + return logger.WriteFile(basename, fmt.Sprintf(format, args...)) + } + + logger.Logf(format, args...) + return "" +} + // RunNemesis generates and applies a series of Operations to exercise the KV // api. It returns a slice of the logical failures encountered. func RunNemesis( @@ -29,10 +74,13 @@ func RunNemesis( rng *rand.Rand, env *Env, config GeneratorConfig, + concurrency int, numSteps int, dbs ...*kv.DB, ) ([]error, error) { - const concurrency = 5 + if env.L != nil { + ctx = context.WithValue(ctx, loggerKey{}, env.L) + } if numSteps <= 0 { return nil, fmt.Errorf("numSteps must be >0, got %v", numSteps) } @@ -55,28 +103,34 @@ func RunNemesis( workerFn := func(ctx context.Context, workerIdx int) error { workerName := fmt.Sprintf(`%d`, workerIdx) - var buf strings.Builder + stepIdx := -1 for atomic.AddInt64(&stepsStartedAtomic, 1) <= int64(numSteps) { + stepIdx++ step := g.RandStep(rng) - - buf.Reset() - fmt.Fprintf(&buf, "step:") - step.format(&buf, formatCtx{indent: ` ` + workerName + ` PRE `}) trace, err := a.Apply(ctx, &step) - buf.WriteString(trace.String()) - step.Trace = buf.String() + + stepPrefix := fmt.Sprintf("w%d_step%d", workerIdx, stepIdx) + step.Trace = l(ctx, fmt.Sprintf("%s_trace", stepPrefix), "%s", trace.String()) + + stepsByWorker[workerIdx] = append(stepsByWorker[workerIdx], step) + + prefix := ` OP ` + if err != nil { + prefix = ` ERR ` + } + + { + var buf strings.Builder + fmt.Fprintf(&buf, " before: %s", step.Before) + step.format(&buf, formatCtx{indent: ` ` + workerName + prefix}) + fmt.Fprintf(&buf, "\n after: %s", step.After) + basename := fmt.Sprintf("%s_%T", stepPrefix, reflect.Indirect(reflect.ValueOf(step.Op.GetValue())).Interface()) + l(ctx, basename, "%s", &buf) + } + if err != nil { - buf.Reset() - step.format(&buf, formatCtx{indent: ` ` + workerName + ` ERR `}) - log.Infof(ctx, "error: %+v\n\n%s", err, buf.String()) return err } - buf.Reset() - fmt.Fprintf(&buf, "\n before: %s", step.Before) - step.format(&buf, formatCtx{indent: ` ` + workerName + ` OP `}) - fmt.Fprintf(&buf, "\n after: %s", step.After) - log.Infof(ctx, "%v", buf.String()) - stepsByWorker[workerIdx] = append(stepsByWorker[workerIdx], step) } return nil } @@ -96,26 +150,43 @@ func RunNemesis( } kvs := w.Finish() defer kvs.Close() - failures := Validate(allSteps, kvs) + + failures := Validate(allSteps, kvs, env.Tracker) // Run consistency checks across the data span, primarily to check the // accuracy of evaluated MVCC stats. failures = append(failures, env.CheckConsistency(ctx, dataSpan)...) if len(failures) > 0 { - log.Infof(ctx, "reproduction steps:\n%s", printRepro(stepsByWorker)) - log.Infof(ctx, "kvs (recorded from rangefeed):\n%s", kvs.DebugPrint(" ")) + var failuresFile string + { + var buf strings.Builder + for _, err := range failures { + l(ctx, "", "%s", err) + fmt.Fprintf(&buf, "%+v\n", err) + fmt.Fprintln(&buf, strings.Repeat("=", 80)) + } + failuresFile = l(ctx, "failures", "%s", &buf) + } + reproFile := l(ctx, "repro.go", "// Reproduction steps:\n%s", printRepro(stepsByWorker)) + rangefeedFile := l(ctx, "kvs-rangefeed.txt", "kvs (recorded from rangefeed):\n%s", kvs.DebugPrint(" ")) + var kvsFile string scanKVs, err := dbs[0].Scan(ctx, dataSpan.Key, dataSpan.EndKey, -1) if err != nil { - log.Infof(ctx, "could not scan actual latest values: %+v", err) + l(ctx, "", "could not scan actual latest values: %+v", err) } else { var kvsBuf strings.Builder for _, kv := range scanKVs { fmt.Fprintf(&kvsBuf, " %s %s -> %s\n", kv.Key, kv.Value.Timestamp, kv.Value.PrettyPrint()) } - log.Infof(ctx, "kvs (scan of latest values according to crdb):\n%s", kvsBuf.String()) + kvsFile = l(ctx, "kvs-scan.txt", "kvs (scan of latest values according to crdb):\n%s", kvsBuf.String()) } + l(ctx, "", `failures(verbose): %s +repro steps: %s +rangefeed KVs: %s +scan KVs: %s`, + failuresFile, reproFile, rangefeedFile, kvsFile) } return failures, nil @@ -132,12 +203,14 @@ func printRepro(stepsByWorker [][]Step) string { buf.WriteString("\n") buf.WriteString(fctx.indent) step.Op.format(&buf, fctx) - buf.WriteString(step.Trace) + if len(step.Trace) > 0 { + fmt.Fprintf(&buf, "\n // ^-- trace in: %s\n", step.Trace) + } buf.WriteString("\n") } buf.WriteString("\n return nil\n") - buf.WriteString("})\n") + buf.WriteString("})\n\n") } - buf.WriteString("g.Wait()\n") + buf.WriteString("require.NoError(t, g.Wait())\n") return buf.String() } diff --git a/pkg/kv/kvnemesis/kvnemesis_test.go b/pkg/kv/kvnemesis/kvnemesis_test.go index a23c9498251b..af8984975206 100644 --- a/pkg/kv/kvnemesis/kvnemesis_test.go +++ b/pkg/kv/kvnemesis/kvnemesis_test.go @@ -13,101 +13,208 @@ package kvnemesis import ( "context" gosql "database/sql" + "math/rand" + "os" "testing" "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/envutil" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/stretchr/testify/require" ) -var numSteps int +var defaultNumSteps = envutil.EnvOrDefaultInt("COCKROACH_KVNEMESIS_STEPS", 50) -func init() { - numSteps = envutil.EnvOrDefaultInt("COCKROACH_KVNEMESIS_STEPS", 50) -} - -func TestKVNemesisSingleNode(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - skip.UnderRace(t) +func testClusterArgs(tr *SeqTracker) base.TestClusterArgs { + storeKnobs := &kvserver.StoreTestingKnobs{ + // Drop the clock MaxOffset to reduce commit-wait time for + // transactions that write to global_read ranges. + MaxOffset: 10 * time.Millisecond, + // Make sure we know the seq for each of our writes when they come out of + // the rangefeed. We do this via an interceptor to avoid having to change + // RangeFeed's APIs. + RangefeedValueHeaderFilter: func(key, endKey roachpb.Key, ts hlc.Timestamp, vh enginepb.MVCCValueHeader) { + if seq := vh.KVNemesisSeq.Get(); seq > 0 { + tr.Add(key, endKey, ts, seq) + } + }, + } - ctx := context.Background() - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + return base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{ - Store: &kvserver.StoreTestingKnobs{ - // Drop the clock MaxOffset to reduce commit-wait time for - // transactions that write to global_read ranges. - MaxOffset: 10 * time.Millisecond, + Store: storeKnobs, + KVClient: &kvcoord.ClientTestingKnobs{ + // Don't let DistSender split DeleteRangeUsingTombstone across range boundaries. + // This does happen in real CRDB, but leads to separate atomic subunits, which + // would add complexity to kvnemesis that isn't worth it. Instead, the operation + // generator for the most part tries to avoid range-spanning requests, and the + // ones that do end up happening get a hard error. + OnRangeSpanningNonTxnalBatch: func(ba *roachpb.BatchRequest) *roachpb.Error { + for _, req := range ba.Requests { + if req.GetInner().Method() != roachpb.DeleteRange { + continue + } + if req.GetDeleteRange().UseRangeTombstone == true { + return roachpb.NewError(errDelRangeUsingTombstoneStraddlesRangeBoundary) + } + } + return nil + }, }, }, }, - }) - defer tc.Stopper().Stop(ctx) - db := tc.Server(0).DB() - sqlDB := tc.ServerConn(0) - sqlutils.MakeSQLRunner(sqlDB).Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) + } +} - config := NewDefaultConfig() - config.NumNodes, config.NumReplicas = 1, 1 - rng, _ := randutil.NewTestRand() - env := &Env{sqlDBs: []*gosql.DB{sqlDB}} - failures, err := RunNemesis(ctx, rng, env, config, numSteps, db) - require.NoError(t, err, `%+v`, err) +func randWithSeed( + t interface { + Logf(string, ...interface{}) + Helper() + }, seedOrZero int64, +) *rand.Rand { + t.Helper() + var rng *rand.Rand + if seedOrZero > 0 { + rng = rand.New(rand.NewSource(seedOrZero)) + } else { + rng, seedOrZero = randutil.NewTestRand() + } + t.Logf("seed: %d", seedOrZero) + return rng +} + +type ti interface { + Helper() + Logf(string, ...interface{}) +} + +type tBridge struct { + ti + ll logLogger +} - for _, failure := range failures { - t.Errorf("failure:\n%+v", failure) +func newTBridge(t *testing.T) *tBridge { + td, err := os.MkdirTemp("", "kvnemesis") + if err != nil { + td = os.TempDir() } + t.Logf("kvnemesis logging to %s", td) + return &tBridge{ + ti: t, + ll: logLogger{ + dir: td, + }, + } +} + +func (t *tBridge) WriteFile(basename string, contents string) string { + return t.ll.WriteFile(basename, contents) +} + +type kvnemesisTestCfg struct { + numNodes int + numSteps int + concurrency int + seedOverride int64 +} + +func TestKVNemesisSingleNode(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testKVNemesisImpl(t, kvnemesisTestCfg{ + numNodes: 1, + numSteps: defaultNumSteps, + concurrency: 5, + seedOverride: 0, + }) } func TestKVNemesisMultiNode(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + + testKVNemesisImpl(t, kvnemesisTestCfg{ + numNodes: 4, + numSteps: defaultNumSteps, + concurrency: 5, + seedOverride: 0, + }) +} + +func testKVNemesisImpl(t *testing.T, cfg kvnemesisTestCfg) { skip.UnderRace(t) + if !buildutil.CrdbTestBuild { + // `roachpb.RequestHeader` and `MVCCValueHeader` have a KVNemesisSeq field + // that is zero-sized outside test builds. We could revisit that should + // a need arise to run kvnemesis against production binaries. + skip.IgnoreLint(t, "kvnemesis must be run with the crdb_test build tag") + } + + // Can set a seed here for determinism. This works best when the seed was + // obtained with cfg.concurrency=1. + rng := randWithSeed(t, cfg.seedOverride) + // 4 nodes so we have somewhere to move 3x replicated ranges to. - const numNodes = 4 ctx := context.Background() - tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ - ReplicationMode: base.ReplicationManual, - ServerArgs: base.TestServerArgs{ - Knobs: base.TestingKnobs{ - Store: &kvserver.StoreTestingKnobs{ - // Drop the clock MaxOffset to reduce commit-wait time for - // transactions that write to global_read ranges. - MaxOffset: 10 * time.Millisecond, - }, - }, - }, - }) + tr := &SeqTracker{} + tc := testcluster.StartTestCluster(t, cfg.numNodes, testClusterArgs(tr)) defer tc.Stopper().Stop(ctx) - dbs, sqlDBs := make([]*kv.DB, numNodes), make([]*gosql.DB, numNodes) - for i := 0; i < numNodes; i++ { + dbs, sqlDBs := make([]*kv.DB, cfg.numNodes), make([]*gosql.DB, cfg.numNodes) + for i := 0; i < cfg.numNodes; i++ { dbs[i] = tc.Server(i).DB() sqlDBs[i] = tc.ServerConn(i) } sqlutils.MakeSQLRunner(sqlDBs[0]).Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`) + // TODO(arul): remove this line when #92189 is addressed. + // + // See: https://github.com/cockroachdb/cockroach/issues/93164. + sqlutils.MakeSQLRunner(sqlDBs[0]).Exec(t, `SET CLUSTER SETTING kv.transaction.dropping_latches_before_eval.enabled = false`) // Turn net/trace on, which results in real trace spans created throughout. // This gives kvnemesis a chance to hit NPEs related to tracing. sqlutils.MakeSQLRunner(sqlDBs[0]).Exec(t, `SET CLUSTER SETTING trace.debug.enable = true`) config := NewDefaultConfig() - config.NumNodes, config.NumReplicas = numNodes, 3 - rng, _ := randutil.NewTestRand() - env := &Env{sqlDBs: sqlDBs} - failures, err := RunNemesis(ctx, rng, env, config, numSteps, dbs...) + config.NumNodes = cfg.numNodes + config.NumReplicas = 3 + if config.NumReplicas > cfg.numNodes { + config.NumReplicas = cfg.numNodes + } + logger := newTBridge(t) + env := &Env{SQLDBs: sqlDBs, Tracker: tr, L: logger} + failures, err := RunNemesis(ctx, rng, env, config, cfg.concurrency, cfg.numSteps, dbs...) require.NoError(t, err, `%+v`, err) + require.Zero(t, len(failures), "kvnemesis detected failures") // they've been logged already +} - for _, failure := range failures { - t.Errorf("failure:\n%+v", failure) - } +// TestRunReproductionSteps is a helper that allows quickly running a kvnemesis +// history. +func TestRunReproductionSteps(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + skip.IgnoreLint(t, "test unskipped only on demand") + ctx := context.Background() + + const n = 1 // number of nodes + + tc := testcluster.StartTestCluster(t, n, base.TestClusterArgs{}) + db0 := tc.Server(0).DB() + _, _ = db0, ctx + + // Paste a repro as printed by kvnemesis here. } diff --git a/pkg/kv/kvnemesis/kvnemesisutil/BUILD.bazel b/pkg/kv/kvnemesis/kvnemesisutil/BUILD.bazel new file mode 100644 index 000000000000..e8cdabe98659 --- /dev/null +++ b/pkg/kv/kvnemesis/kvnemesisutil/BUILD.bazel @@ -0,0 +1,41 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +# gazelle:exclude gen-crdb_test_off.go +# gazelle:exclude gen-crdb_test_on.go + +# keep +go_library( + name = "kvnemesisutil", + srcs = select({ + "//build/toolchains:crdb_test": [":gen-crdb-test-on"], + "//conditions:default": [":gen-crdb-test-off"], + }) + [ + "seq.go", + "context.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil", + visibility = ["//visibility:public"], + deps = [ + "//pkg/util/protoutil", + "@com_github_gogo_protobuf//proto", + ], +) + +REMOVE_GO_BUILD_CONSTRAINTS = "cat $< | grep -v '//go:build' | grep -v '// +build' > $@" + +genrule( + name = "gen-crdb-test-on", + srcs = ["crdb_test_on.go"], + outs = ["gen-crdb_test_on.go"], + cmd = REMOVE_GO_BUILD_CONSTRAINTS, +) + +genrule( + name = "gen-crdb-test-off", + srcs = ["crdb_test_off.go"], + outs = ["gen-crdb_test_off.go"], + cmd = REMOVE_GO_BUILD_CONSTRAINTS, +) + +get_x_data(name = "get_x_data") diff --git a/pkg/kv/kvnemesis/kvnemesisutil/context.go b/pkg/kv/kvnemesis/kvnemesisutil/context.go new file mode 100644 index 000000000000..70083fb27a65 --- /dev/null +++ b/pkg/kv/kvnemesis/kvnemesisutil/context.go @@ -0,0 +1,27 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. +// + +package kvnemesisutil + +import "context" + +type seqKey struct{} + +// WithSeq wraps the Context with a Seq. +func WithSeq(ctx context.Context, seq Seq) context.Context { + return context.WithValue(ctx, seqKey{}, seq) +} + +// FromContext extracts a Seq from the Context if there is one. +func FromContext(ctx context.Context) (Seq, bool) { + v, ok := ctx.Value(seqKey{}).(Seq) + return v, ok +} diff --git a/pkg/kv/kvnemesis/kvnemesisutil/crdb_test_off.go b/pkg/kv/kvnemesis/kvnemesisutil/crdb_test_off.go new file mode 100644 index 000000000000..bae5ed4760c5 --- /dev/null +++ b/pkg/kv/kvnemesis/kvnemesisutil/crdb_test_off.go @@ -0,0 +1,16 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +//go:build !crdb_test || crdb_test_off +// +build !crdb_test crdb_test_off + +package kvnemesisutil + +type Container = NoopContainer diff --git a/pkg/kv/kvnemesis/kvnemesisutil/crdb_test_on.go b/pkg/kv/kvnemesis/kvnemesisutil/crdb_test_on.go new file mode 100644 index 000000000000..e66ed76af5af --- /dev/null +++ b/pkg/kv/kvnemesis/kvnemesisutil/crdb_test_on.go @@ -0,0 +1,16 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +//go:build crdb_test && !crdb_test_off +// +build crdb_test,!crdb_test_off + +package kvnemesisutil + +type Container = SeqContainer diff --git a/pkg/kv/kvnemesis/kvnemesisutil/seq.go b/pkg/kv/kvnemesis/kvnemesisutil/seq.go new file mode 100644 index 000000000000..553b0eb9228c --- /dev/null +++ b/pkg/kv/kvnemesis/kvnemesisutil/seq.go @@ -0,0 +1,144 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvnemesisutil + +import ( + "fmt" + "reflect" + + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + gogoproto "github.com/gogo/protobuf/proto" +) + +// Seq is a unique identifier used to associate MVCC versions with the kvnemesis +// operation that wrote them. +type Seq uint32 + +func (s Seq) String() string { + return fmt.Sprintf("s%d", s) +} + +// SeqContainer is an uint32 with methods that allow it to be used as a +// `gogoproto.casttype`, and which has a getter/setter. See +type SeqContainer uint32 + +var _ protoutil.Message = (*SeqContainer)(nil) + +// Reset implements (a part of) protoutil.Message. +func (m *SeqContainer) Reset() { + *m = 0 +} + +// ProtoMessage implements (a part of) protoutil.Message. +func (m SeqContainer) String() string { + return fmt.Sprintf("s%d", m) +} + +// ProtoMessage implements (a part of) protoutil.Message. +func (m *SeqContainer) ProtoMessage() { +} + +// MarshalTo implements (a part of) protoutil.Message. +func (m *SeqContainer) MarshalTo(buf []byte) (int, error) { + if *m == 0 { + return 0, nil + } + sl := gogoproto.EncodeVarint(uint64(*m)) + _ = append(buf[:0], sl...) + return len(sl), nil +} + +type errT string + +func (err errT) Error() string { + return string(err) +} + +// Unmarshal implements (a part of) protoutil.Message. +func (m *SeqContainer) Unmarshal(buf []byte) error { + if len(buf) == 0 { + *m = 0 + return nil + } + x, n := gogoproto.DecodeVarint(buf) + if n == 0 { + return errT(fmt.Sprintf("unable to unmarshal %x as varint", buf)) + } + *m = SeqContainer(x) + return nil +} + +// MarshalToSizedBuffer implements (a part of) protoutil.Message. +func (m *SeqContainer) MarshalToSizedBuffer(buf []byte) (int, error) { + if *m == 0 { + return 0, nil + } + sl := gogoproto.EncodeVarint(uint64(*m)) + _ = append(buf[:len(buf)-len(sl)], sl...) + return len(sl), nil +} + +// Size implements (a part of) protoutil.Message. +func (m *SeqContainer) Size() int { + if *m == 0 { + return 0 + } + return len(gogoproto.EncodeVarint(uint64(*m))) +} + +// Equal implements (gogoproto.equal). +func (m *SeqContainer) Equal(n interface{}) bool { + return reflect.DeepEqual(m, n) +} + +// Set updates the receiver. Not thread safe. +func (m *SeqContainer) Set(seq Seq) { + *m = SeqContainer(seq) +} + +// Get reads the receiver. Not thread safe. +func (m SeqContainer) Get() Seq { + return Seq(m) +} + +// NoopContainer is an empty struct that can be used as a `gogoproto.casttype` in +// proto messages. It uses no space. When the crdb_test build tag is set, this +// type is instead represented by a NoopContainer. +type NoopContainer struct{} + +func (m *NoopContainer) Reset() {} + +// String implements (a part of) protoutil.Message. +func (m *NoopContainer) String() string { return "0" } + +// ProtoMessage implements (a part of) protoutil.Message. +func (m *NoopContainer) ProtoMessage() {} + +// MarshalTo implements (a part of) protoutil.Message. +func (m *NoopContainer) MarshalTo(buf []byte) (int, error) { return 0, nil } + +// Unmarshal implements (a part of) protoutil.Message. +func (m *NoopContainer) Unmarshal(buf []byte) error { return nil } + +// MarshalToSizedBuffer implements (a part of) protoutil.Message. +func (m *NoopContainer) MarshalToSizedBuffer(buf []byte) (int, error) { return 0, nil } + +// Size implements (a part of) protoutil.Message. +func (m *NoopContainer) Size() int { return 0 } + +// Equal implements (gogoproto.equal). +func (m *NoopContainer) Equal(n interface{}) bool { return reflect.DeepEqual(m, n) } + +// Set is a no-op. +func (m *NoopContainer) Set(Seq) {} + +// Get returns zero. +func (m NoopContainer) Get() Seq { return 0 } diff --git a/pkg/kv/kvnemesis/operations.go b/pkg/kv/kvnemesis/operations.go index 6d8ad552614f..0f451f973787 100644 --- a/pkg/kv/kvnemesis/operations.go +++ b/pkg/kv/kvnemesis/operations.go @@ -13,8 +13,10 @@ package kvnemesis import ( "context" "fmt" + "strconv" "strings" + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" @@ -33,6 +35,8 @@ func (op Operation) Result() *Result { return &o.Result case *DeleteRangeOperation: return &o.Result + case *DeleteRangeUsingTombstoneOperation: + return &o.Result case *SplitOperation: return &o.Result case *MergeOperation: @@ -75,6 +79,13 @@ type formatCtx struct { // TODO(dan): error handling. } +func (fctx formatCtx) maybeCtx() string { + if fctx.receiver == `b` { + return "" + } + return "ctx, " +} + func (s Step) format(w *strings.Builder, fctx formatCtx) { if fctx.receiver != `` { panic(`cannot specify receiver in Step.format fctx`) @@ -112,6 +123,8 @@ func (op Operation) format(w *strings.Builder, fctx formatCtx) { o.format(w, fctx) case *DeleteRangeOperation: o.format(w, fctx) + case *DeleteRangeUsingTombstoneOperation: + o.format(w, fctx) case *SplitOperation: o.format(w, fctx) case *MergeOperation: @@ -169,37 +182,42 @@ func (op Operation) format(w *strings.Builder, fctx formatCtx) { w.WriteString(`})`) o.Result.format(w) if o.Txn != nil { - fmt.Fprintf(w, ` txnpb:(%s)`, o.Txn) + fmt.Fprintf(w, "\n%s// ^-- txnpb:(%s)", fctx.indent, o.Txn) } default: fmt.Fprintf(w, "%v", op.GetValue()) } } +func fmtKey(k []byte) string { + return fmt.Sprintf(`tk(%d)`, fk(string(k))) +} + func (op GetOperation) format(w *strings.Builder, fctx formatCtx) { methodName := `Get` if op.ForUpdate { methodName = `GetForUpdate` } - fmt.Fprintf(w, `%s.%s(ctx, %s)`, fctx.receiver, methodName, roachpb.Key(op.Key)) - switch op.Result.Type { - case ResultType_Error: - err := errors.DecodeError(context.TODO(), *op.Result.Err) - fmt.Fprintf(w, ` // (nil, %s)`, err.Error()) - case ResultType_Value: - v := `nil` - if len(op.Result.Value) > 0 { - v = `"` + mustGetStringValue(op.Result.Value) + `"` - } - fmt.Fprintf(w, ` // (%s, nil)`, v) - } + fmt.Fprintf(w, `%s.%s(%s%s)`, fctx.receiver, methodName, fctx.maybeCtx(), fmtKey(op.Key)) + op.Result.format(w) } func (op PutOperation) format(w *strings.Builder, fctx formatCtx) { - fmt.Fprintf(w, `%s.Put(ctx, %s, %s)`, fctx.receiver, roachpb.Key(op.Key), op.Value) + fmt.Fprintf(w, `%s.Put(%s%s, sv(%d))`, fctx.receiver, fctx.maybeCtx(), fmtKey(op.Key), op.Seq) op.Result.format(w) } +// sv stands for sequence value, i.e. the value dictated by the given sequence number. +func sv(seq kvnemesisutil.Seq) string { + return `v` + strconv.Itoa(int(seq)) +} + +// Value returns the value written by this put. This is a function of the +// sequence number. +func (op PutOperation) Value() string { + return sv(op.Seq) +} + func (op ScanOperation) format(w *strings.Builder, fctx formatCtx) { methodName := `Scan` if op.ForUpdate { @@ -213,76 +231,49 @@ func (op ScanOperation) format(w *strings.Builder, fctx formatCtx) { if fctx.receiver == `b` { maxRowsArg = `` } - fmt.Fprintf(w, `%s.%s(ctx, %s, %s%s)`, fctx.receiver, methodName, roachpb.Key(op.Key), roachpb.Key(op.EndKey), maxRowsArg) - switch op.Result.Type { - case ResultType_Error: - err := errors.DecodeError(context.TODO(), *op.Result.Err) - fmt.Fprintf(w, ` // (nil, %s)`, err.Error()) - case ResultType_Values: - var kvs strings.Builder - for i, kv := range op.Result.Values { - if i > 0 { - kvs.WriteString(`, `) - } - kvs.WriteByte('"') - kvs.WriteString(string(kv.Key)) - kvs.WriteString(`":"`) - kvs.WriteString(mustGetStringValue(kv.Value)) - kvs.WriteByte('"') - } - fmt.Fprintf(w, ` // ([%s], nil)`, kvs.String()) - } + fmt.Fprintf(w, `%s.%s(%s%s, %s%s)`, fctx.receiver, methodName, fctx.maybeCtx(), fmtKey(op.Key), fmtKey(op.EndKey), maxRowsArg) + op.Result.format(w) } func (op DeleteOperation) format(w *strings.Builder, fctx formatCtx) { - fmt.Fprintf(w, `%s.Del(ctx, %s)`, fctx.receiver, roachpb.Key(op.Key)) + fmt.Fprintf(w, `%s.Del(%s%s /* @%s */)`, fctx.receiver, fctx.maybeCtx(), fmtKey(op.Key), op.Seq) op.Result.format(w) } func (op DeleteRangeOperation) format(w *strings.Builder, fctx formatCtx) { - fmt.Fprintf(w, `%s.DelRange(ctx, %s, %s, true)`, fctx.receiver, roachpb.Key(op.Key), roachpb.Key(op.EndKey)) - switch op.Result.Type { - case ResultType_Error: - err := errors.DecodeError(context.TODO(), *op.Result.Err) - fmt.Fprintf(w, ` // (nil, %s)`, err.Error()) - case ResultType_Keys: - var keysW strings.Builder - for i, key := range op.Result.Keys { - if i > 0 { - keysW.WriteString(`, `) - } - keysW.WriteByte('"') - keysW.WriteString(string(key)) - keysW.WriteString(`"`) - } - fmt.Fprintf(w, ` // ([%s], nil)`, keysW.String()) - } + fmt.Fprintf(w, `%s.DelRange(%s%s, %s, true /* @%s */)`, fctx.receiver, fctx.maybeCtx(), fmtKey(op.Key), fmtKey(op.EndKey), op.Seq) + op.Result.format(w) +} + +func (op DeleteRangeUsingTombstoneOperation) format(w *strings.Builder, fctx formatCtx) { + fmt.Fprintf(w, `%s.DelRangeUsingTombstone(ctx, %s, %s /* @%s */)`, fctx.receiver, fmtKey(op.Key), fmtKey(op.EndKey), op.Seq) + op.Result.format(w) } func (op SplitOperation) format(w *strings.Builder, fctx formatCtx) { - fmt.Fprintf(w, `%s.AdminSplit(ctx, %s)`, fctx.receiver, roachpb.Key(op.Key)) + fmt.Fprintf(w, `%s.AdminSplit(ctx, %s)`, fctx.receiver, fmtKey(op.Key)) op.Result.format(w) } func (op MergeOperation) format(w *strings.Builder, fctx formatCtx) { - fmt.Fprintf(w, `%s.AdminMerge(ctx, %s)`, fctx.receiver, roachpb.Key(op.Key)) + fmt.Fprintf(w, `%s.AdminMerge(ctx, %s)`, fctx.receiver, fmtKey(op.Key)) op.Result.format(w) } func (op BatchOperation) format(w *strings.Builder, fctx formatCtx) { w.WriteString("\n") w.WriteString(fctx.indent) - w.WriteString(`b := &Batch{}`) + w.WriteString(`b := &kv.Batch{}`) formatOps(w, fctx, op.Ops) } func (op ChangeReplicasOperation) format(w *strings.Builder, fctx formatCtx) { - fmt.Fprintf(w, `%s.AdminChangeReplicas(ctx, %s, %s)`, fctx.receiver, roachpb.Key(op.Key), op.Changes) + fmt.Fprintf(w, `%s.AdminChangeReplicas(ctx, %s, %s)`, fctx.receiver, fmtKey(op.Key), op.Changes) op.Result.format(w) } func (op TransferLeaseOperation) format(w *strings.Builder, fctx formatCtx) { - fmt.Fprintf(w, `%s.TransferLeaseOperation(ctx, %s, %d)`, fctx.receiver, roachpb.Key(op.Key), op.Target) + fmt.Fprintf(w, `%s.TransferLeaseOperation(ctx, %s, %d)`, fctx.receiver, fmtKey(op.Key), op.Target) op.Result.format(w) } @@ -292,11 +283,55 @@ func (op ChangeZoneOperation) format(w *strings.Builder, fctx formatCtx) { } func (r Result) format(w *strings.Builder) { + if r.Type == ResultType_Unknown { + return + } + fmt.Fprintf(w, ` //`) + if r.OptionalTimestamp.IsSet() { + fmt.Fprintf(w, ` @%s`, r.OptionalTimestamp) + } + + var sl []string + errString := "" switch r.Type { case ResultType_NoError: - fmt.Fprintf(w, ` // nil`) case ResultType_Error: err := errors.DecodeError(context.TODO(), *r.Err) - fmt.Fprintf(w, ` // %s`, err.Error()) + errString = fmt.Sprint(err) + case ResultType_Keys: + for _, k := range r.Keys { + sl = append(sl, roachpb.Key(k).String()) + } + case ResultType_Value: + sl = append(sl, mustGetStringValue(r.Value)) + case ResultType_Values: + for _, kv := range r.Values { + sl = append(sl, fmt.Sprintf(`%s:%s`, roachpb.Key(kv.Key), mustGetStringValue(kv.Value))) + } + default: + panic("unhandled ResultType") + } + + w.WriteString(" ") + + sl = append(sl, errString) + if len(sl) > 1 { + w.WriteString("(") + } + w.WriteString(strings.Join(sl, ", ")) + if len(sl) > 1 { + w.WriteString(")") + } + +} + +// Error decodes and returns the r.Err if it is set. +func (r Result) Error() error { + if r.Err == nil { + return nil + } + if !r.Err.IsSet() { + return nil } + return errors.DecodeError(context.Background(), *r.Err) } diff --git a/pkg/kv/kvnemesis/operations.proto b/pkg/kv/kvnemesis/operations.proto index e5ac38c0f913..f3e996ddc664 100644 --- a/pkg/kv/kvnemesis/operations.proto +++ b/pkg/kv/kvnemesis/operations.proto @@ -54,19 +54,28 @@ message ScanOperation { message PutOperation { bytes key = 1; - bytes value = 2; + uint32 seq = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil.Seq"]; Result result = 3 [(gogoproto.nullable) = false]; } message DeleteOperation { bytes key = 1; - Result result = 2 [(gogoproto.nullable) = false]; + uint32 seq = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil.Seq"]; + Result result = 3 [(gogoproto.nullable) = false]; } message DeleteRangeOperation { bytes key = 1; bytes end_key = 2; - Result result = 3 [(gogoproto.nullable) = false]; + uint32 seq = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil.Seq"]; + Result result = 4 [(gogoproto.nullable) = false]; +} + +message DeleteRangeUsingTombstoneOperation { + bytes key = 1; + bytes end_key = 2; + uint32 seq = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil.Seq"]; + Result result = 4 [(gogoproto.nullable) = false]; } message SplitOperation { @@ -117,11 +126,12 @@ message Operation { ScanOperation scan = 9; DeleteOperation delete = 10; DeleteRangeOperation delete_range = 11; - SplitOperation split = 12; - MergeOperation merge = 13; - ChangeReplicasOperation change_replicas = 14; - TransferLeaseOperation transfer_lease = 15; - ChangeZoneOperation change_zone = 16; + DeleteRangeUsingTombstoneOperation delete_range_using_tombstone = 12; + SplitOperation split = 13; + MergeOperation merge = 14; + ChangeReplicasOperation change_replicas = 15; + TransferLeaseOperation transfer_lease = 16; + ChangeZoneOperation change_zone = 17; } enum ResultType { @@ -158,7 +168,7 @@ message Step { util.hlc.Timestamp before = 2 [(gogoproto.nullable) = false]; util.hlc.Timestamp after = 3 [(gogoproto.nullable) = false]; int32 db_id = 4 [(gogoproto.customname) = 'DBID']; - // Trace will contain the recording of the Step's execution. Used to print it - // out when the test fails. + // Trace will contain (a reference on where to find) the recording of the + // Step's execution. Used to print it out when the test fails. string trace = 5; } diff --git a/pkg/kv/kvnemesis/operations_test.go b/pkg/kv/kvnemesis/operations_test.go index 92fd862e90ea..fa7a053fdd12 100644 --- a/pkg/kv/kvnemesis/operations_test.go +++ b/pkg/kv/kvnemesis/operations_test.go @@ -11,12 +11,28 @@ package kvnemesis import ( + "fmt" "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/stretchr/testify/assert" +) + +var ( + k1 = tk(1) + k2 = tk(2) + k3 = tk(3) + k4 = tk(4) + k5 = tk(5) + k6 = tk(6) + k7 = tk(7) + k8 = tk(8) + k9 = tk(9) + k10 = tk(10) + k11 = tk(11) ) func TestOperationsFormat(t *testing.T) { @@ -24,48 +40,28 @@ func TestOperationsFormat(t *testing.T) { defer log.Scope(t).Close(t) tests := []struct { - step Step - expected string + step Step }{ - {step: step(get(`a`)), expected: `db0.Get(ctx, "a")`}, - {step: step(del(`a`)), expected: `db0.Del(ctx, "a")`}, - {step: step(batch(get(`b`), reverseScanForUpdate(`c`, `e`), get(`f`))), expected: ` - { - b := &Batch{} - b.Get(ctx, "b") - b.ReverseScanForUpdate(ctx, "c", "e") - b.Get(ctx, "f") - db0.Run(ctx, b) - } - `}, + {step: step(get(k1))}, + {step: step(del(k1, 1))}, + {step: step(batch(get(k2), reverseScanForUpdate(k3, k5), get(k6)))}, { step: step( closureTxn(ClosureTxnType_Commit, - batch(get(`g`), get(`h`), del(`i`)), - delRange(`j`, `k`), - put(`k`, `l`), + batch(get(k7), get(k8), del(k9, 1)), + delRange(k10, k11, 2), + put(k11, 3), )), - expected: ` - db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - { - b := &Batch{} - b.Get(ctx, "g") - b.Get(ctx, "h") - b.Del(ctx, "i") - txn.Run(ctx, b) - } - txn.DelRange(ctx, "j", "k", true) - txn.Put(ctx, "k", l) - return nil - }) - `, }, } - for _, test := range tests { - expected := strings.TrimSpace(test.expected) - var actual strings.Builder - test.step.format(&actual, formatCtx{indent: "\t\t\t"}) - assert.Equal(t, expected, strings.TrimSpace(actual.String())) + w := echotest.NewWalker(t, testutils.TestDataPath(t, t.Name())) + for i, test := range tests { + name := fmt.Sprint(i) + t.Run(name, w.Run(t, name, func(t *testing.T) string { + var actual strings.Builder + test.step.format(&actual, formatCtx{indent: "···"}) + return strings.TrimLeft(actual.String(), "\n") + })) } } diff --git a/pkg/kv/kvnemesis/seq_tracker.go b/pkg/kv/kvnemesis/seq_tracker.go new file mode 100644 index 000000000000..70925784e911 --- /dev/null +++ b/pkg/kv/kvnemesis/seq_tracker.go @@ -0,0 +1,98 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. +// + +package kvnemesis + +import ( + "fmt" + "sort" + "strings" + + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" +) + +// SeqTracker is a container that helps kvnemesis map MVCC versions to +// operations as identified by their Seq. +// +// SeqTracker is threadsafe. +type SeqTracker struct { + syncutil.Mutex + seen map[keyTS]kvnemesisutil.Seq +} + +type keyTS struct { + key, endKey string + ts hlc.Timestamp +} + +func (tr *SeqTracker) String() string { + tr.Lock() + defer tr.Unlock() + + var sl []keyTS + for k := range tr.seen { + sl = append(sl, k) + } + sort.Slice(sl, func(i, j int) bool { + return fmt.Sprintf("%v", sl[i]) < fmt.Sprintf("%v", sl[j]) + }) + + var buf strings.Builder + for _, el := range sl { + fmt.Fprintf(&buf, "%s %s -> %s\n", roachpb.Span{Key: roachpb.Key(el.key), EndKey: roachpb.Key(el.endKey)}, el.ts, tr.seen[el]) + } + return buf.String() +} + +// Add associates key@ts with the provided Seq. +func (tr *SeqTracker) Add(key, endKey roachpb.Key, ts hlc.Timestamp, seq kvnemesisutil.Seq) { + tr.Lock() + defer tr.Unlock() + + if tr.seen == nil { + tr.seen = map[keyTS]kvnemesisutil.Seq{} + } + + tr.seen[keyTS{key: string(key), endKey: string(endKey), ts: ts}] = seq +} + +// Lookup checks whether the version key@ts is associated with a Seq. +func (tr *SeqTracker) Lookup(key, endKey roachpb.Key, ts hlc.Timestamp) (kvnemesisutil.Seq, bool) { + tr.Lock() + defer tr.Unlock() + // Rangedels can be split, but the tracker will always see the pre-split + // value (since it's reported by the operation's BatchRequest). So this + // method checks whether the input span is contained in any span seen + // by the tracker. + if seq, fastPathOK := tr.seen[keyTS{ + key: string(key), + endKey: string(endKey), + ts: ts, + }]; fastPathOK { + // Fast path - exact match. Should be the common case outside of MVCC range + // deletions. + return seq, true + } + + for kts := range tr.seen { + if kts.ts != ts { + continue + } + cur := roachpb.Span{Key: roachpb.Key(kts.key), EndKey: roachpb.Key(kts.endKey)} + if cur.Contains(roachpb.Span{Key: key, EndKey: endKey}) { + return tr.seen[kts], true + } + } + return 0, false +} diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/batch b/pkg/kv/kvnemesis/testdata/TestApplier/batch new file mode 100644 index 000000000000..481b14de5360 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/batch @@ -0,0 +1,8 @@ +echo +---- +{ + b := &kv.Batch{} + b.Put(tk(1), sv(21)) // + b.DelRange(tk(2), tk(3), true /* @s22 */) // + db1.Run(ctx, b) // @ +} diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/batch-mixed b/pkg/kv/kvnemesis/testdata/TestApplier/batch-mixed new file mode 100644 index 000000000000..09cb988a4f19 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/batch-mixed @@ -0,0 +1,12 @@ +echo +---- +{ + b := &kv.Batch{} + b.Put(tk(2), sv(2)) // + b.Get(tk(1)) // (, ) + b.Del(tk(2) /* @s1 */) // + b.Del(tk(3) /* @s1 */) // + b.Scan(tk(1), tk(3)) // + b.ReverseScanForUpdate(tk(1), tk(5)) // + db1.Run(ctx, b) // @ +} diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/batch-mixed-err b/pkg/kv/kvnemesis/testdata/TestApplier/batch-mixed-err new file mode 100644 index 000000000000..b4334851e08d --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/batch-mixed-err @@ -0,0 +1,10 @@ +echo +---- +{ + b := &kv.Batch{} + b.Put(tk(2), sv(2)) // context canceled + b.GetForUpdate(tk(1)) // context canceled + b.ScanForUpdate(tk(1), tk(3)) // context canceled + b.ReverseScan(tk(1), tk(3)) // context canceled + db0.Run(ctx, b) // context canceled +} diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/del b/pkg/kv/kvnemesis/testdata/TestApplier/del new file mode 100644 index 000000000000..5d4a4e911950 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/del @@ -0,0 +1,3 @@ +echo +---- +db0.Del(ctx, tk(2) /* @s1 */) // @ diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/del-err b/pkg/kv/kvnemesis/testdata/TestApplier/del-err new file mode 100644 index 000000000000..0933233e8d68 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/del-err @@ -0,0 +1,3 @@ +echo +---- +db0.Del(ctx, tk(2) /* @s1 */) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/delrange b/pkg/kv/kvnemesis/testdata/TestApplier/delrange new file mode 100644 index 000000000000..6ab1dcb6e835 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/delrange @@ -0,0 +1,3 @@ +echo +---- +db1.DelRange(ctx, tk(1), tk(3), true /* @s6 */) // @ (/Table/100/"0000000000000001", ) diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/delrange-err b/pkg/kv/kvnemesis/testdata/TestApplier/delrange-err new file mode 100644 index 000000000000..96bd7e0acbd4 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/delrange-err @@ -0,0 +1,3 @@ +echo +---- +db1.DelRange(ctx, tk(2), tk(3), true /* @s12 */) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/get b/pkg/kv/kvnemesis/testdata/TestApplier/get new file mode 100644 index 000000000000..0dd56fd52825 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/get @@ -0,0 +1,3 @@ +echo +---- +db0.Get(ctx, tk(1)) // @ (, ) diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/get-err b/pkg/kv/kvnemesis/testdata/TestApplier/get-err new file mode 100644 index 000000000000..740b8866b63e --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/get-err @@ -0,0 +1,3 @@ +echo +---- +db1.Get(ctx, tk(1)) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/get-for-update b/pkg/kv/kvnemesis/testdata/TestApplier/get-for-update new file mode 100644 index 000000000000..11059b7c4083 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/get-for-update @@ -0,0 +1,3 @@ +echo +---- +db1.GetForUpdate(ctx, tk(1)) // @ (v1, ) diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/merge b/pkg/kv/kvnemesis/testdata/TestApplier/merge new file mode 100644 index 000000000000..262fa20e40e6 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/merge @@ -0,0 +1,3 @@ +echo +---- +db0.AdminMerge(ctx, tk(1)) // diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/merge-again b/pkg/kv/kvnemesis/testdata/TestApplier/merge-again new file mode 100644 index 000000000000..097de87b6380 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/merge-again @@ -0,0 +1,3 @@ +echo +---- +db0.AdminMerge(ctx, tk(1)) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/put b/pkg/kv/kvnemesis/testdata/TestApplier/put new file mode 100644 index 000000000000..263d1f7aee1f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/put @@ -0,0 +1,3 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @ diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/put-err b/pkg/kv/kvnemesis/testdata/TestApplier/put-err new file mode 100644 index 000000000000..42f9864a4cbe --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/put-err @@ -0,0 +1,3 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/rscan b/pkg/kv/kvnemesis/testdata/TestApplier/rscan new file mode 100644 index 000000000000..c27e7094a186 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/rscan @@ -0,0 +1,3 @@ +echo +---- +db0.ReverseScan(ctx, tk(1), tk(3), 0) // @ (/Table/100/"0000000000000001":v21, ) diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/rscan-err b/pkg/kv/kvnemesis/testdata/TestApplier/rscan-err new file mode 100644 index 000000000000..99791cf475a6 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/rscan-err @@ -0,0 +1,3 @@ +echo +---- +db0.ReverseScan(ctx, tk(1), tk(3), 0) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-update b/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-update new file mode 100644 index 000000000000..3e64ff6dfb1a --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-update @@ -0,0 +1,3 @@ +echo +---- +db1.ReverseScanForUpdate(ctx, tk(1), tk(2), 0) // @ (/Table/100/"0000000000000001":v21, ) diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-update-err b/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-update-err new file mode 100644 index 000000000000..96e356762ea8 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/rscan-for-update-err @@ -0,0 +1,3 @@ +echo +---- +db1.ReverseScanForUpdate(ctx, tk(1), tk(3), 0) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/scan b/pkg/kv/kvnemesis/testdata/TestApplier/scan new file mode 100644 index 000000000000..3c20226a7b83 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/scan @@ -0,0 +1,3 @@ +echo +---- +db1.Scan(ctx, tk(1), tk(3), 0) // @ diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-update b/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-update new file mode 100644 index 000000000000..84b5fab70f91 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-update @@ -0,0 +1,3 @@ +echo +---- +db0.ScanForUpdate(ctx, tk(1), tk(3), 0) // @ (/Table/100/"0000000000000001":v1, ) diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-update-err b/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-update-err new file mode 100644 index 000000000000..54d4e3c75df9 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/scan-for-update-err @@ -0,0 +1,3 @@ +echo +---- +db1.ScanForUpdate(ctx, tk(1), tk(3), 0) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/split b/pkg/kv/kvnemesis/testdata/TestApplier/split new file mode 100644 index 000000000000..2c98be6da1ff --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/split @@ -0,0 +1,3 @@ +echo +---- +db1.AdminSplit(ctx, tk(2)) // diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/split-again b/pkg/kv/kvnemesis/testdata/TestApplier/split-again new file mode 100644 index 000000000000..916303dcfc73 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/split-again @@ -0,0 +1,3 @@ +echo +---- +db1.AdminSplit(ctx, tk(2)) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/transfer b/pkg/kv/kvnemesis/testdata/TestApplier/transfer new file mode 100644 index 000000000000..2c9becff21d0 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/transfer @@ -0,0 +1,3 @@ +echo +---- +db1.TransferLeaseOperation(ctx, tk(6), 1) // diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/transfer-again b/pkg/kv/kvnemesis/testdata/TestApplier/transfer-again new file mode 100644 index 000000000000..d34cc822e962 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/transfer-again @@ -0,0 +1,3 @@ +echo +---- +db0.TransferLeaseOperation(ctx, tk(6), 1) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/txn-commit-batch b/pkg/kv/kvnemesis/testdata/TestApplier/txn-commit-batch new file mode 100644 index 000000000000..cc4f3fa83845 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/txn-commit-batch @@ -0,0 +1,11 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(5), sv(5)) // @ + b := &kv.Batch{} + b.Get(tk(1)) // (, ) + b.Put(tk(6), sv(6)) // + txn.CommitInBatch(ctx, b) // @ + return nil +}) // @ +// ^-- txnpb: diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/txn-commit-mixed b/pkg/kv/kvnemesis/testdata/TestApplier/txn-commit-mixed new file mode 100644 index 000000000000..e5b8f606fe1f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/txn-commit-mixed @@ -0,0 +1,13 @@ +echo +---- +db1.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(5), sv(5)) // @ + { + b := &kv.Batch{} + b.Put(tk(6), sv(6)) // + b.DelRange(tk(3), tk(5), true /* @s1 */) // + txn.Run(ctx, b) // @ + } + return nil +}) // @ +// ^-- txnpb: diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/txn-delrange b/pkg/kv/kvnemesis/testdata/TestApplier/txn-delrange new file mode 100644 index 000000000000..45c812639499 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/txn-delrange @@ -0,0 +1,7 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(2), tk(4), true /* @s1 */) // @ + return nil +}) // @ +// ^-- txnpb: diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/txn-err b/pkg/kv/kvnemesis/testdata/TestApplier/txn-err new file mode 100644 index 000000000000..2d6bad71ab5c --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/txn-err @@ -0,0 +1,6 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(2), tk(4), true /* @s1 */) + return nil +}) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/txn-error b/pkg/kv/kvnemesis/testdata/TestApplier/txn-error new file mode 100644 index 000000000000..406c0d55dbd6 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/txn-error @@ -0,0 +1,6 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(5), sv(5)) // @ + return errors.New("rollback") +}) // rollback diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/txn-rollback b/pkg/kv/kvnemesis/testdata/TestApplier/txn-rollback new file mode 100644 index 000000000000..0b92c1010dbc --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/txn-rollback @@ -0,0 +1,6 @@ +echo +---- +db1.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(5), sv(5)) // @ + return errors.New("rollback") +}) // rollback diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/zcfg b/pkg/kv/kvnemesis/testdata/TestApplier/zcfg new file mode 100644 index 000000000000..a06cdd0d5bce --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/zcfg @@ -0,0 +1,3 @@ +echo +---- +env.UpdateZoneConfig(ctx, ToggleGlobalReads) // diff --git a/pkg/kv/kvnemesis/testdata/TestApplier/zcfg-again b/pkg/kv/kvnemesis/testdata/TestApplier/zcfg-again new file mode 100644 index 000000000000..4e3dfb6c3023 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestApplier/zcfg-again @@ -0,0 +1,3 @@ +echo +---- +env.UpdateZoneConfig(ctx, ToggleGlobalReads) // context canceled diff --git a/pkg/kv/kvnemesis/testdata/TestEngine/output.txt b/pkg/kv/kvnemesis/testdata/TestEngine/output.txt new file mode 100644 index 000000000000..1c50f8610efa --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestEngine/output.txt @@ -0,0 +1,11 @@ +echo +---- +"a" 0.000000002,0 -> /BYTES/a-2 +"a" 0.000000001,0 -> /BYTES/a-1 +"b" 0.000000003,0 -> / +"b" 0.000000002,0 -> /BYTES/b-2 +"c" 0.000000004,0 -> / +"d" 0.000000005,0 -> / +"d" 0.000000004,0 -> /BYTES/d-4 +"e" 0.000000004,0 -> /BYTES/e-4 +"f"-"g" 0.000000007,0 -> / diff --git a/pkg/kv/kvnemesis/testdata/TestOperationsFormat/0 b/pkg/kv/kvnemesis/testdata/TestOperationsFormat/0 new file mode 100644 index 000000000000..b64e4cf873d1 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestOperationsFormat/0 @@ -0,0 +1,3 @@ +echo +---- +···db0.Get(ctx, tk(1)) diff --git a/pkg/kv/kvnemesis/testdata/TestOperationsFormat/1 b/pkg/kv/kvnemesis/testdata/TestOperationsFormat/1 new file mode 100644 index 000000000000..fabbf92108cc --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestOperationsFormat/1 @@ -0,0 +1,3 @@ +echo +---- +···db0.Del(ctx, tk(1) /* @s1 */) diff --git a/pkg/kv/kvnemesis/testdata/TestOperationsFormat/2 b/pkg/kv/kvnemesis/testdata/TestOperationsFormat/2 new file mode 100644 index 000000000000..37a7d1270d56 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestOperationsFormat/2 @@ -0,0 +1,9 @@ +echo +---- +···{ +··· b := &kv.Batch{} +··· b.Get(tk(2)) +··· b.ReverseScanForUpdate(tk(3), tk(5)) +··· b.Get(tk(6)) +··· db0.Run(ctx, b) +···} diff --git a/pkg/kv/kvnemesis/testdata/TestOperationsFormat/3 b/pkg/kv/kvnemesis/testdata/TestOperationsFormat/3 new file mode 100644 index 000000000000..a0b522398085 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestOperationsFormat/3 @@ -0,0 +1,14 @@ +echo +---- +···db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { +··· { +··· b := &kv.Batch{} +··· b.Get(tk(7)) +··· b.Get(tk(8)) +··· b.Del(tk(9) /* @s1 */) +··· txn.Run(ctx, b) +··· } +··· txn.DelRange(ctx, tk(10), tk(11), true /* @s2 */) +··· txn.Put(ctx, tk(11), sv(3)) +··· return nil +···}) diff --git a/pkg/kv/kvnemesis/testdata/TestRandDelRangeUsingTombstone.txt b/pkg/kv/kvnemesis/testdata/TestRandDelRangeUsingTombstone.txt new file mode 100644 index 000000000000..c7a52a304e2b --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestRandDelRangeUsingTombstone.txt @@ -0,0 +1,15 @@ +echo +---- +point: 0.064 n=64 +cross-range: 0.026 n=26 +single-range: 0.910 n=910 + ^---- 0.102 [0,5000) + ^---- 0.122 [5000,10000) + ^---- 0.123 [10000,15000) + ^---- 0.121 [15000,20000) + ^---- 0.116 [20000,25000) + ^---- 0.132 [25000,30000) + ^---- 0.124 [30000,9223372036854775807) + ^---- 0.159 [9223372036854775807,18446744073709551615) +------------------ +total 1.000 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_del-del_transaction_committed b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_del-del_transaction_committed new file mode 100644 index 000000000000..505c256d8a3c --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_del-del_transaction_committed @@ -0,0 +1,8 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + txn.Del(ctx, tk(1) /* @s2 */) // + return nil +}) // result is ambiguous: boom +/Table/100/"0000000000000001"/0.000000001,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_del-del_transaction_committed_but_wrong_seq b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_del-del_transaction_committed_but_wrong_seq new file mode 100644 index 000000000000..6805fda502f0 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_del-del_transaction_committed_but_wrong_seq @@ -0,0 +1,9 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + txn.Del(ctx, tk(1) /* @s2 */) // + return nil +}) // result is ambiguous: boom +/Table/100/"0000000000000001"/0.000000001,0 @ s1 +committed txn overwritten key had write: [d]/Table/100/"0000000000000001":0.000000001,0->@s1 [d]/Table/100/"0000000000000001":missing->@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-del_transaction_committed b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-del_transaction_committed new file mode 100644 index 000000000000..95766b7a9b17 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-del_transaction_committed @@ -0,0 +1,9 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Del(ctx, tk(2) /* @s2 */) // + return nil +}) // result is ambiguous: boom +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000001,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-del_transaction_committed_but_has_validation_error b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-del_transaction_committed_but_has_validation_error new file mode 100644 index 000000000000..acbbd7b25de5 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-del_transaction_committed_but_has_validation_error @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Del(ctx, tk(2) /* @s2 */) // + return nil +}) // result is ambiguous: boom +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 +ambiguous txn non-atomic timestamps: [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 [d]/Table/100/"0000000000000002":0.000000002,0->@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-del_transaction_did_not_commit b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-del_transaction_did_not_commit new file mode 100644 index 000000000000..aca32a7990a0 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-del_transaction_did_not_commit @@ -0,0 +1,7 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Del(ctx, tk(2) /* @s2 */) // + return nil +}) // result is ambiguous: boom diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-put_transaction_committed b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-put_transaction_committed new file mode 100644 index 000000000000..eec3d9dff950 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-put_transaction_committed @@ -0,0 +1,9 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Put(ctx, tk(2), sv(2)) // + return nil +}) // result is ambiguous: boom +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000001,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-put_transaction_committed_but_has_validation_error b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-put_transaction_committed_but_has_validation_error new file mode 100644 index 000000000000..8d9818a960e1 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-put_transaction_committed_but_has_validation_error @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Put(ctx, tk(2), sv(2)) // + return nil +}) // result is ambiguous: boom +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +ambiguous txn non-atomic timestamps: [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 [w]/Table/100/"0000000000000002":0.000000002,0->v2@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-put_transaction_did_not_commit b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-put_transaction_did_not_commit new file mode 100644 index 000000000000..128c9f377804 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/ambiguous_put-put_transaction_did_not_commit @@ -0,0 +1,7 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Put(ctx, tk(2), sv(2)) // + return nil +}) // result is ambiguous: boom diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes new file mode 100644 index 000000000000..3b2c82d0fe1a --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes @@ -0,0 +1,13 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +{ + b := &kv.Batch{} + b.Get(tk(1)) // (v1, ) + b.Get(tk(2)) // (v2, ) + b.Get(tk(3)) // (, ) + db0.Run(ctx, b) // @0.000000003,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes new file mode 100644 index 000000000000..2e3adcfc1a9b --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes @@ -0,0 +1,17 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Del(ctx, tk(1) /* @s3 */) // @0.000000003,0 +db0.Del(ctx, tk(2) /* @s4 */) // @0.000000004,0 +{ + b := &kv.Batch{} + b.Get(tk(1)) // (v1, ) + b.Get(tk(2)) // (v2, ) + b.Get(tk(3)) // (, ) + db0.Run(ctx, b) // @0.000000003,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 +/Table/100/"0000000000000002"/0.000000004,0 @ s4 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes_returning_tombstones b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes_returning_tombstones new file mode 100644 index 000000000000..7f9fb363b0dc --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes_returning_tombstones @@ -0,0 +1,19 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Del(ctx, tk(1) /* @s3 */) // @0.000000003,0 +db0.Del(ctx, tk(2) /* @s3 */) // @0.000000004,0 +{ + b := &kv.Batch{} + b.Get(tk(1)) // (, ) + b.Get(tk(2)) // (, ) + b.Get(tk(3)) // (, ) + db0.Run(ctx, b) // @0.000000005,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 +/Table/100/"0000000000000002"/0.000000004,0 @ s4 +committed delete missing write at seq s3: [d]/Table/100/"0000000000000002":missing->@s3 +unclaimed writes: [d]/Table/100/"0000000000000002":0.000000004,0->@s4 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes_returning_wrong_values b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes_returning_wrong_values new file mode 100644 index 000000000000..3f0a9c686745 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes_returning_wrong_values @@ -0,0 +1,18 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Del(ctx, tk(1) /* @s3 */) // @0.000000003,0 +db0.Del(ctx, tk(2) /* @s4 */) // @0.000000004,0 +{ + b := &kv.Batch{} + b.Get(tk(1)) // (, ) + b.Get(tk(2)) // (v1, ) + b.Get(tk(3)) // (v2, ) + db0.Run(ctx, b) // @0.000000005,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 +/Table/100/"0000000000000002"/0.000000004,0 @ s4 +committed batch non-atomic timestamps: [r]/Table/100/"0000000000000001":[, 0.000000001,0),[0.000000003,0, )-> [r]/Table/100/"0000000000000002":[0,0, 0,0)->v1 [r]/Table/100/"0000000000000003":[0,0, 0,0)->v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes_with_valid_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes_with_valid_time_overlap new file mode 100644 index 000000000000..698ea629e434 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_and_deletes_with_valid_time_overlap @@ -0,0 +1,17 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Del(ctx, tk(1) /* @s3 */) // @0.000000003,0 +db0.Del(ctx, tk(2) /* @s4 */) // @0.000000004,0 +{ + b := &kv.Batch{} + b.Get(tk(1)) // (, ) + b.Get(tk(2)) // (v2, ) + b.Get(tk(3)) // (, ) + db0.Run(ctx, b) // @0.000000003,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 +/Table/100/"0000000000000002"/0.000000004,0 @ s4 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_returning_wrong_values b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_returning_wrong_values new file mode 100644 index 000000000000..e9f506b017bf --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_returning_wrong_values @@ -0,0 +1,14 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +{ + b := &kv.Batch{} + b.Get(tk(1)) // (, ) + b.Get(tk(2)) // (v1, ) + b.Get(tk(3)) // (v2, ) + db0.Run(ctx, b) // @0.000000003,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +committed batch non-atomic timestamps: [r]/Table/100/"0000000000000001":[, 0.000000001,0)-> [r]/Table/100/"0000000000000002":[0,0, 0,0)->v1 [r]/Table/100/"0000000000000003":[0,0, 0,0)->v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_with_empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_with_empty_time_overlap new file mode 100644 index 000000000000..4231ce3ee202 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_reads_after_writes_with_empty_time_overlap @@ -0,0 +1,14 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +{ + b := &kv.Batch{} + b.Get(tk(1)) // (, ) + b.Get(tk(2)) // (v2, ) + b.Get(tk(3)) // (, ) + db0.Run(ctx, b) // @0.000000003,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +committed batch non-atomic timestamps: [r]/Table/100/"0000000000000001":[, 0.000000001,0)-> [r]/Table/100/"0000000000000002":[0.000000002,0, )->v2 [r]/Table/100/"0000000000000003":[, )-> diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_scans_after_writes b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_scans_after_writes new file mode 100644 index 000000000000..2a5025269f48 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_scans_after_writes @@ -0,0 +1,13 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +{ + b := &kv.Batch{} + b.Scan(tk(1), tk(3)) // (/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v2, ) + b.Scan(tk(2), tk(4)) // (/Table/100/"0000000000000002":v2, ) + b.Scan(tk(3), tk(5)) // + db0.Run(ctx, b) // @0.000000003,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_scans_after_writes_returning_wrong_values b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_scans_after_writes_returning_wrong_values new file mode 100644 index 000000000000..49e4b14bafb3 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_scans_after_writes_returning_wrong_values @@ -0,0 +1,14 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +{ + b := &kv.Batch{} + b.Scan(tk(1), tk(3)) // + b.Scan(tk(2), tk(4)) // (/Table/100/"0000000000000002":v1, ) + b.Scan(tk(3), tk(5)) // (/Table/100/"0000000000000003":v2, ) + db0.Run(ctx, b) // @0.000000003,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +committed batch non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{gap:[, 0.000000001,0)}->[] [s]/Table/100/"000000000000000{2"-4"}:{0:[0,0, 0,0), gap:[, )}->[/Table/100/"0000000000000002":v1] [s]/Table/100/"000000000000000{3"-5"}:{0:[0,0, 0,0), gap:[, )}->[/Table/100/"0000000000000003":v2] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_scans_after_writes_with_non-empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_scans_after_writes_with_non-empty_time_overlap new file mode 100644 index 000000000000..b20b70b45e47 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_scans_after_writes_with_non-empty_time_overlap @@ -0,0 +1,14 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +{ + b := &kv.Batch{} + b.Scan(tk(1), tk(3)) // (/Table/100/"0000000000000002":v1, ) + b.Scan(tk(2), tk(4)) // (/Table/100/"0000000000000002":v1, ) + b.Scan(tk(3), tk(5)) // + db0.Run(ctx, b) // @0.000000003,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +committed batch non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0,0, 0,0), gap:[, 0.000000001,0)}->[/Table/100/"0000000000000002":v1] [s]/Table/100/"000000000000000{2"-4"}:{0:[0,0, 0,0), gap:[, )}->[/Table/100/"0000000000000002":v1] [s]/Table/100/"000000000000000{3"-5"}:{gap:[, )}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_touching_rangedels b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_touching_rangedels new file mode 100644 index 000000000000..c50abdb141ef --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_touching_rangedels @@ -0,0 +1,10 @@ +echo +---- +{ + b := &kv.Batch{} + b.DelRangeUsingTombstone(ctx, tk(1), tk(2) /* @s1 */) // + b.DelRangeUsingTombstone(ctx, tk(2), tk(4) /* @s2 */) // + db0.Run(ctx, b) // @0.000000001,0 +} +/Table/100/"000000000000000{1"-2"}/0.000000001,0 @ s1 +/Table/100/"000000000000000{2"-4"}/0.000000001,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_two_overlapping_rangedels b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_two_overlapping_rangedels new file mode 100644 index 000000000000..46e41175707e --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_of_two_overlapping_rangedels @@ -0,0 +1,10 @@ +echo +---- +{ + b := &kv.Batch{} + b.DelRangeUsingTombstone(ctx, tk(1), tk(3) /* @s1 */) // + b.DelRangeUsingTombstone(ctx, tk(2), tk(4) /* @s2 */) // + db0.Run(ctx, b) // @0.000000001,0 +} +/Table/100/"000000000000000{1"-2"}/0.000000001,0 @ s1 +/Table/100/"000000000000000{2"-4"}/0.000000001,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/batch_with_two_deletes_of_same_key b/pkg/kv/kvnemesis/testdata/TestValidate/batch_with_two_deletes_of_same_key new file mode 100644 index 000000000000..22ab36d8e6b6 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/batch_with_two_deletes_of_same_key @@ -0,0 +1,9 @@ +echo +---- +{ + b := &kv.Batch{} + b.Del(tk(1) /* @s1 */) // + b.Del(tk(1) /* @s2 */) // + db0.Run(ctx, b) // @0.000000001,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/no_ops_and_no_kvs b/pkg/kv/kvnemesis/testdata/TestValidate/no_ops_and_no_kvs new file mode 100644 index 000000000000..ad07fd2183e0 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/no_ops_and_no_kvs @@ -0,0 +1,2 @@ +echo +---- diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/no_ops_with_unexpected_delete b/pkg/kv/kvnemesis/testdata/TestValidate/no_ops_with_unexpected_delete new file mode 100644 index 000000000000..78c0c00cfad9 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/no_ops_with_unexpected_delete @@ -0,0 +1,4 @@ +echo +---- +/Table/100/"0000000000000001"/0.000000001,0 @ s1 +unclaimed writes: [d]/Table/100/"0000000000000001":0.000000001,0->@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/no_ops_with_unexpected_write b/pkg/kv/kvnemesis/testdata/TestValidate/no_ops_with_unexpected_write new file mode 100644 index 000000000000..56544656bdce --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/no_ops_with_unexpected_write @@ -0,0 +1,4 @@ +echo +---- +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +unclaimed writes: [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_delete_with_failed_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_delete_with_failed_write new file mode 100644 index 000000000000..1a7ca61987d6 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_delete_with_failed_write @@ -0,0 +1,3 @@ +echo +---- +db0.Del(ctx, tk(1) /* @s1 */) // result is ambiguous: boom diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_delete_with_failed_write_before_a_later_committed_delete b/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_delete_with_failed_write_before_a_later_committed_delete new file mode 100644 index 000000000000..5e400c98452d --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_delete_with_failed_write_before_a_later_committed_delete @@ -0,0 +1,5 @@ +echo +---- +db0.Del(ctx, tk(1) /* @s1 */) // result is ambiguous: boom +db0.Del(ctx, tk(1) /* @s2 */) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_delete_with_successful_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_delete_with_successful_write new file mode 100644 index 000000000000..cc90b313e7b9 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_delete_with_successful_write @@ -0,0 +1,4 @@ +echo +---- +db0.Del(ctx, tk(1) /* @s1 */) // result is ambiguous: boom +/Table/100/"0000000000000001"/0.000000001,0 @ s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_put_with_failed_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_put_with_failed_write new file mode 100644 index 000000000000..5cd8871b8f94 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_put_with_failed_write @@ -0,0 +1,3 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // result is ambiguous: boom diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_put_with_successful_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_put_with_successful_write new file mode 100644 index 000000000000..3ed8894c1361 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_ambiguous_put_with_successful_write @@ -0,0 +1,4 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // result is ambiguous: boom +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_delete_with_missing_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_delete_with_missing_write new file mode 100644 index 000000000000..a9fec03aa3e5 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_delete_with_missing_write @@ -0,0 +1,8 @@ +echo +---- +{ + b := &kv.Batch{} + b.Del(tk(1) /* @s1 */) // + db0.Run(ctx, b) // @0.000000001,0 +} +committed batch missing write at seq s1: [d]/Table/100/"0000000000000001":missing->@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_delete_with_successful_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_delete_with_successful_write new file mode 100644 index 000000000000..7f7ed17672a4 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_delete_with_successful_write @@ -0,0 +1,8 @@ +echo +---- +{ + b := &kv.Batch{} + b.Del(tk(1) /* @s1 */) // + db0.Run(ctx, b) // @0.000000001,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_put_with_missing_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_put_with_missing_write new file mode 100644 index 000000000000..a0a5e99b7467 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_put_with_missing_write @@ -0,0 +1,8 @@ +echo +---- +{ + b := &kv.Batch{} + b.Put(tk(1), sv(1)) // + db0.Run(ctx, b) // @0.000000001,0 +} +committed batch missing write at seq s1: [w]/Table/100/"0000000000000001":missing->v1@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_put_with_successful_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_put_with_successful_write new file mode 100644 index 000000000000..d42b4c5f4935 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_batch_put_with_successful_write @@ -0,0 +1,8 @@ +echo +---- +{ + b := &kv.Batch{} + b.Put(tk(1), sv(1)) // + db0.Run(ctx, b) // @0.000000001,0 +} +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_delete_with_expected_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_delete_with_expected_write new file mode 100644 index 000000000000..67904a9ee043 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_delete_with_expected_write @@ -0,0 +1,4 @@ +echo +---- +db0.Del(ctx, tk(1) /* @s1 */) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_delete_with_expected_write_after_write_transaction_with_shadowed_delete b/pkg/kv/kvnemesis/testdata/TestValidate/one_delete_with_expected_write_after_write_transaction_with_shadowed_delete new file mode 100644 index 000000000000..40007c673f87 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_delete_with_expected_write_after_write_transaction_with_shadowed_delete @@ -0,0 +1,15 @@ +echo +---- +db0.Del(ctx, tk(1) /* @s1 */) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(3)) // + txn.Del(ctx, tk(1) /* @s4 */) // + txn.Put(ctx, tk(1), sv(5)) // + return nil +}) // @0.000000003,0 +db0.Del(ctx, tk(1) /* @s6 */) // @0.000000004,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000001"/0.000000003,0 @ s5 v5 +/Table/100/"0000000000000001"/0.000000004,0 @ s6 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_delete_with_missing_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_delete_with_missing_write new file mode 100644 index 000000000000..99f4ff05f40d --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_delete_with_missing_write @@ -0,0 +1,4 @@ +echo +---- +db0.Del(ctx, tk(1) /* @s1 */) // @0.000000001,0 +committed delete missing write at seq s1: [d]/Table/100/"0000000000000001":missing->@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write new file mode 100644 index 000000000000..a567d19a14e3 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write @@ -0,0 +1,9 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s2 */) // (/Table/100/"0000000000000001", ) + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_extra_deletion b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_extra_deletion new file mode 100644 index 000000000000..c7bd78b28dab --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_extra_deletion @@ -0,0 +1,11 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000002,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s2 */) // @0.000000002,0 (/Table/100/"0000000000000001", /Table/100/"0000000000000002", ) + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +mismatched write timestamp 0.000000001,0 and exec timestamp 0.000000002,0: [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 +committed txn missing write at seq s2: [dr.d]/Table/100/"0000000000000001":0.000000002,0->@s2 [dr.d]/Table/100/"0000000000000002":missing->@s2 [dr.s]/Table/100/"000000000000000{1"-3"}:{gap:[, 0.000000001,0),[0.000000002,0, )}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_missing_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_missing_write new file mode 100644 index 000000000000..f30530cfb26b --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_missing_write @@ -0,0 +1,9 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s2 */) // @0.000000002,0 (/Table/100/"0000000000000001", ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +committed txn missing write at seq s2: [dr.d]/Table/100/"0000000000000001":missing->@s2 [dr.s]/Table/100/"000000000000000{1"-3"}:{gap:[, 0.000000001,0)}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_returning_wrong_value b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_returning_wrong_value new file mode 100644 index 000000000000..85bf7e692c0f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_returning_wrong_value @@ -0,0 +1,10 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s2 */) // @0.000000002,0 + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +unclaimed writes: [d]/Table/100/"0000000000000001":0.000000002,0->@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_with_spurious_deletion b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_with_spurious_deletion new file mode 100644 index 000000000000..a42c70f783bc --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_write_with_spurious_deletion @@ -0,0 +1,10 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s2 */) // @0.000000002,0 (/Table/100/"0000000000000001", /Table/100/"0000000000000002", ) + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes new file mode 100644 index 000000000000..35b8fc8ffe30 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes @@ -0,0 +1,15 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(3), sv(3)) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s4 */) // (/Table/100/"0000000000000001", /Table/100/"0000000000000002", ) + return nil +}) // @0.000000004,0 +db0.Scan(ctx, tk(1), tk(4), 0) // @0.000000004,0 (/Table/100/"0000000000000003":v3, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000003"/0.000000003,0 @ s3 v3 +/Table/100/"0000000000000001"/0.000000004,0 @ s4 +/Table/100/"0000000000000002"/0.000000004,0 @ s4 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_and_delete b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_and_delete new file mode 100644 index 000000000000..e1e552310d97 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_and_delete @@ -0,0 +1,18 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Del(ctx, tk(1) /* @s3 */) // @0.000000004,0 +db0.Put(ctx, tk(1), sv(4)) // @0.000000005,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s5 */) // (/Table/100/"0000000000000001", /Table/100/"0000000000000002", ) + return nil +}) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000001"/0.000000003,0 @ s5 +/Table/100/"0000000000000002"/0.000000003,0 @ s5 +/Table/100/"0000000000000001"/0.000000004,0 @ s3 +/Table/100/"0000000000000002"/0.000000005,0 @ s4 v4 +committed put missing write at seq s4: [w]/Table/100/"0000000000000001":missing->v4@s4 +unclaimed writes: [w]/Table/100/"0000000000000002":0.000000005,0->v4@s4 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_incorrectly_deleting_keys_outside_span_boundary b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_incorrectly_deleting_keys_outside_span_boundary new file mode 100644 index 000000000000..20f93b91c4e6 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_incorrectly_deleting_keys_outside_span_boundary @@ -0,0 +1,12 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(4), sv(2)) // @0.000000002,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s3 */) // (/Table/100/"0000000000000001", /Table/100/"0000000000000004", ) + return nil +}) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 +/Table/100/"0000000000000004"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000004"/0.000000003,0 @ s3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_returning_keys_outside_span_boundary b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_returning_keys_outside_span_boundary new file mode 100644 index 000000000000..73d4d9767a5f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_returning_keys_outside_span_boundary @@ -0,0 +1,12 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(4), sv(2)) // @0.000000002,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s3 */) // (/Table/100/"0000000000000001", /Table/100/"0000000000000004", ) + return nil +}) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 +/Table/100/"0000000000000004"/0.000000002,0 @ s2 v2 +committed txn missing write at seq s3: [dr.d]/Table/100/"0000000000000001":0.000000003,0->@s3 [dr.d]/Table/100/"0000000000000004":missing->@s3 [dr.s]/Table/100/"000000000000000{1"-3"}:{gap:[, 0.000000001,0),[0.000000003,0, )}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_with_missing_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_with_missing_write new file mode 100644 index 000000000000..25d9843cbd42 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_with_missing_write @@ -0,0 +1,16 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(3), sv(3)) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s4 */) // (/Table/100/"0000000000000001", /Table/100/"0000000000000002", ) + return nil +}) // @0.000000004,0 +db0.Scan(ctx, tk(1), tk(4), 0) // @0.000000005,0 (/Table/100/"0000000000000003":v3, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000003"/0.000000003,0 @ s3 v3 +/Table/100/"0000000000000001"/0.000000004,0 @ s4 +committed txn missing write at seq s4: [dr.d]/Table/100/"0000000000000001":0.000000004,0->@s4 [dr.d]/Table/100/"0000000000000002":missing->@s4 [dr.s]/Table/100/"000000000000000{1"-3"}:{gap:[, 0.000000001,0),[0.000000004,0, )}->[] +committed scan non-atomic timestamps: [s]/Table/100/"000000000000000{1"-4"}:{0:[0.000000003,0, ), gap:[, 0.000000001,0)}->[/Table/100/"0000000000000003":v3] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_with_write_timestamp_disagreement b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_with_write_timestamp_disagreement new file mode 100644 index 000000000000..9029aad3f854 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_after_writes_with_write_timestamp_disagreement @@ -0,0 +1,16 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(3), sv(3)) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s4 */) // (/Table/100/"0000000000000001", /Table/100/"0000000000000002", /Table/100/"0000000000000003", ) + return nil +}) // @0.000000004,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000003"/0.000000003,0 @ s3 v3 +/Table/100/"0000000000000001"/0.000000003,0 @ s4 +/Table/100/"0000000000000002"/0.000000004,0 @ s4 +/Table/100/"0000000000000003"/0.000000004,0 @ s4 +committed txn non-atomic timestamps: [dr.d]/Table/100/"0000000000000001":0.000000003,0->@s4 [dr.d]/Table/100/"0000000000000002":0.000000004,0->@s4 [dr.d]/Table/100/"0000000000000003":0.000000004,0->@s4 [dr.s]/Table/100/"000000000000000{1"-3"}:{gap:[, 0.000000001,0),[0.000000004,0, )}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_before_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_before_write new file mode 100644 index 000000000000..1f54c286d69c --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_before_write @@ -0,0 +1,5 @@ +echo +---- +db0.DelRange(ctx, tk(1), tk(3), true /* @s1 */) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_before_write_returning_wrong_value b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_before_write_returning_wrong_value new file mode 100644 index 000000000000..b2bfe819e6dc --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_deleterange_before_write_returning_wrong_value @@ -0,0 +1,6 @@ +echo +---- +db0.DelRange(ctx, tk(1), tk(3), true /* @s1 */) // @0.000000001,0 (/Table/100/"0000000000000001", ) +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +committed deleteRange missing write at seq s1: [dr.d]/Table/100/"0000000000000001":missing->@s1 [dr.s]/Table/100/"000000000000000{1"-3"}:{gap:[, 0.000000002,0)}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_put_with_expected_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_put_with_expected_write new file mode 100644 index 000000000000..78135507c677 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_put_with_expected_write @@ -0,0 +1,4 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_put_with_missing_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_put_with_missing_write new file mode 100644 index 000000000000..678e6eb9b359 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_put_with_missing_write @@ -0,0 +1,4 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +committed put missing write at seq s1: [w]/Table/100/"0000000000000001":missing->v1@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write new file mode 100644 index 000000000000..1451d31e529a --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write @@ -0,0 +1,5 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Get(ctx, tk(1)) // @0.000000002,0 (v1, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write_and_delete b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write_and_delete new file mode 100644 index 000000000000..b71c9ca8db6d --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write_and_delete @@ -0,0 +1,7 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Del(ctx, tk(1) /* @s2 */) // @0.000000002,0 +db0.Get(ctx, tk(1)) // @0.000000001,0 (v1, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write_and_delete_returning_tombstone b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write_and_delete_returning_tombstone new file mode 100644 index 000000000000..059b0b537a44 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write_and_delete_returning_tombstone @@ -0,0 +1,7 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Del(ctx, tk(1) /* @s2 */) // @0.000000002,0 +db0.Get(ctx, tk(1)) // @0.000000003,0 (, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write_returning_wrong_value b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write_returning_wrong_value new file mode 100644 index 000000000000..50947bda6207 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_after_write_returning_wrong_value @@ -0,0 +1,6 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Get(ctx, tk(1)) // @0.000000002,0 (v2, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +committed get non-atomic timestamps: [r]/Table/100/"0000000000000001":[0,0, 0,0)->v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_delete b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_delete new file mode 100644 index 000000000000..6bb53a6abb84 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_delete @@ -0,0 +1,5 @@ +echo +---- +db0.Get(ctx, tk(1)) // @0.000000001,0 (, ) +db0.Del(ctx, tk(1) /* @s1 */) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000002,0 @ s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_write new file mode 100644 index 000000000000..04e0292bb9a5 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_write @@ -0,0 +1,5 @@ +echo +---- +db0.Get(ctx, tk(1)) // @0.000000001,0 (, ) +db0.Put(ctx, tk(1), sv(1)) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000002,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_write_and_delete b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_write_and_delete new file mode 100644 index 000000000000..f183fc26685e --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_write_and_delete @@ -0,0 +1,7 @@ +echo +---- +db0.Get(ctx, tk(1)) // @0.000000001,0 (, ) +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Del(ctx, tk(1) /* @s2 */) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_write_returning_wrong_value b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_write_returning_wrong_value new file mode 100644 index 000000000000..27479f01dccc --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_before_write_returning_wrong_value @@ -0,0 +1,5 @@ +echo +---- +db0.Get(ctx, tk(1)) // @0.000000001,0 (v1, ) +db0.Put(ctx, tk(1), sv(1)) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000002,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_read_in_between_write_and_delete b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_in_between_write_and_delete new file mode 100644 index 000000000000..33db56a3906a --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_in_between_write_and_delete @@ -0,0 +1,7 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Get(ctx, tk(1)) // @0.000000002,0 (v1, ) +db0.Del(ctx, tk(1) /* @s2 */) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000003,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_read_in_between_writes b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_in_between_writes new file mode 100644 index 000000000000..b4c4ac064e7d --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_read_in_between_writes @@ -0,0 +1,7 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Get(ctx, tk(1)) // @0.000000002,0 (v1, ) +db0.Put(ctx, tk(1), sv(2)) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000003,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_delete_with_write_correctly_missing b/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_delete_with_write_correctly_missing new file mode 100644 index 000000000000..b0eed16f1532 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_delete_with_write_correctly_missing @@ -0,0 +1,3 @@ +echo +---- +db0.Del(ctx, tk(1) /* @s1 */) // TransactionRetryWithProtoRefreshError: diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_delete_with_write_incorrectly_present b/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_delete_with_write_incorrectly_present new file mode 100644 index 000000000000..9aa60e0e1cd3 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_delete_with_write_incorrectly_present @@ -0,0 +1,5 @@ +echo +---- +db0.Del(ctx, tk(1) /* @s1 */) // TransactionRetryWithProtoRefreshError: +/Table/100/"0000000000000001"/0.000000001,0 @ s1 +unclaimed writes: [d]/Table/100/"0000000000000001":0.000000001,0->@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_put_with_write_correctly_missing b/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_put_with_write_correctly_missing new file mode 100644 index 000000000000..ab767141682f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_put_with_write_correctly_missing @@ -0,0 +1,3 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // TransactionRetryWithProtoRefreshError: diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_put_with_write_incorrectly_present b/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_put_with_write_incorrectly_present new file mode 100644 index 000000000000..1bbaf288dbf5 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_retryable_put_with_write_incorrectly_present @@ -0,0 +1,5 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 TransactionRetryWithProtoRefreshError: +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +unclaimed writes: [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_write_returning_extra_key b/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_write_returning_extra_key new file mode 100644 index 000000000000..1313b6b3b80c --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_write_returning_extra_key @@ -0,0 +1,8 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(3), sv(2)) // @0.000000002,0 +db0.ReverseScan(ctx, tk(1), tk(4), 0) // @0.000000003,0 (/Table/100/"0000000000000003":v2, /Table/100/"0000000000000002":v3, /Table/100/"0000000000000001":v1, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000003"/0.000000002,0 @ s2 v2 +committed reverse scan non-atomic timestamps: [rs]/Table/100/"000000000000000{1"-4"}:{0:[0.000000002,0, ), 1:[0,0, 0,0), 2:[0.000000001,0, ), gap:[, )}->[/Table/100/"0000000000000003":v2, /Table/100/"0000000000000002":v3, /Table/100/"0000000000000001":v1] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_write_returning_missing_key b/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_write_returning_missing_key new file mode 100644 index 000000000000..a2c304c28064 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_write_returning_missing_key @@ -0,0 +1,8 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.ReverseScan(ctx, tk(1), tk(3), 0) // @0.000000003,0 (/Table/100/"0000000000000002":v2, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +committed reverse scan non-atomic timestamps: [rs]/Table/100/"000000000000000{1"-3"}:{0:[0.000000002,0, ), gap:[, 0.000000001,0)}->[/Table/100/"0000000000000002":v2] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_writes b/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_writes new file mode 100644 index 000000000000..2d304676341f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_writes @@ -0,0 +1,7 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.ReverseScan(ctx, tk(1), tk(3), 0) // @0.000000003,0 (/Table/100/"0000000000000002":v2, /Table/100/"0000000000000001":v1, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_writes_returning_results_in_wrong_order b/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_writes_returning_results_in_wrong_order new file mode 100644 index 000000000000..494297c9a5b8 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_writes_returning_results_in_wrong_order @@ -0,0 +1,8 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.ReverseScan(ctx, tk(1), tk(3), 0) // @0.000000003,0 (/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v2, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +scan result not ordered correctly: [rs]/Table/100/"000000000000000{1"-3"}:{0:[0.000000001,0, ), 1:[0.000000002,0, ), gap:[, )}->[/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v2] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_writes_returning_results_outside_scan_boundary b/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_writes_returning_results_outside_scan_boundary new file mode 100644 index 000000000000..20a8412fd066 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_reverse_scan_after_writes_returning_results_outside_scan_boundary @@ -0,0 +1,10 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(3), sv(3)) // @0.000000003,0 +db0.ReverseScan(ctx, tk(1), tk(3), 0) // @0.000000004,0 (/Table/100/"0000000000000003":v3, /Table/100/"0000000000000002":v2, /Table/100/"0000000000000001":v1, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000003"/0.000000003,0 @ s3 v3 +key /Table/100/"0000000000000003" outside scan bounds: [rs]/Table/100/"000000000000000{1"-3"}:{0:[0.000000003,0, ), 1:[0.000000002,0, ), 2:[0.000000001,0, ), gap:[, )}->[/Table/100/"0000000000000003":v3, /Table/100/"0000000000000002":v2, /Table/100/"0000000000000001":v1] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write new file mode 100644 index 000000000000..9f1736202593 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write @@ -0,0 +1,5 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000002,0 (/Table/100/"0000000000000001":v1, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write_returning_extra_key b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write_returning_extra_key new file mode 100644 index 000000000000..6d0d536c5b0f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write_returning_extra_key @@ -0,0 +1,8 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(3), sv(2)) // @0.000000002,0 +db0.Scan(ctx, tk(1), tk(4), 0) // @0.000000003,0 (/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v3, /Table/100/"0000000000000002":v2, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000003"/0.000000002,0 @ s2 v2 +committed scan non-atomic timestamps: [s]/Table/100/"000000000000000{1"-4"}:{0:[0.000000001,0, ), 1:[0,0, 0,0), 2:[0,0, 0,0), gap:[, 0.000000002,0)}->[/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v3, /Table/100/"0000000000000002":v2] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write_returning_missing_key b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write_returning_missing_key new file mode 100644 index 000000000000..9b1953bd4a54 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write_returning_missing_key @@ -0,0 +1,8 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000003,0 (/Table/100/"0000000000000002":v2, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +committed scan non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000002,0, ), gap:[, 0.000000001,0)}->[/Table/100/"0000000000000002":v2] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write_returning_wrong_value b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write_returning_wrong_value new file mode 100644 index 000000000000..d4cac13c02a8 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_write_returning_wrong_value @@ -0,0 +1,6 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000002,0 (/Table/100/"0000000000000001":v2, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +committed scan non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0,0, 0,0), gap:[, )}->[/Table/100/"0000000000000001":v2] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes new file mode 100644 index 000000000000..eedab2b43811 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes @@ -0,0 +1,7 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000003,0 (/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v2, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_and_delete b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_and_delete new file mode 100644 index 000000000000..8c6d5e1eb15a --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_and_delete @@ -0,0 +1,11 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Del(ctx, tk(1) /* @s3 */) // @0.000000003,0 +db0.Put(ctx, tk(1), sv(4)) // @0.000000004,0 +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000005,0 (/Table/100/"0000000000000002":v2, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 +/Table/100/"0000000000000001"/0.000000004,0 @ s4 v4 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_and_delete_returning_missing_key b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_and_delete_returning_missing_key new file mode 100644 index 000000000000..4afe769b2714 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_and_delete_returning_missing_key @@ -0,0 +1,20 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Put(ctx, tk(2), sv(2)) // + return nil +}) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000002":v2, ) + txn.Del(ctx, tk(1) /* @s3 */) // + return nil +}) // @0.000000002,0 +db0.Put(ctx, tk(1), sv(4)) // @0.000000003,0 +db0.Del(ctx, tk(1) /* @s5 */) // @0.000000004,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000001,0 @ s2 v2 +/Table/100/"0000000000000001"/0.000000002,0 @ s3 +/Table/100/"0000000000000001"/0.000000003,0 @ s4 v4 +/Table/100/"0000000000000001"/0.000000004,0 @ s5 +committed txn non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000001,0, ), gap:[, 0.000000001,0),[0.000000004,0, )}->[/Table/100/"0000000000000002":v2] [d]/Table/100/"0000000000000001":0.000000002,0->@s3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_returning_results_in_wrong_order b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_returning_results_in_wrong_order new file mode 100644 index 000000000000..9680d73df1c0 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_returning_results_in_wrong_order @@ -0,0 +1,8 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000003,0 (/Table/100/"0000000000000002":v2, /Table/100/"0000000000000001":v1, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +scan result not ordered correctly: [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000002,0, ), 1:[0.000000001,0, ), gap:[, )}->[/Table/100/"0000000000000002":v2, /Table/100/"0000000000000001":v1] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_returning_results_outside_scan_boundary b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_returning_results_outside_scan_boundary new file mode 100644 index 000000000000..a28499b79fa9 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_after_writes_returning_results_outside_scan_boundary @@ -0,0 +1,10 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(3), sv(3)) // @0.000000003,0 +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000004,0 (/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v2, /Table/100/"0000000000000003":v3, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000003"/0.000000003,0 @ s3 v3 +key /Table/100/"0000000000000003" outside scan bounds: [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000001,0, ), 1:[0.000000002,0, ), 2:[0.000000003,0, ), gap:[, )}->[/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v2, /Table/100/"0000000000000003":v3] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_before_write b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_before_write new file mode 100644 index 000000000000..c092257a6b75 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_before_write @@ -0,0 +1,5 @@ +echo +---- +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(1)) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000002,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_before_write_returning_wrong_value b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_before_write_returning_wrong_value new file mode 100644 index 000000000000..96c7e52cb018 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_before_write_returning_wrong_value @@ -0,0 +1,6 @@ +echo +---- +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000001,0 (/Table/100/"0000000000000001":v2, ) +db0.Put(ctx, tk(1), sv(1)) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000002,0 @ s1 v1 +committed scan non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0,0, 0,0), gap:[, )}->[/Table/100/"0000000000000001":v2] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_in_between_writes b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_in_between_writes new file mode 100644 index 000000000000..3e48031d83bc --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_scan_in_between_writes @@ -0,0 +1,7 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000002,0 (/Table/100/"0000000000000001":v1, ) +db0.Put(ctx, tk(1), sv(2)) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000003,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_tranactional_scan_after_write_and_delete_returning_extra_key b/pkg/kv/kvnemesis/testdata/TestValidate/one_tranactional_scan_after_write_and_delete_returning_extra_key new file mode 100644 index 000000000000..a37ed9ea44cd --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_tranactional_scan_after_write_and_delete_returning_extra_key @@ -0,0 +1,13 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(2), sv(2)) // + txn.Del(ctx, tk(1) /* @s3 */) // + return nil +}) // @0.000000002,0 +db0.Scan(ctx, tk(1), tk(3), 0) // @0.000000003,0 (/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v2, ) +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s3 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +committed scan non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000001,0, 0.000000002,0), 1:[0.000000002,0, ), gap:[, )}->[/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v2] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_delete_with_write_on_another_key_after_delete b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_delete_with_write_on_another_key_after_delete new file mode 100644 index 000000000000..b6b2796ce047 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_delete_with_write_on_another_key_after_delete @@ -0,0 +1,11 @@ +echo +---- +db0.Del(ctx, tk(1) /* @s1 */) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(2), sv(2)) // + txn.Del(ctx, tk(1) /* @s3 */) // + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000002,0 @ s3 +/Table/100/"0000000000000001"/0.000000003,0 @ s1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_deleterange_followed_by_put_after_writes b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_deleterange_followed_by_put_after_writes new file mode 100644 index 000000000000..990e212014f2 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_deleterange_followed_by_put_after_writes @@ -0,0 +1,11 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s2 */) // (/Table/100/"0000000000000001", ) + txn.Put(ctx, tk(2), sv(3)) // + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_deleterange_followed_by_put_after_writes_with_write_timestamp_disagreement b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_deleterange_followed_by_put_after_writes_with_write_timestamp_disagreement new file mode 100644 index 000000000000..148d4b5b59e5 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_deleterange_followed_by_put_after_writes_with_write_timestamp_disagreement @@ -0,0 +1,12 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.DelRange(ctx, tk(1), tk(3), true /* @s2 */) // (/Table/100/"0000000000000001", ) + txn.Put(ctx, tk(2), sv(3)) // + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +/Table/100/"0000000000000002"/0.000000003,0 @ s3 v3 +committed txn non-atomic timestamps: [dr.d]/Table/100/"0000000000000001":0.000000002,0->@s2 [dr.s]/Table/100/"000000000000000{1"-3"}:{gap:[, 0.000000001,0),[0.000000003,0, )}->[] [w]/Table/100/"0000000000000002":0.000000003,0->v3@s3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_shadowed_by_deleterange_after_writes b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_shadowed_by_deleterange_after_writes new file mode 100644 index 000000000000..e47bed228b03 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_shadowed_by_deleterange_after_writes @@ -0,0 +1,11 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(2), sv(2)) // + txn.DelRange(ctx, tk(1), tk(3), true /* @s3 */) // (/Table/100/"0000000000000001", /Table/100/"0000000000000002", ) + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s3 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_shadowed_by_deleterange_after_writes_with_write_timestamp_disagreement b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_shadowed_by_deleterange_after_writes_with_write_timestamp_disagreement new file mode 100644 index 000000000000..66eaf67ae945 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_shadowed_by_deleterange_after_writes_with_write_timestamp_disagreement @@ -0,0 +1,12 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(2), sv(2)) // + txn.DelRange(ctx, tk(1), tk(3), true /* @s3 */) // (/Table/100/"0000000000000001", /Table/100/"0000000000000002", ) + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s3 +/Table/100/"0000000000000002"/0.000000003,0 @ s3 +committed txn non-atomic timestamps: [w]/Table/100/"0000000000000002":missing->v2@s2 [dr.d]/Table/100/"0000000000000001":0.000000002,0->@s3 [dr.d]/Table/100/"0000000000000002":0.000000003,0->@s3 [dr.s]/Table/100/"000000000000000{1"-3"}:{gap:[, 0.000000001,0),[0.000000003,0, )}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_with_correct_commit_time b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_with_correct_commit_time new file mode 100644 index 000000000000..e6627602ef30 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_with_correct_commit_time @@ -0,0 +1,7 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_with_incorrect_commit_time b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_with_incorrect_commit_time new file mode 100644 index 000000000000..95d9561a726d --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_put_with_incorrect_commit_time @@ -0,0 +1,8 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000002,0 @ s1 v1 +mismatched write timestamp 0.000000002,0 and exec timestamp 0.000000001,0: [w]/Table/100/"0000000000000001":0.000000002,0->v1@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_scan_followed_by_delete_outside_time_range b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_scan_followed_by_delete_outside_time_range new file mode 100644 index 000000000000..4f9dc09aa499 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_scan_followed_by_delete_outside_time_range @@ -0,0 +1,13 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + txn.Del(ctx, tk(1) /* @s2 */) // + return nil +}) // @0.000000004,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000004,0 @ s2 +/Table/100/"0000000000000002"/0.000000003,0 @ s3 v3 +committed txn non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000001,0, ), gap:[, 0.000000003,0)}->[/Table/100/"0000000000000001":v1] [d]/Table/100/"0000000000000001":0.000000004,0->@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_scan_followed_by_delete_within_time_range b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_scan_followed_by_delete_within_time_range new file mode 100644 index 000000000000..bae4e4c7e557 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactional_scan_followed_by_delete_within_time_range @@ -0,0 +1,12 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + txn.Del(ctx, tk(1) /* @s2 */) // + return nil +}) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +/Table/100/"0000000000000002"/0.000000003,0 @ s3 v3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_first_write_missing b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_first_write_missing new file mode 100644 index 000000000000..e0bb716a3638 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_first_write_missing @@ -0,0 +1,9 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + txn.Del(ctx, tk(2) /* @s2 */) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000002"/0.000000001,0 @ s2 +committed txn missing write at seq s1: [d]/Table/100/"0000000000000001":missing->@s1 [d]/Table/100/"0000000000000002":0.000000001,0->@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_second_write_missing b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_second_write_missing new file mode 100644 index 000000000000..fc08f8073806 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_second_write_missing @@ -0,0 +1,9 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + txn.Del(ctx, tk(2) /* @s2 */) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 +committed txn missing write at seq s2: [d]/Table/100/"0000000000000001":0.000000001,0->@s1 [d]/Table/100/"0000000000000002":missing->@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_the_correct_writes b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_the_correct_writes new file mode 100644 index 000000000000..4ad7f08fa6e4 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_the_correct_writes @@ -0,0 +1,7 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_write_timestamp_disagreement b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_write_timestamp_disagreement new file mode 100644 index 000000000000..02cff9ad0e9e --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_delete_with_write_timestamp_disagreement @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + txn.Del(ctx, tk(2) /* @s2 */) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 +committed txn non-atomic timestamps: [d]/Table/100/"0000000000000001":0.000000001,0->@s1 [d]/Table/100/"0000000000000002":0.000000002,0->@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_first_write_missing b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_first_write_missing new file mode 100644 index 000000000000..1510a600518e --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_first_write_missing @@ -0,0 +1,9 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Put(ctx, tk(2), sv(2)) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000002"/0.000000001,0 @ s2 v2 +committed txn missing write at seq s1: [w]/Table/100/"0000000000000001":missing->v1@s1 [w]/Table/100/"0000000000000002":0.000000001,0->v2@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_second_write_missing b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_second_write_missing new file mode 100644 index 000000000000..c3e0a8c116cf --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_second_write_missing @@ -0,0 +1,9 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Put(ctx, tk(2), sv(2)) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +committed txn missing write at seq s2: [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 [w]/Table/100/"0000000000000002":missing->v2@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_the_correct_writes b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_the_correct_writes new file mode 100644 index 000000000000..e6627602ef30 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_the_correct_writes @@ -0,0 +1,7 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_write_timestamp_disagreement b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_write_timestamp_disagreement new file mode 100644 index 000000000000..6c6c6c7ffc56 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_committed_put_with_write_timestamp_disagreement @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Put(ctx, tk(2), sv(2)) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +committed txn non-atomic timestamps: [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 [w]/Table/100/"0000000000000002":0.000000002,0->v2@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_batch_delete_with_write_correctly_missing b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_batch_delete_with_write_correctly_missing new file mode 100644 index 000000000000..6bf6550d9b18 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_batch_delete_with_write_correctly_missing @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + { + b := &kv.Batch{} + b.Del(tk(1) /* @s1 */) // + txn.Run(ctx, b) // + } + return errors.New("rollback") +}) // rollback diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_batch_put_with_write_correctly_missing b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_batch_put_with_write_correctly_missing new file mode 100644 index 000000000000..a90b9e6a4d1f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_batch_put_with_write_correctly_missing @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + { + b := &kv.Batch{} + b.Put(tk(1), sv(1)) // + txn.Run(ctx, b) // + } + return errors.New("rollback") +}) // rollback diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_delete_with_write_correctly_missing b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_delete_with_write_correctly_missing new file mode 100644 index 000000000000..53f647f981d4 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_delete_with_write_correctly_missing @@ -0,0 +1,6 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + return errors.New("rollback") +}) // rollback diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_delete_with_write_incorrectly_present b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_delete_with_write_incorrectly_present new file mode 100644 index 000000000000..55a5583afe82 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_delete_with_write_incorrectly_present @@ -0,0 +1,8 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + return errors.New("rollback") +}) // rollback +/Table/100/"0000000000000001"/0.000000001,0 @ s1 +uncommitted txn had writes: [d]/Table/100/"0000000000000001":0.000000001,0->@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_put_with_write_correctly_missing b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_put_with_write_correctly_missing new file mode 100644 index 000000000000..323a30b5d435 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_put_with_write_correctly_missing @@ -0,0 +1,6 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + return errors.New("rollback") +}) // rollback diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_put_with_write_incorrectly_present b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_put_with_write_incorrectly_present new file mode 100644 index 000000000000..01907c3b220f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/one_transactionally_rolled_back_put_with_write_incorrectly_present @@ -0,0 +1,8 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + return errors.New("rollback") +}) // rollback +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +uncommitted txn had writes: [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/rangedel_shadowing_scan b/pkg/kv/kvnemesis/testdata/TestValidate/rangedel_shadowing_scan new file mode 100644 index 000000000000..bc1724906397 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/rangedel_shadowing_scan @@ -0,0 +1,7 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.DelRangeUsingTombstone(ctx, tk(1), tk(2) /* @s2 */) // @0.000000002,0 +db0.Scan(ctx, tk(1), tk(2), 0) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"000000000000000{1"-2"}/0.000000002,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/rangedel_with_range_split b/pkg/kv/kvnemesis/testdata/TestValidate/rangedel_with_range_split new file mode 100644 index 000000000000..1e65af86a375 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/rangedel_with_range_split @@ -0,0 +1,6 @@ +echo +---- +db0.DelRangeUsingTombstone(ctx, tk(1), tk(3) /* @s1 */) // @0.000000002,0 +/Table/100/"000000000000000{1"-2"}/0.000000002,0 @ s1 +/Table/100/"000000000000000{2"-3"}/0.000000001,0 @ s1 +committed deleteRangeUsingTombstone non-atomic timestamps: [d][/Table/100/"0000000000000001",/Table/100/"0000000000000002"):0.000000002,0->@s1 [d][/Table/100/"0000000000000002",/Table/100/"0000000000000003"):0.000000001,0->@s1 [s]/Table/100/"000000000000000{1"-3"}:{gap:[, )}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/read_before_rangedel b/pkg/kv/kvnemesis/testdata/TestValidate/read_before_rangedel new file mode 100644 index 000000000000..020be533df63 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/read_before_rangedel @@ -0,0 +1,7 @@ +echo +---- +db0.Put(ctx, tk(2), sv(1)) // @0.000000001,0 +db0.Get(ctx, tk(2)) // @0.000000002,0 (v1, ) +db0.DelRangeUsingTombstone(ctx, tk(1), tk(3) /* @s3 */) // @0.000000003,0 +/Table/100/"0000000000000002"/0.000000001,0 @ s1 v1 +/Table/100/"000000000000000{1"-3"}/0.000000003,0 @ s3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/single_mvcc_rangedel b/pkg/kv/kvnemesis/testdata/TestValidate/single_mvcc_rangedel new file mode 100644 index 000000000000..718bd7888d1f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/single_mvcc_rangedel @@ -0,0 +1,4 @@ +echo +---- +db0.DelRangeUsingTombstone(ctx, tk(1), tk(2) /* @s1 */) // @0.000000001,0 +/Table/100/"000000000000000{1"-2"}/0.000000001,0 @ s1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/single_mvcc_rangedel_after_put b/pkg/kv/kvnemesis/testdata/TestValidate/single_mvcc_rangedel_after_put new file mode 100644 index 000000000000..f08e919f619b --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/single_mvcc_rangedel_after_put @@ -0,0 +1,6 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.DelRangeUsingTombstone(ctx, tk(1), tk(2) /* @s2 */) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"000000000000000{1"-2"}/0.000000002,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/single_mvcc_rangedel_before_put b/pkg/kv/kvnemesis/testdata/TestValidate/single_mvcc_rangedel_before_put new file mode 100644 index 000000000000..c97a4652e8ce --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/single_mvcc_rangedel_before_put @@ -0,0 +1,6 @@ +echo +---- +db0.DelRangeUsingTombstone(ctx, tk(1), tk(2) /* @s1 */) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +/Table/100/"000000000000000{1"-2"}/0.000000001,0 @ s1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_after_delete b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_after_delete new file mode 100644 index 000000000000..dd35166d4f69 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_after_delete @@ -0,0 +1,12 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (v1, ) + txn.Del(ctx, tk(1) /* @s2 */) // + txn.Get(ctx, tk(1)) // (v1, ) + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +committed txn non-atomic timestamps: [r]/Table/100/"0000000000000001":[0.000000001,0, )->v1 [d]/Table/100/"0000000000000001":0.000000002,0->@s2 [r]/Table/100/"0000000000000001":[0.000000001,0, 0.000000002,0)->v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_after_write b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_after_write new file mode 100644 index 000000000000..75beff83bb12 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_after_write @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (, ) + txn.Put(ctx, tk(1), sv(1)) // + txn.Get(ctx, tk(1)) // (, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +committed txn non-atomic timestamps: [r]/Table/100/"0000000000000001":[, )-> [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 [r]/Table/100/"0000000000000001":[, 0.000000001,0)-> diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_before_delete b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_before_delete new file mode 100644 index 000000000000..bd0ec75041ea --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_before_delete @@ -0,0 +1,12 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (, ) + txn.Del(ctx, tk(1) /* @s2 */) // + txn.Get(ctx, tk(1)) // (, ) + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +committed txn non-atomic timestamps: [r]/Table/100/"0000000000000001":[, 0.000000001,0)-> [d]/Table/100/"0000000000000001":0.000000002,0->@s2 [r]/Table/100/"0000000000000001":[, 0.000000001,0),[0.000000002,0, )-> diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_before_write b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_before_write new file mode 100644 index 000000000000..09805740bb49 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_read_before_write @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (v1, ) + txn.Put(ctx, tk(1), sv(1)) // + txn.Get(ctx, tk(1)) // (v1, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +committed txn non-atomic timestamps: [r]/Table/100/"0000000000000001":[0,0, 0,0)->v1 [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 [r]/Table/100/"0000000000000001":[0.000000001,0, )->v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_scan_after_write b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_scan_after_write new file mode 100644 index 000000000000..8779e943e407 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_scan_after_write @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // + txn.Put(ctx, tk(1), sv(1)) // + txn.Scan(ctx, tk(1), tk(3), 0) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +committed txn non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{gap:[, )}->[] [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 [s]/Table/100/"000000000000000{1"-3"}:{gap:[, 0.000000001,0)}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_scan_before_write b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_scan_before_write new file mode 100644 index 000000000000..45114c7023f9 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_incorrect_scan_before_write @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + txn.Put(ctx, tk(1), sv(1)) // + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +committed txn non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0,0, 0,0), gap:[, )}->[/Table/100/"0000000000000001":v1] [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000001,0, ), gap:[, )}->[/Table/100/"0000000000000001":v1] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_read_before_and_after_delete b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_read_before_and_after_delete new file mode 100644 index 000000000000..401159f597a4 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_read_before_and_after_delete @@ -0,0 +1,11 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (v1, ) + txn.Del(ctx, tk(1) /* @s2 */) // + txn.Get(ctx, tk(1)) // (, ) + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_read_before_and_after_write b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_read_before_and_after_write new file mode 100644 index 000000000000..bb0a9589c7b0 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_read_before_and_after_write @@ -0,0 +1,9 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (, ) + txn.Put(ctx, tk(1), sv(1)) // + txn.Get(ctx, tk(1)) // (v1, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_scan_before_and_after_write b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_scan_before_and_after_write new file mode 100644 index 000000000000..98adf607f9b8 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transaction_with_scan_before_and_after_write @@ -0,0 +1,9 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // + txn.Put(ctx, tk(1), sv(1)) // + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_read_and_write_with_empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_read_and_write_with_empty_time_overlap new file mode 100644 index 000000000000..c5e02610f071 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_read_and_write_with_empty_time_overlap @@ -0,0 +1,13 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (v1, ) + txn.Put(ctx, tk(2), sv(3)) // + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 +committed txn non-atomic timestamps: [r]/Table/100/"0000000000000001":[0.000000001,0, 0.000000002,0)->v1 [w]/Table/100/"0000000000000002":0.000000002,0->v3@s3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_read_and_write_with_non-empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_read_and_write_with_non-empty_time_overlap new file mode 100644 index 000000000000..c28e899c9f15 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_read_and_write_with_non-empty_time_overlap @@ -0,0 +1,12 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (v1, ) + txn.Put(ctx, tk(2), sv(3)) // + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000003,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_after_writes_and_deletes_with_empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_after_writes_and_deletes_with_empty_time_overlap new file mode 100644 index 000000000000..362d831edfee --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_after_writes_and_deletes_with_empty_time_overlap @@ -0,0 +1,20 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s3 */) // + txn.Del(ctx, tk(2) /* @s4 */) // + return nil +}) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (, ) + txn.Get(ctx, tk(2)) // (v2, ) + txn.Get(ctx, tk(3)) // (, ) + return nil +}) // @0.000000004,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 +/Table/100/"0000000000000002"/0.000000003,0 @ s4 +committed txn non-atomic timestamps: [r]/Table/100/"0000000000000001":[, 0.000000001,0),[0.000000003,0, )-> [r]/Table/100/"0000000000000002":[0.000000002,0, 0.000000003,0)->v2 [r]/Table/100/"0000000000000003":[, )-> diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_after_writes_and_deletes_with_non-empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_after_writes_and_deletes_with_non-empty_time_overlap new file mode 100644 index 000000000000..cb4c766c5327 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_after_writes_and_deletes_with_non-empty_time_overlap @@ -0,0 +1,16 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(2), sv(2)) // @0.000000002,0 +db0.Del(ctx, tk(1) /* @s3 */) // @0.000000003,0 +db0.Del(ctx, tk(2) /* @s4 */) // @0.000000004,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (, ) + txn.Get(ctx, tk(2)) // (v2, ) + txn.Get(ctx, tk(3)) // (, ) + return nil +}) // @0.000000004,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000002"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 +/Table/100/"0000000000000002"/0.000000004,0 @ s4 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_and_deletes_after_write_with_empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_and_deletes_after_write_with_empty_time_overlap new file mode 100644 index 000000000000..1d5f79132f1e --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_and_deletes_after_write_with_empty_time_overlap @@ -0,0 +1,16 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (, ) + txn.Del(ctx, tk(1) /* @s2 */) // + txn.Get(ctx, tk(1)) // (, ) + return nil +}) // @0.000000002,0 +db0.Put(ctx, tk(1), sv(3)) // @0.000000003,0 +db0.Del(ctx, tk(1) /* @s4 */) // @0.000000004,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 v3 +/Table/100/"0000000000000001"/0.000000004,0 @ s4 +committed txn non-atomic timestamps: [r]/Table/100/"0000000000000001":[, 0.000000001,0),[0.000000004,0, )-> [d]/Table/100/"0000000000000001":0.000000002,0->@s2 [r]/Table/100/"0000000000000001":[, 0.000000001,0),[0.000000004,0, ),[0.000000002,0, 0.000000003,0)-> diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_and_deletes_after_write_with_non-empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_and_deletes_after_write_with_non-empty_time_overlap new file mode 100644 index 000000000000..4fc0b7b3f529 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_and_deletes_after_write_with_non-empty_time_overlap @@ -0,0 +1,15 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (v1, ) + txn.Del(ctx, tk(1) /* @s2 */) // + txn.Get(ctx, tk(1)) // (, ) + return nil +}) // @0.000000002,0 +db0.Put(ctx, tk(1), sv(3)) // @0.000000003,0 +db0.Del(ctx, tk(1) /* @s4 */) // @0.000000004,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +/Table/100/"0000000000000001"/0.000000003,0 @ s3 v3 +/Table/100/"0000000000000001"/0.000000004,0 @ s4 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_one_missing_with_empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_one_missing_with_empty_time_overlap new file mode 100644 index 000000000000..709315e98514 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_one_missing_with_empty_time_overlap @@ -0,0 +1,14 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (v1, ) + txn.Get(ctx, tk(2)) // (, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000001,0 @ s3 v3 +committed txn non-atomic timestamps: [r]/Table/100/"0000000000000001":[0.000000001,0, 0.000000002,0)->v1 [r]/Table/100/"0000000000000002":[, 0.000000001,0)-> diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_one_missing_with_non-empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_one_missing_with_non-empty_time_overlap new file mode 100644 index 000000000000..b175ef04ede1 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_one_missing_with_non-empty_time_overlap @@ -0,0 +1,13 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000002,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (v1, ) + txn.Get(ctx, tk(2)) // (, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_with_empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_with_empty_time_overlap new file mode 100644 index 000000000000..fafd5f21e22b --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_with_empty_time_overlap @@ -0,0 +1,16 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(4)) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (v1, ) + txn.Get(ctx, tk(2)) // (v3, ) + return nil +}) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 +/Table/100/"0000000000000002"/0.000000003,0 @ s4 v4 +committed txn non-atomic timestamps: [r]/Table/100/"0000000000000001":[0.000000001,0, 0.000000002,0)->v1 [r]/Table/100/"0000000000000002":[0.000000002,0, 0.000000003,0)->v3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_with_non-empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_with_non-empty_time_overlap new file mode 100644 index 000000000000..6b26967055fb --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_reads_with_non-empty_time_overlap @@ -0,0 +1,15 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000003,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(4)) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (v1, ) + txn.Get(ctx, tk(2)) // (v3, ) + return nil +}) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000003,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 +/Table/100/"0000000000000002"/0.000000003,0 @ s4 v4 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scan_and_write_with_empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scan_and_write_with_empty_time_overlap new file mode 100644 index 000000000000..b52d86e9c2e9 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scan_and_write_with_empty_time_overlap @@ -0,0 +1,13 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + txn.Put(ctx, tk(2), sv(3)) // + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 +committed txn non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000001,0, 0.000000002,0), gap:[, )}->[/Table/100/"0000000000000001":v1] [w]/Table/100/"0000000000000002":0.000000002,0->v3@s3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scan_and_write_with_non-empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scan_and_write_with_non-empty_time_overlap new file mode 100644 index 000000000000..ea93007aafb0 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scan_and_write_with_non-empty_time_overlap @@ -0,0 +1,12 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + txn.Put(ctx, tk(2), sv(3)) // + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000003,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_after_delete_with_empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_after_delete_with_empty_time_overlap new file mode 100644 index 000000000000..5430bd1c14cb --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_after_delete_with_empty_time_overlap @@ -0,0 +1,16 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000001,0 +db0.Del(ctx, tk(2) /* @s4 */) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + txn.Scan(ctx, tk(2), tk(4), 0) // + return nil +}) // @0.000000003,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000001,0 @ s3 v3 +/Table/100/"0000000000000002"/0.000000003,0 @ s4 +committed txn non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000001,0, 0.000000002,0), gap:[, 0.000000001,0),[0.000000003,0, )}->[/Table/100/"0000000000000001":v1] [s]/Table/100/"000000000000000{2"-4"}:{gap:[, 0.000000001,0),[0.000000003,0, )}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_after_delete_with_non-empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_after_delete_with_non-empty_time_overlap new file mode 100644 index 000000000000..905d7d31b24b --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_after_delete_with_non-empty_time_overlap @@ -0,0 +1,17 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000003,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000001,0 +db0.Del(ctx, tk(2) /* @s4 */) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(5)) // @0.000000004,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + txn.Scan(ctx, tk(2), tk(4), 0) // + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000003,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000001,0 @ s3 v3 +/Table/100/"0000000000000002"/0.000000002,0 @ s4 +/Table/100/"0000000000000002"/0.000000004,0 @ s5 v5 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_one_missing_with_empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_one_missing_with_empty_time_overlap new file mode 100644 index 000000000000..3f310a978a3c --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_one_missing_with_empty_time_overlap @@ -0,0 +1,14 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000001,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + txn.Scan(ctx, tk(2), tk(4), 0) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000001,0 @ s3 v3 +committed txn non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000001,0, 0.000000002,0), gap:[, 0.000000001,0)}->[/Table/100/"0000000000000001":v1] [s]/Table/100/"000000000000000{2"-4"}:{gap:[, 0.000000001,0)}->[] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_one_missing_with_non-empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_one_missing_with_non-empty_time_overlap new file mode 100644 index 000000000000..517ea2b1df17 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_one_missing_with_non-empty_time_overlap @@ -0,0 +1,13 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000002,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + txn.Scan(ctx, tk(2), tk(4), 0) // + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_with_empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_with_empty_time_overlap new file mode 100644 index 000000000000..f5f62dc52c7e --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_with_empty_time_overlap @@ -0,0 +1,16 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(4)) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v3, ) + txn.Scan(ctx, tk(2), tk(4), 0) // (/Table/100/"0000000000000002":v3, ) + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 +/Table/100/"0000000000000002"/0.000000003,0 @ s4 v4 +committed txn non-atomic timestamps: [s]/Table/100/"000000000000000{1"-3"}:{0:[0.000000001,0, 0.000000002,0), 1:[0.000000002,0, 0.000000003,0), gap:[, )}->[/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v3] [s]/Table/100/"000000000000000{2"-4"}:{0:[0.000000002,0, 0.000000003,0), gap:[, )}->[/Table/100/"0000000000000002":v3] diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_with_non-empty_time_overlap b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_with_non-empty_time_overlap new file mode 100644 index 000000000000..a36ea1f95c5f --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/transactional_scans_with_non-empty_time_overlap @@ -0,0 +1,15 @@ +echo +---- +db0.Put(ctx, tk(1), sv(1)) // @0.000000001,0 +db0.Put(ctx, tk(1), sv(2)) // @0.000000003,0 +db0.Put(ctx, tk(2), sv(3)) // @0.000000002,0 +db0.Put(ctx, tk(2), sv(4)) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, /Table/100/"0000000000000002":v3, ) + txn.Scan(ctx, tk(2), tk(4), 0) // (/Table/100/"0000000000000002":v3, ) + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000003,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000002,0 @ s3 v3 +/Table/100/"0000000000000002"/0.000000003,0 @ s4 v4 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_overlapping_rangedels b/pkg/kv/kvnemesis/testdata/TestValidate/two_overlapping_rangedels new file mode 100644 index 000000000000..aee9afc6ac0d --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_overlapping_rangedels @@ -0,0 +1,8 @@ +echo +---- +db0.DelRangeUsingTombstone(ctx, tk(1), tk(3) /* @s1 */) // @0.000000001,0 +db0.DelRangeUsingTombstone(ctx, tk(2), tk(4) /* @s2 */) // @0.000000002,0 +/Table/100/"000000000000000{1"-2"}/0.000000001,0 @ s1 +/Table/100/"000000000000000{2"-3"}/0.000000001,0 @ s1 +/Table/100/"000000000000000{2"-3"}/0.000000002,0 @ s2 +/Table/100/"000000000000000{3"-4"}/0.000000002,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactional_deletes_with_out_of_order_commit_times b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactional_deletes_with_out_of_order_commit_times new file mode 100644 index 000000000000..a4b2cf65a8cb --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactional_deletes_with_out_of_order_commit_times @@ -0,0 +1,13 @@ +echo +---- +db0.Del(ctx, tk(1) /* @s1 */) // @0.000000002,0 +db0.Del(ctx, tk(2) /* @s2 */) // @0.000000003,0 +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s3 */) // + txn.Del(ctx, tk(2) /* @s4 */) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s3 +/Table/100/"0000000000000001"/0.000000002,0 @ s1 +/Table/100/"0000000000000002"/0.000000001,0 @ s4 +/Table/100/"0000000000000002"/0.000000003,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_deletes_of_the_same_key b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_deletes_of_the_same_key new file mode 100644 index 000000000000..7b231af8f913 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_deletes_of_the_same_key @@ -0,0 +1,8 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + txn.Del(ctx, tk(1) /* @s2 */) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_deletes_of_the_same_key_with_extra_write b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_deletes_of_the_same_key_with_extra_write new file mode 100644 index 000000000000..1e2be15eb796 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_deletes_of_the_same_key_with_extra_write @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + txn.Del(ctx, tk(1) /* @s2 */) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +committed txn overwritten key had write: [d]/Table/100/"0000000000000001":0.000000001,0->@s1 [d]/Table/100/"0000000000000001":0.000000002,0->@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_put_delete_ops_of_the_same_key_with_incorrect_read b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_put_delete_ops_of_the_same_key_with_incorrect_read new file mode 100644 index 000000000000..dfaf45accf6a --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_put_delete_ops_of_the_same_key_with_incorrect_read @@ -0,0 +1,12 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (, ) + txn.Put(ctx, tk(1), sv(1)) // + txn.Get(ctx, tk(1)) // (v1, ) + txn.Del(ctx, tk(1) /* @s2 */) // + txn.Get(ctx, tk(1)) // (v1, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s2 +committed txn non-atomic timestamps: [r]/Table/100/"0000000000000001":[, )-> [w]/Table/100/"0000000000000001":missing->v1@s1 [r]/Table/100/"0000000000000001":[0.000000001,0, )->v1 [d]/Table/100/"0000000000000001":0.000000001,0->@s2 [r]/Table/100/"0000000000000001":[0,0, 0,0)->v1 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_put_delete_ops_of_the_same_key_with_reads b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_put_delete_ops_of_the_same_key_with_reads new file mode 100644 index 000000000000..60bfa9e14217 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_put_delete_ops_of_the_same_key_with_reads @@ -0,0 +1,11 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (, ) + txn.Put(ctx, tk(1), sv(1)) // + txn.Get(ctx, tk(1)) // (v1, ) + txn.Del(ctx, tk(1) /* @s2 */) // + txn.Get(ctx, tk(1)) // (, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key new file mode 100644 index 000000000000..eb34541d3476 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key @@ -0,0 +1,8 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Put(ctx, tk(1), sv(2)) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key_with_extra_write b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key_with_extra_write new file mode 100644 index 000000000000..90fa22e01ca4 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key_with_extra_write @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Put(ctx, tk(1), sv(2)) // + return nil +}) // @0.000000002,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 v2 +committed txn overwritten key had write: [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 [w]/Table/100/"0000000000000001":0.000000002,0->v2@s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key_with_reads b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key_with_reads new file mode 100644 index 000000000000..b0231a889823 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key_with_reads @@ -0,0 +1,11 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Get(ctx, tk(1)) // (, ) + txn.Put(ctx, tk(1), sv(1)) // + txn.Get(ctx, tk(1)) // (v1, ) + txn.Put(ctx, tk(1), sv(2)) // + txn.Get(ctx, tk(1)) // (v2, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key_with_scans b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key_with_scans new file mode 100644 index 000000000000..fa9977249b22 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_puts_of_the_same_key_with_scans @@ -0,0 +1,14 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Scan(ctx, tk(1), tk(3), 0) // + txn.Put(ctx, tk(1), sv(1)) // + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v1, ) + txn.Put(ctx, tk(1), sv(2)) // + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v2, ) + txn.Put(ctx, tk(2), sv(3)) // + txn.Scan(ctx, tk(1), tk(3), 0) // (/Table/100/"0000000000000001":v2, /Table/100/"0000000000000002":v3, ) + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s2 v2 +/Table/100/"0000000000000002"/0.000000001,0 @ s3 v3 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_writes_delete_put_of_the_same_key b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_writes_delete_put_of_the_same_key new file mode 100644 index 000000000000..fdfeda1c821a --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_writes_delete_put_of_the_same_key @@ -0,0 +1,8 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Del(ctx, tk(1) /* @s1 */) // + txn.Put(ctx, tk(1), sv(2)) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s2 v2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_writes_put_delete_of_the_same_key b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_writes_put_delete_of_the_same_key new file mode 100644 index 000000000000..7c45a32997a7 --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_writes_put_delete_of_the_same_key @@ -0,0 +1,8 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Del(ctx, tk(1) /* @s2 */) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s2 diff --git a/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_writes_put_delete_of_the_same_key_with_extra_write b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_writes_put_delete_of_the_same_key_with_extra_write new file mode 100644 index 000000000000..8a81d1c9976b --- /dev/null +++ b/pkg/kv/kvnemesis/testdata/TestValidate/two_transactionally_committed_writes_put_delete_of_the_same_key_with_extra_write @@ -0,0 +1,10 @@ +echo +---- +db0.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + txn.Put(ctx, tk(1), sv(1)) // + txn.Del(ctx, tk(1) /* @s2 */) // + return nil +}) // @0.000000001,0 +/Table/100/"0000000000000001"/0.000000001,0 @ s1 v1 +/Table/100/"0000000000000001"/0.000000002,0 @ s2 +committed txn overwritten key had write: [w]/Table/100/"0000000000000001":0.000000001,0->v1@s1 [d]/Table/100/"0000000000000001":0.000000002,0->@s2 diff --git a/pkg/kv/kvnemesis/validator.go b/pkg/kv/kvnemesis/validator.go index 622962a88f9e..f0d47a922f0d 100644 --- a/pkg/kv/kvnemesis/validator.go +++ b/pkg/kv/kvnemesis/validator.go @@ -13,10 +13,12 @@ package kvnemesis import ( "context" "fmt" + "reflect" "regexp" "sort" "strings" + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -46,8 +48,8 @@ import ( // // Splits and merges are not verified for anything other than that they did not // return an error. -func Validate(steps []Step, kvs *Engine) []error { - v, err := makeValidator(kvs) +func Validate(steps []Step, kvs *Engine, dt *SeqTracker) []error { + v, err := makeValidator(kvs, dt) if err != nil { return []error{err} } @@ -63,39 +65,34 @@ func Validate(steps []Step, kvs *Engine) []error { // by `After` timestamp is sufficient to get us the necessary ordering. This // is because txns cannot be used concurrently, so none of the (Begin,After) // timespans for a given transaction can overlap. - // - // TODO(tbg): if, as we should, we order all operations by the timestamp at - // which they executed (at least for MVCC-aware operations), then we would - // sort here by that timestamp. sort.Slice(steps, func(i, j int) bool { return steps[i].After.Less(steps[j].After) }) for _, s := range steps { - v.processOp(notBuffering, s.Op) + v.processOp(s.Op) } var extraKVs []observedOp - for _, kv := range v.kvByValue { - kv := &observedWrite{ - Key: kv.Key.Key, - Value: roachpb.Value{RawBytes: kv.Value}, - Timestamp: kv.Key.Timestamp, - Materialized: true, - } - extraKVs = append(extraKVs, kv) - } - for key, tombstones := range v.tombstonesForKey { - numExtraWrites := len(tombstones) - v.committedDeletesForKey[key] - for i := 0; i < numExtraWrites; i++ { + for seq, svs := range v.kvBySeq { + for _, sv := range svs { + mvccV, err := storage.DecodeMVCCValue(sv.Value) + if err != nil { + v.failures = append(v.failures, err) + continue + } kv := &observedWrite{ - Key: []byte(key), - Value: roachpb.Value{}, - // NB: As it's unclear which are "extra", timestamp is left missing. - Materialized: true, + Key: sv.Key, + EndKey: sv.EndKey, + Value: mvccV.Value, + Timestamp: sv.Timestamp, + Seq: seq, } extraKVs = append(extraKVs, kv) } } + + // These are writes that we saw in MVCC, but they weren't matched up to any + // operation kvnemesis ran. if len(extraKVs) > 0 { - err := errors.Errorf(`extra writes: %s`, printObserved(extraKVs...)) + err := errors.Errorf(`unclaimed writes: %s`, printObserved(extraKVs...)) v.failures = append(v.failures, err) } @@ -209,12 +206,13 @@ type observedOp interface { observedMarker() } +// An observedWrite is an effect of an operation. type observedWrite struct { - Key roachpb.Key - Value roachpb.Value - // Timestamp will only be filled if Materialized is true. + Key, EndKey roachpb.Key + Value roachpb.Value + Seq kvnemesisutil.Seq + // A write is materialized if it has a timestamp. Timestamp hlc.Timestamp - Materialized bool IsDeleteRange bool } @@ -249,94 +247,105 @@ type validator struct { // checkAtomic, which then calls processOp (which might recurse owing to the // existence of txn closures, batches, etc). curObservations []observedOp + buffering bufferingType // NB: The Generator carefully ensures that each value written is unique // globally over a run, so there's a 1:1 relationship between a value that was // written and the operation that wrote it. - kvByValue map[string]storage.MVCCKeyValue - - // Unfortunately, with tombstones there is no 1:1 relationship between the nil - // value and the delete operation that wrote it, so we must store all tombstones - // for a given key. When validating committed delete operations, we validate - // that there is a tombstone with a timestamp that would be valid, similar - // to how reads are evaluated. At the end of validation, we also validate - // that we have seen a correct number of materialized delete operations - // given the number of tombstones for each key; thus, we can see if we have - // any "missing" or "extra" writes at the end. - // Each key has a map of all the tombstone timestamps, stored with a boolean - // flag indicating if it has been matched to a transactional delete or not. - tombstonesForKey map[string]map[hlc.Timestamp]bool - committedDeletesForKey map[string]int + // kvsByKeyAndSeq map[keySeq]storage.MVCCKeyValue // TODO remove + kvBySeq map[kvnemesisutil.Seq][]tsSpanVal failures []error } -func makeValidator(kvs *Engine) (*validator, error) { - kvByValue := make(map[string]storage.MVCCKeyValue) - tombstonesForKey := make(map[string]map[hlc.Timestamp]bool) +type tsSpanVal struct { + roachpb.Span + hlc.Timestamp + Value []byte +} + +func makeValidator(kvs *Engine, tr *SeqTracker) (*validator, error) { + kvBySeq := make(map[kvnemesisutil.Seq][]tsSpanVal) var err error - kvs.Iterate(func(key storage.MVCCKey, value []byte, iterErr error) { + kvs.Iterate(func(key, endKey roachpb.Key, ts hlc.Timestamp, value []byte, iterErr error) { + if err != nil { + return + } if iterErr != nil { err = errors.CombineErrors(err, iterErr) return } - v, decodeErr := storage.DecodeMVCCValue(value) - if err != nil { - err = errors.CombineErrors(err, decodeErr) + seq, ok := tr.Lookup(key, endKey, ts) + if !ok { + err = errors.AssertionFailedf("no seqno found for [%s,%s) @ %s, tracker is %v", key, endKey, ts, tr) return } - if v.Value.GetTag() != roachpb.ValueType_UNKNOWN { - valueStr := mustGetStringValue(value) - if existing, ok := kvByValue[valueStr]; ok { - // TODO(dan): This may be too strict. Some operations (db.Run on a - // Batch) seem to be double-committing. See #46374. - panic(errors.AssertionFailedf( - `invariant violation: value %s was written by two operations %s and %s`, - valueStr, existing.Key, key)) - } - // NB: The Generator carefully ensures that each value written is unique - // globally over a run, so there's a 1:1 relationship between a value that - // was written and the operation that wrote it. - kvByValue[valueStr] = storage.MVCCKeyValue{Key: key, Value: value} - } else if !v.Value.IsPresent() { - rawKey := string(key.Key) - if _, ok := tombstonesForKey[rawKey]; !ok { - tombstonesForKey[rawKey] = make(map[hlc.Timestamp]bool) - } - tombstonesForKey[rawKey][key.Timestamp] = false + v := tsSpanVal{ + Span: roachpb.Span{Key: key, EndKey: endKey}, + Timestamp: ts, + Value: value, } + kvBySeq[seq] = append(kvBySeq[seq], v) }) if err != nil { return nil, err } return &validator{ - kvs: kvs, - kvByValue: kvByValue, - tombstonesForKey: tombstonesForKey, - committedDeletesForKey: make(map[string]int), + kvs: kvs, + kvBySeq: kvBySeq, }, nil } -// getDeleteForKey looks up a stored tombstone for a given key (if it -// exists) from tombstonesForKey, returning the tombstone (i.e. MVCCKey) along -// with a `true` boolean value if found, or the empty key and `false` if not. -func (v *validator) getDeleteForKey(key string, optOpTS hlc.Timestamp) (storage.MVCCKey, bool) { - if optOpTS.IsEmpty() { - panic(errors.AssertionFailedf(`transaction required to look up delete for key: %v`, key)) +func (v *validator) tryConsumeWrite(key roachpb.Key, seq kvnemesisutil.Seq) (tsSpanVal, bool) { + svs, ok := v.tryConsumeRangedWrite(seq, key, nil) + if !ok { + return tsSpanVal{}, false + } + if len(svs) != 1 { + panic(fmt.Sprintf("expected exactly one element: %+v", svs)) } + return svs[0], true +} - if used, ok := v.tombstonesForKey[key][optOpTS]; !used && ok { - v.tombstonesForKey[key][optOpTS] = true - return storage.MVCCKey{Key: []byte(key), Timestamp: optOpTS}, true +func (v *validator) tryConsumeRangedWrite( + seq kvnemesisutil.Seq, key, endKey roachpb.Key, +) ([]tsSpanVal, bool) { + svs, ok := v.kvBySeq[seq] + if !ok || len(svs) == 0 { + return nil, false + } + opSpan := roachpb.Span{Key: key, EndKey: endKey} + + var consumed []tsSpanVal + var remaining []tsSpanVal + for i := range svs { + cur := svs[i] + if !opSpan.Contains(cur.Span) { + // Operation must have written this write but doesn't want to consume it + // right now, so skip it. For example, DeleteRange decomposes into point + // deletes and will look these deletes up here one by one. If an operation + // truly wrote outside of its span, this will cause a failure in + // validation. + remaining = append(remaining, cur) + continue + } + consumed = append(consumed, cur) } - return storage.MVCCKey{}, false + if len(remaining) == 0 { + delete(v.kvBySeq, seq) + } else { + v.kvBySeq[seq] = remaining + } + return consumed, len(consumed) > 0 } +type bufferingType byte + const ( - notBuffering = false - isBuffering = true + bufferingSingle bufferingType = iota + bufferingBatchOrTxn ) // processOp turns the result of an operation into its observations (which are @@ -346,124 +355,249 @@ const ( // itself processOp, with the operation to handle being the batch or txn). // Whenever it is `false`, processOp invokes the validator's checkAtomic method // for the operation. -func (v *validator) processOp(buffering bool, op Operation) { - // We don't need an execution timestamp when buffering (the caller will need - // an execution timestamp for the combined operation, though). Additionally, - // some operations supported by kvnemesis aren't MVCC-aware (splits, etc) and - // thus also don't need an execution timestamp. - execTimestampStrictlyOptional := buffering +func (v *validator) processOp(op Operation) { + // We will validate the presence of execution timestamps below, i.e. we verify + // that the KV API will tell the caller at which MVCC timestamp operations have + // executed. It won't do this in all situations (such as non-mvcc ops such as + // splits, etc) but should do so for all MVCC ops. + // + // To start, we don't need an execution timestamp when buffering (the caller will need + // an execution timestamp for the combined operation, though), or when the operation + // didn't succeed. + execTimestampStrictlyOptional := v.buffering == bufferingBatchOrTxn || op.Result().Error() != nil switch t := op.GetValue().(type) { case *GetOperation: - v.failIfError(op, t.Result) - if !buffering { - v.checkAtomic(`get`, t.Result, op) - } else { - read := &observedRead{ - Key: t.Key, - Value: roachpb.Value{RawBytes: t.Result.Value}, - } - v.curObservations = append(v.curObservations, read) + if _, isErr := v.checkError(op, t.Result); isErr { + break + } + read := &observedRead{ + Key: t.Key, + Value: roachpb.Value{RawBytes: t.Result.Value}, + } + v.curObservations = append(v.curObservations, read) + + if v.buffering == bufferingSingle { + v.checkAtomic(`get`, t.Result) } case *PutOperation: - if !buffering { - v.checkAtomic(`put`, t.Result, op) - } else { - // Accumulate all the writes for this transaction. - kv, ok := v.kvByValue[string(t.Value)] - delete(v.kvByValue, string(t.Value)) + if v.checkNonAmbError(op, t.Result) { + break + } + // Accumulate all the writes for this transaction. + write := &observedWrite{ + Key: t.Key, + Seq: t.Seq, + Value: roachpb.MakeValueFromString(t.Value()), + } + if sv, ok := v.tryConsumeWrite(t.Key, t.Seq); ok { + write.Timestamp = sv.Timestamp + } + v.curObservations = append(v.curObservations, write) + + if v.buffering == bufferingSingle { + v.checkAtomic(`put`, t.Result) + } + case *DeleteOperation: + if v.checkNonAmbError(op, t.Result) { + break + } + sv, _ := v.tryConsumeWrite(t.Key, t.Seq) + write := &observedWrite{ + Key: t.Key, + Seq: t.Seq, + Timestamp: sv.Timestamp, + } + v.curObservations = append(v.curObservations, write) + + if v.buffering == bufferingSingle { + v.checkAtomic(`delete`, t.Result) + } + case *DeleteRangeOperation: + if v.checkNonAmbError(op, t.Result) { + break + } + // We express DeleteRange as point deletions on all of the keys it claimed + // to have deleted and (atomically post-ceding the deletions) a scan that + // sees an empty span. If DeleteRange places a tombstone it didn't report, + // validation will fail with an unclaimed write. If it fails to delete a + // key, the scan will not validate. If it reports that it deleted a key + // that didn't have a non-nil value (i.e. didn't get a new tombstone), + // then validation will fail with a missing write. If it reports & places + // a tombstone that wasn't necessary (i.e. a combination of the above), + // validation will succeed. This is arguably incorrect; we had code in + // the past that handled this at the expense of additional complexity[^1]. + // See the `one deleterange after write with spurious deletion` test case + // in TestValidate. + // + // [^1]: https://github.com/cockroachdb/cockroach/pull/68003/files#diff-804b6fefcb2b7ae68fab388e6dcbaf7dbc3937a266b14b79c330b703ea9d0d95R382-R388 + deleteOps := make([]observedOp, len(t.Result.Keys)) + for i, key := range t.Result.Keys { + sv, _ := v.tryConsumeWrite(key, t.Seq) write := &observedWrite{ - Key: t.Key, - Value: roachpb.MakeValueFromBytes(t.Value), - Materialized: ok, + Key: key, + Seq: t.Seq, + Value: roachpb.Value{}, + IsDeleteRange: true, // only for String(), no semantics attached + Timestamp: sv.Timestamp, } - if write.Materialized { - write.Timestamp = kv.Key.Timestamp + deleteOps[i] = write + } + v.curObservations = append(v.curObservations, deleteOps...) + // The span ought to be empty right after the DeleteRange. + v.curObservations = append(v.curObservations, &observedScan{ + Span: roachpb.Span{ + Key: t.Key, + EndKey: t.EndKey, + }, + IsDeleteRange: true, // just for printing + KVs: nil, + }) + + if v.buffering == bufferingSingle { + v.checkAtomic(`deleteRange`, t.Result) + } + case *DeleteRangeUsingTombstoneOperation: + if v.checkNonAmbError(op, t.Result, exceptUnhandledRetry) { + // If there was an error and it's not an ambiguous result, since we don't + // allow this operation to span ranges (which might otherwise allow for + // partial execution, since it's not atomic across ranges), we know none + // of the writes must have become visible. Check this by not emitting them + // in the first place. (Otherwise, we could get an error but the write + // would still be there and kvnemesis would pass). + break + } + // NB: MVCC range deletions aren't allowed in transactions (and can't be + // overwritten in the same non-txn'al batch), so we currently will only + // ever see one write to consume. With transactions (or self-overlapping + // batches) we could get the following: + // + // txn.DelRangeUsingTombstone(a, c) + // txn.Put(b, v) + // txn.Commit + // + // The resulting atomic unit would emit two MVCC range deletions. [a,b) + // and [b\x00, c). + // + // The code here handles this, and it is unit tested, so that if and when + // we do support rangedels in transactions, kvnemesis will be ready. + // + // However, DeleteRangeUsingTombstone is a ranged non-txnal request type + // that will be split in DistSender, and so it is *not* atomic[^1]. An + // earlier attempt at letting `kvnemesis` handle this fact by treating each + // individual written piece that we see as an atomic unit led to too much + // complexity (in particular, we have to validate/tolerate partial + // executions). Instead, we *disable* DistSender's splitting of + // DeleteRangeUsingTombstone when run with kvnemesis, and attempt to create + // only operations for it that respect the likely range splits. + // + // In theory this code here supports any kind of atomic batched or + // transactional MVCC range deletions, assuming the KV API started to + // support them as well. + // + // [^1]: https://github.com/cockroachdb/cockroach/issues/46081 + svs, _ := v.tryConsumeRangedWrite(t.Seq, t.Key, t.EndKey) + var unobserved roachpb.SpanGroup + unobserved.Add(roachpb.Span{Key: t.Key, EndKey: t.EndKey}) + for _, sv := range svs { + unobserved.Sub(sv.Span) + write := &observedWrite{ + Key: sv.Key, + EndKey: sv.EndKey, + Seq: t.Seq, + Timestamp: sv.Timestamp, } v.curObservations = append(v.curObservations, write) } - case *DeleteOperation: - if !buffering { - v.checkAtomic(`delete`, t.Result, op) - } else { - // NB: While Put operations can be identified as having materialized - // (or not) in the storage engine because the Generator guarantees each - // value to be unique (and thus, if a MVCC key/value pair exists in the - // storage engine with a value matching that of a write operation, it - // materialized), the same cannot be done for Delete operations, which - // all write the same tombstone value. Thus, Delete operations can only - // be identified as materialized by determining if the final write - // operation for a key in a given transaction was a Delete, and - // validating that a potential tombstone for that key was stored. - // This validation must be done at the end of the transaction; - // specifically, in the function `checkAtomicCommitted(..)` where it looks - // up a corresponding tombstone with `getDeleteForKey(..)`. + // Add unmaterialized versions of the write for any gaps. If !atomicAcrossSplits, + // the batch might've partially succeeded (and so there might be gaps), but in + // this case we ought to have received an error. + for _, sp := range unobserved.Slice() { write := &observedWrite{ - Key: t.Key, - Value: roachpb.Value{}, + Key: sp.Key, + EndKey: sp.EndKey, + Seq: t.Seq, } v.curObservations = append(v.curObservations, write) } - case *DeleteRangeOperation: - if !buffering { - v.checkAtomic(`deleteRange`, t.Result, op) - } else { - // For the purposes of validation, DelRange operations decompose into - // a specialized scan for keys with non-nil values, followed by - // writes for each key, with a span to validate that the keys we are - // deleting are within the proper bounds. See above comment for how - // the individual deletion tombstones for each key are validated. - scan := &observedScan{ - Span: roachpb.Span{ - Key: t.Key, - EndKey: t.EndKey, - }, - IsDeleteRange: true, - KVs: make([]roachpb.KeyValue, len(t.Result.Keys)), - } - deleteOps := make([]observedOp, len(t.Result.Keys)) - for i, key := range t.Result.Keys { - scan.KVs[i] = roachpb.KeyValue{ - Key: key, - Value: roachpb.Value{}, - } - write := &observedWrite{ - Key: key, - Value: roachpb.Value{}, - IsDeleteRange: true, - } - deleteOps[i] = write - } - v.curObservations = append(v.curObservations, scan) - v.curObservations = append(v.curObservations, deleteOps...) + + // The span ought to be empty right after the DeleteRange, even if parts of + // the DeleteRange that didn't materialize due to a shadowing operation. + v.curObservations = append(v.curObservations, &observedScan{ + Span: roachpb.Span{ + Key: t.Key, + EndKey: t.EndKey, + }, + }) + + if v.buffering == bufferingSingle { + v.checkAtomic(`deleteRangeUsingTombstone`, t.Result) } case *ScanOperation: - v.failIfError(op, t.Result) - if !buffering { + if _, isErr := v.checkError(op, t.Result); isErr { + break + } + scan := &observedScan{ + Span: roachpb.Span{ + Key: t.Key, + EndKey: t.EndKey, + }, + KVs: make([]roachpb.KeyValue, len(t.Result.Values)), + Reverse: t.Reverse, + } + for i, kv := range t.Result.Values { + scan.KVs[i] = roachpb.KeyValue{ + Key: kv.Key, + Value: roachpb.Value{RawBytes: kv.Value}, + } + } + v.curObservations = append(v.curObservations, scan) + + if v.buffering == bufferingSingle { atomicScanType := `scan` if t.Reverse { atomicScanType = `reverse scan` } - v.checkAtomic(atomicScanType, t.Result, op) - } else { - scan := &observedScan{ - Span: roachpb.Span{ - Key: t.Key, - EndKey: t.EndKey, - }, - KVs: make([]roachpb.KeyValue, len(t.Result.Values)), - Reverse: t.Reverse, - } - for i, kv := range t.Result.Values { - scan.KVs[i] = roachpb.KeyValue{ - Key: kv.Key, - Value: roachpb.Value{RawBytes: kv.Value}, - } - } - v.curObservations = append(v.curObservations, scan) + v.checkAtomic(atomicScanType, t.Result) + } + case *BatchOperation: + // Intentionally don't check the error here. An error on the Batch becomes + // that error on each individual operation. + + // Only call checkAtomic if we're in bufferingSingle here. We could have + // been a batch inside a txn. + wasBuffering := v.buffering + v.buffering = bufferingBatchOrTxn + for _, op := range t.Ops { + v.processOp(op) + } + if wasBuffering == bufferingSingle { + v.checkAtomic(`batch`, t.Result) + } + case *ClosureTxnOperation: + // A txn can only fail with an intentional rollback or ambiguous result. + // Retry and omitted errors can only happen inside of the txn, but not + // inform its result. + // For ambiguous results, we must continue validating the txn (since writes + // may be there). For all other errors, it still makes sense to do so in case + // since a problem at the txn level is likely due to something weird in an + // individual operation, so it makes sense to try to emit more failures. + // + // So we ignore the results of failIfError, calling it only for its side + // effect of perhaps registering a failure with the validator. + v.failIfError(op, t.Result, exceptRollback, exceptAmbiguous) + + ops := t.Ops + if t.CommitInBatch != nil { + ops = append(ops, t.CommitInBatch.Ops...) } + v.buffering = bufferingBatchOrTxn + for _, op := range ops { + v.processOp(op) + } + v.checkAtomic(`txn`, t.Result) case *SplitOperation: execTimestampStrictlyOptional = true - v.failIfError(op, t.Result) + v.failIfError(op, t.Result) // splits should never return *any* error case *MergeOperation: execTimestampStrictlyOptional = true if resultIsErrorStr(t.Result, `cannot merge final range`) { @@ -504,7 +638,7 @@ func (v *validator) processOp(buffering bool, op Operation) { } else if resultIsErrorStr(t.Result, `merge failed: RHS range bounds do not match`) { // Probably should be transparently retried. } else { - v.failIfError(op, t.Result) + v.failIfError(op, t.Result) // fail on all other errors } case *ChangeReplicasOperation: execTimestampStrictlyOptional = true @@ -512,10 +646,11 @@ func (v *validator) processOp(buffering bool, op Operation) { if err := errorFromResult(t.Result); err != nil { ignore = kvserver.IsRetriableReplicationChangeError(err) || kvserver.IsIllegalReplicationChangeError(err) || - kvserver.IsReplicationChangeInProgressError(err) + kvserver.IsReplicationChangeInProgressError(err) || + errors.Is(err, roachpb.ErrReplicaCannotHoldLease) } if !ignore { - v.failIfError(op, t.Result) + v.failIfError(op, t.Result) // fail on all other errors } case *TransferLeaseOperation: execTimestampStrictlyOptional = true @@ -538,40 +673,25 @@ func (v *validator) processOp(buffering bool, op Operation) { resultIsErrorStr(t.Result, `cannot transfer lease while merge in progress`) || // If the existing leaseholder has not yet heard about the transfer // target's liveness record through gossip, it will return an error. - resultIsError(t.Result, liveness.ErrRecordCacheMiss) || + exceptLivenessCacheMiss(errorFromResult(t.Result)) || // Same as above, but matches cases where ErrRecordCacheMiss is // passed through a LeaseRejectedError. This is necessary until // LeaseRejectedErrors works with errors.Cause. resultIsErrorStr(t.Result, liveness.ErrRecordCacheMiss.Error()) } if !ignore { - v.failIfError(op, t.Result) + v.failIfError(op, t.Result) // fail on all other errors } case *ChangeZoneOperation: execTimestampStrictlyOptional = true - v.failIfError(op, t.Result) - case *BatchOperation: - if !resultIsRetryable(t.Result) { - v.failIfError(op, t.Result) - if !buffering { - v.checkAtomic(`batch`, t.Result, t.Ops...) - } else { - for _, op := range t.Ops { - v.processOp(buffering, op) - } - } - } - case *ClosureTxnOperation: - ops := t.Ops - if t.CommitInBatch != nil { - ops = append(ops, t.CommitInBatch.Ops...) - } - v.checkAtomic(`txn`, t.Result, ops...) + v.failIfError(op, t.Result) // fail on all errors default: panic(errors.AssertionFailedf(`unknown operation type: %T %v`, t, t)) } - if !execTimestampStrictlyOptional && !buffering && op.Result().Type != ResultType_Error && op.Result().OptionalTimestamp.IsEmpty() { + // If the current operation is expected to have an operation timestamp but + // didn't have one, emit a failure. + if !execTimestampStrictlyOptional && op.Result().OptionalTimestamp.IsEmpty() { v.failures = append(v.failures, errors.Errorf("execution timestamp missing for %s", op)) } } @@ -579,26 +699,28 @@ func (v *validator) processOp(buffering bool, op Operation) { // checkAtomic verifies a set of operations that should be atomic by trying to find // a timestamp at which the observed reads and writes of the operations (as executed // in the order in which they appear in the arguments) match the MVCC history. -func (v *validator) checkAtomic(atomicType string, result Result, ops ...Operation) { - for _, op := range ops { - // NB: we're not really necessarily in a txn, but passing true here means that - // we have an atomic unit, which is also the case if we are called here by a - // non-transactional Put, for example. - v.processOp(isBuffering, op) - } +func (v *validator) checkAtomic(atomicType string, result Result) { observations := v.curObservations v.curObservations = nil + v.buffering = bufferingSingle + // Only known-uncommitted results may come without a timestamp. Whenever we + // actually tried to commit, there is a timestamp. if result.Type != ResultType_Error { // The timestamp is not optional in this case. Note however that at the time // of writing, checkAtomicCommitted doesn't capitalize on this unconditional // presence yet, and most unit tests don't specify it for reads. - if result.OptionalTimestamp.IsEmpty() { + if !result.OptionalTimestamp.IsSet() { err := errors.AssertionFailedf("operation has no execution timestamp: %s", result) v.failures = append(v.failures, err) } v.checkAtomicCommitted(`committed `+atomicType, observations, result.OptionalTimestamp) } else if resultIsAmbiguous(result) { + // An ambiguous result shouldn't have an execution timestamp. + if result.OptionalTimestamp.IsSet() { + err := errors.AssertionFailedf("OptionalTimestamp set for ambiguous result: %s", result) + v.failures = append(v.failures, err) + } v.checkAtomicAmbiguous(`ambiguous `+atomicType, observations) } else { v.checkAtomicUncommitted(`uncommitted `+atomicType, observations) @@ -613,7 +735,7 @@ func (v *validator) checkAtomic(atomicType string, result Result, ops ...Operati // succeeded in a "normal" way. However, for ambiguous results, it is not always // present. This limitation could be lifted, see checkAtomicAmbiguous. func (v *validator) checkAtomicCommitted( - atomicType string, txnObservations []observedOp, optOpsTimestamp hlc.Timestamp, + atomicType string, txnObservations []observedOp, execTimestamp hlc.Timestamp, ) { // The following works by verifying that there is at least one time at which // it was valid to see all the reads and writes that we saw in this @@ -685,84 +807,141 @@ func (v *validator) checkAtomicCommitted( batch := v.kvs.kvs.NewIndexedBatch() defer func() { _ = batch.Close() }() - // If the same key is written multiple times in a transaction, only the last - // one makes it to kv. - lastWriteIdxByKey := make(map[string]int, len(txnObservations)) + var failure string + // writeTS is populated with the timestamp of the materialized observed writes + // (if there are any). We'll use it below to maintain the "view" of prefixes + // of atomic unit. + var writeTS hlc.Timestamp + // First, hide all of our writes from the view. Remember the index of the last + // ('most recent') write to each key so that we can check below whether any + // shadowed writes erroneously materialized. Recall that writes can be ranged + // (mvcc range deletions), but these writes cannot be transactional. At the + // time of writing, we also don't do non-transactional batches (see + // DefaultConfig) which means in effect we'll only ever see ranged operations + // alone in an atomic unit. This code still handles these cases, and they are + // unit tested. + lastWritesByIdx := map[int]struct{}{} + var lastWrites roachpb.SpanGroup for idx := len(txnObservations) - 1; idx >= 0; idx-- { observation := txnObservations[idx] switch o := observation.(type) { case *observedWrite: - if _, ok := lastWriteIdxByKey[string(o.Key)]; !ok { - lastWriteIdxByKey[string(o.Key)] = idx - - // Mark which deletes are materialized and match them with a stored - // tombstone, since this cannot be done before the end of the txn. - // This is because materialized deletes do not write unique values, - // but must be the final write in a txn for that key. - if o.isDelete() { - key := string(o.Key) - v.committedDeletesForKey[key]++ - if optOpsTimestamp.IsEmpty() { - // Special case: our operation doesn't know at which timestamp - // it wrote and so we're unable to match it to a particular tombstone - // and can only check the cardinality - if there was a tombstone left, - // we assume it's ours. - // - // We leave the Timestamp field empty as a result (if there are - // multiple tombstones left, how do we know which one is ours?) and - // everyone else needs to be able to handle this special case. - // - // TODO(tbg): see checkAtomicAmbiguous about letting ambiguously - // committed operations learn their commit timestamp. - o.Materialized = v.committedDeletesForKey[key] <= len(v.tombstonesForKey[key]) - } else if storedDelete, ok := v.getDeleteForKey(key, optOpsTimestamp); ok { - o.Materialized = true - o.Timestamp = storedDelete.Timestamp - } + sp := roachpb.Span{Key: o.Key, EndKey: o.EndKey} + // Check if the last writes set already covers the current write. + // + // Writes are fragmented in the sense that they are either fully the + // last write or not, since all (materialized) writes happened at the + // same MVCC timestamp, at least in the absence of bugs. + // + // For example, a Put A that gets shadowed by an MVCC rangedel B that + // then gets overlaid by a Put C and then intersected by another + // rangedel D should give an "incremental history" (as we construct it + // further down below) + // [-----D----) + // C + // [----------B------------) + // A + // + // and lastWrites will be + // + // [-----B-----)C[--B--)[-----D----) + // + // In particular, when we constructed the observedWrite for our rangedels, + // we construct them for the actual spans from the rangefeed, not the span + // of the operation. + var lastWrite bool + { + var g roachpb.SpanGroup + g.Add(lastWrites.Slice()...) + lastWrite = !g.Sub(sp) // if subtracting did nothing, it's a most recent write + if !lastWrite { + // Otherwise, add it back in, which should restore the old set. If it + // didn't, there was partial overlap, which shouldn't be possible. + g.Add(sp) + } + if then, now := lastWrites.Slice(), g.Slice(); !reflect.DeepEqual(then, now) { + v.failures = append(v.failures, + errors.AssertionFailedf("%s has write %q partially overlapping %+v; subtracting and re-adding gave %+v", atomicType, sp, then, now)) + return } } - if !o.Timestamp.IsEmpty() { + + if lastWrite { + lastWritesByIdx[idx] = struct{}{} + lastWrites.Add(sp) + } + + if o.Timestamp.IsEmpty() { + // This write didn't materialize (say a superseded write in + // a txn), so it's not present here. + continue + } + + // NB: we allow writeTS to change here, since that will be caught by + // validation below anyway, and then we can produce better errors since + // read timestamps will be filled in. + if writeTS.IsEmpty() { + writeTS = o.Timestamp + } + + if len(o.EndKey) == 0 { // point write mvccKey := storage.MVCCKey{Key: o.Key, Timestamp: o.Timestamp} if err := batch.Delete(storage.EncodeMVCCKey(mvccKey), nil); err != nil { panic(err) } + } else { // ranged write + suffix := storage.EncodeMVCCTimestampSuffix(o.Timestamp) + if err := batch.RangeKeyUnset(o.Key, o.EndKey, suffix, nil); err != nil { + panic(err) + } } } } - // Check if any key that was written twice in the txn had the overwritten - // writes materialize in kv. Also fill in all the read timestamps first so - // they show up in the failure message. - var failure string + // Iterate through the observations, building up the snapshot visible at each + // point in the atomic unit and filling in the valid read times (validating + // them later, in a separate loop, for better errors). We also check that only + // the most recent writes materialized (i.e. showed up in MVCC). Check if any + // key that was written twice in the txn had the overwritten writes + // materialize in kv. for idx, observation := range txnObservations { if failure != `` { break } switch o := observation.(type) { case *observedWrite: - var mvccKey storage.MVCCKey - if lastWriteIdx := lastWriteIdxByKey[string(o.Key)]; idx == lastWriteIdx { - // The last write of a given key in the txn wins and should have made it - // to kv. - mvccKey = storage.MVCCKey{Key: o.Key, Timestamp: o.Timestamp} - } else { - if o.Materialized { - failure = `committed txn overwritten key had write` + // Only the most recent write between overlapping mutations makes it into MVCC. + // writeTS was populated above as the unique timestamp at which the writes became + // visible. We know the operation had writes (we're looking at one now) and so + // this operation has either materialized or is covered by a later one that did, + // and so we must have a timestamp here. We defer the failure to the next for + // loop, as we will have filled in the read timestamps at that time. + if writeTS.IsEmpty() { + continue + } + + _, isLastWrite := lastWritesByIdx[idx] + + if !isLastWrite && o.Timestamp.IsSet() { + failure = `committed txn overwritten key had write` + break + } + + // Make this write visible (at writeTS, regardless of whether it's the + // last write or not, since that's the snapshot at which our operation + // wrote). + if len(o.EndKey) == 0 { + if err := batch.Set(storage.EncodeMVCCKey(storage.MVCCKey{Key: o.Key, Timestamp: writeTS}), o.Value.RawBytes, nil); err != nil { + panic(err) } - // This write was never materialized in KV because the key got - // overwritten later in the txn. But reads in the txn could have seen - // it, so we put in the batch being maintained for validReadTimes using - // the timestamp of the write for this key that eventually "won". - mvccKey = storage.MVCCKey{ - Key: o.Key, - Timestamp: txnObservations[lastWriteIdx].(*observedWrite).Timestamp, + } else { + suffix := storage.EncodeMVCCTimestampSuffix(writeTS) + if err := batch.RangeKeySet(o.Key, o.EndKey, suffix, o.Value.RawBytes, nil); err != nil { + panic(err) } } - if err := batch.Set(storage.EncodeMVCCKey(mvccKey), o.Value.RawBytes, nil); err != nil { - panic(err) - } case *observedRead: - o.ValidTimes = validReadTimes(batch, o.Key, o.Value.RawBytes, false) + o.ValidTimes = validReadTimes(batch, o.Key, o.Value.RawBytes) case *observedScan: // All kvs should be within scan boundary. for _, kv := range o.KVs { @@ -783,7 +962,7 @@ func (v *validator) checkAtomicCommitted( if !sort.IsSorted(orderedKVs) { failure = `scan result not ordered correctly` } - o.Valid = validScanTime(batch, o.Span, o.KVs, o.IsDeleteRange) + o.Valid = validScanTime(batch, o.Span, o.KVs) default: panic(errors.AssertionFailedf(`unknown observedOp: %T %s`, observation, observation)) } @@ -797,21 +976,14 @@ func (v *validator) checkAtomicCommitted( var opValid disjointTimeSpans switch o := observation.(type) { case *observedWrite: - isLastWriteForKey := idx == lastWriteIdxByKey[string(o.Key)] - if !isLastWriteForKey { - continue - } - if !o.Materialized { - failure = atomicType + ` missing write` + _, isLastWrite := lastWritesByIdx[idx] + if !isLastWrite { continue } - - if o.isDelete() && len(txnObservations) == 1 { - // For delete operations outside of transactions, it is not possible to - // identify the precise tombstone, so we skip timestamp validation. + if o.Timestamp.IsEmpty() { + failure = atomicType + ` missing write at seq ` + o.Seq.String() continue } - opValid = disjointTimeSpans{{Start: o.Timestamp, End: o.Timestamp.Next()}} case *observedRead: opValid = o.ValidTimes @@ -829,16 +1001,8 @@ func (v *validator) checkAtomicCommitted( // Finally, validate that the write timestamp of the transaction matches the // write timestamp of each write within that transaction. - for _, observation := range txnObservations { - if failure != `` { - break - } - switch o := observation.(type) { - case *observedWrite: - if optOpsTimestamp.IsSet() && o.Materialized && optOpsTimestamp != o.Timestamp { - failure = fmt.Sprintf(`mismatched write timestamp %s`, optOpsTimestamp) - } - } + if failure == `` && writeTS.IsSet() && execTimestamp.IsSet() && writeTS != execTimestamp { + failure = fmt.Sprintf(`mismatched write timestamp %s and exec timestamp %s`, writeTS, execTimestamp) } if failure != `` { @@ -848,58 +1012,36 @@ func (v *validator) checkAtomicCommitted( } func (v *validator) checkAtomicAmbiguous(atomicType string, txnObservations []observedOp) { - var somethingCommitted bool - deletedKeysInTxn := make(map[string]int) - var hadWrite bool - var maybeExecTS hlc.Timestamp + // If the atomic unit hasn't observed any writes (i.e. it's a read-only/admin + // op) or any part of it has materialized, treat it as committed. + // + // TODO(tbg): even when there's no materialized write, we could treat the + // prefix of pure reads as a committed operation. This is probably most + // relevant for aborted txns, which must have still seen a consistent snapshot + // before they realized they were aborted, and which had bugs in the past. + var execTimestamp hlc.Timestamp + var isRW bool for _, observation := range txnObservations { - switch o := observation.(type) { - case *observedWrite: - hadWrite = true - if o.Materialized { - somethingCommitted = true - maybeExecTS.Forward(o.Timestamp) // use Forward() just in case o.Timestamp is zero - break - } - if o.isDelete() && len(v.tombstonesForKey[string(o.Key)]) > v.committedDeletesForKey[string(o.Key)] { - deletedKeysInTxn[string(o.Key)]++ - break - } + o, ok := observation.(*observedWrite) + if !ok { + continue + } + isRW = true + if o.Timestamp.IsSet() { + execTimestamp = o.Timestamp + break } } - if len(deletedKeysInTxn) > 0 { - // TODO(sarkesian): Since we can't rely on the transaction write timestamp - // in an ambiguous transaction, and therefore cannot identify the tombstone - // resulting from a delete operation, it is impossible to validate if the - // transaction was actually atomic. For now, we have chosen to fail loudly, - // though if we are able to validate properly, this should be removed. - // - // TODO(tbg): this might be addressable. For an ambiguous transaction we - // should still be able to salvage the timestamp at which the transaction - // would have committed if it did, because kvnemesis always has a local - // TxnCoordSender which always knows the one possible commit timestamp - // and so it's simply a matter of making sure this information is - // guaranteed to flow back with the AmbiguousResultError. - err := errors.Errorf( - `unable to validate delete operations in ambiguous transactions: %s`, - printObserved(txnObservations...), - ) - v.failures = append(v.failures, err) - - for key := range deletedKeysInTxn { - // NB: We don't know for sure if these delete committed, but we know we - // still have tombstones for the keys. If we are incorrect in assuming it - // committed, it will affect delete counting in subsequent transactions; - // note that when dealing with ambiguous deletes that fail to commit, - // later deletes may show "committed delete missing write" errors. - v.committedDeletesForKey[key]++ - } - } else if !hadWrite || somethingCommitted { - v.checkAtomicCommitted(atomicType, txnObservations, maybeExecTS) + if !isRW || execTimestamp.IsSet() { + v.checkAtomicCommitted(atomicType, txnObservations, execTimestamp) } else { // This is a writing transaction but not a single one of its writes // showed up in KV, so verify that it is uncommitted. + // + // NB: if there's ever a way for a writing transaction to not leave + // a trace on the rangefeed (DeleteRange comes to mind) then it's + // fine to treat that transaction as uncommitted as well. v.checkAtomicUncommitted(atomicType, txnObservations) } } @@ -912,7 +1054,7 @@ func (v *validator) checkAtomicUncommitted(atomicType string, txnObservations [] } switch o := observed.(type) { case *observedWrite: - if o.Materialized { + if o.Timestamp.IsSet() { failure = atomicType + ` had writes` } // NB: While we don't check deletes here, as we cannot uniquely identify @@ -937,7 +1079,57 @@ func (v *validator) checkAtomicUncommitted(atomicType string, txnObservations [] } } -func (v *validator) failIfError(op Operation, r Result) { +// checkError returns true if the operation resulted in an error. It also registers a failure +// with the validator unless the error is an ambiguous result, an omitted error, or a retry +// error. Additional exceptions may be passed in. +// Exceptions don't influence the return value, which simply indicates whether there +// was *any* error. This is useful because any error usually means the rest of the validation +// for the command ought to be skipped. +// +// Writing operations usually want to call checkNonAmbError instead. +// +// Note that in a Batch each operation inherits the outcome of the Batch as a +// whole, which means that a read operation may result in an +// AmbiguousResultError (which it presumably didn't cause). Ideally Batch would +// track this more precisely but until it does we're just going to allow +// ambiguous results everywhere. We could also work around this in kvnemesis by +// tracking which context we're in, i.e. allow ambiguous results for, say, a Get +// but only if in a batched context. Similarly, we shouldn't allow retry errors +// when in a non-batched context, since these ought to be retried away +// internally. +func (v *validator) checkError( + op Operation, r Result, extraExceptions ...func(err error) bool, +) (ambiguous, hadError bool) { + sl := []func(error) bool{ + exceptAmbiguous, exceptOmitted, exceptRetry, + exceptDelRangeUsingTombstoneStraddlesRangeBoundary, + } + sl = append(sl, extraExceptions...) + return v.failIfError(op, r, sl...) +} + +// checkNonAmbError returns true if the result has an error, but the error +// is not an ambiguous result. It applies the same exceptions as checkError, +// which won't cause a failure to be emitted to the validator for certain +// "excepted" error types. True will be returned for those nevertheless. +// +// This is typically called by operations that may write, since these always +// want to emit observedWrites in case the operation actually did commit. +func (v *validator) checkNonAmbError( + op Operation, r Result, extraExceptions ...func(error) bool, +) bool { + isAmb, isErr := v.checkError(op, r, extraExceptions...) + return isErr && !isAmb +} + +// failIfError is the lower-level version of checkError that requires all +// exceptions to be passed in. This includes exceptAmbiguous, if desired. +// The first bool will be true if the error is an ambiguous result. Note +// that for this ambiguous result to *not* also be registered as a failure, +// exceptAmbiguous must be passed in. +func (v *validator) failIfError( + op Operation, r Result, exceptions ...func(err error) bool, +) (ambiguous, hasError bool) { switch r.Type { case ResultType_Unknown: err := errors.AssertionFailedf(`unknown result %s`, op) @@ -945,9 +1137,15 @@ func (v *validator) failIfError(op Operation, r Result) { case ResultType_Error: ctx := context.Background() err := errors.DecodeError(ctx, *r.Err) + for _, fn := range exceptions { + if fn(err) { + return exceptAmbiguous(err), true + } + } err = errors.Wrapf(err, `error applying %s`, op) v.failures = append(v.failures, err) } + return false, false } func errorFromResult(r Result) error { @@ -958,16 +1156,14 @@ func errorFromResult(r Result) error { return errors.DecodeError(ctx, *r.Err) } -func resultIsError(r Result, reference error) bool { - return errors.Is(errorFromResult(r), reference) -} - -func resultIsRetryable(r Result) bool { - return errors.HasInterface(errorFromResult(r), (*roachpb.ClientVisibleRetryError)(nil)) +func exceptLivenessCacheMiss(err error) bool { + return errors.Is(err, liveness.ErrRecordCacheMiss) } func resultIsAmbiguous(r Result) bool { - return errors.HasInterface(errorFromResult(r), (*roachpb.ClientVisibleAmbiguousError)(nil)) + resErr := errorFromResult(r) + hasClientVisibleAE := errors.HasInterface(resErr, (*roachpb.ClientVisibleAmbiguousError)(nil)) + return hasClientVisibleAE } // TODO(dan): Checking errors using string containment is fragile at best and a @@ -998,45 +1194,97 @@ func mustGetStringValue(value []byte) string { return string(b) } -func validReadTimes( - b *pebble.Batch, key roachpb.Key, value []byte, anyValueAccepted bool, -) disjointTimeSpans { - var validTimes disjointTimeSpans - end := hlc.MaxTimestamp - - iter := b.NewIter(nil) +func validReadTimes(b *pebble.Batch, key roachpb.Key, value []byte) disjointTimeSpans { + var hist []storage.MVCCValue + lowerBound := storage.EncodeMVCCKey(storage.MVCCKey{Key: key}) + upperBound := storage.EncodeMVCCKey(storage.MVCCKey{Key: key.Next()}) + iter := b.NewIter(&pebble.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, + LowerBound: lowerBound, + UpperBound: upperBound, + }) defer func() { _ = iter.Close() }() - iter.SeekGE(storage.EncodeMVCCKey(storage.MVCCKey{Key: key})) + + iter.SeekGE(lowerBound) for ; iter.Valid(); iter.Next() { + hasPoint, hasRange := iter.HasPointAndRange() + if hasRange && iter.RangeKeyChanged() { + encK, encEK := iter.RangeBounds() + k, err := storage.DecodeMVCCKey(encK) + if err != nil { + panic(err) + } + ek, err := storage.DecodeMVCCKey(encEK) + if err != nil { + panic(err) + } + + sp := roachpb.Span{Key: k.Key, EndKey: ek.Key} + if !sp.ContainsKey(key) { + // We used bounds that should make this impossible. + panic(fmt.Sprintf("iterator for %s on non-overlapping range key %s", key, sp)) + } + // Range key contains the key. Emit a point deletion on the key + // at the tombstone's timestamp for each active range key. + for _, rk := range iter.RangeKeys() { + ts, err := storage.DecodeMVCCTimestampSuffix(rk.Suffix) + if err != nil { + panic(err) + } + hist = append(hist, storage.MVCCValue{Value: roachpb.Value{Timestamp: ts}}) + } + } + + if !hasPoint { + continue + } + mvccKey, err := storage.DecodeMVCCKey(iter.Key()) if err != nil { panic(err) } + if !mvccKey.Key.Equal(key) { - break + // We used bounds that should make this impossible. + panic("iterator on non-overlapping key") + } + + // Handle a point key - put it into `hist`. + valB, err := iter.ValueAndErr() + if err != nil { + panic(err) } - if (anyValueAccepted && len(iter.Value()) > 0) || - (!anyValueAccepted && mustGetStringValue(iter.Value()) == mustGetStringValue(value)) { - validTimes = append(validTimes, timeSpan{Start: mvccKey.Timestamp, End: end}) + v, err := storage.DecodeMVCCValue(valB) + if err != nil { + panic(err) } - end = mvccKey.Timestamp + v.Value.Timestamp = mvccKey.Timestamp + hist = append(hist, v) } - if !anyValueAccepted && len(value) == 0 { + // The slice isn't sorted due to MVCC rangedels. Sort in descending order. + sort.Slice(hist, func(i, j int) bool { + return hist[j].Value.Timestamp.Less(hist[i].Value.Timestamp) + }) + + sv := mustGetStringValue(value) + var validTimes disjointTimeSpans + end := hlc.MaxTimestamp + for i := range hist { + v := hist[i].Value + if mustGetStringValue(v.RawBytes) == sv { + validTimes = append(validTimes, timeSpan{Start: v.Timestamp, End: end}) + } + end = v.Timestamp + } + + if len(value) == 0 { validTimes = append(disjointTimeSpans{{Start: hlc.MinTimestamp, End: end}}, validTimes...) } - // NB: With the exception of deletes, the "only write each value once" - // property of the generator means that we have a 1:1 mapping between some - // `(key, non-nil-value)` observation and a time span in which it was valid. - // With deletes, there multiple disjoint spans for a `(key, nil-value)` - // observation (i.e. before the key existed, after it was deleted). - // This means that for each read, we must consider all possibly valid times. return validTimes } -func validScanTime( - b *pebble.Batch, span roachpb.Span, kvs []roachpb.KeyValue, isDeleteRange bool, -) multiKeyTimeSpan { +func validScanTime(b *pebble.Batch, span roachpb.Span, kvs []roachpb.KeyValue) multiKeyTimeSpan { valid := multiKeyTimeSpan{ Gaps: disjointTimeSpans{{Start: hlc.MinTimestamp, End: hlc.MaxTimestamp}}, } @@ -1046,14 +1294,16 @@ func validScanTime( // Since scan results don't include deleted keys, there should only ever // be 0 or 1 valid read time span for each `(key, specific-non-nil-value)` // returned, given that the values are guaranteed to be unique by the - // Generator. However, in the DeleteRange case where we are looking for - // `(key, any-non-nil-value)`, it is of course valid for there to be - // multiple disjoint time spans. - validTimes := validReadTimes(b, kv.Key, kv.Value.RawBytes, isDeleteRange) - if !isDeleteRange && len(validTimes) > 1 { + // Generator. + // + // NB: we use value uniqueness here, but we could also use seqnos, so this + // is only a left-over of past times rather than an actual reliance on + // unique values. + validTimes := validReadTimes(b, kv.Key, kv.Value.RawBytes) + if len(validTimes) > 1 { panic(errors.AssertionFailedf( - `invalid number of read time spans for a (key,non-nil-value) pair in scan results: %s->%s`, - kv.Key, mustGetStringValue(kv.Value.RawBytes))) + `invalid number of read time spans for a (key,non-nil-value) pair in scan results: %s->%s: %v`, + kv.Key, mustGetStringValue(kv.Value.RawBytes), validTimes)) } if len(validTimes) == 0 { validTimes = append(validTimes, timeSpan{}) @@ -1069,9 +1319,18 @@ func validScanTime( } missingKeys := make(map[string]disjointTimeSpans) + + // Next, discover all of the keys that were *not* returned but overlap the + // scan span and compute validReadTimes for them. + // + // Note that this iterator ignores MVCC range deletions. We use this iterator + // only to *discover* point keys; we then invoke validReadTimes for each of + // them which *does* take into account MVCC range deletions. iter := b.NewIter(nil) defer func() { _ = iter.Close() }() + iter.SeekGE(storage.EncodeMVCCKey(storage.MVCCKey{Key: span.Key})) + for ; iter.Valid(); iter.Next() { mvccKey, err := storage.DecodeMVCCKey(iter.Key()) if err != nil { @@ -1088,7 +1347,7 @@ func validScanTime( if _, ok := missingKeys[string(mvccKey.Key)]; !ok { // Key not in scan response. Only valid if scan was before key's time, or // at a time when the key was deleted. - missingKeys[string(mvccKey.Key)] = validReadTimes(b, mvccKey.Key, nil, false) + missingKeys[string(mvccKey.Key)] = validReadTimes(b, mvccKey.Key, nil) } } @@ -1110,21 +1369,26 @@ func printObserved(observedOps ...observedOp) string { opCode := "w" if o.isDelete() { if o.IsDeleteRange { - opCode = "dr.d" + if len(o.EndKey) == 0 { + opCode = "dr.d" + } else { + opCode = "rd" // mvcc range del + } } else { opCode = "d" } } ts := `missing` - if o.Materialized { - if o.isDelete() && o.Timestamp.IsEmpty() { - ts = `uncertain` - } else { - ts = o.Timestamp.String() - } + if o.Timestamp.IsSet() { + ts = o.Timestamp.String() + } + if len(o.EndKey) == 0 { + fmt.Fprintf(&buf, "[%s]%s:%s->%s@%s", + opCode, o.Key, ts, mustGetStringValue(o.Value.RawBytes), o.Seq) + } else { + fmt.Fprintf(&buf, "[%s][%s,%s):%s->%s@%s", + opCode, o.Key, o.EndKey, ts, mustGetStringValue(o.Value.RawBytes), o.Seq) } - fmt.Fprintf(&buf, "[%s]%s:%s->%s", - opCode, o.Key, ts, mustGetStringValue(o.Value.RawBytes)) case *observedRead: fmt.Fprintf(&buf, "[r]%s:", o.Key) validTimes := o.ValidTimes diff --git a/pkg/kv/kvnemesis/validator_test.go b/pkg/kv/kvnemesis/validator_test.go index 34583daf18a1..e05171770bad 100644 --- a/pkg/kv/kvnemesis/validator_test.go +++ b/pkg/kv/kvnemesis/validator_test.go @@ -12,16 +12,23 @@ package kvnemesis import ( "context" + "fmt" + "strings" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -45,18 +52,23 @@ func withResult(op Operation) Operation { return withResultErr(op, nil /* err */) } +func withAmbResult(op Operation) Operation { + err := roachpb.NewAmbiguousResultErrorf("boom") + op = withResultErr(op, err) + return op +} + func withResultErr(op Operation, err error) Operation { *op.Result() = resultInit(context.Background(), err) - // Most operations in tests use timestamp 1, so use that and any test cases - // that differ from that can use withTimestamp(). - if op.Result().OptionalTimestamp.IsEmpty() { - op.Result().OptionalTimestamp = hlc.Timestamp{WallTime: 1} - } return op } func withReadResult(op Operation, value string) Operation { - op = withResult(op) + return withReadResultTS(op, value, 0) +} + +func withReadResultTS(op Operation, value string, ts int) Operation { + op = withResultTS(op, ts) get := op.GetValue().(*GetOperation) get.Result.Type = ResultType_Value if value != `` { @@ -65,45 +77,107 @@ func withReadResult(op Operation, value string) Operation { return op } -func withScanResult(op Operation, kvs ...KeyValue) Operation { - op = withResult(op) +func withScanResultTS(op Operation, ts int, kvs ...KeyValue) Operation { + op = withTimestamp(withResult(op), ts) scan := op.GetValue().(*ScanOperation) scan.Result.Type = ResultType_Values scan.Result.Values = kvs return op } -func withDeleteRangeResult(op Operation, keys ...[]byte) Operation { - op = withResult(op) +func withDeleteRangeResult(op Operation, ts int, keys ...[]byte) Operation { + op = withTimestamp(withResult(op), ts) delRange := op.GetValue().(*DeleteRangeOperation) delRange.Result.Type = ResultType_Keys delRange.Result.Keys = keys return op } +type seqKV struct { + key, endKey roachpb.Key + val []byte // contains seq + ts hlc.Timestamp +} + +func (kv *seqKV) seq() kvnemesisutil.Seq { + mvccV, err := storage.DecodeMVCCValue(kv.val) + if err != nil { + panic(err) + } + return mvccV.KVNemesisSeq.Get() +} + func TestValidate(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - kv := func(key string, ts int, value string) storage.MVCCKeyValue { - return storage.MVCCKeyValue{ - Key: storage.MVCCKey{ - Key: []byte(key), - Timestamp: hlc.Timestamp{WallTime: int64(ts)}, - }, - Value: roachpb.MakeValueFromString(value).RawBytes, + if !buildutil.CrdbTestBuild { + // `roachpb.RequestHeader` and `MVCCValueHeader` have a KVNemesisSeq field + // that is zero-sized outside test builds. We could revisit that should + // a need arise to run kvnemesis against production binaries. + skip.IgnoreLint(t, "kvnemesis must be run with the crdb_test build tag") + } + + const ( + s1 = kvnemesisutil.Seq(1 + iota) + s2 + s3 + s4 + s5 + s6 + ) + + const ( + noTS = iota + t1 + t2 + t3 + t4 + t5 + ) + + vi := func(s kvnemesisutil.Seq) string { + return PutOperation{Seq: s}.Value() + } + var ( + v1 = vi(s1) + v2 = vi(s2) + v3 = vi(s3) + ) + + valWithSeq := func(seq kvnemesisutil.Seq, v roachpb.Value) []byte { + var vh enginepb.MVCCValueHeader + vh.KVNemesisSeq.Set(seq) + sl, err := storage.EncodeMVCCValue(storage.MVCCValue{ + MVCCValueHeader: vh, + Value: v, + }) + if err != nil { + panic(err) } + return sl } - tombstone := func(key string, ts int) storage.MVCCKeyValue { - return storage.MVCCKeyValue{ - Key: storage.MVCCKey{ - Key: []byte(key), - Timestamp: hlc.Timestamp{WallTime: int64(ts)}, - }, - Value: nil, + kv := func(key string, ts int, seq kvnemesisutil.Seq) seqKV { + return seqKV{ + key: roachpb.Key(key), + ts: hlc.Timestamp{WallTime: int64(ts)}, + val: valWithSeq(seq, roachpb.MakeValueFromString(PutOperation{Seq: seq}.Value())), } } - kvs := func(kvs ...storage.MVCCKeyValue) []storage.MVCCKeyValue { + tombstone := func(key string, ts int, seq kvnemesisutil.Seq) seqKV { + r := kv(key, ts, seq) + r.val = valWithSeq(seq, roachpb.Value{}) + return r + } + rd := func(key, endKey string, ts int, seq kvnemesisutil.Seq) seqKV { + return seqKV{ + key: roachpb.Key(key), + endKey: roachpb.Key(endKey), + ts: hlc.Timestamp{WallTime: int64(ts)}, + val: valWithSeq(seq, roachpb.Value{}), + } + } + kvs := func(kvs ...seqKV) []seqKV { return kvs } scanKV := func(key, value string) KeyValue { @@ -114,1792 +188,1673 @@ func TestValidate(t *testing.T) { } tests := []struct { - name string - steps []Step - kvs []storage.MVCCKeyValue - expected []string + name string + steps []Step + kvs []seqKV }{ { - name: "no ops and no kvs", - steps: nil, - kvs: nil, - expected: nil, + name: "no ops and no kvs", + steps: nil, + kvs: nil, }, { - name: "no ops with unexpected write", - steps: nil, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{`extra writes: [w]"a":0.000000001,0->v1`}, + name: "no ops with unexpected write", + steps: nil, + kvs: kvs(kv(k1, t1, s1)), }, { - name: "no ops with unexpected delete", - steps: nil, - kvs: kvs(tombstone(`a`, 1)), - expected: []string{`extra writes: [d]"a":uncertain->`}, + name: "no ops with unexpected delete", + steps: nil, + kvs: kvs(tombstone(k1, t1, s1)), }, { - name: "one put with expected write", - steps: []Step{step(withResult(put(`a`, `v1`)))}, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + name: "one put with expected write", + steps: []Step{step(withResultTS(put(k1, s1), t1))}, + kvs: kvs(kv(k1, t1, s1)), }, { - name: "one delete with expected write", - steps: []Step{step(withResult(del(`a`)))}, - kvs: kvs(tombstone(`a`, 1)), - expected: nil, + name: "one delete with expected write", + steps: []Step{step(withResultTS(del(k1, s1), t1))}, + kvs: kvs(tombstone(k1, t1, s1)), }, { - name: "one put with missing write", - steps: []Step{step(withResult(put(`a`, `v1`)))}, - kvs: nil, - expected: []string{`committed put missing write: [w]"a":missing->v1`}, + name: "one put with missing write", + steps: []Step{step(withResultTS(put(k1, s1), t1))}, + kvs: nil, }, { - name: "one delete with missing write", - steps: []Step{step(withResult(del(`a`)))}, - kvs: nil, - expected: []string{`committed delete missing write: [d]"a":missing->`}, + name: "one delete with missing write", + steps: []Step{step(withResultTS(del(k1, s1), t1))}, + kvs: nil, }, { - name: "one ambiguous put with successful write", - steps: []Step{step(withResultErr(put(`a`, `v1`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + name: "one ambiguous put with successful write", + steps: []Step{step(withAmbResult(put(k1, s1)))}, + kvs: kvs(kv(k1, t1, s1)), }, { - name: "one ambiguous delete with successful write", - steps: []Step{step(withResultErr(del(`a`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, - kvs: kvs(tombstone(`a`, 1)), - expected: []string{`unable to validate delete operations in ambiguous transactions: [d]"a":missing->`}, + name: "one ambiguous delete with successful write", + steps: []Step{step(withAmbResult(del(k1, s1)))}, + kvs: kvs(tombstone(k1, t1, s1)), }, + { - name: "one ambiguous put with failed write", - steps: []Step{step(withResultErr(put(`a`, `v1`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, - kvs: nil, - expected: nil, + name: "one ambiguous put with failed write", + steps: []Step{step(withAmbResult(put(k1, s1)))}, + kvs: nil, }, { - name: "one ambiguous delete with failed write", - steps: []Step{step(withResultErr(del(`a`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, - kvs: nil, - expected: nil, + name: "one ambiguous delete with failed write", + steps: []Step{step(withAmbResult(del(k1, s1)))}, + kvs: nil, }, { name: "one ambiguous delete with failed write before a later committed delete", steps: []Step{ - step(withResultErr(del(`a`), roachpb.NewAmbiguousResultError(errors.New("boom")))), - step(withResultTS(del(`a`), 2)), - }, - kvs: kvs(tombstone(`a`, 2)), - expected: []string{ - `unable to validate delete operations in ambiguous transactions: [d]"a":missing->`, + step(withAmbResult(del(k1, s1))), + step(withResultTS(del(k1, s2), t2)), }, + kvs: kvs(tombstone(k1, t2, s2)), }, { - name: "one retryable put with write (correctly) missing", - steps: []Step{step(withResultErr(put(`a`, `v1`), retryableError))}, - kvs: nil, - expected: nil, + name: "one retryable put with write (correctly) missing", + steps: []Step{step(withResultErr(put(k1, s1), retryableError))}, + kvs: nil, }, { - name: "one retryable delete with write (correctly) missing", - steps: []Step{step(withResultErr(del(`a`), retryableError))}, - kvs: nil, - expected: nil, + name: "one retryable delete with write (correctly) missing", + steps: []Step{step(withResultErr(del(k1, s1), retryableError))}, + kvs: nil, }, { - name: "one retryable put with write (incorrectly) present", - steps: []Step{step(withResultErr(put(`a`, `v1`), retryableError))}, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{`uncommitted put had writes: [w]"a":0.000000001,0->v1`}, + name: "one retryable put with write (incorrectly) present", + steps: []Step{step(withTimestamp(withResultErr(put(k1, s1), retryableError), t1))}, + kvs: kvs(kv(k1, t1, s1)), }, { name: "one retryable delete with write (incorrectly) present", - steps: []Step{step(withResultErr(del(`a`), retryableError))}, - kvs: kvs(tombstone(`a`, 1)), + steps: []Step{step(withResultErr(del(k1, s1), retryableError))}, + kvs: kvs(tombstone(k1, t1, s1)), // NB: Error messages are different because we can't match an uncommitted // delete op to a stored kv like above. - expected: []string{`extra writes: [d]"a":uncertain->`}, + }, { name: "one delete with expected write after write transaction with shadowed delete", steps: []Step{ - step(withResultTS(del(`a`), 1)), - step(withResultTS(put(`a`, `v1`), 2)), + step(withResultTS(del(k1, s1), t1)), + step(withResultTS(put(k1, s2), t2)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResultOK(put(`a`, `v2`)), - withResultOK(del(`a`)), - withResultOK(put(`a`, `v3`)), - ), 3)), - step(withResultTS(del(`a`), 4)), + withResultOK(put(k1, s3)), + withResultOK(del(k1, s4)), + withResultOK(put(k1, s5)), + ), t3)), + step(withResultTS(del(k1, s6), t4)), }, kvs: kvs( - tombstone(`a`, 1), - kv(`a`, 2, `v1`), - kv(`a`, 3, `v3`), - tombstone(`a`, 4)), - expected: nil, + tombstone(k1, t1, s1), + kv(k1, t2, s2), + kv(k1, t3, s5), + tombstone(k1, t4, s6)), }, { - name: "one batch put with successful write", - steps: []Step{step(withResult(batch(withResult(put(`a`, `v1`)))))}, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + name: "one batch put with successful write", + steps: []Step{step(withResultTS(batch(withResult(put(k1, s1))), t1))}, + kvs: kvs(kv(k1, t1, s1)), }, { - name: "one batch delete with successful write", - steps: []Step{step(withResult(batch(withResult(del(`a`)))))}, - kvs: kvs(tombstone(`a`, 1)), - expected: nil, + name: "one batch delete with successful write", + steps: []Step{step(withResultTS(batch(withResult(del(k1, s1))), t1))}, + kvs: kvs(tombstone(k1, t1, s1)), }, { - name: "one batch put with missing write", - steps: []Step{step(withResult(batch(withResult(put(`a`, `v1`)))))}, - kvs: nil, - expected: []string{`committed batch missing write: [w]"a":missing->v1`}, + name: "one batch put with missing write", + steps: []Step{step(withResultTS(batch(withResult(put(k1, s1))), t1))}, + kvs: nil, }, { - name: "one batch delete with missing write", - steps: []Step{step(withResult(batch(withResult(del(`a`)))))}, - kvs: nil, - expected: []string{`committed batch missing write: [d]"a":missing->`}, + name: "one batch delete with missing write", + steps: []Step{step(withResultTS(batch(withResult(del(k1, s1))), t1))}, + kvs: nil, }, { name: "one transactionally committed put with the correct writes", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + ), t1)), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1)), }, + { name: "one transactionally committed delete with the correct writes", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(del(k1, s1)), + ), t1)), }, - kvs: kvs(tombstone(`a`, 1)), - expected: nil, + kvs: kvs(tombstone(k1, t1, s1)), }, { name: "one transactionally committed put with first write missing", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(put(`b`, `v2`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(put(k2, s2)), + ), t1)), }, - kvs: kvs(kv(`b`, 1, `v2`)), - expected: []string{`committed txn missing write: [w]"a":missing->v1 [w]"b":0.000000001,0->v2`}, + kvs: kvs(kv(k2, t1, s2)), }, { name: "one transactionally committed delete with first write missing", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`)), - withResult(del(`b`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(del(k1, s1)), + withResult(del(k2, s2)), + ), t1)), }, - kvs: kvs(tombstone(`b`, 1)), - expected: []string{`committed txn missing write: [d]"a":missing-> [d]"b":0.000000001,0->`}, + kvs: kvs(tombstone(k2, t1, s2)), }, { name: "one transactionally committed put with second write missing", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(put(`b`, `v2`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(put(k2, s2)), + ), t1)), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{`committed txn missing write: [w]"a":0.000000001,0->v1 [w]"b":missing->v2`}, + kvs: kvs(kv(k1, t1, s1)), }, { name: "one transactionally committed delete with second write missing", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`)), - withResult(del(`b`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(del(k1, s1)), + withResult(del(k2, s2)), + ), t1)), }, - kvs: kvs(tombstone(`a`, 1)), - expected: []string{`committed txn missing write: [d]"a":0.000000001,0-> [d]"b":missing->`}, + kvs: kvs(tombstone(k1, t1, s1)), }, { name: "one transactionally committed put with write timestamp disagreement", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(put(`b`, `v2`)), - ), 1))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `committed txn non-atomic timestamps: [w]"a":0.000000001,0->v1 [w]"b":0.000000002,0->v2`, + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(put(k2, s2)), + ), t1)), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "one transactionally committed delete with write timestamp disagreement", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`)), - withResult(del(`b`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(del(k1, s1)), + withResult(del(k2, s2)), + ), t1)), }, - kvs: kvs(tombstone(`a`, 1), tombstone(`b`, 2)), + kvs: kvs(tombstone(k1, t1, s1), tombstone(k2, t2, s2)), // NB: Error messages are different because we can't match an uncommitted // delete op to a stored kv like above. - expected: []string{ - `committed txn missing write: [d]"a":0.000000001,0-> [d]"b":missing->`, - }, }, { name: "one transactionally rolled back put with write (correctly) missing", steps: []Step{ step(withResultErr(closureTxn(ClosureTxnType_Rollback, - withResult(put(`a`, `v1`)), + withResult(put(k1, s1)), ), errors.New(`rollback`))), }, - kvs: nil, - expected: nil, + kvs: nil, }, { name: "one transactionally rolled back delete with write (correctly) missing", steps: []Step{ step(withResultErr(closureTxn(ClosureTxnType_Rollback, - withResult(del(`a`)), + withResult(del(k1, s1)), ), errors.New(`rollback`))), }, - kvs: nil, - expected: nil, + kvs: nil, }, { name: "one transactionally rolled back put with write (incorrectly) present", steps: []Step{ step(withResultErr(closureTxn(ClosureTxnType_Rollback, - withResult(put(`a`, `v1`)), + withResult(put(k1, s1)), ), errors.New(`rollback`))), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{`uncommitted txn had writes: [w]"a":0.000000001,0->v1`}, + kvs: kvs(kv(k1, t1, s1)), }, { name: "one transactionally rolled back delete with write (incorrectly) present", steps: []Step{ step(withResultErr(closureTxn(ClosureTxnType_Rollback, - withResult(del(`a`)), + withResult(del(k1, s1)), ), errors.New(`rollback`))), }, - kvs: kvs(tombstone(`a`, 1)), - expected: []string{`extra writes: [d]"a":uncertain->`}, + kvs: kvs(tombstone(k1, t1, s1)), }, { name: "one transactionally rolled back batch put with write (correctly) missing", steps: []Step{ step(withResultErr(closureTxn(ClosureTxnType_Rollback, withResult(batch( - withResult(put(`a`, `v1`)), + withResult(put(k1, s1)), )), ), errors.New(`rollback`))), }, - kvs: nil, - expected: nil, + kvs: nil, }, { name: "one transactionally rolled back batch delete with write (correctly) missing", steps: []Step{ step(withResultErr(closureTxn(ClosureTxnType_Rollback, withResult(batch( - withResult(del(`a`)), + withResult(del(k1, s1)), )), ), errors.New(`rollback`))), }, - kvs: nil, - expected: nil, + kvs: nil, }, { name: "two transactionally committed puts of the same key", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(put(`a`, `v2`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(put(k1, s2)), + ), t1)), + }, + kvs: kvs(kv(k1, t1, s2)), + }, + { + // NB: this can't happen in practice since KV would throw a WriteTooOldError. + // But transactionally this works, see below. + name: "batch with two deletes of same key", + steps: []Step{ + step(withResultTS(batch( + withResult(del(k1, s1)), + withResult(del(k1, s2)), + ), t1)), }, - kvs: kvs(kv(`a`, 1, `v2`)), - expected: nil, + kvs: kvs(tombstone(k1, t1, s2)), }, { name: "two transactionally committed deletes of the same key", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`)), - withResult(del(`a`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(del(k1, s1)), + withResult(del(k1, s2)), + ), t1)), }, - kvs: kvs(tombstone(`a`, 1)), - expected: nil, + kvs: kvs(tombstone(k1, t1, s2)), }, { name: "two transactionally committed writes (put, delete) of the same key", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(del(`a`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(del(k1, s2)), + ), t1)), }, - kvs: kvs(tombstone(`a`, 1)), - expected: nil, + kvs: kvs(tombstone(k1, t1, s2)), }, { name: "two transactionally committed writes (delete, put) of the same key", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`)), - withResult(put(`a`, `v2`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(del(k1, s1)), + withResult(put(k1, s2)), + ), t1)), }, - kvs: kvs(kv(`a`, 1, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s2)), }, { name: "two transactionally committed puts of the same key with extra write", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(put(`a`, `v2`)), - ), 2))), - }, - // HACK: These should be the same timestamp. See the TODO in - // watcher.processEvents. - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`)), - expected: []string{ - `committed txn overwritten key had write: [w]"a":0.000000001,0->v1 [w]"a":0.000000002,0->v2`, + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(put(k1, s2)), + ), t2)), }, + kvs: kvs(kv(k1, t1, s1), kv(k1, t2, s2)), }, { name: "two transactionally committed deletes of the same key with extra write", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`)), - withResult(del(`a`)), - ), 1))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(del(k1, s1)), + withResult(del(k1, s2)), + ), t1)), }, - // HACK: These should be the same timestamp. See the TODO in - // watcher.processEvents. - kvs: kvs(tombstone(`a`, 1), tombstone(`a`, 2)), - expected: []string{`extra writes: [d]"a":uncertain->`}, + kvs: kvs(tombstone(k1, t1, s1), tombstone(k1, t2, s2)), }, { name: "two transactionally committed writes (put, delete) of the same key with extra write", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResultOK(put(`a`, `v1`)), - withResultOK(del(`a`)), - ), 1)), - }, - // HACK: These should be the same timestamp. See the TODO in - // watcher.processEvents. - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), - expected: []string{ - // NB: the deletion is marked as "missing" because we are using timestamp 1 for the - // txn and the tombstone is at 2; so it isn't marked as materialized in the verifier. - `committed txn overwritten key had write: [w]"a":0.000000001,0->v1 [d]"a":missing->`, + withResultOK(put(k1, s1)), + withResultOK(del(k1, s2)), + ), t1)), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2)), }, { - name: "ambiguous transaction committed", + name: "ambiguous put-put transaction committed", steps: []Step{ - step(withResultErr(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(put(`b`, `v2`)), - ), roachpb.NewAmbiguousResultError(errors.New("boom")))), + step(withAmbResult(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(put(k2, s2)), + ))), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 1, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t1, s2)), }, { - name: "ambiguous transaction with delete committed", + name: "ambiguous put-del transaction committed", steps: []Step{ - step(withResultErr(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(del(`b`)), - ), roachpb.NewAmbiguousResultError(errors.New("boom")))), - }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`b`, 1)), - // TODO(sarkesian): If able to determine the tombstone resulting from a - // delete in an ambiguous txn, this should pass without error. - // For now we fail validation on all ambiguous transactions with deletes. - expected: []string{ - `unable to validate delete operations in ambiguous transactions: [w]"a":0.000000001,0->v1 [d]"b":missing->`, + step(withAmbResult(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(del(k2, s2)), + ))), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k2, t1, s2)), }, { - name: "ambiguous transaction did not commit", + // NB: this case is a tough nut to crack if we rely on timestamps since we + // don't have a single timestamp result here and no unique values. But we + // use sequence numbers so no problem! We learn the commit timestamp from + // them if any of the writes show up. + name: "ambiguous del-del transaction committed", steps: []Step{ - step(withResultErr(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(put(`b`, `v2`)), - ), roachpb.NewAmbiguousResultError(errors.New("boom")))), + step(withAmbResult(closureTxn(ClosureTxnType_Commit, + withResult(del(k1, s1)), + withResult(del(k1, s2)), + ))), }, - kvs: nil, - expected: nil, + kvs: kvs(tombstone(k1, t1, s2)), }, { - name: "ambiguous transaction with delete did not commit", + name: "ambiguous del-del transaction committed but wrong seq", steps: []Step{ - step(withResultErr(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(del(`b`)), - ), roachpb.NewAmbiguousResultError(errors.New("boom")))), + step(withAmbResult(closureTxn(ClosureTxnType_Commit, + withResult(del(k1, s1)), + withResult(del(k1, s2)), + ))), }, - kvs: nil, - expected: nil, + kvs: kvs(tombstone(k1, t1, s1)), }, { - name: "ambiguous transaction committed but has validation error", + name: "ambiguous put-put transaction did not commit", steps: []Step{ - step(withResultErr(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(put(`b`, `v2`)), - ), roachpb.NewAmbiguousResultError(errors.New("boom")))), + step(withAmbResult(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(put(k2, s2)), + ))), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `ambiguous txn non-atomic timestamps: [w]"a":0.000000001,0->v1 [w]"b":0.000000002,0->v2`, + kvs: nil, + }, + { + name: "ambiguous put-del transaction did not commit", + steps: []Step{ + step(withAmbResult(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(del(k2, s2)), + ))), }, + kvs: nil, }, { - name: "ambiguous transaction with delete committed but has validation error", + name: "ambiguous put-put transaction committed but has validation error", steps: []Step{ - step(withResultErr(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(del(`b`)), - ), 2), roachpb.NewAmbiguousResultError(errors.New("boom")))), + step(withAmbResult(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(put(k2, s2)), + ))), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`b`, 2)), - // TODO(sarkesian): If able to determine the tombstone resulting from a - // delete in an ambiguous txn, we should get the following error: - // `ambiguous txn non-atomic timestamps: [w]"a":0.000000001,0->v1 [w]"b":0.000000002,0->v2` - // For now we fail validation on all ambiguous transactions with deletes. - expected: []string{ - `unable to validate delete operations in ambiguous transactions: [w]"a":0.000000001,0->v1 [d]"b":missing->`, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), + }, + { + name: "ambiguous put-del transaction committed but has validation error", + steps: []Step{ + step(withAmbResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + withResult(put(k1, s1)), + withResult(del(k2, s2)), + ), t2))), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k2, t2, s2)), }, { name: "one read before write", steps: []Step{ - step(withReadResult(get(`a`), ``)), - step(withResult(put(`a`, `v1`))), + step(withReadResultTS(get(k1), ``, t1)), + step(withResultTS(put(k1, s1), t2)), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + kvs: kvs(kv(k1, t2, s1)), }, { name: "one read before delete", steps: []Step{ - step(withReadResult(get(`a`), ``)), - step(withResult(del(`a`))), + step(withReadResultTS(get(k1), ``, t1)), + step(withResultTS(del(k1, s1), t2)), }, - kvs: kvs(tombstone(`a`, 1)), - expected: nil, + kvs: kvs(tombstone(k1, t2, s1)), }, { name: "one read before write and delete", steps: []Step{ - step(withReadResult(get(`a`), ``)), - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(del(`a`), 2)), + step(withReadResultTS(get(k1), ``, t1)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(del(k1, s2), t2)), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2)), }, { name: "one read before write returning wrong value", steps: []Step{ - step(withReadResult(get(`a`), `v2`)), - step(withResult(put(`a`, `v1`))), - }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{ - `committed get non-atomic timestamps: [r]"a":[0,0, 0,0)->v2`, + step(withReadResultTS(get(k1), v1, t1)), + step(withResultTS(put(k1, s1), t2)), }, + kvs: kvs(kv(k1, t2, s1)), }, { name: "one read after write", steps: []Step{ - step(withResult(put(`a`, `v1`))), - step(withReadResult(get(`a`), `v1`)), + step(withResultTS(put(k1, s1), t1)), + step(withReadResultTS(get(k1), v1, t2)), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1)), }, { name: "one read after write and delete", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(withTimestamp(del(`a`), 2), 2)), - step(withResultTS(withReadResult(get(`a`), `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(withTimestamp(del(k1, s2), t2), t2)), + step(withReadResultTS(get(k1), v1, t1)), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2)), }, { name: "one read after write and delete returning tombstone", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(del(`a`), 2)), - step(withReadResult(get(`a`), ``)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(del(k1, s2), t2)), + step(withReadResultTS(get(k1), ``, t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2)), }, { name: "one read after write returning wrong value", steps: []Step{ - step(withResult(put(`a`, `v1`))), - step(withReadResult(get(`a`), `v2`)), - }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{ - `committed get non-atomic timestamps: [r]"a":[0,0, 0,0)->v2`, + step(withResultTS(put(k1, s1), t1)), + step(withReadResultTS(get(k1), v2, t2)), }, + kvs: kvs(kv(k1, t1, s1)), }, { name: "one read in between writes", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withReadResult(get(`a`), `v1`)), - step(withResultTS(put(`a`, `v2`), 2)), + step(withResultTS(put(k1, s1), t1)), + step(withReadResultTS(get(k1), v1, t2)), + step(withResultTS(put(k1, s2), t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k1, t3, s2)), }, { name: "one read in between write and delete", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withReadResult(get(`a`), `v1`)), - step(withResultTS(del(`a`), 2)), + step(withResultTS(put(k1, s1), t1)), + step(withReadResultTS(get(k1), v1, t2)), + step(withResultTS(del(k1, s2), t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t3, s2)), }, { name: "batch of reads after writes", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResult(batch( - withReadResult(get(`a`), `v1`), - withReadResult(get(`b`), `v2`), - withReadResult(get(`c`), ``), - ))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(batch( + withReadResult(get(k1), v1), + withReadResult(get(k2), v2), + withReadResult(get(k3), ``), + ), t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "batch of reads after writes and deletes", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(del(`a`), 3)), - step(withResultTS(del(`b`), 4)), - step(withResult(batch( - withReadResult(get(`a`), `v1`), - withReadResult(get(`b`), `v2`), - withReadResult(get(`c`), ``), - ))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(del(k1, s3), t3)), + step(withResultTS(del(k2, s4), t4)), + step(withResultTS(batch( + withReadResult(get(k1), v1), + withReadResult(get(k2), v2), + withReadResult(get(k3), ``), + ), t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 4)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), tombstone(k1, t3, s3), tombstone(k2, t4, s4)), }, { name: "batch of reads after writes and deletes returning tombstones", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(del(`a`), 3)), - step(withResultTS(del(`b`), 4)), - step(withResult(batch( - withReadResult(get(`a`), ``), - withReadResult(get(`b`), ``), - withReadResult(get(`c`), ``), - ))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(del(k1, s3), t3)), + step(withResultTS(del(k2, s3), t4)), + step(withResultTS(batch( + withReadResult(get(k1), ``), + withReadResult(get(k2), ``), + withReadResult(get(k3), ``), + ), t5)), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 4)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), tombstone(k1, t3, s3), tombstone(k2, t4, s4)), }, { name: "batch of reads after writes returning wrong values", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResult(batch( - withReadResult(get(`a`), ``), - withReadResult(get(`b`), `v1`), - withReadResult(get(`c`), `v2`), - ))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `committed batch non-atomic timestamps: ` + - `[r]"a":[, 0.000000001,0)-> [r]"b":[0,0, 0,0)->v1 [r]"c":[0,0, 0,0)->v2`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(batch( + withReadResult(get(k1), ``), + withReadResult(get(k2), v1), + withReadResult(get(k3), v2), + ), t3)), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "batch of reads after writes and deletes returning wrong values", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(del(`a`), 3)), - step(withResultTS(del(`b`), 4)), - step(withResult(batch( - withReadResult(get(`a`), ``), - withReadResult(get(`b`), `v1`), - withReadResult(get(`c`), `v2`), - ))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 4)), - expected: []string{ - `committed batch non-atomic timestamps: ` + - `[r]"a":[, 0.000000001,0),[0.000000003,0, )-> [r]"b":[0,0, 0,0)->v1 [r]"c":[0,0, 0,0)->v2`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(del(k1, s3), t3)), + step(withResultTS(del(k2, s4), t4)), + step(withResultTS(batch( + withReadResult(get(k1), ``), + withReadResult(get(k2), v1), + withReadResult(get(k3), v2), + ), t5)), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), tombstone(k1, t3, s3), tombstone(k2, t4, s4)), }, { - name: "batch of reads after writes with non-empty time overlap", + name: "batch of reads after writes with empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResult(batch( - withReadResult(get(`a`), ``), - withReadResult(get(`b`), `v2`), - withReadResult(get(`c`), ``), - ))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `committed batch non-atomic timestamps: ` + - `[r]"a":[, 0.000000001,0)-> [r]"b":[0.000000002,0, )->v2 [r]"c":[, )->`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(batch( + withReadResult(get(k1), ``), + withReadResult(get(k2), v2), + withReadResult(get(k3), ``), + ), t3)), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "batch of reads after writes and deletes with valid time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(del(`a`), 3)), - step(withResultTS(del(`b`), 4)), - step(withResult(batch( - withReadResult(get(`a`), ``), - withReadResult(get(`b`), `v2`), - withReadResult(get(`c`), ``), - ))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(del(k1, s3), t3)), + step(withResultTS(del(k2, s4), t4)), + step(withResultTS(batch( + withReadResult(get(k1), ``), + withReadResult(get(k2), v2), + withReadResult(get(k3), ``), + ), t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 4)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), tombstone(k1, t3, s3), tombstone(k2, t4, s4)), }, { name: "transactional reads with non-empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 3)), - step(withResultTS(put(`b`, `v3`), 2)), - step(withResultTS(put(`b`, `v4`), 3)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), `v1`), - withReadResult(get(`b`), `v3`), - ), 3))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t3)), + step(withResultTS(put(k2, s3), t2)), + step(withResultTS(put(k2, s4), t3)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withReadResult(get(k1), v1), + withReadResult(get(k2), v3), + ), t3)), }, // Reading v1 is valid from 1-3 and v3 is valid from 2-3: overlap 2-3 - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 3, `v2`), kv(`b`, 2, `v3`), kv(`b`, 3, `v4`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k1, t3, s2), kv(k2, t2, s3), kv(k2, t3, s4)), }, { name: "transactional reads after writes and deletes with non-empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(del(`a`), 3)), - step(withResultTS(del(`b`), 4)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), ``), - withReadResult(get(`b`), `v2`), - withReadResult(get(`c`), ``), - ), 4))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(del(k1, s3), t3)), + step(withResultTS(del(k2, s4), t4)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withReadResult(get(k1), ``), + withReadResult(get(k2), v2), + withReadResult(get(k3), ``), + ), t4)), }, // Reading (a, ) is valid from min-1 or 3-max, and (b, v2) is valid from 2-4: overlap 3-4 - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 4)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), tombstone(k1, t3, s3), tombstone(k2, t4, s4)), }, { name: "transactional reads with empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 2)), - step(withResultTS(put(`b`, `v3`), 2)), - step(withResultTS(put(`b`, `v4`), 3)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), `v1`), - withReadResult(get(`b`), `v3`), - ), 3))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t2)), + step(withResultTS(put(k2, s3), t2)), + step(withResultTS(put(k2, s4), t3)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withReadResult(get(k1), v1), + withReadResult(get(k2), v3), + ), t3)), }, // Reading v1 is valid from 1-2 and v3 is valid from 2-3: no overlap - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`), kv(`b`, 2, `v3`), kv(`b`, 3, `v4`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[r]"a":[0.000000001,0, 0.000000002,0)->v1 [r]"b":[0.000000002,0, 0.000000003,0)->v3`, - }, + kvs: kvs(kv(k1, t1, s1), kv(k1, t2, s2), kv(k2, t2, s3), kv(k2, t3, s4)), }, { name: "transactional reads after writes and deletes with empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResultOK(del(`a`)), - withResultOK(del(`b`)), - ), 3)), + withResultOK(del(k1, s3)), + withResultOK(del(k2, s4)), + ), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), ``), - withReadResult(get(`b`), `v2`), - withReadResult(get(`c`), ``), - ), 4)), + withReadResult(get(k1), ``), + withReadResult(get(k2), v2), + withReadResult(get(k3), ``), + ), t4)), }, // Reading (a, ) is valid from min-1 or 3-max, and (b, v2) is valid from 2-3: no overlap - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 3)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[r]"a":[, 0.000000001,0),[0.000000003,0, )-> [r]"b":[0.000000002,0, 0.000000003,0)->v2 [r]"c":[, )->`, - }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), tombstone(k1, t3, s3), tombstone(k2, t3, s4)), }, { name: "transactional reads and deletes after write with non-empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), `v1`), - withResult(del(`a`)), - withReadResult(get(`a`), ``), - ), 2)), - step(withResultTS(put(`a`, `v2`), 3)), - step(withResultTS(del(`a`), 4)), + withReadResult(get(k1), v1), + withResult(del(k1, s2)), + withReadResult(get(k1), ``), + ), t2)), + step(withResultTS(put(k1, s3), t3)), + step(withResultTS(del(k1, s4), t4)), }, // Reading (a, v1) is valid from 1-2, reading (a, ) is valid from min-1, 2-3, or 4-max: overlap in txn view at 2 - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), kv(`a`, 3, `v2`), tombstone(`a`, 4)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2), kv(k1, t3, s3), tombstone(k1, t4, s4)), }, { name: "transactional reads and deletes after write with empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`))), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), ``), - withResult(del(`a`)), - withReadResult(get(`a`), ``), - ), 2)), - step(withResultTS(put(`a`, `v2`), 3)), - step(withResultTS(del(`a`), 4)), + withReadResult(get(k1), ``), + withResult(del(k1, s2)), + withReadResult(get(k1), ``), + ), t2)), + step(withResultTS(put(k1, s3), t3)), + step(withResultTS(del(k1, s4), t4)), }, // First read of (a, ) is valid from min-1 or 4-max, delete is valid at 2: no overlap - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), kv(`a`, 3, `v2`), tombstone(`a`, 4)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[r]"a":[, 0.000000001,0),[0.000000004,0, )-> [d]"a":0.000000002,0-> [r]"a":[, 0.000000001,0),[0.000000004,0, ),[0.000000002,0, 0.000000003,0)->`, - }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2), kv(k1, t3, s3), tombstone(k1, t4, s4)), }, { name: "transactional reads one missing with non-empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 2)), - step(withResultTS(put(`b`, `v3`), 2)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t2)), + step(withResultTS(put(k2, s3), t2)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), `v1`), - withReadResult(get(`b`), ``), - ), 1)), + withReadResult(get(k1), v1), + withReadResult(get(k2), ``), + ), t1)), }, // Reading v1 is valid from 1-2 and v3 is valid from 0-2: overlap 1-2 - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`), kv(`b`, 2, `v3`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k1, t2, s2), kv(k2, t2, s3)), }, { name: "transactional reads one missing with empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 2)), - step(withResultTS(put(`b`, `v3`), 1)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t2)), + step(withResultTS(put(k2, s3), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), `v1`), - withReadResult(get(`b`), ``), - ), 1)), + withReadResult(get(k1), v1), + withReadResult(get(k2), ``), + ), t1)), }, // Reading v1 is valid from 1-2 and v3 is valid from 0-1: no overlap - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`), kv(`b`, 1, `v3`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[r]"a":[0.000000001,0, 0.000000002,0)->v1 [r]"b":[, 0.000000001,0)->`, - }, + kvs: kvs(kv(k1, t1, s1), kv(k1, t2, s2), kv(k2, t1, s3)), }, { name: "transactional read and write with non-empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 3)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), `v1`), - withResult(put(`b`, `v3`)), - ), 2)), + withReadResult(get(k1), v1), + withResult(put(k2, s3)), + ), t2)), }, // Reading v1 is valid from 1-3 and v3 is valid at 2: overlap @2 - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 3, `v2`), kv(`b`, 2, `v3`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k1, t3, s2), kv(k2, t2, s3)), }, { name: "transactional read and write with empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 2)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t2)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), `v1`), - withResultOK(put(`b`, `v3`)), - ), 2)), + withReadResult(get(k1), v1), + withResultOK(put(k2, s3)), + ), t2)), }, // Reading v1 is valid from 1-2 and v3 is valid at 2: no overlap - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`), kv(`b`, 2, `v3`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[r]"a":[0.000000001,0, 0.000000002,0)->v1 [w]"b":0.000000002,0->v3`, - }, + kvs: kvs(kv(k1, t1, s1), kv(k1, t2, s2), kv(k2, t2, s3)), }, { name: "transaction with read before and after write", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), ``), - withResult(put(`a`, `v1`)), - withReadResult(get(`a`), `v1`), - ), 1)), + withReadResult(get(k1), ``), + withResult(put(k1, s1)), + withReadResult(get(k1), v1), + ), t1)), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1)), }, { name: "transaction with read before and after delete", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), `v1`), - withResult(del(`a`)), - withReadResult(get(`a`), ``), - ), 2)), + withReadResult(get(k1), v1), + withResult(del(k1, s2)), + withReadResult(get(k1), ``), + ), t2)), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2)), }, { name: "transaction with incorrect read before write", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), `v1`), - withResult(put(`a`, `v1`)), - withReadResult(get(`a`), `v1`), - ), 1)), - }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[r]"a":[0,0, 0,0)->v1 [w]"a":0.000000001,0->v1 [r]"a":[0.000000001,0, )->v1`, + withReadResult(get(k1), v1), + withResult(put(k1, s1)), + withReadResult(get(k1), v1), + ), t1)), }, + kvs: kvs(kv(k1, t1, s1)), }, { name: "transaction with incorrect read before delete", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), ``), - withResult(del(`a`)), - withReadResult(get(`a`), ``), - ), 2)), - }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[r]"a":[, 0.000000001,0)-> [d]"a":0.000000002,0-> [r]"a":[, 0.000000001,0),[0.000000002,0, )->`, + withReadResult(get(k1), ``), + withResult(del(k1, s2)), + withReadResult(get(k1), ``), + ), t2)), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2)), }, { name: "transaction with incorrect read after write", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), ``), - withResult(put(`a`, `v1`)), - withReadResult(get(`a`), ``), - ), 1)), - }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[r]"a":[, )-> [w]"a":0.000000001,0->v1 [r]"a":[, 0.000000001,0)->`, + withReadResult(get(k1), ``), + withResult(put(k1, s1)), + withReadResult(get(k1), ``), + ), t1)), }, + kvs: kvs(kv(k1, t1, s1)), }, { name: "transaction with incorrect read after delete", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), `v1`), - withResultOK(del(`a`)), - withReadResult(get(`a`), `v1`), - ), 2)), - }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[r]"a":[0.000000001,0, )->v1 [d]"a":0.000000002,0-> [r]"a":[0.000000001,0, 0.000000002,0)->v1`, + withReadResult(get(k1), v1), + withResultOK(del(k1, s2)), + withReadResult(get(k1), v1), + ), t2)), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2)), }, { name: "two transactionally committed puts of the same key with reads", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), ``), - withResult(put(`a`, `v1`)), - withReadResult(get(`a`), `v1`), - withResult(put(`a`, `v2`)), - withReadResult(get(`a`), `v2`), - ), 1)), + withReadResult(get(k1), ``), + withResult(put(k1, s1)), + withReadResult(get(k1), v1), + withResult(put(k1, s2)), + withReadResult(get(k1), v2), + ), t1)), }, - kvs: kvs(kv(`a`, 1, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s2)), }, { name: "two transactionally committed put/delete ops of the same key with reads", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), ``), - withResult(put(`a`, `v1`)), - withReadResult(get(`a`), `v1`), - withResult(del(`a`)), - withReadResult(get(`a`), ``), - ), 1)), + withReadResult(get(k1), ``), + withResult(put(k1, s1)), + withReadResult(get(k1), v1), + withResult(del(k1, s2)), + withReadResult(get(k1), ``), + ), t1)), }, - kvs: kvs(tombstone(`a`, 1)), - expected: nil, + kvs: kvs(tombstone(k1, t1, s2)), }, { name: "two transactionally committed put/delete ops of the same key with incorrect read", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withReadResult(get(`a`), ``), - withResult(put(`a`, `v1`)), - withReadResult(get(`a`), `v1`), - withResult(del(`a`)), - withReadResult(get(`a`), `v1`), - ), 1)), - }, - kvs: kvs(tombstone(`a`, 1)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[r]"a":[, )-> [w]"a":missing->v1 [r]"a":[0.000000001,0, )->v1 [d]"a":0.000000001,0-> [r]"a":[0,0, 0,0)->v1`, + withReadResult(get(k1), ``), + withResult(put(k1, s1)), + withReadResult(get(k1), v1), + withResult(del(k1, s2)), + withReadResult(get(k1), v1), + ), t1)), }, + kvs: kvs(tombstone(k1, t1, s2)), }, { name: "one transactional put with correct commit time", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - ), 1)), + withResult(put(k1, s1)), + ), t1)), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1)), }, { name: "one transactional put with incorrect commit time", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - ), 1)), - }, - kvs: kvs(kv(`a`, 2, `v1`)), - expected: []string{ - `mismatched write timestamp 0.000000001,0: [w]"a":0.000000002,0->v1`, + withResult(put(k1, s1)), + ), t1)), }, + kvs: kvs(kv(k1, t2, s1)), }, { name: "one transactional delete with write on another key after delete", steps: []Step{ // NB: this Delete comes first in operation order, but the write is delayed. - step(withResultTS(del(`a`), 3)), + step(withResultTS(del(k1, s1), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResult(put(`b`, `v1`)), - withResult(del(`a`)), - ), 2)), + withResult(put(k2, s2)), + withResult(del(k1, s3)), + ), t2)), }, - kvs: kvs(tombstone(`a`, 2), tombstone(`a`, 3), kv(`b`, 2, `v1`)), - // This should fail validation if we match delete operations to tombstones by operation order, - // and should pass if we correctly use the transaction timestamp. While the first delete is - // an earlier operation, the transactional delete actually commits first. - expected: nil, + kvs: kvs(tombstone(k1, t2, s3), tombstone(k1, t3, s1), kv(k2, t2, s2)), }, { name: "two transactional deletes with out of order commit times", steps: []Step{ - step(withResultTS(del(`a`), 2)), - step(withResultTS(del(`b`), 3)), + step(withResultTS(del(k1, s1), t2)), + step(withResultTS(del(k2, s2), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`)), - withResult(del(`b`)), - ), 1)), + withResult(del(k1, s3)), + withResult(del(k2, s4)), + ), t1)), }, - kvs: kvs(tombstone(`a`, 1), tombstone(`a`, 2), tombstone(`b`, 1), tombstone(`b`, 3)), - // This should fail validation if we match delete operations to tombstones by operation order, - // and should pass if we correctly use the transaction timestamp. While the first two deletes are - // earlier operations, the transactional deletes actually commits first. - expected: nil, + kvs: kvs(tombstone(k1, t1, s3), tombstone(k1, t2, s1), tombstone(k2, t1, s4), tombstone(k2, t3, s2)), }, { name: "one transactional scan followed by delete within time range", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(del(`a`)), - ), 2)), - step(withResultTS(put(`b`, `v2`), 3)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + withResult(del(k1, s2)), + ), t2)), + step(withResultTS(put(k2, s3), t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), kv(`b`, 3, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2), kv(k2, t3, s3)), }, { name: "one transactional scan followed by delete outside time range", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(del(`a`)), - ), 4)), - step(withResultTS(put(`b`, `v2`), 3)), - }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 4), kv(`b`, 3, `v2`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[s]{a-c}:{0:[0.000000001,0, ), gap:[, 0.000000003,0)}->["a":v1] [d]"a":0.000000004,0->`, + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + withResult(del(k1, s2)), + ), t4)), + step(withResultTS(put(k2, s3), t3)), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t4, s2), kv(k2, t3, s3)), }, { name: "one scan before write", steps: []Step{ - step(withScanResult(scan(`a`, `c`))), - step(withResult(put(`a`, `v1`))), + step(withScanResultTS(scan(k1, k3), t1)), + step(withResultTS(put(k1, s1), t2)), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + kvs: kvs(kv(k1, t2, s1)), }, { name: "one scan before write returning wrong value", steps: []Step{ - step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v2`))), - step(withResult(put(`a`, `v1`))), - }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{ - `committed scan non-atomic timestamps: ` + - `[s]{a-c}:{0:[0,0, 0,0), gap:[, )}->["a":v2]`, + step(withScanResultTS(scan(k1, k3), t1, scanKV(k1, v2))), + step(withResultTS(put(k1, s1), t2)), }, + kvs: kvs(kv(k1, t2, s1)), }, { name: "one scan after write", steps: []Step{ - step(withResult(put(`a`, `v1`))), - step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`))), + step(withResultTS(put(k1, s1), t1)), + step(withScanResultTS(scan(k1, k3), t2, scanKV(k1, v1))), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1)), }, { name: "one scan after write returning wrong value", steps: []Step{ - step(withResult(put(`a`, `v1`))), - step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v2`))), - }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{ - `committed scan non-atomic timestamps: ` + - `[s]{a-c}:{0:[0,0, 0,0), gap:[, )}->["a":v2]`, + step(withResultTS(put(k1, s1), t1)), + step(withScanResultTS(scan(k1, k3), t2, scanKV(k1, v2))), }, + kvs: kvs(kv(k1, t1, s1)), }, { name: "one scan after writes", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`b`, `v2`))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withScanResultTS(scan(k1, k3), t3, scanKV(k1, v1), scanKV(k2, v2))), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "one reverse scan after writes", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withScanResult(reverseScan(`a`, `c`), scanKV(`b`, `v2`), scanKV(`a`, `v1`))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withScanResultTS(reverseScan(k1, k3), t3, scanKV(k2, v2), scanKV(k1, v1))), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "one scan after writes and delete", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(del(`a`), 3)), - step(withResultTS(put(`a`, `v3`), 4)), - step(withScanResult(scan(`a`, `c`), scanKV(`b`, `v2`))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(del(k1, s3), t3)), + step(withResultTS(put(k1, s4), t4)), + step(withScanResultTS(scan(k1, k3), t5, scanKV(k2, v2))), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), kv(`a`, 4, `v3`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), tombstone(k1, t3, s3), kv(k1, t4, s4)), }, { name: "one scan after write returning extra key", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`a2`, `v3`), scanKV(`b`, `v2`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `committed scan non-atomic timestamps: ` + - `[s]{a-c}:{0:[0.000000001,0, ), 1:[0,0, 0,0), 2:[0.000000002,0, ), gap:[, )}->["a":v1, "a2":v3, "b":v2]`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k3, s2), t2)), + step(withScanResultTS(scan(k1, k4), t3, scanKV(k1, v1), scanKV(k2, v3), scanKV(k2, v2))), }, + kvs: kvs(kv(k1, t1, s1), kv(k3, t2, s2)), }, { name: "one tranactional scan after write and delete returning extra key", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResult(put(`b`, `v2`)), - withResult(del(`a`)), - ), 2)), - step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`b`, `v2`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), kv(`b`, 2, `v2`)), - expected: []string{ - `committed scan non-atomic timestamps: ` + - `[s]{a-c}:{0:[0.000000001,0, 0.000000002,0), 1:[0.000000002,0, ), gap:[, )}->["a":v1, "b":v2]`, + withResult(put(k2, s2)), + withResult(del(k1, s3)), + ), t2)), + step(withScanResultTS(scan(k1, k3), t3, scanKV(k1, v1), scanKV(k2, v2))), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s3), kv(k2, t2, s2)), }, { name: "one reverse scan after write returning extra key", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withScanResult(reverseScan(`a`, `c`), scanKV(`b`, `v2`), scanKV(`a2`, `v3`), scanKV(`a`, `v1`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `committed reverse scan non-atomic timestamps: ` + - `[rs]{a-c}:{0:[0.000000002,0, ), 1:[0,0, 0,0), 2:[0.000000001,0, ), gap:[, )}->["b":v2, "a2":v3, "a":v1]`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k3, s2), t2)), + step(withScanResultTS(reverseScan(k1, k4), t3, + scanKV(k3, v2), + scanKV(k2, v3), + scanKV(k1, v1), + )), }, + kvs: kvs(kv(k1, t1, s1), kv(k3, t2, s2)), }, { name: "one scan after write returning missing key", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withScanResult(scan(`a`, `c`), scanKV(`b`, `v2`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `committed scan non-atomic timestamps: ` + - `[s]{a-c}:{0:[0.000000002,0, ), gap:[, 0.000000001,0)}->["b":v2]`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withScanResultTS(scan(k1, k3), t3, scanKV(k2, v2))), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "one scan after writes and delete returning missing key", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`)), - withResult(put(`b`, `v2`)), - ), 1)), + withResult(put(k1, s1)), + withResult(put(k2, s2)), + ), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`b`, `v2`)), - withResult(del(`a`)), - ), 2)), - step(withResultTS(put(`a`, `v3`), 3)), - step(withResultTS(del(`a`), 4)), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 1, `v2`), tombstone(`a`, 2), kv(`a`, 3, `v3`), tombstone(`a`, 4)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[s]{a-c}:{0:[0.000000001,0, ), gap:[, 0.000000001,0),[0.000000004,0, )}->["b":v2] [d]"a":0.000000002,0->`, + withScanResultTS(scan(k1, k3), noTS, scanKV(k2, v2)), + withResult(del(k1, s3)), + ), t2)), + step(withResultTS(put(k1, s4), t3)), + step(withResultTS(del(k1, s5), t4)), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t1, s2), tombstone(k1, t2, s3), kv(k1, t3, s4), tombstone(k1, t4, s5)), }, { name: "one reverse scan after write returning missing key", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withScanResult(reverseScan(`a`, `c`), scanKV(`b`, `v2`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `committed reverse scan non-atomic timestamps: ` + - `[rs]{a-c}:{0:[0.000000002,0, ), gap:[, 0.000000001,0)}->["b":v2]`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withScanResultTS(reverseScan(k1, k3), t3, scanKV(k2, v2))), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "one scan after writes returning results in wrong order", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withScanResult(scan(`a`, `c`), scanKV(`b`, `v2`), scanKV(`a`, `v1`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `scan result not ordered correctly: ` + - `[s]{a-c}:{0:[0.000000002,0, ), 1:[0.000000001,0, ), gap:[, )}->["b":v2, "a":v1]`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withScanResultTS(scan(k1, k3), t3, scanKV(k2, v2), scanKV(k1, v1))), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "one reverse scan after writes returning results in wrong order", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withScanResult(reverseScan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`b`, `v2`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `scan result not ordered correctly: ` + - `[rs]{a-c}:{0:[0.000000001,0, ), 1:[0.000000002,0, ), gap:[, )}->["a":v1, "b":v2]`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withScanResultTS(reverseScan(k1, k3), t3, scanKV(k1, v1), scanKV(k2, v2))), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "one scan after writes returning results outside scan boundary", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(put(`c`, `v3`), 3)), - step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`b`, `v2`), scanKV(`c`, `v3`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), kv(`c`, 3, `v3`)), - expected: []string{ - `key "c" outside scan bounds: ` + - `[s]{a-c}:{0:[0.000000001,0, ), 1:[0.000000002,0, ), 2:[0.000000003,0, ), gap:[, )}->["a":v1, "b":v2, "c":v3]`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(put(k3, s3), t3)), + step(withScanResultTS(scan(k1, k3), t4, scanKV(k1, v1), scanKV(k2, v2), scanKV(k3, v3))), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), kv(k3, t3, s3)), }, { name: "one reverse scan after writes returning results outside scan boundary", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(put(`c`, `v3`), 3)), - step(withScanResult(reverseScan(`a`, `c`), scanKV(`c`, `v3`), scanKV(`b`, `v2`), scanKV(`a`, `v1`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), kv(`c`, 3, `v3`)), - expected: []string{ - `key "c" outside scan bounds: ` + - `[rs]{a-c}:{0:[0.000000003,0, ), 1:[0.000000002,0, ), 2:[0.000000001,0, ), gap:[, )}->["c":v3, "b":v2, "a":v1]`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(put(k3, s3), t3)), + step(withScanResultTS(reverseScan(k1, k3), t4, scanKV(k3, v3), scanKV(k2, v2), scanKV(k1, v1))), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), kv(k3, t3, s3)), }, { name: "one scan in between writes", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`))), - step(withResultTS(put(`a`, `v2`), 2)), + step(withResultTS(put(k1, s1), t1)), + step(withScanResultTS(scan(k1, k3), t2, scanKV(k1, v1))), + step(withResultTS(put(k1, s2), t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k1, t3, s2)), }, { name: "batch of scans after writes", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResult(batch( - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`b`, `v2`)), - withScanResult(scan(`b`, `d`), scanKV(`b`, `v2`)), - withScanResult(scan(`c`, `e`)), - ))), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(batch( + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1), scanKV(k2, v2)), + withScanResultTS(scan(k2, k4), noTS, scanKV(k2, v2)), + withScanResultTS(scan(k3, k5), noTS), + ), t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "batch of scans after writes returning wrong values", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResult(batch( - withScanResult(scan(`a`, `c`)), - withScanResult(scan(`b`, `d`), scanKV(`b`, `v1`)), - withScanResult(scan(`c`, `e`), scanKV(`c`, `v2`)), - ))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `committed batch non-atomic timestamps: ` + - `[s]{a-c}:{gap:[, 0.000000001,0)}->[] ` + - `[s]{b-d}:{0:[0,0, 0,0), gap:[, )}->["b":v1] ` + - `[s]{c-e}:{0:[0,0, 0,0), gap:[, )}->["c":v2]`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(batch( + withScanResultTS(scan(k1, k3), noTS), + withScanResultTS(scan(k2, k4), noTS, scanKV(k2, v1)), + withScanResultTS(scan(k3, k5), noTS, scanKV(k3, v2)), + ), t3)), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "batch of scans after writes with non-empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResult(batch( - withScanResult(scan(`a`, `c`), scanKV(`b`, `v1`)), - withScanResult(scan(`b`, `d`), scanKV(`b`, `v1`)), - withScanResult(scan(`c`, `e`)), - ))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), - expected: []string{ - `committed batch non-atomic timestamps: ` + - `[s]{a-c}:{0:[0,0, 0,0), gap:[, 0.000000001,0)}->["b":v1] ` + - `[s]{b-d}:{0:[0,0, 0,0), gap:[, )}->["b":v1] ` + - `[s]{c-e}:{gap:[, )}->[]`, + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(batch( + withScanResultTS(scan(k1, k3), noTS, scanKV(k2, v1)), + withScanResultTS(scan(k2, k4), noTS, scanKV(k2, v1)), + withScanResultTS(scan(k3, k5), noTS), + ), t3)), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2)), }, { name: "transactional scans with non-empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 3)), - step(withResultTS(put(`b`, `v3`), 2)), - step(withResultTS(put(`b`, `v4`), 3)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t3)), + step(withResultTS(put(k2, s3), t2)), + step(withResultTS(put(k2, s4), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`b`, `v3`)), - withScanResult(scan(`b`, `d`), scanKV(`b`, `v3`)), - ), 2)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1), scanKV(k2, v3)), + withScanResultTS(scan(k2, k4), noTS, scanKV(k2, v3)), + ), t2)), }, // Reading v1 is valid from 1-3 and v3 is valid from 2-3: overlap 2-3 - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 3, `v2`), kv(`b`, 2, `v3`), kv(`b`, 3, `v4`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k1, t3, s2), kv(k2, t2, s3), kv(k2, t3, s4)), }, { name: "transactional scans after delete with non-empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 3)), - step(withResultTS(put(`b`, `v3`), 1)), - step(withResultTS(del(`b`), 2)), - step(withResultTS(put(`b`, `v4`), 4)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t3)), + step(withResultTS(put(k2, s3), t1)), + step(withResultTS(del(k2, s4), t2)), + step(withResultTS(put(k2, s5), t4)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withScanResult(scan(`b`, `d`)), - ), 2)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + withScanResultTS(scan(k2, k4), noTS), + ), t2)), }, - // Reading v1 is valid from 1-3 and for `b` is valid -1 and 2-4: overlap 2-3 - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 3, `v2`), kv(`b`, 1, `v3`), tombstone(`b`, 2), kv(`b`, 4, `v4`)), - expected: nil, + // Reading v1 is valid from 1-3 and for k2 is valid -1 and 2-4: overlap 2-3 + kvs: kvs(kv(k1, t1, s1), kv(k1, t3, s2), kv(k2, t1, s3), tombstone(k2, t2, s4), kv(k2, t4, s5)), }, { name: "transactional scans with empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 2)), - step(withResultTS(put(`b`, `v3`), 2)), - step(withResultTS(put(`b`, `v4`), 3)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t2)), + step(withResultTS(put(k2, s3), t2)), + step(withResultTS(put(k2, s4), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`b`, `v3`)), - withScanResult(scan(`b`, `d`), scanKV(`b`, `v3`)), - ), 2)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1), scanKV(k2, v3)), + withScanResultTS(scan(k2, k4), noTS, scanKV(k2, v3)), + ), t2)), }, // Reading v1 is valid from 1-2 and v3 is valid from 2-3: no overlap - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`), kv(`b`, 2, `v3`), kv(`b`, 3, `v4`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[s]{a-c}:{0:[0.000000001,0, 0.000000002,0), 1:[0.000000002,0, 0.000000003,0), gap:[, )}->["a":v1, "b":v3] ` + - `[s]{b-d}:{0:[0.000000002,0, 0.000000003,0), gap:[, )}->["b":v3]`, - }, + kvs: kvs(kv(k1, t1, s1), kv(k1, t2, s2), kv(k2, t2, s3), kv(k2, t3, s4)), }, { name: "transactional scans after delete with empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 2)), - step(withResultTS(put(`b`, `v3`), 1)), - step(withResultTS(del(`b`), 3)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t2)), + step(withResultTS(put(k2, s3), t1)), + step(withResultTS(del(k2, s4), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withScanResult(scan(`b`, `d`)), - ), 3)), - }, - // Reading v1 is valid from 1-2 and for `b` is valid from -1, 3-: no overlap - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`), kv(`b`, 1, `v3`), tombstone(`b`, 3)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[s]{a-c}:{0:[0.000000001,0, 0.000000002,0), gap:[, 0.000000001,0),[0.000000003,0, )}->["a":v1] ` + - `[s]{b-d}:{gap:[, 0.000000001,0),[0.000000003,0, )}->[]`, + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + withScanResultTS(scan(k2, k4), noTS), + ), t3)), }, + // Reading v1 is valid from 1-2 and for k2 is valid from -1, 3-: no overlap + kvs: kvs(kv(k1, t1, s1), kv(k1, t2, s2), kv(k2, t1, s3), tombstone(k2, t3, s4)), }, { name: "transactional scans one missing with non-empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 2)), - step(withResultTS(put(`b`, `v3`), 2)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t2)), + step(withResultTS(put(k2, s3), t2)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withScanResult(scan(`b`, `d`)), - ), 2)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + withScanResultTS(scan(k2, k4), noTS), + ), t2)), }, // Reading v1 is valid from 1-2 and v3 is valid from 0-2: overlap 1-2 - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`), kv(`b`, 2, `v3`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k1, t2, s2), kv(k2, t2, s3)), }, { name: "transactional scans one missing with empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 2)), - step(withResultTS(put(`b`, `v3`), 1)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t2)), + step(withResultTS(put(k2, s3), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withScanResult(scan(`b`, `d`)), - ), 1)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + withScanResultTS(scan(k2, k4), noTS), + ), t1)), }, // Reading v1 is valid from 1-2 and v3 is valid from 0-1: no overlap - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`), kv(`b`, 1, `v3`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[s]{a-c}:{0:[0.000000001,0, 0.000000002,0), gap:[, 0.000000001,0)}->["a":v1] ` + - `[s]{b-d}:{gap:[, 0.000000001,0)}->[]`, - }, + kvs: kvs(kv(k1, t1, s1), kv(k1, t2, s2), kv(k2, t1, s3)), }, { name: "transactional scan and write with non-empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 3)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(put(`b`, `v3`)), - ), 2)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + withResult(put(k2, s3)), + ), t2)), }, // Reading v1 is valid from 1-3 and v3 is valid at 2: overlap @2 - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 3, `v2`), kv(`b`, 2, `v3`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k1, t3, s2), kv(k2, t2, s3)), }, { name: "transactional scan and write with empty time overlap", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`a`, `v2`), 2)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k1, s2), t2)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(put(`b`, `v3`)), - ), 2)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + withResult(put(k2, s3)), + ), t2)), }, // Reading v1 is valid from 1-2 and v3 is valid at 2: no overlap - kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`), kv(`b`, 2, `v3`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[s]{a-c}:{0:[0.000000001,0, 0.000000002,0), gap:[, )}->["a":v1] [w]"b":0.000000002,0->v3`, - }, + kvs: kvs(kv(k1, t1, s1), kv(k1, t2, s2), kv(k2, t2, s3)), }, { name: "transaction with scan before and after write", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`)), - withResult(put(`a`, `v1`)), - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - ), 1)), + withScanResultTS(scan(k1, k3), noTS), + withResult(put(k1, s1)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + ), t1)), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1)), }, { name: "transaction with incorrect scan before write", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(put(`a`, `v1`)), - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - ), 1)), - }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[s]{a-c}:{0:[0,0, 0,0), gap:[, )}->["a":v1] ` + - `[w]"a":0.000000001,0->v1 ` + - `[s]{a-c}:{0:[0.000000001,0, ), gap:[, )}->["a":v1]`, + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + withResult(put(k1, s1)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + ), t1)), }, + kvs: kvs(kv(k1, t1, s1)), }, { name: "transaction with incorrect scan after write", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`)), - withResult(put(`a`, `v1`)), - withScanResult(scan(`a`, `c`)), - ), 1)), - }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[s]{a-c}:{gap:[, )}->[] [w]"a":0.000000001,0->v1 [s]{a-c}:{gap:[, 0.000000001,0)}->[]`, + withScanResultTS(scan(k1, k3), noTS), + withResult(put(k1, s1)), + withScanResultTS(scan(k1, k3), noTS), + ), t1)), }, + kvs: kvs(kv(k1, t1, s1)), }, { name: "two transactionally committed puts of the same key with scans", steps: []Step{ step(withResultTS(closureTxn(ClosureTxnType_Commit, - withScanResult(scan(`a`, `c`)), - withResult(put(`a`, `v1`)), - withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(put(`a`, `v2`)), - withScanResult(scan(`a`, `c`), scanKV(`a`, `v2`)), - withResult(put(`b`, `v3`)), - withScanResult(scan(`a`, `c`), scanKV(`a`, `v2`), scanKV(`b`, `v3`)), - ), 1)), + withScanResultTS(scan(k1, k3), noTS), + withResult(put(k1, s1)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v1)), + withResult(put(k1, s2)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v2)), + withResult(put(k2, s3)), + withScanResultTS(scan(k1, k3), noTS, scanKV(k1, v2), scanKV(k2, v3)), + ), t1)), }, - kvs: kvs(kv(`a`, 1, `v2`), kv(`b`, 1, `v3`)), - expected: nil, + kvs: kvs(kv(k1, t1, s2), kv(k2, t1, s3)), }, { name: "one deleterange before write", steps: []Step{ - step(withDeleteRangeResult(delRange(`a`, `c`))), - step(withResult(put(`a`, `v1`))), + step(withDeleteRangeResult(delRange(k1, k3, s1), t1)), + step(withResultTS(put(k1, s2), t2)), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: nil, + kvs: kvs(kv(k1, t2, s2)), }, { name: "one deleterange before write returning wrong value", steps: []Step{ - step(withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`))), - step(withResult(put(`a`, `v1`))), - }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{ - `committed deleteRange missing write: ` + - `[dr.s]{a-c}:{0:[0.000000001,0, ), gap:[, )}->["a"] ` + - `[dr.d]"a":missing->`, + step(withDeleteRangeResult(delRange(k1, k3, s1), t1, roachpb.Key(k1))), + step(withResultTS(put(k1, s2), t2)), }, + kvs: kvs(kv(k1, t2, s2)), }, { name: "one deleterange after write", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`)), - ), 2)), + withDeleteRangeResult(delRange(k1, k3, s2), noTS, roachpb.Key(k1)), + ), t2)), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2)), }, { name: "one deleterange after write returning wrong value", steps: []Step{ - step(withResult(put(`a`, `v1`))), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`)), - ), 2)), - }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), - expected: []string{ - `extra writes: [d]"a":uncertain->`, + withDeleteRangeResult(delRange(k1, k3, s2), t2), + ), t2)), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2)), }, { name: "one deleterange after write missing write", steps: []Step{ - step(withResult(put(`a`, `v1`))), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`)), - ), 1)), + withDeleteRangeResult(delRange(k1, k3, s2), t2, roachpb.Key(k1)), + ), t1)), }, - kvs: kvs(kv(`a`, 1, `v1`)), - expected: []string{ - `committed txn missing write: ` + - `[dr.s]{a-c}:{0:[0.000000001,0, ), gap:[, )}->["a"] ` + - `[dr.d]"a":missing->`, + kvs: kvs(kv(k1, t1, s1)), + }, + { + name: "one deleterange after write extra deletion", + steps: []Step{ + step(withResultTS(put(k1, s1), t2)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withDeleteRangeResult(delRange(k1, k3, s2), t2, roachpb.Key(k1), roachpb.Key(k2)), + ), t2)), + }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2)), + }, + { + name: "one deleterange after write with spurious deletion", + steps: []Step{ + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withDeleteRangeResult(delRange(k1, k3, s2), t2, roachpb.Key(k1), roachpb.Key(k2)), + ), t2)), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2), tombstone(k2, t2, s2)), }, { name: "one deleterange after writes", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(put(`c`, `v3`), 3)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(put(k3, s3), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 4)), - step(withScanResult(scan(`a`, `d`), scanKV(`c`, `v3`))), + withDeleteRangeResult(delRange(k1, k3, s4), noTS, roachpb.Key(k1), roachpb.Key(k2)), + ), t4)), + step(withScanResultTS(scan(k1, k4), t4, scanKV(k3, v3))), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), kv(`c`, 3, `v3`), tombstone(`a`, 4), tombstone(`b`, 4)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), kv(k3, t3, s3), tombstone(k1, t4, s4), tombstone(k2, t4, s4)), }, { name: "one deleterange after writes with write timestamp disagreement", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(put(`c`, `v3`), 3)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(put(k3, s3), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 4)), - step(withScanResult(scan(`a`, `d`), scanKV(`c`, `v3`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), kv(`c`, 3, `v3`), tombstone(`a`, 4), tombstone(`b`, 5)), - expected: []string{ - `committed txn missing write: ` + - `[dr.s]{a-c}:{0:[0.000000001,0, ), 1:[0.000000002,0, 0.000000005,0), gap:[, )}->["a", "b"] ` + - `[dr.d]"a":0.000000004,0-> [dr.d]"b":missing->`, + withDeleteRangeResult(delRange(k1, k3, s4), noTS, roachpb.Key(k1), roachpb.Key(k2), roachpb.Key(k3)), + ), t4)), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), kv(k3, t3, s3), tombstone(k1, t3, s4), tombstone(k2, t4, s4), tombstone(k3, t4, s4)), }, { name: "one deleterange after writes with missing write", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(put(`c`, `v3`), 3)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(put(k3, s3), t3)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 4)), - step(withScanResult(scan(`a`, `d`), scanKV(`c`, `v3`))), - }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), kv(`c`, 3, `v3`), tombstone(`a`, 4)), - expected: []string{ - `committed txn missing write: ` + - `[dr.s]{a-c}:{0:[0.000000001,0, ), 1:[0.000000002,0, ), gap:[, )}->["a", "b"] ` + - `[dr.d]"a":0.000000004,0-> [dr.d]"b":missing->`, - `committed scan non-atomic timestamps: [s]{a-d}:{0:[0.000000003,0, ), gap:[, 0.000000001,0)}->["c":v3]`, + withDeleteRangeResult(delRange(k1, k3, s4), noTS, roachpb.Key(k1), roachpb.Key(k2)), + ), t4)), + step(withScanResultTS(scan(k1, k4), t5, scanKV(k3, v3))), }, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), kv(k3, t3, s3), tombstone(k1, t4, s4)), }, { name: "one deleterange after writes and delete", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`b`, `v2`), 2)), - step(withResultTS(del(`a`), 4)), - step(withResultTS(put(`a`, `v3`), 5)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k2, s2), t2)), + step(withResultTS(del(k1, s3), t4)), + step(withResultTS(put(k1, s4), t5)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 3)), + withDeleteRangeResult(delRange(k1, k3, s5), noTS, roachpb.Key(k1), roachpb.Key(k2)), + ), t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 3), tombstone(`a`, 4), kv(`a`, 5, `v3`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), kv(k2, t2, s2), tombstone(k1, t3, s5), tombstone(k2, t3, s5), tombstone(k1, t4, s3), kv(k2, t5, s4)), }, { name: "one transactional deleterange followed by put after writes", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`)), - withResult(put(`b`, `v2`)), - ), 2)), + withDeleteRangeResult(delRange(k1, k3, s2), noTS, roachpb.Key(k1)), + withResult(put(k2, s3)), + ), t2)), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), kv(`b`, 2, `v2`)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2), kv(k2, t2, s3)), }, { name: "one transactional deleterange followed by put after writes with write timestamp disagreement", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`)), - withResult(put(`b`, `v2`)), - ), 2)), - }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), kv(`b`, 3, `v2`)), - expected: []string{ - `committed txn non-atomic timestamps: ` + - `[dr.s]{a-c}:{0:[0.000000001,0, ), gap:[, )}->["a"] ` + - `[dr.d]"a":0.000000002,0-> [w]"b":0.000000003,0->v2`, + withDeleteRangeResult(delRange(k1, k3, s2), noTS, roachpb.Key(k1)), + withResult(put(k2, s3)), + ), t2)), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s2), kv(k2, t3, s3)), }, { name: "one transactional put shadowed by deleterange after writes", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResult(put(`b`, `v2`)), - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 2)), + withResult(put(k2, s2)), + withDeleteRangeResult(delRange(k1, k3, s3), noTS, roachpb.Key(k1), roachpb.Key(k2)), + ), t2)), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), tombstone(`b`, 2)), - expected: nil, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s3), tombstone(k2, t2, s3)), }, { name: "one transactional put shadowed by deleterange after writes with write timestamp disagreement", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(k1, s1), t1)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withResult(put(`b`, `v2`)), - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 2)), - }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), tombstone(`b`, 3)), - expected: []string{ - `committed txn missing write: ` + - `[w]"b":missing->v2 ` + - `[dr.s]{a-c}:{0:[0.000000001,0, ), 1:[0,0, ), gap:[, )}->["a", "b"] ` + - `[dr.d]"a":0.000000002,0-> [dr.d]"b":missing->`, + withResult(put(k2, s2)), + withDeleteRangeResult(delRange(k1, k3, s3), noTS, roachpb.Key(k1), roachpb.Key(k2)), + ), t2)), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t2, s3), tombstone(k2, t3, s3)), }, { name: "one deleterange after writes returning keys outside span boundary", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`d`, `v2`), 2)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k4, s2), t2)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`d`)), - ), 3)), - }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 3), kv(`d`, 2, `v2`)), - expected: []string{ - `key "d" outside delete range bounds: ` + - `[dr.s]{a-c}:{0:[0.000000001,0, ), 1:[0.000000002,0, ), gap:[, )}->["a", "d"] ` + - `[dr.d]"a":0.000000003,0-> [dr.d]"d":missing->`, + withDeleteRangeResult(delRange(k1, k3, s3), noTS, roachpb.Key(k1), roachpb.Key(k4)), + ), t3)), }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t3, s3), kv(k4, t2, s2)), }, { name: "one deleterange after writes incorrectly deleting keys outside span boundary", steps: []Step{ - step(withResultTS(put(`a`, `v1`), 1)), - step(withResultTS(put(`d`, `v2`), 2)), + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(put(k4, s2), t2)), step(withResultTS(closureTxn(ClosureTxnType_Commit, - withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`d`)), - ), 3)), + withDeleteRangeResult(delRange(k1, k3, s3), noTS, roachpb.Key(k1), roachpb.Key(k4)), + ), t3)), + }, + kvs: kvs(kv(k1, t1, s1), tombstone(k1, t3, s3), kv(k4, t2, s2), tombstone(k4, t3, s3)), + }, + { + name: "single mvcc rangedel", + steps: []Step{ + step(withResultTS(delRangeUsingTombstone(k1, k2, s1), t1)), + }, + kvs: kvs(rd(k1, k2, t1, s1)), + }, + { + name: "single mvcc rangedel after put", + steps: []Step{ + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(delRangeUsingTombstone(k1, k2, s2), t2)), + }, + kvs: kvs(kv(k1, t1, s1), rd(k1, k2, t2, s2)), + }, + { + name: "single mvcc rangedel before put", + steps: []Step{ + step(withResultTS(delRangeUsingTombstone(k1, k2, s1), t1)), + step(withResultTS(put(k1, s2), t2)), + }, + kvs: kvs(rd(k1, k2, t1, s1), kv(k1, t2, s2)), + }, + { + name: "two overlapping rangedels", + steps: []Step{ + step(withResultTS(delRangeUsingTombstone(k1, k3, s1), t1)), + step(withResultTS(delRangeUsingTombstone(k2, k4, s2), t2)), + }, + // Note: you see rangedel fragmentation in action here, which has to + // happen. Even if we decided to hand pebble overlapping rangedels, it + // would fragment them for us, and we'd get what you see below back when + // we read. + kvs: kvs( + rd(k1, k2, t1, s1), + rd(k2, k3, t1, s1), + rd(k2, k3, t2, s2), + rd(k3, k4, t2, s2), + ), + }, + { + name: "batch of touching rangedels", + steps: []Step{step(withResultTS(batch( + withResult(delRangeUsingTombstone(k1, k2, s1)), + withResult(delRangeUsingTombstone(k2, k4, s2)), + ), t1)), + }, + // Note that the tombstones aren't merged. In fact, our use of sequence numbers + // embedded in MVCCValueHeader implies that pebble can never merge adjacent + // tombstones from the same batch/txn. + kvs: kvs( + rd(k1, k2, t1, s1), + rd(k2, k4, t1, s2), + ), + }, + { + // Note also that self-overlapping batches or rangedels in txns aren't + // allowed today, so this particular example exists in this unit test but + // not in real CRDB. But we can have "touching" rangedels today, see + // above. + name: "batch of two overlapping rangedels", + steps: []Step{step(withResultTS(batch( + withResult(delRangeUsingTombstone(k1, k3, s1)), + withResult(delRangeUsingTombstone(k2, k4, s2)), + ), t1)), + }, + // Note that the tombstones aren't merged. In fact, our use of sequence numbers + // embedded in MVCCValueHeader implies that pebble can never merge adjacent + // tombstones from the same batch/txn. + // Note also that self-overlapping batches or rangedels in txns aren't + // allowed today, so this particular example exists in this unit test but + // not in real CRDB. But we can have "touching" rangedels today. + kvs: kvs( + rd(k1, k2, t1, s1), + rd(k2, k4, t1, s2), + ), + }, + { + name: "read before rangedel", + steps: []Step{ + step(withResultTS(put(k2, s1), t1)), + step(withReadResultTS(get(k2), v1, t2)), + step(withResultTS(delRangeUsingTombstone(k1, k3, s3), t3)), }, - kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 3), kv(`d`, 2, `v2`), tombstone(`d`, 3)), - expected: []string{ - `key "d" outside delete range bounds: ` + - `[dr.s]{a-c}:{0:[0.000000001,0, ), 1:[0.000000002,0, ), gap:[, )}->["a", "d"] ` + - `[dr.d]"a":0.000000003,0-> [dr.d]"d":0.000000003,0->`, + kvs: kvs( + kv(k2, t1, s1), + rd(k1, k3, t3, s3), + ), + }, + { + // MVCC range deletions are executed individually when the range is split, + // and if this happens kvnemesis will report a failure since the writes + // will in all likelihood have non-atomic timestamps. + // In an actual run we avoid this by adding a test hook to DistSender to + // avoid splitting MVCC rangedels across ranges, instead failing with a + // hard error, and the generator attempts - imperfectly - to respect the + // split points. + name: "rangedel with range split", + steps: []Step{ + step(withResultTS(delRangeUsingTombstone(k1, k3, s1), t2)), }, + kvs: kvs( + rd(k1, k2, t2, s1), + rd(k2, k3, t1, s1), + ), + }, + { + name: "rangedel shadowing scan", + steps: []Step{ + step(withResultTS(put(k1, s1), t1)), + step(withResultTS(delRangeUsingTombstone(k1, k2, s2), t2)), + step(withScanResultTS(scan(k1, k2), t3)), // no rows returned + }, + kvs: kvs( + kv(k1, t1, s1), + rd(k1, k2, t2, s2), + ), }, } + w := echotest.NewWalker(t, testutils.TestDataPath(t, t.Name())) for _, test := range tests { - t.Run(test.name, func(t *testing.T) { + t.Run(test.name, w.Run(t, test.name, func(t *testing.T) string { e, err := MakeEngine() require.NoError(t, err) defer e.Close() + + var buf strings.Builder + for _, step := range test.steps { + fmt.Fprintln(&buf, strings.TrimSpace(step.String())) + } + + tr := &SeqTracker{} for _, kv := range test.kvs { - e.Put(kv.Key, kv.Value) + seq := kv.seq() + tr.Add(kv.key, kv.endKey, kv.ts, seq) + // NB: we go a little beyond what is truly necessary by embedding the + // sequence numbers (inside kv.val) unconditionally, as they would be in + // a real run. But we *do* need to embed them in `e.DeleteRange`, for + // otherwise pebble might start merging adjacent MVCC range dels (since + // they could have the same timestamp and empty value, where the seqno + // would really produce unique values). + if len(kv.endKey) == 0 { + k := storage.MVCCKey{ + Key: kv.key, + Timestamp: kv.ts, + } + e.Put(k, kv.val) + fmt.Fprintln(&buf, k, "@", seq, mustGetStringValue(kv.val)) + } else { + k := storage.MVCCRangeKey{ + StartKey: kv.key, + EndKey: kv.endKey, + Timestamp: kv.ts, + } + e.DeleteRange(kv.key, kv.endKey, kv.ts, kv.val) + fmt.Fprintln(&buf, k, "@", seq, mustGetStringValue(kv.val)) + } } - var actual []string - if failures := Validate(test.steps, e); len(failures) > 0 { - actual = make([]string, len(failures)) + + if failures := Validate(test.steps, e, tr); len(failures) > 0 { for i := range failures { - actual[i] = failures[i].Error() + fmt.Fprintln(&buf, failures[i]) } } - assert.Equal(t, test.expected, actual) - }) + return buf.String() + })) } } diff --git a/pkg/kv/kvnemesis/watcher.go b/pkg/kv/kvnemesis/watcher.go index 908bfa1394b6..89696ad1e7ed 100644 --- a/pkg/kv/kvnemesis/watcher.go +++ b/pkg/kv/kvnemesis/watcher.go @@ -12,13 +12,19 @@ package kvnemesis import ( "context" + "fmt" + "math" + "math/rand" "reflect" + "strings" "time" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -157,52 +163,18 @@ func (w *Watcher) processEvents(ctx context.Context, eventC chan kvcoord.RangeFe case *roachpb.RangeFeedError: return e.Error.GoError() case *roachpb.RangeFeedValue: - log.Infof(ctx, `rangefeed Put %s %s -> %s (prev %s)`, - e.Key, e.Value.Timestamp, e.Value.PrettyPrint(), e.PrevValue.PrettyPrint()) - w.mu.Lock() - // TODO(dan): If the exact key+ts is put into kvs more than once, the - // Engine will keep the last. This matches our txn semantics (if a key - // is written in a transaction more than once, only the last is kept) - // but it means that we'll won't catch it if we violate those semantics. - // Consider first doing a Get and somehow failing if this exact key+ts - // has previously been put with a different value. - if len(e.Value.RawBytes) == 0 { - w.mu.kvs.Delete(storage.MVCCKey{Key: e.Key, Timestamp: e.Value.Timestamp}) - } else { - w.mu.kvs.Put(storage.MVCCKey{Key: e.Key, Timestamp: e.Value.Timestamp}, e.Value.RawBytes) - } - prevTs := e.Value.Timestamp.Prev() - prevValue := w.mu.kvs.Get(e.Key, prevTs) - - // RangeFeed doesn't send the timestamps of the previous values back - // because changefeeds don't need them. It would likely be easy to - // implement, but would add unnecessary allocations in changefeeds, - // which don't need them. This means we'd want to make it an option in - // the request, which seems silly to do for only this test. - prevValue.Timestamp = hlc.Timestamp{} - // Additionally, ensure that deletion tombstones and missing keys are - // normalized as the nil slice, so that they can be matched properly - // between the RangeFeed and the Engine. - if len(e.PrevValue.RawBytes) == 0 { - e.PrevValue.RawBytes = nil - } - prevValueMismatch := !reflect.DeepEqual(prevValue, e.PrevValue) - var engineContents string - if prevValueMismatch { - engineContents = w.mu.kvs.DebugPrint(" ") + if err := w.handleValue(ctx, roachpb.Span{Key: e.Key}, e.Value, &e.PrevValue); err != nil { + return err } - w.mu.Unlock() - - if prevValueMismatch { - log.Infof(ctx, "rangefeed mismatch\n%s", engineContents) - panic(errors.Errorf( - `expected (%s, %s) previous value %s got: %s`, e.Key, prevTs, prevValue, e.PrevValue)) + case *roachpb.RangeFeedDeleteRange: + if err := w.handleValue(ctx, e.Span, roachpb.Value{Timestamp: e.Timestamp}, nil /* prevV */); err != nil { + return err } case *roachpb.RangeFeedCheckpoint: w.mu.Lock() frontierAdvanced, err := w.mu.frontier.Forward(e.Span, e.ResolvedTS) if err != nil { - panic(errors.Wrapf(err, "unexpected frontier error advancing to %s@%s", e.Span, e.ResolvedTS)) + return errors.Wrapf(err, "unexpected frontier error advancing to %s@%s", e.Span, e.ResolvedTS) } if frontierAdvanced { frontier := w.mu.frontier.Frontier() @@ -220,7 +192,86 @@ func (w *Watcher) processEvents(ctx context.Context, eventC chan kvcoord.RangeFe } } w.mu.Unlock() + default: + return errors.Errorf("unknown event: %T", e) } } } } + +func (w *Watcher) handleValue( + ctx context.Context, span roachpb.Span, v roachpb.Value, prevV *roachpb.Value, +) error { + w.mu.Lock() + defer w.mu.Unlock() + + var buf strings.Builder + fmt.Fprintf(&buf, `rangefeed %s %s -> %s`, span, v.Timestamp, v.PrettyPrint()) + if prevV != nil { + fmt.Fprintf(&buf, ` (prev %s)`, prevV.PrettyPrint()) + } + // TODO(dan): If the exact key+ts is put into kvs more than once, the + // Engine will keep the last. This matches our txn semantics (if a key + // is written in a transaction more than once, only the last is kept) + // but it means that we'll won't catch it if we violate those semantics. + // Consider first doing a Get and somehow failing if this exact key+ts + // has previously been put with a different value. + if len(span.EndKey) > 0 { + // If we have two operations that are not atomic (i.e. aren't in a batch) + // and they produce touching tombstones at the same timestamp, then + // `.mu.kvs` will merge them but they wouldn't be merged in pebble, since + // their MVCCValueHeader will contain different seqnos (and thus the value + // isn't identical). To work around that, we put random stuff in here. This + // is never interpreted - the seqno is only pulled out via an interceptor at + // the rangefeed boundary, and handed to the tracker. This is merely our + // local copy. + // + // See https://github.com/cockroachdb/cockroach/issues/92822. + var vh enginepb.MVCCValueHeader + vh.KVNemesisSeq.Set(kvnemesisutil.Seq(rand.Int63n(math.MaxUint32))) + mvccV := storage.MVCCValue{ + MVCCValueHeader: vh, + } + + sl, err := storage.EncodeMVCCValue(mvccV) + if err != nil { + return err + } + + w.mu.kvs.DeleteRange(span.Key, span.EndKey, v.Timestamp, sl) + return nil + } + + // Handle a point write. + w.mu.kvs.Put(storage.MVCCKey{Key: span.Key, Timestamp: v.Timestamp}, v.RawBytes) + prevTs := v.Timestamp.Prev() + getPrevV := w.mu.kvs.Get(span.Key, prevTs) + + // RangeFeed doesn't send the timestamps of the previous values back + // because changefeeds don't need them. It would likely be easy to + // implement, but would add unnecessary allocations in changefeeds, + // which don't need them. This means we'd want to make it an option in + // the request, which seems silly to do for only this test. + getPrevV.Timestamp = hlc.Timestamp{} + // Additionally, ensure that deletion tombstones and missing keys are + // normalized as the nil slice, so that they can be matched properly + // between the RangeFeed and the Engine. + if len(prevV.RawBytes) == 0 { + prevV.RawBytes = nil + } + prevValueMismatch := !reflect.DeepEqual(prevV, &getPrevV) + var engineContents string + if prevValueMismatch { + engineContents = w.mu.kvs.DebugPrint(" ") + } + + if prevValueMismatch { + log.Infof(ctx, "rangefeed mismatch\n%s", engineContents) + s := mustGetStringValue(getPrevV.RawBytes) + fmt.Println(s) + return errors.Errorf( + `expected (%s, %s) has previous value %s in kvs, but rangefeed has: %s`, + span, prevTs, mustGetStringValue(getPrevV.RawBytes), mustGetStringValue(prevV.RawBytes)) + } + return nil +} diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 1ac027efcad9..982b1a852655 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -111,6 +111,7 @@ go_library( "//pkg/kv", "//pkg/kv/kvbase", "//pkg/kv/kvclient/rangecache", + "//pkg/kv/kvnemesis/kvnemesisutil", "//pkg/kv/kvserver/abortspan", "//pkg/kv/kvserver/allocator", "//pkg/kv/kvserver/allocator/allocatorimpl", diff --git a/pkg/kv/kvserver/kvserverbase/base.go b/pkg/kv/kvserver/kvserverbase/base.go index cc58738edc0e..2db8d3a843ed 100644 --- a/pkg/kv/kvserver/kvserverbase/base.go +++ b/pkg/kv/kvserver/kvserverbase/base.go @@ -64,7 +64,7 @@ type FilterArgs struct { // ProposalFilterArgs groups the arguments to ReplicaProposalFilter. type ProposalFilterArgs struct { Ctx context.Context - Cmd kvserverpb.RaftCommand + Cmd *kvserverpb.RaftCommand QuotaAlloc *quotapool.IntAlloc CmdID CmdIDKey Req roachpb.BatchRequest diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan.go b/pkg/kv/kvserver/rangefeed/catchup_scan.go index 1726d672187d..cd684b8992d9 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan.go @@ -67,6 +67,7 @@ type CatchUpIterator struct { span roachpb.Span startTime hlc.Timestamp // exclusive pacer *admission.Pacer + OnEmit func(key, endKey roachpb.Key, ts hlc.Timestamp, vh enginepb.MVCCValueHeader) } // NewCatchUpIterator returns a CatchUpIterator for the given Reader over the @@ -189,15 +190,23 @@ func (i *CatchUpIterator) CatchUpScan( var span roachpb.Span a, span.Key = a.Copy(rangeKeys.Bounds.Key, 0) a, span.EndKey = a.Copy(rangeKeys.Bounds.EndKey, 0) + ts := rangeKeys.Versions[j].Timestamp err := outputFn(&roachpb.RangeFeedEvent{ DeleteRange: &roachpb.RangeFeedDeleteRange{ Span: span, - Timestamp: rangeKeys.Versions[j].Timestamp, + Timestamp: ts, }, }) if err != nil { return err } + if i.OnEmit != nil { + v, err := storage.DecodeMVCCValue(rangeKeys.Versions[j].Value) + if err != nil { + return err + } + i.OnEmit(span.Key, span.EndKey, ts, v.MVCCValueHeader) + } } } // If there's no point key here (e.g. we found a bare range key above), then @@ -325,6 +334,9 @@ func (i *CatchUpIterator) CatchUpScan( }, }) reorderBuf = append(reorderBuf, event) + if i.OnEmit != nil { + i.OnEmit(key, nil, ts, mvccVal.MVCCValueHeader) + } } } diff --git a/pkg/kv/kvserver/replica_evaluate.go b/pkg/kv/kvserver/replica_evaluate.go index f6fb82ae8063..9b838c7efa23 100644 --- a/pkg/kv/kvserver/replica_evaluate.go +++ b/pkg/kv/kvserver/replica_evaluate.go @@ -15,6 +15,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" @@ -268,6 +269,10 @@ func evaluateBatch( reply := br.Responses[index].GetInner() + if seq := args.Header().KVNemesisSeq.Get(); seq != 0 { + ctx = kvnemesisutil.WithSeq(ctx, seq) + } + // Note that `reply` is populated even when an error is returned: it // may carry a response transaction and in the case of WriteTooOldError // (which is sometimes deferred) it is fully populated. diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 35ff3b131658..25ee5f8ad568 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -275,7 +275,7 @@ func (r *Replica) evalAndPropose( if filter := r.store.TestingKnobs().TestingProposalFilter; filter != nil { filterArgs := kvserverbase.ProposalFilterArgs{ Ctx: ctx, - Cmd: *proposal.command, + Cmd: proposal.command, QuotaAlloc: proposal.quotaAlloc, CmdID: idKey, Req: *ba, diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index d31a687ac57e..0934730b98ea 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -226,7 +226,11 @@ func (r *Replica) rangeFeedWithRangeID( // Assert that we still hold the raftMu when this is called to ensure // that the catchUpIter reads from the current snapshot. r.raftMu.AssertHeld() - return rangefeed.NewCatchUpIterator(r.Engine(), span, startTime, iterSemRelease, pacer) + i := rangefeed.NewCatchUpIterator(r.Engine(), span, startTime, iterSemRelease, pacer) + if f := r.store.TestingKnobs().RangefeedValueHeaderFilter; f != nil { + i.OnEmit = f + } + return i } } p := r.registerWithRangefeedRaftMuLocked( @@ -578,6 +582,8 @@ func (r *Replica) handleLogicalOpLogRaftMuLocked( return } + vhf := r.store.TestingKnobs().RangefeedValueHeaderFilter + // When reading straight from the Raft log, some logical ops will not be // fully populated. Read from the Reader to populate all fields. for _, op := range ops.Ops { @@ -592,10 +598,24 @@ func (r *Replica) handleLogicalOpLogRaftMuLocked( case *enginepb.MVCCWriteIntentOp, *enginepb.MVCCUpdateIntentOp, *enginepb.MVCCAbortIntentOp, - *enginepb.MVCCAbortTxnOp, - *enginepb.MVCCDeleteRangeOp: + *enginepb.MVCCAbortTxnOp: // Nothing to do. continue + case *enginepb.MVCCDeleteRangeOp: + if vhf == nil { + continue + } + valBytes, err := storage.MVCCLookupRangeKeyValue(reader, t.StartKey, t.EndKey, t.Timestamp) + if err != nil { + panic(err) + } + + v, err := storage.DecodeMVCCValue(valBytes) + if err != nil { + panic(err) + } + vhf(t.StartKey, t.EndKey, t.Timestamp, v.MVCCValueHeader) + continue default: panic(errors.AssertionFailedf("unknown logical op %T", t)) } @@ -615,7 +635,7 @@ func (r *Replica) handleLogicalOpLogRaftMuLocked( // Read the value directly from the Reader. This is performed in the // same raftMu critical section that the logical op's corresponding // WriteBatch is applied, so the value should exist. - val, _, err := storage.MVCCGet(ctx, reader, key, ts, storage.MVCCGetOptions{Tombstones: true}) + val, _, vh, err := storage.MVCCGetWithValueHeader(ctx, reader, key, ts, storage.MVCCGetOptions{Tombstones: true}) if val == nil && err == nil { err = errors.New("value missing in reader") } @@ -625,6 +645,10 @@ func (r *Replica) handleLogicalOpLogRaftMuLocked( )) return } + + if vhf != nil { + vhf(key, nil, ts, vh) + } *valPtr = val.RawBytes } diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index da7d2a8931dc..31fbced01414 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) @@ -403,6 +404,12 @@ type StoreTestingKnobs struct { // renewing expiration based leases. LeaseRenewalDurationOverride time.Duration + // RangefeedValueHeaderFilter, if set, is invoked before each value emitted on + // the rangefeed, be it in steady state or during the catch-up scan. + // + // TODO(before merge): plumb the seqno through the rangefeed. + RangefeedValueHeaderFilter func(key, endKey roachpb.Key, ts hlc.Timestamp, vh enginepb.MVCCValueHeader) + // MakeSystemConfigSpanUnavailableToQueues makes the system config span // unavailable to queues that ask for it. MakeSystemConfigSpanUnavailableToQueues bool diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index 3f0b6ef41496..4958fc888cf7 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -11,6 +11,7 @@ go_library( srcs = [ "ambiguous_result_error.go", "api.go", + "api_requestheader.go", "app_stats.go", "batch.go", "data.go", @@ -36,10 +37,12 @@ go_library( deps = [ "//pkg/geo/geopb", "//pkg/keysbase", + "//pkg/kv/kvnemesis/kvnemesisutil", "//pkg/kv/kvserver/concurrency/lock", "//pkg/storage/enginepb", "//pkg/util", "//pkg/util/bitarray", + "//pkg/util/buildutil", "//pkg/util/caller", "//pkg/util/duration", "//pkg/util/encoding", @@ -89,12 +92,14 @@ go_test( deps = [ "//pkg/cli/exit", "//pkg/keys", + "//pkg/kv/kvnemesis/kvnemesisutil", "//pkg/kv/kvserver/concurrency/lock", "//pkg/storage/enginepb", "//pkg/testutils/echotest", "//pkg/testutils/zerofields", "//pkg/util", "//pkg/util/bitarray", + "//pkg/util/buildutil", "//pkg/util/duration", "//pkg/util/encoding", "//pkg/util/hlc", diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index a8fe3074064e..3556d1adcd99 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -14,6 +14,7 @@ import ( "context" "fmt" + _ "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" // see RequestHeader "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 9876956bf4c5..2dbf6e25a496 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -81,9 +81,75 @@ enum ResumeReason { RESUME_RANGE_BOUNDARY = 4; } +// RequestHeaderPure is not to be used directly. It's generated only for use of +// its marshaling methods by RequestHeader. See the comment there. +message RequestHeaderPure { + bytes key = 3 [(gogoproto.casttype) = "Key"]; + bytes end_key = 4 [(gogoproto.casttype) = "Key"]; + int32 sequence = 5 [ + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnSeq"]; +} + +// RequestHeaderPure is not to be used directly. It's generated only for use of +// its marshaling methods by RequestHeader. See the comment there. +message RequestHeaderCrdbTest { + message Empty{}; + Empty kvnemesis_seq = 6 [ + (gogoproto.customname) = "KVNemesisSeq", + (gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil.Container"]; + bytes key = 3 [(gogoproto.casttype) = "Key"]; + bytes end_key = 4 [(gogoproto.casttype) = "Key"]; + int32 sequence = 5 [ + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnSeq"]; +} + // RequestHeader is supplied with every storage node request. +// +// This message has some trickery attached to it to allow it to carry a +// testing-only field (KVNemesisSeq) in a way that avoids overhead in production +// code. The field is nominally an empty Message, but is cast to a Go type that +// is an int64 when under the crdb_test tag, and an empty struct otherwise. This +// ensures that production code does not have to pay for the increased struct +// size that would result from having the field always present. Additionally, we +// want to avoid encoding extra bytes on the wire; (gogoproto.nullable) will +// still encode the field tag even when the referenced field is empty. To work +// around this, we don't auto-generate the Size() and various marshal methods +// for RequestHeader. Instead, we have two sibling messages, RequestHeaderPure +// and RequestHeaderCrdbTest. The former is RequestHeader without the +// KVNemesisSeq field, and the latter is identical to RequestHeader but does +// have the generated Size and Marshal methods. We then implement the +// corresponding methods on RequestHeader by delegating to either, making sure +// to only delegate to RequestHeaderCrdbTest when we're under the crdb_test +// build tag *and* the KVNemesisSeq is nonzero. This effectively simulates a +// version of (gogoproto.nullable) in which the zero value is not represented on +// the wire. No change to unmarshalling code is necessary, as the generated +// unmarshaler on RequestHeader can deal with the presence of the field just +// fine. +// +// NB: unfortunately there isn't a way to have the {Pure,CrdbTest} messages +// unexported. +// +// NB: ensure that RequestHeader{,Pure,CrdbTest} stay in sync when adding +// or removing fields. message RequestHeader { + option (gogoproto.marshaler) = false; + option (gogoproto.sizer) = false; + reserved 1, 2; + message Empty{}; + // Empty is zero-size in production. It's an int64 under the crdb_test build tag. + // This is used to enable kvnemesis testing, which builds on uniqueness of values + // in the MVCC history. Deletions don't have a user-definable value, so we need + // an extra identifier, which is provided by this field. + // + // NB: it's important that this isn't at the end of the message, see: + // https://dave.cheney.net/2015/10/09/padding-is-hard + Empty kvnemesis_seq = 6 [ + (gogoproto.customname) = "KVNemesisSeq", + (gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil.Container"]; + // The key for request. If the request operates on a range, this // represents the starting key for the range. bytes key = 3 [(gogoproto.casttype) = "Key"]; diff --git a/pkg/roachpb/api_requestheader.go b/pkg/roachpb/api_requestheader.go new file mode 100644 index 000000000000..57892b4babb6 --- /dev/null +++ b/pkg/roachpb/api_requestheader.go @@ -0,0 +1,65 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package roachpb + +import "github.com/cockroachdb/cockroach/pkg/util/buildutil" + +func (r *RequestHeader) pure() RequestHeaderPure { + return RequestHeaderPure{ + Key: r.Key, + EndKey: r.EndKey, + Sequence: r.Sequence, + } +} + +func (r *RequestHeader) crdbTest() RequestHeaderCrdbTest { + return RequestHeaderCrdbTest(*r) +} + +// Size implements protoutil.Message. +func (r *RequestHeader) Size() int { + if buildutil.CrdbTestBuild && r.KVNemesisSeq.Get() != 0 { + rt := r.crdbTest() //gcassert:noescape + return rt.Size() + } + p := r.pure() //gcassert:noescape + return p.Size() +} + +// Marshal implements protoutil.Message. +func (r *RequestHeader) Marshal() ([]byte, error) { + if buildutil.CrdbTestBuild && r.KVNemesisSeq.Get() != 0 { + rt := r.crdbTest() + return rt.Marshal() + } + p := r.pure() + return p.Marshal() +} + +// MarshalTo implements protoutil.Message. +func (r *RequestHeader) MarshalTo(buf []byte) (int, error) { + if buildutil.CrdbTestBuild && r.KVNemesisSeq.Get() != 0 { + rt := r.crdbTest() //gcassert:noescape + return rt.MarshalTo(buf) + } + p := r.pure() //gcassert:noescape + return p.MarshalTo(buf) +} + +// MarshalToSizedBuffer implements protoutil.Message. +func (r *RequestHeader) MarshalToSizedBuffer(buf []byte) (int, error) { + if buildutil.CrdbTestBuild && r.KVNemesisSeq.Get() != 0 { + rt := r.crdbTest() //gcassert:noescape + return rt.MarshalToSizedBuffer(buf) + } + p := r.pure() //gcassert:noescape + return p.MarshalToSizedBuffer(buf) +} diff --git a/pkg/roachpb/api_test.go b/pkg/roachpb/api_test.go index 36ce4b0fd15b..561fd12149ee 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/roachpb/api_test.go @@ -14,9 +14,12 @@ import ( "reflect" "testing" + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/redact" gogoproto "github.com/gogo/protobuf/proto" @@ -406,3 +409,20 @@ func TestFlagCombinations(t *testing.T) { } } } + +func TestRequestHeaderRoundTrip(t *testing.T) { + var seq kvnemesisutil.Container + seq.Set(123) + exp := seq.Get() + if buildutil.CrdbTestBuild { + require.EqualValues(t, 123, exp) + } + rh := RequestHeader{KVNemesisSeq: seq} + sl, err := protoutil.Marshal(&rh) + require.NoError(t, err) + + rh.Reset() + require.NoError(t, protoutil.Unmarshal(sl, &rh)) + + require.Equal(t, exp, rh.KVNemesisSeq.Get()) +} diff --git a/pkg/roachpb/metadata_replicas.go b/pkg/roachpb/metadata_replicas.go index 2b874d082cdd..4950adac01aa 100644 --- a/pkg/roachpb/metadata_replicas.go +++ b/pkg/roachpb/metadata_replicas.go @@ -508,7 +508,11 @@ func (c ReplicaChangeType) IsRemoval() bool { } var errReplicaNotFound = errors.Errorf(`replica not found in RangeDescriptor`) -var errReplicaCannotHoldLease = errors.Errorf("replica cannot hold lease") + +// ErrReplicaCannotHoldLease can be returned from CheckCanReceiveLease. +// +// See: https://github.com/cockroachdb/cockroach/issues/93163 +var ErrReplicaCannotHoldLease = errors.Errorf("replica cannot hold lease") // CheckCanReceiveLease checks whether `wouldbeLeaseholder` can receive a lease. // Returns an error if the respective replica is not eligible. @@ -546,7 +550,7 @@ func CheckCanReceiveLease( // We allow a demoting / incoming voter to receive the lease if there's an incoming voter. // In this case, when exiting the joint config, we will transfer the lease to the incoming // voter. - return errReplicaCannotHoldLease + return ErrReplicaCannotHoldLease } return nil } diff --git a/pkg/roachpb/span_group.go b/pkg/roachpb/span_group.go index 070a0bda9474..3aa3c9f0e26f 100644 --- a/pkg/roachpb/span_group.go +++ b/pkg/roachpb/span_group.go @@ -101,7 +101,11 @@ func (g *SpanGroup) Slice() []Span { if rg == nil { return nil } - ret := make([]Span, 0, rg.Len()) + n := rg.Len() + if n == 0 { + return nil + } + ret := make([]Span, 0, n) it := rg.Iterator() for { rng, next := it.Next() diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 2b2737ef4571..d4ff71159f06 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -51,6 +51,7 @@ go_library( "//pkg/cli/exit", "//pkg/clusterversion", "//pkg/keys", + "//pkg/kv/kvnemesis/kvnemesisutil", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/diskmap", "//pkg/kv/kvserver/uncertainty", @@ -153,6 +154,7 @@ go_test( "//pkg/storage/enginepb", "//pkg/storage/fs", "//pkg/testutils", + "//pkg/testutils/echotest", "//pkg/testutils/skip", "//pkg/testutils/zerofields", "//pkg/util", diff --git a/pkg/storage/enginepb/BUILD.bazel b/pkg/storage/enginepb/BUILD.bazel index baf83f62d193..16da46ef6457 100644 --- a/pkg/storage/enginepb/BUILD.bazel +++ b/pkg/storage/enginepb/BUILD.bazel @@ -41,11 +41,14 @@ go_library( "file_registry.go", "mvcc.go", "mvcc3.go", + "mvcc3_valueheader.go", ], embed = [":enginepb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/storage/enginepb", visibility = ["//visibility:public"], deps = [ + "//pkg/kv/kvnemesis/kvnemesisutil", + "//pkg/util/buildutil", "//pkg/util/hlc", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", @@ -67,6 +70,7 @@ go_test( "//pkg/storage", "//pkg/testutils/zerofields", "//pkg/util/hlc", + "//pkg/util/protoutil", "//pkg/util/uuid", "@com_github_cockroachdb_redact//:redact", "@com_github_stretchr_testify//assert", diff --git a/pkg/storage/enginepb/mvcc3.go b/pkg/storage/enginepb/mvcc3.go index 4db670f1699b..69fe1e128193 100644 --- a/pkg/storage/enginepb/mvcc3.go +++ b/pkg/storage/enginepb/mvcc3.go @@ -10,7 +10,10 @@ package enginepb -import "github.com/cockroachdb/errors" +import ( + _ "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" // see MVCCValueHeader + "github.com/cockroachdb/errors" +) // SafeValue implements the redact.SafeValue interface. func (MVCCStatsDelta) SafeValue() {} @@ -51,12 +54,3 @@ func (op *MVCCLogicalOp) MustSetValue(value interface{}) { panic(errors.AssertionFailedf("%T excludes %T", op, value)) } } - -// IsEmpty returns true if the header is empty. -// gcassert:inline -func (h MVCCValueHeader) IsEmpty() bool { - // NB: We don't use a struct comparison like h == MVCCValueHeader{} due to a - // Go 1.19 performance regression, see: - // https://github.com/cockroachdb/cockroach/issues/88818 - return h.LocalTimestamp.IsEmpty() -} diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index df98c642236e..99c201416d68 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -144,8 +144,31 @@ message IgnoredSeqNumRange { // Used by storage.MVCCValue. // // NB: Make sure to update MVCCValueHeader.IsEmpty() when adding fields. +// +// NB: a shallow copy of this value has to equal a deep copy, i.e. there +// must be (recursively) no pointers in this type. Should this need to +// change, need to update mvccGetWithValueHeader. +// +// NB: ensure that this struct stays in sync with MVCCValueHeader{Pure,CrdbTest}. +// See roachpb.RequestHeader for details on how they help us keep the KVNemesisSeq +// field out of production code. message MVCCValueHeader { option (gogoproto.equal) = true; + option (gogoproto.marshaler) = false; + option (gogoproto.sizer) = false; + + message Empty{}; + // Empty is zero-size in production. It's an int64 under the crdb_test build tag. + // This is used to enable kvnemesis testing, which builds on uniqueness of values + // in the MVCC history. Deletions don't have a user-definable value, so we need + // an extra identifier, which is provided by this field. + // + // NB: it's important that this isn't at the end of the message, see: + // https://dave.cheney.net/2015/10/09/padding-is-hard + Empty kvnemesis_seq = 2 [ + (gogoproto.customname) = "KVNemesisSeq", + (gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil.Container"]; // The local clock timestamp records the value of the local HLC clock on the // leaseholder when the key was originally written. It is used to make claims @@ -174,6 +197,24 @@ message MVCCValueHeader { (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; } +// MVCCValueHeaderPure is not to be used directly. It's generated only for use of +// its marshaling methods by MVCCValueHeader. See the comment there. +message MVCCValueHeaderPure { + util.hlc.Timestamp local_timestamp = 1 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; +} +// MVCCValueHeaderCrdbTest is not to be used directly. It's generated only for use of +// its marshaling methods by MVCCValueHeader. See the comment there. +message MVCCValueHeaderCrdbTest { + message Empty{}; + Empty kvnemesis_seq = 2 [ + (gogoproto.customname) = "KVNemesisSeq", + (gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil.Container"]; + util.hlc.Timestamp local_timestamp = 1 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; +} + // MVCCStatsDelta is convertible to MVCCStats, but uses signed variable width // encodings for most fields that make it more efficient to store negative // values. This makes the encodings incompatible. diff --git a/pkg/storage/enginepb/mvcc3_test.go b/pkg/storage/enginepb/mvcc3_test.go index 408c47d21323..135b3dc4049d 100644 --- a/pkg/storage/enginepb/mvcc3_test.go +++ b/pkg/storage/enginepb/mvcc3_test.go @@ -12,18 +12,35 @@ package enginepb import ( + "reflect" "testing" "github.com/cockroachdb/cockroach/pkg/testutils/zerofields" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/stretchr/testify/require" ) -func TestMVCCValueHeader_IsEmpty(t *testing.T) { +func populatedMVCCValueHeader() MVCCValueHeader { allFieldsSet := MVCCValueHeader{ LocalTimestamp: hlc.ClockTimestamp{WallTime: 1, Logical: 1, Synthetic: true}, } + allFieldsSet.KVNemesisSeq.Set(123) + return allFieldsSet +} + +func TestMVCCValueHeader_IsEmpty(t *testing.T) { + allFieldsSet := populatedMVCCValueHeader() require.NoError(t, zerofields.NoZeroField(allFieldsSet), "make sure you update the IsEmpty method") require.True(t, MVCCValueHeader{}.IsEmpty()) require.False(t, allFieldsSet.IsEmpty()) } + +func TestMVCCValueHeader_MarshalUnmarshal(t *testing.T) { + vh := populatedMVCCValueHeader() + b, err := protoutil.Marshal(&vh) + require.NoError(t, err) + var vh2 MVCCValueHeader + require.NoError(t, protoutil.Unmarshal(b, &vh2)) + reflect.DeepEqual(vh, vh2) +} diff --git a/pkg/storage/enginepb/mvcc3_valueheader.go b/pkg/storage/enginepb/mvcc3_valueheader.go new file mode 100644 index 000000000000..59f829433886 --- /dev/null +++ b/pkg/storage/enginepb/mvcc3_valueheader.go @@ -0,0 +1,72 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package enginepb + +import "github.com/cockroachdb/cockroach/pkg/util/buildutil" + +// IsEmpty returns true if the header is empty. +// gcassert:inline +func (h MVCCValueHeader) IsEmpty() bool { + // NB: We don't use a struct comparison like h == MVCCValueHeader{} due to a + // Go 1.19 performance regression, see: + // https://github.com/cockroachdb/cockroach/issues/88818 + return h.LocalTimestamp.IsEmpty() && h.KVNemesisSeq.Get() == 0 +} + +func (h *MVCCValueHeader) pure() MVCCValueHeaderPure { + return MVCCValueHeaderPure{ + LocalTimestamp: h.LocalTimestamp, + } +} + +func (h *MVCCValueHeader) crdbTest() MVCCValueHeaderCrdbTest { + return (MVCCValueHeaderCrdbTest)(*h) +} + +// Size implements protoutil.Message. +func (h *MVCCValueHeader) Size() int { + if buildutil.CrdbTestBuild && h.KVNemesisSeq.Get() != 0 { + ht := h.crdbTest() //gcassert:noescape + return ht.Size() + } + p := h.pure() //gcassert:noescape + return p.Size() +} + +// Marshal implements protoutil.Message. +func (h *MVCCValueHeader) Marshal() ([]byte, error) { + if buildutil.CrdbTestBuild && h.KVNemesisSeq.Get() != 0 { + ht := h.crdbTest() //gcassert:noescape + return ht.Marshal() + } + p := h.pure() //gcassert:noescape + return p.Marshal() +} + +// MarshalTo implements protoutil.Message. +func (h *MVCCValueHeader) MarshalTo(buf []byte) (int, error) { + if buildutil.CrdbTestBuild && h.KVNemesisSeq.Get() != 0 { + ht := h.crdbTest() //gcassert:noescape + return ht.MarshalTo(buf) + } + p := h.pure() //gcassert:noescape + return p.MarshalTo(buf) +} + +// MarshalToSizedBuffer implements protoutil.Message. +func (h *MVCCValueHeader) MarshalToSizedBuffer(buf []byte) (int, error) { + if buildutil.CrdbTestBuild && h.KVNemesisSeq.Get() != 0 { + ht := h.crdbTest() //gcassert:noescape + return ht.MarshalToSizedBuffer(buf) + } + p := h.pure() //gcassert:noescape + return p.MarshalToSizedBuffer(buf) +} diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 93a00fd5f6aa..eb0979eb769d 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -32,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/admission" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" @@ -991,6 +993,15 @@ func newMVCCIterator( func MVCCGet( ctx context.Context, reader Reader, key roachpb.Key, timestamp hlc.Timestamp, opts MVCCGetOptions, ) (*roachpb.Value, *roachpb.Intent, error) { + value, intent, _, err := MVCCGetWithValueHeader(ctx, reader, key, timestamp, opts) + return value, intent, err +} + +// MVCCGetWithValueHeader is like MVCCGet, but in addition returns the +// MVCCValueHeader for the value. +func MVCCGetWithValueHeader( + ctx context.Context, reader Reader, key roachpb.Key, timestamp hlc.Timestamp, opts MVCCGetOptions, +) (*roachpb.Value, *roachpb.Intent, enginepb.MVCCValueHeader, error) { iter := newMVCCIterator( reader, timestamp, false /* rangeKeyMasking */, opts.DontInterleaveIntents, IterOptions{ KeyTypes: IterKeyTypePointsAndRanges, @@ -998,10 +1009,11 @@ func MVCCGet( }, ) defer iter.Close() - value, intent, err := mvccGet(ctx, iter, key, timestamp, opts) - return value.ToPointer(), intent, err + value, intent, vh, err := mvccGetWithValueHeader(ctx, iter, key, timestamp, opts) + return value.ToPointer(), intent, vh, err } +// gcassert:inline func mvccGet( ctx context.Context, iter MVCCIterator, @@ -1009,17 +1021,28 @@ func mvccGet( timestamp hlc.Timestamp, opts MVCCGetOptions, ) (value optionalValue, intent *roachpb.Intent, err error) { + value, intent, _, err = mvccGetWithValueHeader(ctx, iter, key, timestamp, opts) + return value, intent, err +} + +func mvccGetWithValueHeader( + ctx context.Context, + iter MVCCIterator, + key roachpb.Key, + timestamp hlc.Timestamp, + opts MVCCGetOptions, +) (value optionalValue, intent *roachpb.Intent, vh enginepb.MVCCValueHeader, err error) { if len(key) == 0 { - return optionalValue{}, nil, emptyKeyError() + return optionalValue{}, nil, enginepb.MVCCValueHeader{}, emptyKeyError() } if timestamp.WallTime < 0 { - return optionalValue{}, nil, errors.Errorf("cannot write to %q at timestamp %s", key, timestamp) + return optionalValue{}, nil, enginepb.MVCCValueHeader{}, errors.Errorf("cannot write to %q at timestamp %s", key, timestamp) } if util.RaceEnabled && !iter.IsPrefix() { - return optionalValue{}, nil, errors.AssertionFailedf("mvccGet called with non-prefix iterator") + return optionalValue{}, nil, enginepb.MVCCValueHeader{}, errors.AssertionFailedf("mvccGet called with non-prefix iterator") } if err := opts.validate(); err != nil { - return optionalValue{}, nil, err + return optionalValue{}, nil, enginepb.MVCCValueHeader{}, err } mvccScanner := pebbleMVCCScannerPool.Get().(*pebbleMVCCScanner) @@ -1049,36 +1072,39 @@ func mvccGet( recordIteratorStats(ctx, mvccScanner.parent) if mvccScanner.err != nil { - return optionalValue{}, nil, mvccScanner.err + return optionalValue{}, nil, enginepb.MVCCValueHeader{}, mvccScanner.err } intents, err := buildScanIntents(mvccScanner.intentsRepr()) if err != nil { - return optionalValue{}, nil, err + return optionalValue{}, nil, enginepb.MVCCValueHeader{}, err } if opts.errOnIntents() && len(intents) > 0 { - return optionalValue{}, nil, &roachpb.WriteIntentError{Intents: intents} + return optionalValue{}, nil, enginepb.MVCCValueHeader{}, &roachpb.WriteIntentError{Intents: intents} } if len(intents) > 1 { - return optionalValue{}, nil, errors.Errorf("expected 0 or 1 intents, got %d", len(intents)) + return optionalValue{}, nil, enginepb.MVCCValueHeader{}, errors.Errorf("expected 0 or 1 intents, got %d", len(intents)) } else if len(intents) == 1 { intent = &intents[0] } if len(mvccScanner.results.repr) == 0 { - return optionalValue{}, intent, nil + return optionalValue{}, intent, enginepb.MVCCValueHeader{}, nil } mvccKey, rawValue, _, err := MVCCScanDecodeKeyValue(mvccScanner.results.repr) if err != nil { - return optionalValue{}, nil, err + return optionalValue{}, nil, enginepb.MVCCValueHeader{}, err } value = makeOptionalValue(roachpb.Value{ RawBytes: rawValue, Timestamp: mvccKey.Timestamp, }) - return value, intent, nil + // NB: we may return MVCCValueHeader out of curUnsafeValue because that + // type does not contain any pointers. A comment on MVCCValueHeader ensures + // that this stays true. + return value, intent, mvccScanner.curUnsafeValue.MVCCValueHeader, nil } // MVCCGetAsTxn constructs a temporary transaction from the given transaction @@ -2003,6 +2029,13 @@ func mvccPutInternal( versionValue := MVCCValue{} versionValue.Value = value versionValue.LocalTimestamp = localTimestamp + + if buildutil.CrdbTestBuild { + if seq, seqOK := kvnemesisutil.FromContext(ctx); seqOK { + versionValue.KVNemesisSeq.Set(seq) + } + } + if !versionValue.LocalTimestampNeeded(versionKey.Timestamp) || !writer.ShouldWriteLocalTimestamps(ctx) { versionValue.LocalTimestamp = hlc.ClockTimestamp{} @@ -3263,6 +3296,11 @@ func MVCCDeleteRangeUsingTombstone( if !value.LocalTimestampNeeded(timestamp) || !rw.ShouldWriteLocalTimestamps(ctx) { value.LocalTimestamp = hlc.ClockTimestamp{} } + if buildutil.CrdbTestBuild { + if seq, ok := kvnemesisutil.FromContext(ctx); ok { + value.KVNemesisSeq.Set(seq) + } + } valueRaw, err := EncodeMVCCValue(value) if err != nil { return err @@ -6544,3 +6582,67 @@ func isWatchedSystemTable(key roachpb.Key) bool { return false } } + +// MVCCLookupRangeKeyValue reads the value header for a range deletion on +// [key,endKey) at the specified timestamp. The range deletion is allowed to be +// fragmented (with identical value) and is allowed to extend out of +// [key,endKey). An error is returned if a matching range deletion cannot be +// found. +func MVCCLookupRangeKeyValue( + reader Reader, key, endKey roachpb.Key, ts hlc.Timestamp, +) ([]byte, error) { + it := reader.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ + LowerBound: key, + UpperBound: endKey, + KeyTypes: IterKeyTypeRangesOnly, + }) + defer it.Close() + + it.SeekGE(MVCCKey{Key: key}) + + // Start by assuming that we've already seen [min, key) and now we're iterating + // to fill this up to [min, endKey). + span := roachpb.Span{ + Key: roachpb.KeyMin, + EndKey: append([]byte(nil), key...), // copy since we'll mutate this memory + } + first := true + var val []byte + for ; ; it.Next() { + ok, err := it.Valid() + if err != nil { + return nil, err + } + if !ok { + break + } + rkv, ok := it.RangeKeys().FirstAtOrAbove(ts) + if !ok || rkv.Timestamp != ts { + return nil, errors.Errorf( + "gap [%s,...) in expected range deletion [%s,%s)", span.EndKey, key, endKey) + } + + unsafeBounds := it.RangeBounds() // only valid until next call to iterator + if !span.EndKey.Equal(unsafeBounds.Key) { + return nil, errors.Errorf( + "gap [%s,%s) in expected range deletion [%s,%s)", span.EndKey, unsafeBounds.Key, key, endKey, + ) + } + + if first { + val = append(val, rkv.Value...) + first = false + } else if !bytes.Equal(val, rkv.Value) { + return nil, errors.Errorf( + "value change at %s in expected range deletion [%s,%s)", unsafeBounds.Key, key, endKey) + } + + span.EndKey = append(span.EndKey[:0], unsafeBounds.EndKey...) + } + if !span.EndKey.Equal(endKey) { + return nil, errors.Errorf( + "gap [%s,...) in expected range deletion [%s,%s)", span.EndKey, key, endKey) + } + // Made it! + return val, nil +} diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 8022ee2f8e42..64cf802693c1 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -375,14 +375,14 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { } mKeySize := int64(mvccKey(key).EncodedSize()) - require.EqualValues(t, mKeySize, 2) + require.EqualValues(t, 2, mKeySize) mVal1Size := int64((&enginepb.MVCCMetadata{ Timestamp: ts1.ToLegacyTimestamp(), Deleted: false, Txn: &txn.TxnMeta, }).Size()) - require.EqualValues(t, mVal1Size, 46) + require.EqualValues(t, 46, mVal1Size) mVal1Size += 2 // TODO(sumeer): this is the first put at ts1, so why are we using this m1ValSize @@ -392,17 +392,17 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { Deleted: false, Txn: &txn.TxnMeta, }).Size()) - require.EqualValues(t, m1ValSize, 46) + require.EqualValues(t, 46, m1ValSize) m1ValSize += 2 vKeySize := MVCCVersionTimestampSize - require.EqualValues(t, vKeySize, 12) + require.EqualValues(t, 12, vKeySize) vValSize := int64(len(value.RawBytes)) - require.EqualValues(t, vValSize, 10) + require.EqualValues(t, 10, vValSize) if disableSimpleValueEncoding { vValSize += emptyMVCCValueHeaderSize - require.EqualValues(t, vValSize, 17) + require.EqualValues(t, 17, vValSize) } expMS := enginepb.MVCCStats{ @@ -443,7 +443,7 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { if disableSimpleValueEncoding { expM2ValSize = 71 } - require.EqualValues(t, m2ValSize, expM2ValSize) + require.EqualValues(t, expM2ValSize, m2ValSize) if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) @@ -562,7 +562,7 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { if disableSimpleValueEncoding { expMVal2Size = 61 } - require.EqualValues(t, mVal2Size, expMVal2Size) + require.EqualValues(t, expMVal2Size, mVal2Size) value := roachpb.MakeValueFromString("value") @@ -892,7 +892,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { require.EqualValues(t, vVal2Size, 14) if disableSimpleValueEncoding { vVal2Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal2Size, 21) + require.EqualValues(t, 21, vVal2Size) } txn.WriteTimestamp.Forward(ts3) @@ -915,7 +915,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { if disableSimpleValueEncoding { expMVal2Size = 61 } - require.EqualValues(t, mVal2SizeWithHistory, expMVal2Size) + require.EqualValues(t, expMVal2Size, mVal2SizeWithHistory) expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 3e9, @@ -1263,7 +1263,7 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { require.EqualValues(t, vVal2Size, 14) if disableSimpleValueEncoding { vVal2Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal2Size, 21) + require.EqualValues(t, 21, vVal2Size) } expMS := enginepb.MVCCStats{ @@ -1294,7 +1294,7 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { if disableSimpleValueEncoding { expMVal2Size = 71 } - require.EqualValues(t, mVal2Size, expMVal2Size) + require.EqualValues(t, expMVal2Size, mVal2Size) if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { t.Fatal(err) @@ -1363,7 +1363,7 @@ func TestMVCCStatsTxnSysPutAbort(t *testing.T) { require.EqualValues(t, vVal2Size, 14) if disableSimpleValueEncoding { vVal2Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal2Size, 21) + require.EqualValues(t, 21, vVal2Size) } expMS := enginepb.MVCCStats{ @@ -1429,7 +1429,7 @@ func TestMVCCStatsSysPutPut(t *testing.T) { require.EqualValues(t, vVal2Size, 14) if disableSimpleValueEncoding { vVal2Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal2Size, 21) + require.EqualValues(t, 21, vVal2Size) } expMS := enginepb.MVCCStats{ diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 05a63fb86356..ed73b6854738 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/zerofields" "github.com/cockroachdb/cockroach/pkg/util/encoding" @@ -279,7 +280,7 @@ func TestMVCCGetNoMoreOldVersion(t *testing.T) { } } -func TestMVCCGetAndDelete(t *testing.T) { +func TestMVCCGetWithValueHeader(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -287,32 +288,34 @@ func TestMVCCGetAndDelete(t *testing.T) { engine := NewDefaultInMemForTesting() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1, Logical: 1}, hlc.ClockTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) + value, _, vh, err := MVCCGetWithValueHeader(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } if value == nil { t.Fatal("the value should not be empty") } + require.Equal(t, hlc.ClockTimestamp{WallTime: 1}, vh.LocalTimestamp) - _, err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil) + _, err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{WallTime: 2, Logical: 1}, nil) if err != nil { t.Fatal(err) } // Read the latest version which should be deleted. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{}) + value, _, vh, err = MVCCGetWithValueHeader(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } if value != nil { t.Fatal("the value should be empty") } + require.Zero(t, vh.LocalTimestamp) // Read the latest version with tombstone. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, + value, _, vh, err = MVCCGetWithValueHeader(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{Tombstones: true}) if err != nil { t.Fatal(err) @@ -320,9 +323,11 @@ func TestMVCCGetAndDelete(t *testing.T) { t.Fatalf("the value should be non-nil with empty RawBytes; got %+v", value) } + require.Equal(t, hlc.ClockTimestamp{WallTime: 2, Logical: 1}, vh.LocalTimestamp) + // Read the old version which should still exist. for _, logical := range []int32{0, math.MaxInt32} { - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2, Logical: logical}, + value, _, vh, err := MVCCGetWithValueHeader(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2, Logical: logical}, MVCCGetOptions{}) if err != nil { t.Fatal(err) @@ -330,6 +335,7 @@ func TestMVCCGetAndDelete(t *testing.T) { if value == nil { t.Fatal("the value should not be empty") } + require.Equal(t, hlc.ClockTimestamp{WallTime: 1}, vh.LocalTimestamp) } } @@ -6616,3 +6622,133 @@ func mvccGetRawWithError(t *testing.T, r Reader, key MVCCKey) ([]byte, error) { } return iter.Value(), nil } + +func TestMVCCLookupRangeKeyValue(t *testing.T) { + defer leaktest.AfterTest(t)() + + eng := createTestPebbleEngine() + defer eng.Close() + + const diagram = ` +# a b c d +# t=4000 [-----v1----) +# t=2000 [-v2-)[-----v3----) +# +` + + v1 := MVCCValue{ + MVCCValueHeader: enginepb.MVCCValueHeader{ + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1}, + }, + } + v2 := MVCCValue{ + MVCCValueHeader: enginepb.MVCCValueHeader{ + LocalTimestamp: hlc.ClockTimestamp{WallTime: 2}, + }, + } + v3 := MVCCValue{ + MVCCValueHeader: enginepb.MVCCValueHeader{ + LocalTimestamp: hlc.ClockTimestamp{WallTime: 3}, + }, + } + + t2000 := hlc.Timestamp{WallTime: 2000} + t4000 := hlc.Timestamp{WallTime: 4000} + + a, b, c, d := roachpb.Key("a"), roachpb.Key("b"), roachpb.Key("c"), roachpb.Key("d") + + require.NoError(t, eng.PutMVCCRangeKey(MVCCRangeKey{ + StartKey: a, + EndKey: c, + Timestamp: t4000, + }, v1)) + + require.NoError(t, eng.PutMVCCRangeKey(MVCCRangeKey{ + StartKey: a, + EndKey: b, + Timestamp: t2000, + }, v2)) + + require.NoError(t, eng.PutMVCCRangeKey(MVCCRangeKey{ + StartKey: b, + EndKey: d, + Timestamp: t2000, + }, v3)) + + var buf bytes.Buffer + fmt.Fprintln(&buf, strings.TrimSpace(diagram)) + + for _, tc := range []struct { + name string + k, ek roachpb.Key + ts hlc.Timestamp + }{ + { + // Look up the exact rangedel. + name: "ac-valid-full", + k: a, + ek: c, + ts: t4000, + }, + { + // Look up inside of the rangedel. + name: "ac-valid-partial", + k: a.Next(), + ek: b, + ts: t4000, + }, + { + // Correct bounds, but incorrect timestamp, + // will see part of ab and bd which are not compatible and error out. + name: "ac-incompatible-fragments", + k: a, + ek: c, + ts: t2000, + }, + { + // Correct bounds, but timestamp too early. + // Won't see anything and error out. + name: "ac-ts-too-early", + k: a, + ek: b, + ts: t2000, + }, + { + // See ac but with a gap. Start key before rangedel starts. Errors out. + name: "ac-invalid-pre", + k: roachpb.KeyMin, + ek: c, + ts: t4000, + }, + { + // Sees ac but with a gap. End key after rangedel end. Errors out. + name: "ac-invalid-post", + k: a, + ek: d, + ts: t4000, + }, + // Sees cd but wants it longer. Errors. + { + name: "cd-invalid-post", + k: c, + ek: roachpb.Key("f"), + ts: t2000, + }, + } { + fmt.Fprintf(&buf, "# %s\n", tc.name) + fmt.Fprintf(&buf, "lookup([%s,%s) @ %d) = ", tc.k, tc.ek, tc.ts.WallTime) + valBytes, err := MVCCLookupRangeKeyValue(eng, tc.k, tc.ek, tc.ts) + if err != nil { + fmt.Fprintln(&buf, err) + } else { + v, err := DecodeMVCCValue(valBytes) + if err != nil { + fmt.Fprintln(&buf, err) + } else { + fmt.Fprintf(&buf, "v%d\n", v.MVCCValueHeader.LocalTimestamp.WallTime) + } + } + } + path := testutils.TestDataPath(t, t.Name()) + echotest.Require(t, buf.String(), path) +} diff --git a/pkg/storage/mvcc_value.go b/pkg/storage/mvcc_value.go index 98ff415e9ef8..288cf5ed5bdf 100644 --- a/pkg/storage/mvcc_value.go +++ b/pkg/storage/mvcc_value.go @@ -253,14 +253,12 @@ func decodeExtendedMVCCValue(buf []byte) (MVCCValue, error) { if len(buf) < int(headerSize) { return MVCCValue{}, errMVCCValueMissingHeader } - var header enginepb.MVCCValueHeader + var v MVCCValue // NOTE: we don't use protoutil to avoid passing header through an interface, // which would cause a heap allocation and incur the cost of dynamic dispatch. - if err := header.Unmarshal(buf[extendedPreludeSize:headerSize]); err != nil { + if err := v.MVCCValueHeader.Unmarshal(buf[extendedPreludeSize:headerSize]); err != nil { return MVCCValue{}, errors.Wrapf(err, "unmarshaling MVCCValueHeader") } - var v MVCCValue - v.LocalTimestamp = header.LocalTimestamp v.Value.RawBytes = buf[headerSize:] return v, nil } diff --git a/pkg/storage/mvcc_value_test.go b/pkg/storage/mvcc_value_test.go index f4f234cc6028..43d913e4c0eb 100644 --- a/pkg/storage/mvcc_value_test.go +++ b/pkg/storage/mvcc_value_test.go @@ -13,12 +13,16 @@ package storage import ( "bytes" "fmt" + "strings" "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/echotest" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -118,36 +122,39 @@ func TestEncodeDecodeMVCCValue(t *testing.T) { valHeader.LocalTimestamp = hlc.ClockTimestamp{WallTime: 9} testcases := map[string]struct { - val MVCCValue - expect []byte + val MVCCValue }{ - "tombstone": {val: MVCCValue{}, expect: nil}, - "bytes": {val: MVCCValue{Value: strVal}, expect: []byte{0x0, 0x0, 0x0, 0x0, 0x3, 0x66, 0x6f, 0x6f}}, - "int": {val: MVCCValue{Value: intVal}, expect: []byte{0x0, 0x0, 0x0, 0x0, 0x1, 0x22}}, - "header+tombstone": {val: MVCCValue{MVCCValueHeader: valHeader}, expect: []byte{0x0, 0x0, 0x0, 0x4, 0x65, 0xa, 0x2, 0x8, 0x9}}, - "header+bytes": {val: MVCCValue{MVCCValueHeader: valHeader, Value: strVal}, expect: []byte{0x0, 0x0, 0x0, 0x4, 0x65, 0xa, 0x2, 0x8, 0x9, 0x0, 0x0, 0x0, 0x0, 0x3, 0x66, 0x6f, 0x6f}}, - "header+int": {val: MVCCValue{MVCCValueHeader: valHeader, Value: intVal}, expect: []byte{0x0, 0x0, 0x0, 0x4, 0x65, 0xa, 0x2, 0x8, 0x9, 0x0, 0x0, 0x0, 0x0, 0x1, 0x22}}, + "tombstone": {val: MVCCValue{}}, + "bytes": {val: MVCCValue{Value: strVal}}, + "int": {val: MVCCValue{Value: intVal}}, + "header+tombstone": {val: MVCCValue{MVCCValueHeader: valHeader}}, + "header+bytes": {val: MVCCValue{MVCCValueHeader: valHeader, Value: strVal}}, + "header+int": {val: MVCCValue{MVCCValueHeader: valHeader, Value: intVal}}, } + w := echotest.NewWalker(t, testutils.TestDataPath(t, t.Name())) for name, tc := range testcases { - t.Run(name, func(t *testing.T) { - encSize := encodedMVCCValueSize(tc.val) - require.Equal(t, len(tc.expect), encSize) - + t.Run(name, w.Run(t, name, func(t *testing.T) string { + var buf strings.Builder enc, err := EncodeMVCCValue(tc.val) require.NoError(t, err) - require.Equal(t, tc.expect, enc) + fmt.Fprintf(&buf, "encoded: %x", enc) + assert.Equal(t, encodedMVCCValueSize(tc.val), len(enc)) dec, err := DecodeMVCCValue(enc) require.NoError(t, err) + if len(dec.Value.RawBytes) == 0 { dec.Value.RawBytes = nil // normalize } + require.Equal(t, tc.val, dec) require.Equal(t, tc.val.IsTombstone(), dec.IsTombstone()) isTombstone, err := EncodedMVCCValueIsTombstone(enc) require.NoError(t, err) require.Equal(t, tc.val.IsTombstone(), isTombstone) - }) + + return buf.String() + })) } } diff --git a/pkg/storage/testdata/TestEncodeDecodeMVCCValue/bytes b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/bytes new file mode 100644 index 000000000000..958e22d7066b --- /dev/null +++ b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/bytes @@ -0,0 +1,3 @@ +echo +---- +encoded: 0000000003666f6f diff --git a/pkg/storage/testdata/TestEncodeDecodeMVCCValue/header_bytes b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/header_bytes new file mode 100644 index 000000000000..befbcc90dd14 --- /dev/null +++ b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/header_bytes @@ -0,0 +1,3 @@ +echo +---- +encoded: 00000004650a0208090000000003666f6f diff --git a/pkg/storage/testdata/TestEncodeDecodeMVCCValue/header_int b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/header_int new file mode 100644 index 000000000000..0f87c4d707e5 --- /dev/null +++ b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/header_int @@ -0,0 +1,3 @@ +echo +---- +encoded: 00000004650a020809000000000122 diff --git a/pkg/storage/testdata/TestEncodeDecodeMVCCValue/header_tombstone b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/header_tombstone new file mode 100644 index 000000000000..d5deaeff9b9d --- /dev/null +++ b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/header_tombstone @@ -0,0 +1,3 @@ +echo +---- +encoded: 00000004650a020809 diff --git a/pkg/storage/testdata/TestEncodeDecodeMVCCValue/int b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/int new file mode 100644 index 000000000000..7d5b8ea66a4f --- /dev/null +++ b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/int @@ -0,0 +1,3 @@ +echo +---- +encoded: 000000000122 diff --git a/pkg/storage/testdata/TestEncodeDecodeMVCCValue/tombstone b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/tombstone new file mode 100644 index 000000000000..2e9622259a77 --- /dev/null +++ b/pkg/storage/testdata/TestEncodeDecodeMVCCValue/tombstone @@ -0,0 +1,3 @@ +echo +---- +encoded: diff --git a/pkg/storage/testdata/TestMVCCLookupRangeKeyValue b/pkg/storage/testdata/TestMVCCLookupRangeKeyValue new file mode 100644 index 000000000000..139235fa508e --- /dev/null +++ b/pkg/storage/testdata/TestMVCCLookupRangeKeyValue @@ -0,0 +1,20 @@ +echo +---- +# a b c d +# t=4000 [-----v1----) +# t=2000 [-v2-)[-----v3----) +# +# ac-valid-full +lookup(["a","c") @ 4000) = v1 +# ac-valid-partial +lookup(["a\x00","b") @ 4000) = v1 +# ac-incompatible-fragments +lookup(["a","c") @ 2000) = value change at "b" in expected range deletion ["a","c") +# ac-ts-too-early +lookup(["a","b") @ 2000) = v2 +# ac-invalid-pre +lookup([/Min,"c") @ 4000) = gap [/Min,"a") in expected range deletion [/Min,"c") +# ac-invalid-post +lookup(["a","d") @ 4000) = gap ["c",...) in expected range deletion ["a","d") +# cd-invalid-post +lookup(["c","f") @ 2000) = gap ["d",...) in expected range deletion ["c","f") diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index eb8c9b5f3ae1..03999b9efcca 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1076,7 +1076,9 @@ func TestLint(t *testing.T) { ":!rpc/codec.go", ":!rpc/codec_test.go", ":!settings/settings_test.go", + ":!roachpb/api_requestheader.go", ":!storage/mvcc_value.go", + ":!storage/enginepb/mvcc3_valueheader.go", ":!sql/types/types_jsonpb.go", ":!sql/schemachanger/scplan/scviz/maps.go", ) @@ -2000,6 +2002,7 @@ func TestLint(t *testing.T) { "../../col/coldata", "../../keys", "../../kv/kvclient/rangecache", + "../../roachpb", "../../sql/catalog/descs", "../../sql/colcontainer", "../../sql/colconv", diff --git a/pkg/testutils/lint/passes/fmtsafe/functions.go b/pkg/testutils/lint/passes/fmtsafe/functions.go index 80b6e0fe5867..7d2e845c0c88 100644 --- a/pkg/testutils/lint/passes/fmtsafe/functions.go +++ b/pkg/testutils/lint/passes/fmtsafe/functions.go @@ -151,6 +151,9 @@ var requireConstFmt = map[string]bool{ "(github.com/cockroachdb/cockroach/pkg/sql/logictest/logictestbase.stdlogger).Fatalf": true, "(github.com/cockroachdb/cockroach/pkg/sql/logictest/logictestbase.stdlogger).Logf": true, + "github.com/cockroachdb/cockroach/pkg/kv/kvnemesis.l": true, + "(*github.com/cockroachdb/cockroach/pkg/kv/kvnemesis.logLogger).Logf": true, + // Error things are populated in the init() message. } diff --git a/pkg/util/buildutil/BUILD.bazel b/pkg/util/buildutil/BUILD.bazel index 4b8cc18cf7c3..562b6e9a2fde 100644 --- a/pkg/util/buildutil/BUILD.bazel +++ b/pkg/util/buildutil/BUILD.bazel @@ -36,7 +36,10 @@ go_test( srcs = ["crdb_test_test.go"], args = ["-test.timeout=295s"], embed = [":buildutil"], # keep - deps = ["@com_github_stretchr_testify//require"], + deps = [ + "//pkg/build/bazel", + "@com_github_stretchr_testify//require", + ], ) get_x_data(name = "get_x_data") diff --git a/pkg/util/buildutil/crdb_test_test.go b/pkg/util/buildutil/crdb_test_test.go index 784d986b1004..727185b5ae19 100644 --- a/pkg/util/buildutil/crdb_test_test.go +++ b/pkg/util/buildutil/crdb_test_test.go @@ -13,11 +13,14 @@ package buildutil import ( "testing" + "github.com/cockroachdb/cockroach/pkg/build/bazel" "github.com/stretchr/testify/require" ) func TestCrdbTestOn(t *testing.T) { // Sanity-check: make sure CrdbTestBuild is set. This should be true for - // any test. - require.True(t, CrdbTestBuild) + // any test built with bazel. + if bazel.BuiltWithBazel() { + require.True(t, CrdbTestBuild) + } }