diff --git a/pkg/kv/kvnemesis/validator.go b/pkg/kv/kvnemesis/validator.go index 5f44be2053d4..622962a88f9e 100644 --- a/pkg/kv/kvnemesis/validator.go +++ b/pkg/kv/kvnemesis/validator.go @@ -245,10 +245,10 @@ func (*observedScan) observedMarker() {} type validator struct { kvs *Engine - // Ops for the current atomic unit. This is reset between units, in - // checkAtomic, which then calls processOp (which might recurse owing - // to the existence of txn closures, batches, etc). - curOps []observedOp + // Observations for the current atomic unit. This is reset between units, in + // checkAtomic, which then calls processOp (which might recurse owing to the + // existence of txn closures, batches, etc). + curObservations []observedOp // 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 @@ -356,17 +356,17 @@ func (v *validator) processOp(buffering bool, op Operation) { case *GetOperation: v.failIfError(op, t.Result) if !buffering { - v.checkAtomic(`get`, t.Result, hlc.Timestamp{}, op) + v.checkAtomic(`get`, t.Result, op) } else { read := &observedRead{ Key: t.Key, Value: roachpb.Value{RawBytes: t.Result.Value}, } - v.curOps = append(v.curOps, read) + v.curObservations = append(v.curObservations, read) } case *PutOperation: if !buffering { - v.checkAtomic(`put`, t.Result, hlc.Timestamp{}, op) + v.checkAtomic(`put`, t.Result, op) } else { // Accumulate all the writes for this transaction. kv, ok := v.kvByValue[string(t.Value)] @@ -379,11 +379,11 @@ func (v *validator) processOp(buffering bool, op Operation) { if write.Materialized { write.Timestamp = kv.Key.Timestamp } - v.curOps = append(v.curOps, write) + v.curObservations = append(v.curObservations, write) } case *DeleteOperation: if !buffering { - v.checkAtomic(`delete`, t.Result, hlc.Timestamp{}, op) + 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 @@ -395,17 +395,17 @@ func (v *validator) processOp(buffering bool, op Operation) { // 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 `checkCommittedTxn(..)` where it looks + // specifically, in the function `checkAtomicCommitted(..)` where it looks // up a corresponding tombstone with `getDeleteForKey(..)`. write := &observedWrite{ Key: t.Key, Value: roachpb.Value{}, } - v.curOps = append(v.curOps, write) + v.curObservations = append(v.curObservations, write) } case *DeleteRangeOperation: if !buffering { - v.checkAtomic(`deleteRange`, t.Result, hlc.Timestamp{}, op) + 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 @@ -433,8 +433,8 @@ func (v *validator) processOp(buffering bool, op Operation) { } deleteOps[i] = write } - v.curOps = append(v.curOps, scan) - v.curOps = append(v.curOps, deleteOps...) + v.curObservations = append(v.curObservations, scan) + v.curObservations = append(v.curObservations, deleteOps...) } case *ScanOperation: v.failIfError(op, t.Result) @@ -443,7 +443,7 @@ func (v *validator) processOp(buffering bool, op Operation) { if t.Reverse { atomicScanType = `reverse scan` } - v.checkAtomic(atomicScanType, t.Result, hlc.Timestamp{}, op) + v.checkAtomic(atomicScanType, t.Result, op) } else { scan := &observedScan{ Span: roachpb.Span{ @@ -459,7 +459,7 @@ func (v *validator) processOp(buffering bool, op Operation) { Value: roachpb.Value{RawBytes: kv.Value}, } } - v.curOps = append(v.curOps, scan) + v.curObservations = append(v.curObservations, scan) } case *SplitOperation: execTimestampStrictlyOptional = true @@ -554,7 +554,7 @@ func (v *validator) processOp(buffering bool, op Operation) { if !resultIsRetryable(t.Result) { v.failIfError(op, t.Result) if !buffering { - v.checkAtomic(`batch`, t.Result, hlc.Timestamp{}, t.Ops...) + v.checkAtomic(`batch`, t.Result, t.Ops...) } else { for _, op := range t.Ops { v.processOp(buffering, op) @@ -566,15 +566,7 @@ func (v *validator) processOp(buffering bool, op Operation) { if t.CommitInBatch != nil { ops = append(ops, t.CommitInBatch.Ops...) } - var optOpsTimestamp hlc.Timestamp - if t.Result.Err == nil { - if t.Txn == nil { - v.failures = append(v.failures, errors.AssertionFailedf("missing transaction")) - break - } - optOpsTimestamp = t.Txn.WriteTimestamp - } - v.checkAtomic(`txn`, t.Result, optOpsTimestamp, ops...) + v.checkAtomic(`txn`, t.Result, ops...) default: panic(errors.AssertionFailedf(`unknown operation type: %T %v`, t, t)) } @@ -587,28 +579,40 @@ 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, optOpsTimestamp hlc.Timestamp, ops ...Operation, -) { +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) } - txnObservations := v.curOps - v.curOps = nil + observations := v.curObservations + v.curObservations = nil if result.Type != ResultType_Error { - v.checkCommittedTxn(`committed `+atomicType, txnObservations, optOpsTimestamp) + // 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() { + 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) { - v.checkAmbiguousTxn(`ambiguous `+atomicType, txnObservations) + v.checkAtomicAmbiguous(`ambiguous `+atomicType, observations) } else { - v.checkUncommittedTxn(`uncommitted `+atomicType, txnObservations) + v.checkAtomicUncommitted(`uncommitted `+atomicType, observations) } } -func (v *validator) checkCommittedTxn( +// checkAtomicCommitted verifies an atomic unit (i.e. single cmd, batch, or txn) that +// was successful. Its writes thus must be present, and (as is always the case, no +// matter the outcome) its reads must have been valid. +// +// The execution timestamp optOptsTimestamp is always present for operations that +// 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, ) { // The following works by verifying that there is at least one time at which @@ -699,10 +703,17 @@ func (v *validator) checkCommittedTxn( key := string(o.Key) v.committedDeletesForKey[key]++ if optOpsTimestamp.IsEmpty() { - // In the case that the delete is not in a transaction (or in an - // ambiguous transaction), we do not match it to a specific - // tombstone as we cannot be certain which tombstone resulted from - // this operation; hence, we leave the timestamp empty. + // 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 @@ -836,16 +847,18 @@ func (v *validator) checkCommittedTxn( } } -func (v *validator) checkAmbiguousTxn(atomicType string, txnObservations []observedOp) { +func (v *validator) checkAtomicAmbiguous(atomicType string, txnObservations []observedOp) { var somethingCommitted bool deletedKeysInTxn := make(map[string]int) var hadWrite bool + var maybeExecTS hlc.Timestamp 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)] { @@ -861,6 +874,13 @@ func (v *validator) checkAmbiguousTxn(atomicType string, txnObservations []obser // 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...), @@ -875,19 +895,16 @@ func (v *validator) checkAmbiguousTxn(atomicType string, txnObservations []obser // later deletes may show "committed delete missing write" errors. v.committedDeletesForKey[key]++ } - } else if !hadWrite { - // TODO(dan): Is it possible to receive an ambiguous read-only txn? Assume - // committed for now because the committed case has assertions about reads - // but the uncommitted case doesn't and this seems to work. - v.checkCommittedTxn(atomicType, txnObservations, hlc.Timestamp{}) - } else if somethingCommitted { - v.checkCommittedTxn(atomicType, txnObservations, hlc.Timestamp{}) + } else if !hadWrite || somethingCommitted { + v.checkAtomicCommitted(atomicType, txnObservations, maybeExecTS) } else { - v.checkUncommittedTxn(atomicType, txnObservations) + // This is a writing transaction but not a single one of its writes + // showed up in KV, so verify that it is uncommitted. + v.checkAtomicUncommitted(atomicType, txnObservations) } } -func (v *validator) checkUncommittedTxn(atomicType string, txnObservations []observedOp) { +func (v *validator) checkAtomicUncommitted(atomicType string, txnObservations []observedOp) { var failure string for _, observed := range txnObservations { if failure != `` { diff --git a/pkg/kv/kvnemesis/validator_test.go b/pkg/kv/kvnemesis/validator_test.go index ad1cc51e2470..34583daf18a1 100644 --- a/pkg/kv/kvnemesis/validator_test.go +++ b/pkg/kv/kvnemesis/validator_test.go @@ -29,39 +29,34 @@ var retryableError = roachpb.NewTransactionRetryWithProtoRefreshError( ``, uuid.MakeV4(), roachpb.Transaction{}) func withTimestamp(op Operation, ts int) Operation { - txn := roachpb.MakeTransaction( - "test", - nil, // baseKey - roachpb.NormalUserPriority, - hlc.Timestamp{WallTime: int64(ts)}, - 0, // maxOffsetNs - 0, // coordinatorNodeID - ) - switch o := op.GetValue().(type) { - case *ClosureTxnOperation: - o.Txn = &txn - default: - panic("incorrect op type") - } - + op.Result().OptionalTimestamp = hlc.Timestamp{WallTime: int64(ts)} return op } -func withResult(op Operation, err error) Operation { +func withResultTS(op Operation, ts int) Operation { + return withTimestamp(withResultOK(op), ts) +} + +func withResultOK(op Operation) Operation { + return withResult(op) +} + +func withResult(op Operation) Operation { + return withResultErr(op, nil /* err */) +} + +func withResultErr(op Operation, err error) Operation { *op.Result() = resultInit(context.Background(), err) - // A number of operations are expected to report an execution - // timestamp. Sometime down the road we want to use that timestamp - // for validation throughout, but currently we only use it for - // transactions, and only in a limited fashion. - // - // Until we rely on it, we set a bogus timestamp here that will be obvious - // when and if it should start causing errors. - op.Result().OptionalTimestamp = hlc.Timestamp{Logical: 987} + // 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, nil) + op = withResult(op) get := op.GetValue().(*GetOperation) get.Result.Type = ResultType_Value if value != `` { @@ -71,7 +66,7 @@ func withReadResult(op Operation, value string) Operation { } func withScanResult(op Operation, kvs ...KeyValue) Operation { - op = withResult(op, nil) + op = withResult(op) scan := op.GetValue().(*ScanOperation) scan.Result.Type = ResultType_Values scan.Result.Values = kvs @@ -79,7 +74,7 @@ func withScanResult(op Operation, kvs ...KeyValue) Operation { } func withDeleteRangeResult(op Operation, keys ...[]byte) Operation { - op = withResult(op, nil) + op = withResult(op) delRange := op.GetValue().(*DeleteRangeOperation) delRange.Result.Type = ResultType_Keys delRange.Result.Keys = keys @@ -144,85 +139,84 @@ func TestValidate(t *testing.T) { }, { name: "one put with expected write", - steps: []Step{step(withResult(put(`a`, `v1`), nil))}, + steps: []Step{step(withResult(put(`a`, `v1`)))}, kvs: kvs(kv(`a`, 1, `v1`)), expected: nil, }, { name: "one delete with expected write", - steps: []Step{step(withResult(del(`a`), nil))}, + steps: []Step{step(withResult(del(`a`)))}, kvs: kvs(tombstone(`a`, 1)), expected: nil, }, { name: "one put with missing write", - steps: []Step{step(withResult(put(`a`, `v1`), nil))}, + steps: []Step{step(withResult(put(`a`, `v1`)))}, kvs: nil, expected: []string{`committed put missing write: [w]"a":missing->v1`}, }, { name: "one delete with missing write", - steps: []Step{step(withResult(del(`a`), nil))}, + steps: []Step{step(withResult(del(`a`)))}, kvs: nil, expected: []string{`committed delete missing write: [d]"a":missing->`}, }, { name: "one ambiguous put with successful write", - steps: []Step{step(withResult(put(`a`, `v1`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, + steps: []Step{step(withResultErr(put(`a`, `v1`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, kvs: kvs(kv(`a`, 1, `v1`)), expected: nil, }, { name: "one ambiguous delete with successful write", - steps: []Step{step(withResult(del(`a`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, + 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 put with failed write", - steps: []Step{step(withResult(put(`a`, `v1`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, + steps: []Step{step(withResultErr(put(`a`, `v1`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, kvs: nil, expected: nil, }, { name: "one ambiguous delete with failed write", - steps: []Step{step(withResult(del(`a`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, + steps: []Step{step(withResultErr(del(`a`), roachpb.NewAmbiguousResultError(errors.New("boom"))))}, kvs: nil, expected: nil, }, { name: "one ambiguous delete with failed write before a later committed delete", steps: []Step{ - step(withResult(del(`a`), roachpb.NewAmbiguousResultError(errors.New("boom")))), - step(withResult(del(`a`), nil)), + step(withResultErr(del(`a`), roachpb.NewAmbiguousResultError(errors.New("boom")))), + step(withResultTS(del(`a`), 2)), }, - kvs: kvs(tombstone(`a`, 1)), + kvs: kvs(tombstone(`a`, 2)), expected: []string{ `unable to validate delete operations in ambiguous transactions: [d]"a":missing->`, - `committed delete missing write: [d]"a":missing->`, }, }, { name: "one retryable put with write (correctly) missing", - steps: []Step{step(withResult(put(`a`, `v1`), retryableError))}, + steps: []Step{step(withResultErr(put(`a`, `v1`), retryableError))}, kvs: nil, expected: nil, }, { name: "one retryable delete with write (correctly) missing", - steps: []Step{step(withResult(del(`a`), retryableError))}, + steps: []Step{step(withResultErr(del(`a`), retryableError))}, kvs: nil, expected: nil, }, { name: "one retryable put with write (incorrectly) present", - steps: []Step{step(withResult(put(`a`, `v1`), retryableError))}, + 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 delete with write (incorrectly) present", - steps: []Step{step(withResult(del(`a`), retryableError))}, + steps: []Step{step(withResultErr(del(`a`), retryableError))}, kvs: kvs(tombstone(`a`, 1)), // NB: Error messages are different because we can't match an uncommitted // delete op to a stored kv like above. @@ -231,39 +225,43 @@ func TestValidate(t *testing.T) { { name: "one delete with expected write after write transaction with shadowed delete", steps: []Step{ - step(withResult(del(`a`), nil)), - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v2`), nil), - withResult(del(`a`), nil), - withResult(put(`a`, `v3`), nil), - ), 3), nil)), - step(withResult(del(`a`), nil)), + step(withResultTS(del(`a`), 1)), + step(withResultTS(put(`a`, `v1`), 2)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResultOK(put(`a`, `v2`)), + withResultOK(del(`a`)), + withResultOK(put(`a`, `v3`)), + ), 3)), + step(withResultTS(del(`a`), 4)), }, - kvs: kvs(tombstone(`a`, 1), kv(`a`, 2, `v1`), kv(`a`, 3, `v3`), tombstone(`a`, 4)), + kvs: kvs( + tombstone(`a`, 1), + kv(`a`, 2, `v1`), + kv(`a`, 3, `v3`), + tombstone(`a`, 4)), expected: nil, }, { name: "one batch put with successful write", - steps: []Step{step(withResult(batch(withResult(put(`a`, `v1`), nil)), nil))}, + steps: []Step{step(withResult(batch(withResult(put(`a`, `v1`)))))}, kvs: kvs(kv(`a`, 1, `v1`)), expected: nil, }, { name: "one batch delete with successful write", - steps: []Step{step(withResult(batch(withResult(del(`a`), nil)), nil))}, + steps: []Step{step(withResult(batch(withResult(del(`a`)))))}, kvs: kvs(tombstone(`a`, 1)), expected: nil, }, { name: "one batch put with missing write", - steps: []Step{step(withResult(batch(withResult(put(`a`, `v1`), nil)), nil))}, + steps: []Step{step(withResult(batch(withResult(put(`a`, `v1`)))))}, kvs: nil, expected: []string{`committed batch missing write: [w]"a":missing->v1`}, }, { name: "one batch delete with missing write", - steps: []Step{step(withResult(batch(withResult(del(`a`), nil)), nil))}, + steps: []Step{step(withResult(batch(withResult(del(`a`)))))}, kvs: nil, expected: []string{`committed batch missing write: [d]"a":missing->`}, }, @@ -271,8 +269,8 @@ func TestValidate(t *testing.T) { name: "one transactionally committed put with the correct writes", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - ), 1), nil)), + withResult(put(`a`, `v1`)), + ), 1))), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: nil, @@ -281,8 +279,8 @@ func TestValidate(t *testing.T) { name: "one transactionally committed delete with the correct writes", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`), nil), - ), 1), nil)), + withResult(del(`a`)), + ), 1))), }, kvs: kvs(tombstone(`a`, 1)), expected: nil, @@ -291,9 +289,9 @@ func TestValidate(t *testing.T) { name: "one transactionally committed put with first write missing", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(put(`b`, `v2`), nil), - ), 1), nil)), + withResult(put(`a`, `v1`)), + withResult(put(`b`, `v2`)), + ), 1))), }, kvs: kvs(kv(`b`, 1, `v2`)), expected: []string{`committed txn missing write: [w]"a":missing->v1 [w]"b":0.000000001,0->v2`}, @@ -302,9 +300,9 @@ func TestValidate(t *testing.T) { name: "one transactionally committed delete with first write missing", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`), nil), - withResult(del(`b`), nil), - ), 1), nil)), + withResult(del(`a`)), + withResult(del(`b`)), + ), 1))), }, kvs: kvs(tombstone(`b`, 1)), expected: []string{`committed txn missing write: [d]"a":missing-> [d]"b":0.000000001,0->`}, @@ -313,9 +311,9 @@ func TestValidate(t *testing.T) { name: "one transactionally committed put with second write missing", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(put(`b`, `v2`), nil), - ), 1), nil)), + withResult(put(`a`, `v1`)), + withResult(put(`b`, `v2`)), + ), 1))), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: []string{`committed txn missing write: [w]"a":0.000000001,0->v1 [w]"b":missing->v2`}, @@ -324,9 +322,9 @@ func TestValidate(t *testing.T) { name: "one transactionally committed delete with second write missing", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`), nil), - withResult(del(`b`), nil), - ), 1), nil)), + withResult(del(`a`)), + withResult(del(`b`)), + ), 1))), }, kvs: kvs(tombstone(`a`, 1)), expected: []string{`committed txn missing write: [d]"a":0.000000001,0-> [d]"b":missing->`}, @@ -335,9 +333,9 @@ func TestValidate(t *testing.T) { name: "one transactionally committed put with write timestamp disagreement", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(put(`b`, `v2`), nil), - ), 1), nil)), + withResult(put(`a`, `v1`)), + withResult(put(`b`, `v2`)), + ), 1))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), expected: []string{ @@ -348,9 +346,9 @@ func TestValidate(t *testing.T) { name: "one transactionally committed delete with write timestamp disagreement", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`), nil), - withResult(del(`b`), nil), - ), 1), nil)), + withResult(del(`a`)), + withResult(del(`b`)), + ), 1))), }, kvs: kvs(tombstone(`a`, 1), tombstone(`b`, 2)), // NB: Error messages are different because we can't match an uncommitted @@ -362,8 +360,8 @@ func TestValidate(t *testing.T) { { name: "one transactionally rolled back put with write (correctly) missing", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Rollback, - withResult(put(`a`, `v1`), nil), + step(withResultErr(closureTxn(ClosureTxnType_Rollback, + withResult(put(`a`, `v1`)), ), errors.New(`rollback`))), }, kvs: nil, @@ -372,8 +370,8 @@ func TestValidate(t *testing.T) { { name: "one transactionally rolled back delete with write (correctly) missing", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Rollback, - withResult(del(`a`), nil), + step(withResultErr(closureTxn(ClosureTxnType_Rollback, + withResult(del(`a`)), ), errors.New(`rollback`))), }, kvs: nil, @@ -382,8 +380,8 @@ func TestValidate(t *testing.T) { { name: "one transactionally rolled back put with write (incorrectly) present", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Rollback, - withResult(put(`a`, `v1`), nil), + step(withResultErr(closureTxn(ClosureTxnType_Rollback, + withResult(put(`a`, `v1`)), ), errors.New(`rollback`))), }, kvs: kvs(kv(`a`, 1, `v1`)), @@ -392,8 +390,8 @@ func TestValidate(t *testing.T) { { name: "one transactionally rolled back delete with write (incorrectly) present", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Rollback, - withResult(del(`a`), nil), + step(withResultErr(closureTxn(ClosureTxnType_Rollback, + withResult(del(`a`)), ), errors.New(`rollback`))), }, kvs: kvs(tombstone(`a`, 1)), @@ -402,10 +400,10 @@ func TestValidate(t *testing.T) { { name: "one transactionally rolled back batch put with write (correctly) missing", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Rollback, + step(withResultErr(closureTxn(ClosureTxnType_Rollback, withResult(batch( - withResult(put(`a`, `v1`), nil), - ), nil), + withResult(put(`a`, `v1`)), + )), ), errors.New(`rollback`))), }, kvs: nil, @@ -414,10 +412,10 @@ func TestValidate(t *testing.T) { { name: "one transactionally rolled back batch delete with write (correctly) missing", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Rollback, + step(withResultErr(closureTxn(ClosureTxnType_Rollback, withResult(batch( - withResult(del(`a`), nil), - ), nil), + withResult(del(`a`)), + )), ), errors.New(`rollback`))), }, kvs: nil, @@ -427,9 +425,9 @@ func TestValidate(t *testing.T) { name: "two transactionally committed puts of the same key", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(put(`a`, `v2`), nil), - ), 1), nil)), + withResult(put(`a`, `v1`)), + withResult(put(`a`, `v2`)), + ), 1))), }, kvs: kvs(kv(`a`, 1, `v2`)), expected: nil, @@ -438,9 +436,9 @@ func TestValidate(t *testing.T) { name: "two transactionally committed deletes of the same key", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`), nil), - withResult(del(`a`), nil), - ), 1), nil)), + withResult(del(`a`)), + withResult(del(`a`)), + ), 1))), }, kvs: kvs(tombstone(`a`, 1)), expected: nil, @@ -449,9 +447,9 @@ func TestValidate(t *testing.T) { name: "two transactionally committed writes (put, delete) of the same key", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(del(`a`), nil), - ), 1), nil)), + withResult(put(`a`, `v1`)), + withResult(del(`a`)), + ), 1))), }, kvs: kvs(tombstone(`a`, 1)), expected: nil, @@ -460,9 +458,9 @@ func TestValidate(t *testing.T) { name: "two transactionally committed writes (delete, put) of the same key", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`), nil), - withResult(put(`a`, `v2`), nil), - ), 1), nil)), + withResult(del(`a`)), + withResult(put(`a`, `v2`)), + ), 1))), }, kvs: kvs(kv(`a`, 1, `v2`)), expected: nil, @@ -471,9 +469,9 @@ func TestValidate(t *testing.T) { name: "two transactionally committed puts of the same key with extra write", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(put(`a`, `v2`), nil), - ), 2), nil)), + withResult(put(`a`, `v1`)), + withResult(put(`a`, `v2`)), + ), 2))), }, // HACK: These should be the same timestamp. See the TODO in // watcher.processEvents. @@ -486,9 +484,9 @@ func TestValidate(t *testing.T) { name: "two transactionally committed deletes of the same key with extra write", steps: []Step{ step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`), nil), - withResult(del(`a`), nil), - ), 1), nil)), + withResult(del(`a`)), + withResult(del(`a`)), + ), 1))), }, // HACK: These should be the same timestamp. See the TODO in // watcher.processEvents. @@ -498,24 +496,26 @@ func TestValidate(t *testing.T) { { name: "two transactionally committed writes (put, delete) of the same key with extra write", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(del(`a`), nil), - ), 2), nil)), + 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{ - `committed txn overwritten key had write: [w]"a":0.000000001,0->v1 [d]"a":0.000000002,0->`, + // 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->`, }, }, { name: "ambiguous transaction committed", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(put(`b`, `v2`), nil), + step(withResultErr(closureTxn(ClosureTxnType_Commit, + withResult(put(`a`, `v1`)), + withResult(put(`b`, `v2`)), ), roachpb.NewAmbiguousResultError(errors.New("boom")))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 1, `v2`)), @@ -524,9 +524,9 @@ func TestValidate(t *testing.T) { { name: "ambiguous transaction with delete committed", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(del(`b`), nil), + 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)), @@ -540,9 +540,9 @@ func TestValidate(t *testing.T) { { name: "ambiguous transaction did not commit", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(put(`b`, `v2`), nil), + step(withResultErr(closureTxn(ClosureTxnType_Commit, + withResult(put(`a`, `v1`)), + withResult(put(`b`, `v2`)), ), roachpb.NewAmbiguousResultError(errors.New("boom")))), }, kvs: nil, @@ -551,9 +551,9 @@ func TestValidate(t *testing.T) { { name: "ambiguous transaction with delete did not commit", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(del(`b`), nil), + step(withResultErr(closureTxn(ClosureTxnType_Commit, + withResult(put(`a`, `v1`)), + withResult(del(`b`)), ), roachpb.NewAmbiguousResultError(errors.New("boom")))), }, kvs: nil, @@ -562,9 +562,9 @@ func TestValidate(t *testing.T) { { name: "ambiguous transaction committed but has validation error", steps: []Step{ - step(withResult(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(put(`b`, `v2`), nil), + step(withResultErr(closureTxn(ClosureTxnType_Commit, + withResult(put(`a`, `v1`)), + withResult(put(`b`, `v2`)), ), roachpb.NewAmbiguousResultError(errors.New("boom")))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), @@ -575,9 +575,9 @@ func TestValidate(t *testing.T) { { name: "ambiguous transaction with delete committed but has validation error", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(del(`b`), nil), + step(withResultErr(withTimestamp(closureTxn(ClosureTxnType_Commit, + withResult(put(`a`, `v1`)), + withResult(del(`b`)), ), 2), roachpb.NewAmbiguousResultError(errors.New("boom")))), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`b`, 2)), @@ -593,7 +593,7 @@ func TestValidate(t *testing.T) { name: "one read before write", steps: []Step{ step(withReadResult(get(`a`), ``)), - step(withResult(put(`a`, `v1`), nil)), + step(withResult(put(`a`, `v1`))), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: nil, @@ -602,7 +602,7 @@ func TestValidate(t *testing.T) { name: "one read before delete", steps: []Step{ step(withReadResult(get(`a`), ``)), - step(withResult(del(`a`), nil)), + step(withResult(del(`a`))), }, kvs: kvs(tombstone(`a`, 1)), expected: nil, @@ -611,8 +611,8 @@ func TestValidate(t *testing.T) { name: "one read before write and delete", steps: []Step{ step(withReadResult(get(`a`), ``)), - step(withResult(put(`a`, `v1`), nil)), - step(withResult(del(`a`), nil)), + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(del(`a`), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), expected: nil, @@ -621,7 +621,7 @@ func TestValidate(t *testing.T) { name: "one read before write returning wrong value", steps: []Step{ step(withReadResult(get(`a`), `v2`)), - step(withResult(put(`a`, `v1`), nil)), + step(withResult(put(`a`, `v1`))), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: []string{ @@ -631,7 +631,7 @@ func TestValidate(t *testing.T) { { name: "one read after write", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), + step(withResult(put(`a`, `v1`))), step(withReadResult(get(`a`), `v1`)), }, kvs: kvs(kv(`a`, 1, `v1`)), @@ -640,9 +640,9 @@ func TestValidate(t *testing.T) { { name: "one read after write and delete", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(del(`a`), nil)), - step(withReadResult(get(`a`), `v1`)), + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(withTimestamp(del(`a`), 2), 2)), + step(withResultTS(withReadResult(get(`a`), `v1`), 1)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), expected: nil, @@ -650,8 +650,8 @@ func TestValidate(t *testing.T) { { name: "one read after write and delete returning tombstone", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(del(`a`), nil)), + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(del(`a`), 2)), step(withReadResult(get(`a`), ``)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), @@ -660,7 +660,7 @@ func TestValidate(t *testing.T) { { name: "one read after write returning wrong value", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), + step(withResult(put(`a`, `v1`))), step(withReadResult(get(`a`), `v2`)), }, kvs: kvs(kv(`a`, 1, `v1`)), @@ -671,9 +671,9 @@ func TestValidate(t *testing.T) { { name: "one read in between writes", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), + step(withResultTS(put(`a`, `v1`), 1)), step(withReadResult(get(`a`), `v1`)), - step(withResult(put(`a`, `v2`), nil)), + step(withResultTS(put(`a`, `v2`), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`)), expected: nil, @@ -681,9 +681,9 @@ func TestValidate(t *testing.T) { { name: "one read in between write and delete", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), + step(withResultTS(put(`a`, `v1`), 1)), step(withReadResult(get(`a`), `v1`)), - step(withResult(del(`a`), nil)), + step(withResultTS(del(`a`), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), expected: nil, @@ -691,13 +691,13 @@ func TestValidate(t *testing.T) { { name: "batch of reads after writes", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`), ``), - ), nil)), + ))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), expected: nil, @@ -705,15 +705,15 @@ func TestValidate(t *testing.T) { { name: "batch of reads after writes and deletes", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(del(`a`), nil)), - step(withResult(del(`b`), nil)), + 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`), ``), - ), nil)), + ))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 4)), expected: nil, @@ -721,15 +721,15 @@ func TestValidate(t *testing.T) { { name: "batch of reads after writes and deletes returning tombstones", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(del(`a`), nil)), - step(withResult(del(`b`), nil)), + 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`), ``), - ), nil)), + ))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 4)), expected: nil, @@ -737,13 +737,13 @@ func TestValidate(t *testing.T) { { name: "batch of reads after writes returning wrong values", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`), - ), nil)), + ))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), expected: []string{ @@ -754,15 +754,15 @@ func TestValidate(t *testing.T) { { name: "batch of reads after writes and deletes returning wrong values", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(del(`a`), nil)), - step(withResult(del(`b`), nil)), + 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`), - ), nil)), + ))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 4)), expected: []string{ @@ -773,13 +773,13 @@ func TestValidate(t *testing.T) { { name: "batch of reads after writes with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`), ``), - ), nil)), + ))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), expected: []string{ @@ -790,15 +790,15 @@ func TestValidate(t *testing.T) { { name: "batch of reads after writes and deletes with valid time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(del(`a`), nil)), - step(withResult(del(`b`), nil)), + 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`), ``), - ), nil)), + ))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), tombstone(`b`, 4)), expected: nil, @@ -806,14 +806,14 @@ func TestValidate(t *testing.T) { { name: "transactional reads with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(put(`b`, `v3`), nil)), - step(withResult(put(`b`, `v4`), nil)), + 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), nil)), + ), 3))), }, // 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`)), @@ -822,15 +822,15 @@ func TestValidate(t *testing.T) { { name: "transactional reads after writes and deletes with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(del(`a`), nil)), - step(withResult(del(`b`), nil)), + 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), nil)), + ), 4))), }, // 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)), @@ -839,14 +839,14 @@ func TestValidate(t *testing.T) { { name: "transactional reads with empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(put(`b`, `v3`), nil)), - step(withResult(put(`b`, `v4`), nil)), + 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), nil)), + ), 3))), }, // 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`)), @@ -858,17 +858,17 @@ func TestValidate(t *testing.T) { { name: "transactional reads after writes and deletes with empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`), nil), - withResult(del(`b`), nil), - ), 3), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`b`, `v2`), 2)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResultOK(del(`a`)), + withResultOK(del(`b`)), + ), 3)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), ``), withReadResult(get(`b`), `v2`), withReadResult(get(`c`), ``), - ), 4), nil)), + ), 4)), }, // 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)), @@ -880,14 +880,14 @@ func TestValidate(t *testing.T) { { name: "transactional reads and deletes after write with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), `v1`), - withResult(del(`a`), nil), + withResult(del(`a`)), withReadResult(get(`a`), ``), - ), 2), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(del(`a`), nil)), + ), 2)), + step(withResultTS(put(`a`, `v2`), 3)), + step(withResultTS(del(`a`), 4)), }, // 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)), @@ -896,14 +896,14 @@ func TestValidate(t *testing.T) { { name: "transactional reads and deletes after write with empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResult(put(`a`, `v1`))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), ``), - withResult(del(`a`), nil), + withResult(del(`a`)), withReadResult(get(`a`), ``), - ), 2), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(del(`a`), nil)), + ), 2)), + step(withResultTS(put(`a`, `v2`), 3)), + step(withResultTS(del(`a`), 4)), }, // 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)), @@ -915,13 +915,13 @@ func TestValidate(t *testing.T) { { name: "transactional reads one missing with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(put(`b`, `v3`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`a`, `v2`), 2)), + step(withResultTS(put(`b`, `v3`), 2)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), `v1`), withReadResult(get(`b`), ``), - ), 1), nil)), + ), 1)), }, // 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`)), @@ -930,13 +930,13 @@ func TestValidate(t *testing.T) { { name: "transactional reads one missing with empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(put(`b`, `v3`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`a`, `v2`), 2)), + step(withResultTS(put(`b`, `v3`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), `v1`), withReadResult(get(`b`), ``), - ), 1), nil)), + ), 1)), }, // 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`)), @@ -948,12 +948,12 @@ func TestValidate(t *testing.T) { { name: "transactional read and write with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`a`, `v2`), 3)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), `v1`), - withResult(put(`b`, `v3`), nil), - ), 2), nil)), + withResult(put(`b`, `v3`)), + ), 2)), }, // 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`)), @@ -962,12 +962,12 @@ func TestValidate(t *testing.T) { { name: "transactional read and write with empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`a`, `v2`), 2)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), `v1`), - withResult(put(`b`, `v3`), nil), - ), 2), nil)), + withResultOK(put(`b`, `v3`)), + ), 2)), }, // 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`)), @@ -979,11 +979,11 @@ func TestValidate(t *testing.T) { { name: "transaction with read before and after write", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), ``), - withResult(put(`a`, `v1`), nil), + withResult(put(`a`, `v1`)), withReadResult(get(`a`), `v1`), - ), 1), nil)), + ), 1)), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: nil, @@ -991,12 +991,12 @@ func TestValidate(t *testing.T) { { name: "transaction with read before and after delete", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), `v1`), - withResult(del(`a`), nil), + withResult(del(`a`)), withReadResult(get(`a`), ``), - ), 2), nil)), + ), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), expected: nil, @@ -1004,11 +1004,11 @@ func TestValidate(t *testing.T) { { name: "transaction with incorrect read before write", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), `v1`), - withResult(put(`a`, `v1`), nil), + withResult(put(`a`, `v1`)), withReadResult(get(`a`), `v1`), - ), 1), nil)), + ), 1)), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: []string{ @@ -1019,12 +1019,12 @@ func TestValidate(t *testing.T) { { name: "transaction with incorrect read before delete", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), ``), - withResult(del(`a`), nil), + withResult(del(`a`)), withReadResult(get(`a`), ``), - ), 2), nil)), + ), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), expected: []string{ @@ -1035,11 +1035,11 @@ func TestValidate(t *testing.T) { { name: "transaction with incorrect read after write", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), ``), - withResult(put(`a`, `v1`), nil), + withResult(put(`a`, `v1`)), withReadResult(get(`a`), ``), - ), 1), nil)), + ), 1)), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: []string{ @@ -1050,12 +1050,12 @@ func TestValidate(t *testing.T) { { name: "transaction with incorrect read after delete", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), `v1`), - withResult(del(`a`), nil), + withResultOK(del(`a`)), withReadResult(get(`a`), `v1`), - ), 2), nil)), + ), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), expected: []string{ @@ -1066,13 +1066,13 @@ func TestValidate(t *testing.T) { { name: "two transactionally committed puts of the same key with reads", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), ``), - withResult(put(`a`, `v1`), nil), + withResult(put(`a`, `v1`)), withReadResult(get(`a`), `v1`), - withResult(put(`a`, `v2`), nil), + withResult(put(`a`, `v2`)), withReadResult(get(`a`), `v2`), - ), 1), nil)), + ), 1)), }, kvs: kvs(kv(`a`, 1, `v2`)), expected: nil, @@ -1080,13 +1080,13 @@ func TestValidate(t *testing.T) { { name: "two transactionally committed put/delete ops of the same key with reads", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), ``), - withResult(put(`a`, `v1`), nil), + withResult(put(`a`, `v1`)), withReadResult(get(`a`), `v1`), - withResult(del(`a`), nil), + withResult(del(`a`)), withReadResult(get(`a`), ``), - ), 1), nil)), + ), 1)), }, kvs: kvs(tombstone(`a`, 1)), expected: nil, @@ -1094,13 +1094,13 @@ func TestValidate(t *testing.T) { { name: "two transactionally committed put/delete ops of the same key with incorrect read", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, withReadResult(get(`a`), ``), - withResult(put(`a`, `v1`), nil), + withResult(put(`a`, `v1`)), withReadResult(get(`a`), `v1`), - withResult(del(`a`), nil), + withResult(del(`a`)), withReadResult(get(`a`), `v1`), - ), 1), nil)), + ), 1)), }, kvs: kvs(tombstone(`a`, 1)), expected: []string{ @@ -1111,9 +1111,9 @@ func TestValidate(t *testing.T) { { name: "one transactional put with correct commit time", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - ), 1), nil)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(`a`, `v1`)), + ), 1)), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: nil, @@ -1121,9 +1121,9 @@ func TestValidate(t *testing.T) { { name: "one transactional put with incorrect commit time", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - ), 1), nil)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(`a`, `v1`)), + ), 1)), }, kvs: kvs(kv(`a`, 2, `v1`)), expected: []string{ @@ -1134,11 +1134,11 @@ func TestValidate(t *testing.T) { 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(withResult(del(`a`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`b`, `v1`), nil), - withResult(del(`a`), nil), - ), 2), nil)), + step(withResultTS(del(`a`), 3)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(`b`, `v1`)), + withResult(del(`a`)), + ), 2)), }, 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, @@ -1149,12 +1149,12 @@ func TestValidate(t *testing.T) { { name: "two transactional deletes with out of order commit times", steps: []Step{ - step(withResult(del(`a`), nil)), - step(withResult(del(`b`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(del(`a`), nil), - withResult(del(`b`), nil), - ), 1), nil)), + step(withResultTS(del(`a`), 2)), + step(withResultTS(del(`b`), 3)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(del(`a`)), + withResult(del(`b`)), + ), 1)), }, 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, @@ -1165,12 +1165,12 @@ func TestValidate(t *testing.T) { { name: "one transactional scan followed by delete within time range", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(del(`a`), nil), - ), 2), nil)), - step(withResult(put(`b`, `v2`), nil)), + withResult(del(`a`)), + ), 2)), + step(withResultTS(put(`b`, `v2`), 3)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), kv(`b`, 3, `v2`)), expected: nil, @@ -1178,12 +1178,12 @@ func TestValidate(t *testing.T) { { name: "one transactional scan followed by delete outside time range", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(del(`a`), nil), - ), 4), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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{ @@ -1195,7 +1195,7 @@ func TestValidate(t *testing.T) { name: "one scan before write", steps: []Step{ step(withScanResult(scan(`a`, `c`))), - step(withResult(put(`a`, `v1`), nil)), + step(withResult(put(`a`, `v1`))), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: nil, @@ -1204,7 +1204,7 @@ func TestValidate(t *testing.T) { name: "one scan before write returning wrong value", steps: []Step{ step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v2`))), - step(withResult(put(`a`, `v1`), nil)), + step(withResult(put(`a`, `v1`))), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: []string{ @@ -1215,7 +1215,7 @@ func TestValidate(t *testing.T) { { name: "one scan after write", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), + step(withResult(put(`a`, `v1`))), step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`))), }, kvs: kvs(kv(`a`, 1, `v1`)), @@ -1224,7 +1224,7 @@ func TestValidate(t *testing.T) { { name: "one scan after write returning wrong value", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), + step(withResult(put(`a`, `v1`))), step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v2`))), }, kvs: kvs(kv(`a`, 1, `v1`)), @@ -1236,8 +1236,8 @@ func TestValidate(t *testing.T) { { name: "one scan after writes", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`b`, `v2`), 2)), step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`b`, `v2`))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), @@ -1246,8 +1246,8 @@ func TestValidate(t *testing.T) { { name: "one reverse scan after writes", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`b`, `v2`), 2)), step(withScanResult(reverseScan(`a`, `c`), scanKV(`b`, `v2`), scanKV(`a`, `v1`))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), @@ -1256,10 +1256,10 @@ func TestValidate(t *testing.T) { { name: "one scan after writes and delete", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(del(`a`), nil)), - step(withResult(put(`a`, `v3`), nil)), + 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`))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`), tombstone(`a`, 3), kv(`a`, 4, `v3`)), @@ -1268,8 +1268,8 @@ func TestValidate(t *testing.T) { { name: "one scan after write returning extra key", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`)), @@ -1281,11 +1281,11 @@ func TestValidate(t *testing.T) { { name: "one tranactional scan after write and delete returning extra key", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`b`, `v2`), nil), - withResult(del(`a`), nil), - ), 2), nil)), + step(withResultTS(put(`a`, `v1`), 1)), + 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`)), @@ -1297,8 +1297,8 @@ func TestValidate(t *testing.T) { { name: "one reverse scan after write returning extra key", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`)), @@ -1310,8 +1310,8 @@ func TestValidate(t *testing.T) { { name: "one scan after write returning missing key", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`)), @@ -1323,16 +1323,16 @@ func TestValidate(t *testing.T) { { name: "one scan after writes and delete returning missing key", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`a`, `v1`), nil), - withResult(put(`b`, `v2`), nil), - ), 1), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(`a`, `v1`)), + withResult(put(`b`, `v2`)), + ), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`b`, `v2`)), - withResult(del(`a`), nil), - ), 2), nil)), - step(withResult(put(`a`, `v3`), nil)), - step(withResult(del(`a`), nil)), + 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{ @@ -1343,8 +1343,8 @@ func TestValidate(t *testing.T) { { name: "one reverse scan after write returning missing key", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`)), @@ -1356,8 +1356,8 @@ func TestValidate(t *testing.T) { { name: "one scan after writes returning results in wrong order", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`)), @@ -1369,8 +1369,8 @@ func TestValidate(t *testing.T) { { name: "one reverse scan after writes returning results in wrong order", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`)), @@ -1382,9 +1382,9 @@ func TestValidate(t *testing.T) { { name: "one scan after writes returning results outside scan boundary", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(put(`c`, `v3`), nil)), + 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`)), @@ -1396,9 +1396,9 @@ func TestValidate(t *testing.T) { { name: "one reverse scan after writes returning results outside scan boundary", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(put(`c`, `v3`), nil)), + 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`)), @@ -1410,9 +1410,9 @@ func TestValidate(t *testing.T) { { name: "one scan in between writes", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), + step(withResultTS(put(`a`, `v1`), 1)), step(withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`))), - step(withResult(put(`a`, `v2`), nil)), + step(withResultTS(put(`a`, `v2`), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`a`, 2, `v2`)), expected: nil, @@ -1420,13 +1420,13 @@ func TestValidate(t *testing.T) { { name: "batch of scans after writes", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`)), - ), nil)), + ))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), expected: nil, @@ -1434,13 +1434,13 @@ func TestValidate(t *testing.T) { { name: "batch of scans after writes returning wrong values", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`)), - ), nil)), + ))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), expected: []string{ @@ -1453,13 +1453,13 @@ func TestValidate(t *testing.T) { { name: "batch of scans after writes with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), + 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`)), - ), nil)), + ))), }, kvs: kvs(kv(`a`, 1, `v1`), kv(`b`, 2, `v2`)), expected: []string{ @@ -1472,14 +1472,14 @@ func TestValidate(t *testing.T) { { name: "transactional scans with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(put(`b`, `v3`), nil)), - step(withResult(put(`b`, `v4`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + 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(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`b`, `v3`)), withScanResult(scan(`b`, `d`), scanKV(`b`, `v3`)), - ), 2), nil)), + ), 2)), }, // 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`)), @@ -1488,15 +1488,15 @@ func TestValidate(t *testing.T) { { name: "transactional scans after delete with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(put(`b`, `v3`), nil)), - step(withResult(del(`b`), nil)), - step(withResult(put(`b`, `v4`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + 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(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), withScanResult(scan(`b`, `d`)), - ), 2), nil)), + ), 2)), }, // 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`)), @@ -1505,14 +1505,14 @@ func TestValidate(t *testing.T) { { name: "transactional scans with empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(put(`b`, `v3`), nil)), - step(withResult(put(`b`, `v4`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + 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(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`), scanKV(`b`, `v3`)), withScanResult(scan(`b`, `d`), scanKV(`b`, `v3`)), - ), 2), nil)), + ), 2)), }, // 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`)), @@ -1525,14 +1525,14 @@ func TestValidate(t *testing.T) { { name: "transactional scans after delete with empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(put(`b`, `v3`), nil)), - step(withResult(del(`b`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + 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(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), withScanResult(scan(`b`, `d`)), - ), 3), nil)), + ), 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)), @@ -1545,13 +1545,13 @@ func TestValidate(t *testing.T) { { name: "transactional scans one missing with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(put(`b`, `v3`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`a`, `v2`), 2)), + step(withResultTS(put(`b`, `v3`), 2)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), withScanResult(scan(`b`, `d`)), - ), 2), nil)), + ), 2)), }, // 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`)), @@ -1560,13 +1560,13 @@ func TestValidate(t *testing.T) { { name: "transactional scans one missing with empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(put(`b`, `v3`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`a`, `v2`), 2)), + step(withResultTS(put(`b`, `v3`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), withScanResult(scan(`b`, `d`)), - ), 1), nil)), + ), 1)), }, // 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`)), @@ -1579,12 +1579,12 @@ func TestValidate(t *testing.T) { { name: "transactional scan and write with non-empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`a`, `v2`), 3)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(put(`b`, `v3`), nil), - ), 2), nil)), + withResult(put(`b`, `v3`)), + ), 2)), }, // 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`)), @@ -1593,12 +1593,12 @@ func TestValidate(t *testing.T) { { name: "transactional scan and write with empty time overlap", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`a`, `v2`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`a`, `v2`), 2)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(put(`b`, `v3`), nil), - ), 2), nil)), + withResult(put(`b`, `v3`)), + ), 2)), }, // 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`)), @@ -1610,11 +1610,11 @@ func TestValidate(t *testing.T) { { name: "transaction with scan before and after write", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`)), - withResult(put(`a`, `v1`), nil), + withResult(put(`a`, `v1`)), withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - ), 1), nil)), + ), 1)), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: nil, @@ -1622,11 +1622,11 @@ func TestValidate(t *testing.T) { { name: "transaction with incorrect scan before write", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(put(`a`, `v1`), nil), + withResult(put(`a`, `v1`)), withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - ), 1), nil)), + ), 1)), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: []string{ @@ -1639,11 +1639,11 @@ func TestValidate(t *testing.T) { { name: "transaction with incorrect scan after write", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`)), - withResult(put(`a`, `v1`), nil), + withResult(put(`a`, `v1`)), withScanResult(scan(`a`, `c`)), - ), 1), nil)), + ), 1)), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: []string{ @@ -1654,15 +1654,15 @@ func TestValidate(t *testing.T) { { name: "two transactionally committed puts of the same key with scans", steps: []Step{ - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(closureTxn(ClosureTxnType_Commit, withScanResult(scan(`a`, `c`)), - withResult(put(`a`, `v1`), nil), + withResult(put(`a`, `v1`)), withScanResult(scan(`a`, `c`), scanKV(`a`, `v1`)), - withResult(put(`a`, `v2`), nil), + withResult(put(`a`, `v2`)), withScanResult(scan(`a`, `c`), scanKV(`a`, `v2`)), - withResult(put(`b`, `v3`), nil), + withResult(put(`b`, `v3`)), withScanResult(scan(`a`, `c`), scanKV(`a`, `v2`), scanKV(`b`, `v3`)), - ), 1), nil)), + ), 1)), }, kvs: kvs(kv(`a`, 1, `v2`), kv(`b`, 1, `v3`)), expected: nil, @@ -1671,7 +1671,7 @@ func TestValidate(t *testing.T) { name: "one deleterange before write", steps: []Step{ step(withDeleteRangeResult(delRange(`a`, `c`))), - step(withResult(put(`a`, `v1`), nil)), + step(withResult(put(`a`, `v1`))), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: nil, @@ -1680,7 +1680,7 @@ func TestValidate(t *testing.T) { name: "one deleterange before write returning wrong value", steps: []Step{ step(withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`))), - step(withResult(put(`a`, `v1`), nil)), + step(withResult(put(`a`, `v1`))), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: []string{ @@ -1692,10 +1692,10 @@ func TestValidate(t *testing.T) { { name: "one deleterange after write", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`)), - ), 2), nil)), + ), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), expected: nil, @@ -1703,10 +1703,10 @@ func TestValidate(t *testing.T) { { name: "one deleterange after write returning wrong value", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResult(put(`a`, `v1`))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`)), - ), 2), nil)), + ), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2)), expected: []string{ @@ -1716,10 +1716,10 @@ func TestValidate(t *testing.T) { { name: "one deleterange after write missing write", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResult(put(`a`, `v1`))), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`)), - ), 1), nil)), + ), 1)), }, kvs: kvs(kv(`a`, 1, `v1`)), expected: []string{ @@ -1731,12 +1731,12 @@ func TestValidate(t *testing.T) { { name: "one deleterange after writes", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(put(`c`, `v3`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`b`, `v2`), 2)), + step(withResultTS(put(`c`, `v3`), 3)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 4), nil)), + ), 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`, 4)), @@ -1745,12 +1745,12 @@ func TestValidate(t *testing.T) { { name: "one deleterange after writes with write timestamp disagreement", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(put(`c`, `v3`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`b`, `v2`), 2)), + step(withResultTS(put(`c`, `v3`), 3)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 4), nil)), + ), 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)), @@ -1763,12 +1763,12 @@ func TestValidate(t *testing.T) { { name: "one deleterange after writes with missing write", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(put(`c`, `v3`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`b`, `v2`), 2)), + step(withResultTS(put(`c`, `v3`), 3)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 4), nil)), + ), 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)), @@ -1782,13 +1782,13 @@ func TestValidate(t *testing.T) { { name: "one deleterange after writes and delete", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`b`, `v2`), nil)), - step(withResult(del(`a`), nil)), - step(withResult(put(`a`, `v3`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + 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(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 3), nil)), + ), 3)), }, 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, @@ -1796,11 +1796,11 @@ func TestValidate(t *testing.T) { { name: "one transactional deleterange followed by put after writes", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`)), - withResult(put(`b`, `v2`), nil), - ), 2), nil)), + withResult(put(`b`, `v2`)), + ), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), kv(`b`, 2, `v2`)), expected: nil, @@ -1808,11 +1808,11 @@ func TestValidate(t *testing.T) { { name: "one transactional deleterange followed by put after writes with write timestamp disagreement", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`)), - withResult(put(`b`, `v2`), nil), - ), 2), nil)), + withResult(put(`b`, `v2`)), + ), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), kv(`b`, 3, `v2`)), expected: []string{ @@ -1824,11 +1824,11 @@ func TestValidate(t *testing.T) { { name: "one transactional put shadowed by deleterange after writes", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`b`, `v2`), nil), + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(`b`, `v2`)), withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 2), nil)), + ), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), tombstone(`b`, 2)), expected: nil, @@ -1836,11 +1836,11 @@ func TestValidate(t *testing.T) { { name: "one transactional put shadowed by deleterange after writes with write timestamp disagreement", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, - withResult(put(`b`, `v2`), nil), + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, + withResult(put(`b`, `v2`)), withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`b`)), - ), 2), nil)), + ), 2)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 2), tombstone(`b`, 3)), expected: []string{ @@ -1853,11 +1853,11 @@ func TestValidate(t *testing.T) { { name: "one deleterange after writes returning keys outside span boundary", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`d`, `v2`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`d`, `v2`), 2)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`d`)), - ), 3), nil)), + ), 3)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 3), kv(`d`, 2, `v2`)), expected: []string{ @@ -1869,11 +1869,11 @@ func TestValidate(t *testing.T) { { name: "one deleterange after writes incorrectly deleting keys outside span boundary", steps: []Step{ - step(withResult(put(`a`, `v1`), nil)), - step(withResult(put(`d`, `v2`), nil)), - step(withResult(withTimestamp(closureTxn(ClosureTxnType_Commit, + step(withResultTS(put(`a`, `v1`), 1)), + step(withResultTS(put(`d`, `v2`), 2)), + step(withResultTS(closureTxn(ClosureTxnType_Commit, withDeleteRangeResult(delRange(`a`, `c`), roachpb.Key(`a`), roachpb.Key(`d`)), - ), 3), nil)), + ), 3)), }, kvs: kvs(kv(`a`, 1, `v1`), tombstone(`a`, 3), kv(`d`, 2, `v2`), tombstone(`d`, 3)), expected: []string{ diff --git a/pkg/workload/cli/run.go b/pkg/workload/cli/run.go index 2f76776ae22b..f741cb5dc1c7 100644 --- a/pkg/workload/cli/run.go +++ b/pkg/workload/cli/run.go @@ -425,7 +425,7 @@ func runRun(gen workload.Generator, urls []string, dbName string) error { var ops workload.QueryLoad prepareStart := timeutil.Now() log.Infof(ctx, "creating load generator...") - const prepareTimeout = 60 * time.Minute + const prepareTimeout = 90 * time.Minute prepareCtx, cancel := context.WithTimeout(ctx, prepareTimeout) defer cancel() if prepareErr := func(ctx context.Context) error {