diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 5f8b9b34398a..cf3f578704fb 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -719,9 +719,9 @@ func newMVCCIterator( reader Reader, timestamp hlc.Timestamp, rangeKeyMasking bool, opts IterOptions, ) MVCCIterator { // If reading inline then just return a plain MVCCIterator without intents. - // We also disable range keys, since they're not allowed across inline values. + // However, we allow the caller to enable range keys, since they may be needed + // for conflict checks when writing inline values. if timestamp.IsEmpty() { - opts.KeyTypes = IterKeyTypePointsOnly return reader.NewMVCCIterator(MVCCKeyIterKind, opts) } // Enable range key masking if requested. @@ -840,6 +840,20 @@ func mvccGet( return value, intent, nil } +// TODO(erikgrinaker): This is temporary until mvccGet always uses point +// synthesis (which requires read-path optimizations). +func mvccGetWithPointSynthesis( + ctx context.Context, + iter MVCCIterator, + key roachpb.Key, + timestamp hlc.Timestamp, + opts MVCCGetOptions, +) (value optionalValue, intent *roachpb.Intent, err error) { + pointIter := newPointSynthesizingIter(iter, true /* emitOnSeekGE */) + defer pointIter.release() // NB: not Close(), since we don't own iter + return mvccGet(ctx, pointIter, key, timestamp, opts) +} + // MVCCGetAsTxn constructs a temporary transaction from the given transaction // metadata and calls MVCCGet as that transaction. This method is required // only for reading intents of a transaction when only its metadata is known @@ -874,57 +888,91 @@ func MVCCGetAsTxn( // that is the usual contribution of the meta key). The value size returned // will be zero, as there is no stored MVCCMetadata. // ok is set to true. -// The passed in MVCCMetadata must not be nil. +// The passed in MVCCMetadata must not be nil. Any MVCC range tombstones will be +// treated like point tombstones. // // If the supplied iterator is nil, no seek operation is performed. This is // used by the Blind{Put,ConditionalPut} operations to avoid seeking when the -// metadata is known not to exist. If iterAlreadyPositioned is true, the -// iterator has already been seeked to metaKey, so a wasteful seek can be -// avoided. +// metadata is known not to exist. func mvccGetMetadata( - iter MVCCIterator, metaKey MVCCKey, iterAlreadyPositioned bool, meta *enginepb.MVCCMetadata, + iter MVCCIterator, metaKey MVCCKey, meta *enginepb.MVCCMetadata, ) (ok bool, keyBytes, valBytes int64, err error) { if iter == nil { return false, 0, 0, nil } - if !iterAlreadyPositioned { - iter.SeekGE(metaKey) - } + iter.SeekGE(metaKey) if ok, err = iter.Valid(); !ok { return false, 0, 0, err } - unsafeKey := iter.UnsafeKey() if !unsafeKey.Key.Equal(metaKey.Key) { return false, 0, 0, nil } - if !unsafeKey.IsValue() { + hasPoint, hasRange := iter.HasPointAndRange() + + // Check for existing intent metadata. Intents will be emitted colocated with + // a covering range key when seeking to it, so we don't need to handle range + // keys here. + if hasPoint && !unsafeKey.IsValue() { if err := iter.ValueProto(meta); err != nil { return false, 0, 0, err } - return true, int64(unsafeKey.EncodedSize()), - int64(len(iter.UnsafeValue())), nil + return true, int64(unsafeKey.EncodedSize()), int64(len(iter.UnsafeValue())), nil } + // Synthesize point key metadata. For values, the size of keys is always + // accounted for as MVCCVersionTimestampSize. The size of the metadata key is + // accounted for separately. + meta.Reset() + meta.KeyBytes = MVCCVersionTimestampSize + + // If we land on a (bare) range key, step to look for a colocated point key. + if hasRange && !hasPoint { + rkTimestamp := iter.RangeKeys()[0].RangeKey.Timestamp + + iter.Next() + if ok, err = iter.Valid(); err != nil { + return false, 0, 0, err + } else if ok { + // NB: For !ok, hasPoint is already false. + hasPoint, hasRange = iter.HasPointAndRange() + unsafeKey = iter.UnsafeKey() + } + // If only a bare range tombstone was found at the seek key, synthesize + // point tombstone metadata for it. + if !hasPoint || !unsafeKey.Key.Equal(metaKey.Key) { + meta.Deleted = true + meta.Timestamp = rkTimestamp.ToLegacyTimestamp() + return true, int64(encodedMVCCKeyPrefixLength(metaKey.Key)), 0, nil + } + } + + // We're now on a point key. Check if it's covered by an MVCC range tombstone, + // and synthesize point tombstone metadata for it in that case. + if hasRange { + if rkTS := iter.RangeKeys()[0].RangeKey.Timestamp; unsafeKey.Timestamp.LessEq(rkTS) { + meta.Deleted = true + meta.Timestamp = rkTS.ToLegacyTimestamp() + return true, int64(encodedMVCCKeyPrefixLength(metaKey.Key)), 0, nil + } + } + + // Synthesize metadata for a regular point key. + var unsafeVal MVCCValue unsafeValRaw := iter.UnsafeValue() - unsafeVal, ok, err := tryDecodeSimpleMVCCValue(unsafeValRaw) - if !ok && err == nil { + if unsafeVal, ok, err = tryDecodeSimpleMVCCValue(unsafeValRaw); !ok && err == nil { unsafeVal, err = decodeExtendedMVCCValue(unsafeValRaw) } if err != nil { return false, 0, 0, err } - meta.Reset() - // For values, the size of keys is always accounted for as - // MVCCVersionTimestampSize. The size of the metadata key is - // accounted for separately. - meta.KeyBytes = MVCCVersionTimestampSize meta.ValBytes = int64(len(unsafeValRaw)) meta.Deleted = unsafeVal.IsTombstone() meta.Timestamp = unsafeKey.Timestamp.ToLegacyTimestamp() - return true, int64(unsafeKey.EncodedSize()) - meta.KeyBytes, 0, nil + + return true, int64(encodedMVCCKeyPrefixLength(metaKey.Key)), 0, nil } // putBuffer holds pointer data needed by mvccPutInternal. Bundling @@ -1040,7 +1088,10 @@ func MVCCPut( var iter MVCCIterator blind := ms == nil && timestamp.IsEmpty() if !blind { - iter = newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{Prefix: true}) + iter = newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() } return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, value, txn, nil) @@ -1084,7 +1135,10 @@ func MVCCDelete( localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, ) error { - iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{Prefix: true}) + iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, nil) @@ -1140,7 +1194,9 @@ func maybeGetValue( var exVal optionalValue if exists { var err error - exVal, _, err = mvccGet(ctx, iter, key, readTimestamp, MVCCGetOptions{Tombstones: true}) + exVal, _, err = mvccGetWithPointSynthesis(ctx, iter, key, readTimestamp, MVCCGetOptions{ + Tombstones: true, + }) if err != nil { return roachpb.Value{}, err } @@ -1208,7 +1264,10 @@ func replayTransactionalWrite( // This is a special case. This is when the intent hasn't made it // to the intent history yet. We must now assert the value written // in the intent to the value we're trying to write. - writtenValue, _, err = mvccGet(ctx, iter, key, timestamp, MVCCGetOptions{Txn: txn, Tombstones: true}) + writtenValue, _, err = mvccGetWithPointSynthesis(ctx, iter, key, timestamp, MVCCGetOptions{ + Txn: txn, + Tombstones: true, + }) if err != nil { return err } @@ -1260,7 +1319,10 @@ func replayTransactionalWrite( // last committed value on the key. Since we want the last committed // value on the key, we must make an inconsistent read so we ignore // our previous intents here. - exVal, _, err = mvccGet(ctx, iter, key, timestamp, MVCCGetOptions{Inconsistent: true, Tombstones: true}) + exVal, _, err = mvccGetWithPointSynthesis(ctx, iter, key, timestamp, MVCCGetOptions{ + Inconsistent: true, + Tombstones: true, + }) if err != nil { return err } @@ -1339,8 +1401,7 @@ func mvccPutInternal( } metaKey := MakeMVCCMetadataKey(key) - ok, origMetaKeySize, origMetaValSize, err := - mvccGetMetadata(iter, metaKey, false /* iterAlreadyPositioned */, &buf.meta) + ok, origMetaKeySize, origMetaValSize, err := mvccGetMetadata(iter, metaKey, &buf.meta) if err != nil { return err } @@ -1461,9 +1522,13 @@ func mvccPutInternal( if !enginepb.TxnSeqIsIgnored(meta.Txn.Sequence, txn.IgnoredSeqNums) { // Seqnum of last write is not ignored. Retrieve the value. iter.SeekGE(oldVersionKey) + var hasPoint bool if valid, err := iter.Valid(); err != nil { return err - } else if !valid || !iter.UnsafeKey().Equal(oldVersionKey) { + } else if valid { + hasPoint, _ = iter.HasPointAndRange() + } + if !hasPoint || !iter.UnsafeKey().Equal(oldVersionKey) { return errors.Errorf("existing intent value missing: %s", oldVersionKey) } @@ -1495,7 +1560,10 @@ func mvccPutInternal( // // Since we want the last committed value on the key, we must make // an inconsistent read so we ignore our previous intents here. - exVal, _, err = mvccGet(ctx, iter, key, readTimestamp, MVCCGetOptions{Inconsistent: true, Tombstones: true}) + exVal, _, err = mvccGetWithPointSynthesis(ctx, iter, key, readTimestamp, MVCCGetOptions{ + Inconsistent: true, + Tombstones: true, + }) if err != nil { return err } @@ -1524,9 +1592,21 @@ func mvccPutInternal( // MVCCResolveWriteIntent. prevKey := oldVersionKey.Next() iter.SeekGE(prevKey) - if valid, err := iter.Valid(); err != nil { + valid, err := iter.Valid() + if err != nil { return err - } else if valid && iter.UnsafeKey().Key.Equal(prevKey.Key) { + } else if valid { + // TODO(erikgrinaker): We don't handle MVCC range tombstones in MVCC + // stats yet, so if we land on a bare range key just step onto the + // next point key (if any). + if hasPoint, hasRange := iter.HasPointAndRange(); hasRange && !hasPoint { + iter.Next() + if valid, err = iter.Valid(); err != nil { + return err + } + } + } + if valid && iter.UnsafeKey().Key.Equal(prevKey.Key) { prevUnsafeKey := iter.UnsafeKey() if !prevUnsafeKey.IsValue() { return errors.Errorf("expected an MVCC value key: %s", prevUnsafeKey) @@ -1742,7 +1822,10 @@ func MVCCIncrement( txn *roachpb.Transaction, inc int64, ) (int64, error) { - iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{Prefix: true}) + iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() var int64Val int64 @@ -1816,7 +1899,10 @@ func MVCCConditionalPut( allowIfDoesNotExist CPutMissingBehavior, txn *roachpb.Transaction, ) error { - iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{Prefix: true}) + iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() return mvccConditionalPutUsingIter( @@ -1898,7 +1984,10 @@ func MVCCInitPut( failOnTombstones bool, txn *roachpb.Transaction, ) error { - iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{Prefix: true}) + iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() return mvccInitPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, value, failOnTombstones, txn) } @@ -1977,6 +2066,9 @@ func (m mvccKeyFormatter) Format(f fmt.State, c rune) { // concatenates undifferentiated byte slice values, and efficiently // combines time series observations if the roachpb.Value tag value // indicates the value byte slice is of type TIMESERIES. +// +// Merges are not really MVCC operations: they operate on inline values with no +// version, and do not check for conflicts with other MVCC versions. func MVCCMerge( _ context.Context, rw ReadWriter, @@ -2277,7 +2369,10 @@ func MVCCDeleteRange( buf := newPutBuffer() defer buf.release() - iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{Prefix: true}) + iter := newMVCCIterator(rw, timestamp, false /* rangeKeyMasking */, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + Prefix: true, + }) defer iter.Close() var keys []roachpb.Key @@ -3636,8 +3731,7 @@ func MVCCGarbageCollect( meta := &enginepb.MVCCMetadata{} for _, gcKey := range keys { encKey := MakeMVCCMetadataKey(gcKey.Key) - ok, metaKeySize, metaValSize, err := - mvccGetMetadata(iter, encKey, false /* iterAlreadyPositioned */, meta) + ok, metaKeySize, metaValSize, err := mvccGetMetadata(iter, encKey, meta) if err != nil { return err } diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 05f91b216da8..64e3254ece9c 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -56,6 +56,7 @@ import ( // txn_step t= [n=] // txn_advance t= ts=[,] // txn_status t= status= +// txn_ignore_seqs t= seqs=[-[,-...]] // // resolve_intent t= k= [status=] [clockWhilePending=[,]] // check_intent k= [none] @@ -65,6 +66,8 @@ import ( // del_range [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= [end=] [max=] [returnKeys] // del_range_ts [ts=[,]] [localTs=[,]] k= end= // increment [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= [inc=] +// initput [t=] [ts=[,]] [resolve [status=]] k= v= [raw] [failOnTombstones] +// merge [t=] [ts=[,]] [resolve [status=]] k= v= [raw] // put [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] // put_rangekey ts=[,] [localTS=[,]] k= end= // get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] @@ -488,6 +491,7 @@ var commands = map[string]cmd{ "del_range_ts": {typDataUpdate, cmdDeleteRangeTombstone}, "get": {typReadOnly, cmdGet}, "increment": {typDataUpdate, cmdIncrement}, + "initput": {typDataUpdate, cmdInitPut}, "merge": {typDataUpdate, cmdMerge}, "put": {typDataUpdate, cmdPut}, "put_rangekey": {typDataUpdate, cmdPutRangeKey}, @@ -734,6 +738,27 @@ func cmdCPut(e *evalCtx) error { }) } +func cmdInitPut(e *evalCtx) error { + txn := e.getTxn(optional) + ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) + + key := e.getKey() + val := e.getVal() + failOnTombstones := e.hasArg("failOnTombstones") + resolve, resolveStatus := e.getResolve() + + return e.withWriter("initput", func(rw ReadWriter) error { + if err := MVCCInitPut(e.ctx, rw, e.ms, key, ts, localTs, val, failOnTombstones, txn); err != nil { + return err + } + if resolve { + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) + } + return nil + }) +} + func cmdDelete(e *evalCtx) error { txn := e.getTxn(optional) key := e.getKey() diff --git a/pkg/storage/testdata/mvcc_histories/range_key_iter b/pkg/storage/testdata/mvcc_histories/range_key_iter index c0474f775eec..7179e17652de 100644 --- a/pkg/storage/testdata/mvcc_histories/range_key_iter +++ b/pkg/storage/testdata/mvcc_histories/range_key_iter @@ -14,23 +14,23 @@ # run ok put_rangekey k=a end=k ts=1 -put_rangekey k=m end=n ts=3 localTs=2 -put_rangekey k=b end=d ts=3 -put_rangekey k=f end=h ts=3 -put_rangekey k=c end=g ts=5 put k=a ts=2 v=a2 del k=a ts=4 +put_rangekey k=b end=d ts=3 del k=b ts=4 put k=d ts=4 v=d4 put k=e ts=3 v=e3 put k=f ts=2 v=f2 -put k=f ts=4 v=f4 -put k=f ts=6 v=f6 put k=g ts=2 v=g2 +put_rangekey k=f end=h ts=3 +put k=f ts=4 v=f4 put k=g ts=4 v=g4 +put_rangekey k=c end=g ts=5 +put k=f ts=6 v=f6 put k=h ts=3 v=h3 del k=h ts=4 put k=k ts=5 v=k5 +put_rangekey k=m end=n ts=3 localTs=2 with t=A txn_begin ts=7 put k=a v=a7 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts b/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts new file mode 100644 index 000000000000..99c342af8aed --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts @@ -0,0 +1,334 @@ +# Set up some point keys, point tombstones x, range tombstones o--o, +# and intents []. The initial state is: +# +# 7 [g7] +# 6 +# 5 o-------------------------------o +# 4 +# 3 o---------------------------------------o +# 2 x +# 1 d1 e1 f1 g1 1 1 +# a b c d e f g h i j k +run ok +put k=d ts=1 v=d1 +put k=e ts=1 v=e1 +del k=e ts=2 +put k=f ts=1 v=f1 +put k=g ts=1 v=g1 +increment k=i ts=1 +increment k=j ts=1 +del_range_ts k=a end=k ts=3 +del_range_ts k=c end=k ts=5 +with t=A ts=7 + txn_begin + put k=g v=7 +---- +inc: current value = 1 +inc: current value = 1 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 + +# Inline value or tombstone below range tombstone should error. +run error +put k=b ts=0 v=b0 +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*withstack.withStack:) "b"/0,0: put is inline=true, but existing value is inline=false + +run error +del k=b ts=0 +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*withstack.withStack:) "b"/0,0: put is inline=true, but existing value is inline=false + +# DeleteRange at ts=5 should error with WriteTooOldError. +# +# TODO(erikgrinaker): This should error on c rather than d, but won't do so +# until MVCCScan respects MVCC range tombstones. Until it does, the +# put will actually do a write at the new timestamp. +run error +del_range k=a end=f ts=5 +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "d" at timestamp 5.000000000,0 too old; wrote at 5.000000000,1 + +# Point key below range tombstones should error, but is written anyway at a +# higher timestamp. +# +# TODO(erikgrinaker): These should test stats too, once range tombstones are +# correctly accounted for: "Stats are updated correctly, even when there are +# existing point values and tombstones below the range tombstones". +run error +put k=c ts=3 v=c3 +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "c"/5.000000000,1 -> /BYTES/c3 +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "c" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 + +run error +put k=d ts=3 v=d3 +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "c"/5.000000000,1 -> /BYTES/c3 +data: "d"/5.000000000,2 -> /BYTES/d3 +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "d" at timestamp 3.000000000,0 too old; wrote at 5.000000000,2 + +run error +put k=e ts=3 v=e3 +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "c"/5.000000000,1 -> /BYTES/c3 +data: "d"/5.000000000,2 -> /BYTES/d3 +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/5.000000000,1 -> /BYTES/e3 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "e" at timestamp 3.000000000,0 too old; wrote at 5.000000000,1 + +# CPuts expecting a value covered by a range tombstone should error. +run error +cput k=f ts=7 v=f7 cond=f1 +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "c"/5.000000000,1 -> /BYTES/c3 +data: "d"/5.000000000,2 -> /BYTES/d3 +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/5.000000000,1 -> /BYTES/e3 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: + +# A CPut replay of an intent expecting a value covered by a range tombstone +# should error because of the range tombstone covering it. +run error +with t=A ts=7 + cput k=g v=g7 cond=g1 +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "c"/5.000000000,1 -> /BYTES/c3 +data: "d"/5.000000000,2 -> /BYTES/d3 +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/5.000000000,1 -> /BYTES/e3 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: + +# A CPut replacing an existing but ignored intent expecting a value covered +# by a range tombstone should error because of the range tombstone covering it. +run error +with t=A ts=7 + txn_step + txn_ignore_seqs seqs=0-1 + cput k=g v=g7 cond=g1 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=7.000000000,0 wto=false gul=0,0 isn=1 +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "c"/5.000000000,1 -> /BYTES/c3 +data: "d"/5.000000000,2 -> /BYTES/d3 +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/5.000000000,1 -> /BYTES/e3 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: + +# An InitPut with failOnTombstones above a range tombstone should error. +run error +initput k=f ts=7 v=f7 failOnTombstones +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "c"/5.000000000,1 -> /BYTES/c3 +data: "d"/5.000000000,2 -> /BYTES/d3 +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/5.000000000,1 -> /BYTES/e3 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*roachpb.ConditionFailedError:) unexpected value: timestamp: + +# An InitPut with a different value as an existing key should succeed when there's +# a range tombstone covering the existing value. +# +# TODO(erikgrinaker): This should test stats too. +run ok +initput k=f ts=7 v=f7 +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "c"/5.000000000,1 -> /BYTES/c3 +data: "d"/5.000000000,2 -> /BYTES/d3 +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/5.000000000,1 -> /BYTES/e3 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/7.000000000,0 -> /BYTES/f7 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 + +# An increment below a range tombstone should reset to 1 and write above it with +# a WriteTooOldError. +run error +increment k=i ts=2 +---- +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "c"/5.000000000,1 -> /BYTES/c3 +data: "d"/5.000000000,2 -> /BYTES/d3 +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/5.000000000,1 -> /BYTES/e3 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/7.000000000,0 -> /BYTES/f7 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/5.000000000,1 -> /INT/1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "i" at timestamp 2.000000000,0 too old; wrote at 5.000000000,1 + +# An increment above a range tombstone should reset to 1. +run ok +increment k=j ts=7 +---- +inc: current value = 1 +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "c"/5.000000000,1 -> /BYTES/c3 +data: "d"/5.000000000,2 -> /BYTES/d3 +data: "d"/5.000000000,1 -> / +data: "d"/1.000000000,0 -> /BYTES/d1 +data: "e"/5.000000000,1 -> /BYTES/e3 +data: "e"/2.000000000,0 -> / +data: "e"/1.000000000,0 -> /BYTES/e1 +data: "f"/7.000000000,0 -> /BYTES/f7 +data: "f"/1.000000000,0 -> /BYTES/f1 +meta: "g"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=7.000000000,0 min=0,0 seq=0} ts=7.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "g"/7.000000000,0 -> /BYTES/7 +data: "g"/1.000000000,0 -> /BYTES/g1 +data: "i"/5.000000000,1 -> /INT/1 +data: "i"/1.000000000,0 -> /INT/1 +data: "j"/7.000000000,0 -> /INT/1 +data: "j"/1.000000000,0 -> /INT/1