diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index 56eefac61726..496be0da4e12 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -544,14 +544,20 @@ func (b *Batch) PutInline(key, value interface{}) { // expValue needs to correspond to a Value.TagAndDataBytes() - i.e. a key's // value without the checksum (as the checksum includes the key too). func (b *Batch) CPut(key, value interface{}, expValue []byte) { - b.cputInternal(key, value, expValue, false, false) + b.cputInternal(key, value, expValue, false, false, false) } // CPutAllowingIfNotExists is like CPut except it also allows the Put when the // existing entry does not exist -- i.e. it succeeds if there is no existing // entry or the existing entry has the expected value. func (b *Batch) CPutAllowingIfNotExists(key, value interface{}, expValue []byte) { - b.cputInternal(key, value, expValue, true, false) + b.cputInternal(key, value, expValue, true, false, false) +} + +// CPutFailOnTombstones is like CPut except it fails with a +// ConditionedFailedError if it encounters a tombstone. +func (b *Batch) CPutFailOnTombstones(key, value interface{}, expValue []byte) { + b.cputInternal(key, value, expValue, false, false, true) } // CPutWithOriginTimestamp is like CPut except that it also sets the @@ -577,7 +583,7 @@ func (b *Batch) CPutWithOriginTimestamp( b.initResult(0, 1, notRaw, err) return } - r := kvpb.NewConditionalPut(k, v, expValue, false) + r := kvpb.NewConditionalPut(k, v, expValue, false, false) r.(*kvpb.ConditionalPutRequest).OriginTimestamp = ts r.(*kvpb.ConditionalPutRequest).ShouldWinOriginTimestampTie = shouldWinTie b.appendReqs(r) @@ -600,11 +606,11 @@ func (b *Batch) CPutWithOriginTimestamp( // A nil value can be used to delete the respective key, since there is no // DelInline(). This is different from CPut(). func (b *Batch) CPutInline(key, value interface{}, expValue []byte) { - b.cputInternal(key, value, expValue, false, true) + b.cputInternal(key, value, expValue, false, true, false) } func (b *Batch) cputInternal( - key, value interface{}, expValue []byte, allowNotExist bool, inline bool, + key, value interface{}, expValue []byte, allowNotExist bool, inline bool, failOnTombstones bool, ) { k, err := marshalKey(key) if err != nil { @@ -617,9 +623,9 @@ func (b *Batch) cputInternal( return } if inline { - b.appendReqs(kvpb.NewConditionalPutInline(k, v, expValue, allowNotExist)) + b.appendReqs(kvpb.NewConditionalPutInline(k, v, expValue, allowNotExist, failOnTombstones)) } else { - b.appendReqs(kvpb.NewConditionalPut(k, v, expValue, allowNotExist)) + b.appendReqs(kvpb.NewConditionalPut(k, v, expValue, allowNotExist, failOnTombstones)) } b.approxMutationReqBytes += len(k) + len(v.RawBytes) b.initResult(1, 1, notRaw, nil) @@ -711,6 +717,8 @@ func (b *Batch) CPutValuesEmpty(bs BulkSource[roachpb.Value]) { // key can be either a byte slice or a string. value can be any key type, a // protoutil.Message or any Go primitive type (bool, int, etc). It is illegal // to set value to nil. +// TODO(yuzefovich): this can be removed after compatibility with 24.3 is no +// longer needed. func (b *Batch) InitPut(key, value interface{}, failOnTombstones bool) { k, err := marshalKey(key) if err != nil { diff --git a/pkg/kv/kvpb/api.go b/pkg/kv/kvpb/api.go index 228a3dddf1b7..3a4644dadfc0 100644 --- a/pkg/kv/kvpb/api.go +++ b/pkg/kv/kvpb/api.go @@ -1656,7 +1656,7 @@ func NewPutInline(key roachpb.Key, value roachpb.Value) Request { // them. The caller retains ownership of expVal; NewConditionalPut will copy it // into the request. func NewConditionalPut( - key roachpb.Key, value roachpb.Value, expValue []byte, allowNotExist bool, + key roachpb.Key, value roachpb.Value, expValue []byte, allowNotExist bool, failOnTombstones bool, ) Request { value.InitChecksum(key) return &ConditionalPutRequest{ @@ -1666,6 +1666,7 @@ func NewConditionalPut( Value: value, ExpBytes: expValue, AllowIfDoesNotExist: allowNotExist, + FailOnTombstones: failOnTombstones, } } @@ -1676,7 +1677,7 @@ func NewConditionalPut( // them. The caller retains ownership of expVal; NewConditionalPut will copy it // into the request. func NewConditionalPutInline( - key roachpb.Key, value roachpb.Value, expValue []byte, allowNotExist bool, + key roachpb.Key, value roachpb.Value, expValue []byte, allowNotExist bool, failOnTombstones bool, ) Request { value.InitChecksum(key) return &ConditionalPutRequest{ @@ -1687,6 +1688,7 @@ func NewConditionalPutInline( ExpBytes: expValue, AllowIfDoesNotExist: allowNotExist, Inline: true, + FailOnTombstones: failOnTombstones, } } @@ -2555,10 +2557,16 @@ func (writeOptions *WriteOptions) GetOriginTimestamp() hlc.Timestamp { } func (r *ConditionalPutRequest) Validate() error { + if r.AllowIfDoesNotExist && r.FailOnTombstones { + return errors.AssertionFailedf("invalid ConditionalPutRequest: AllowIfDoesNotExist and FailOnTombstones are incompatible") + } if !r.OriginTimestamp.IsEmpty() { if r.AllowIfDoesNotExist { return errors.AssertionFailedf("invalid ConditionalPutRequest: AllowIfDoesNotExist and non-empty OriginTimestamp are incompatible") } + if r.FailOnTombstones { + return errors.AssertionFailedf("invalid ConditionalPutRequest: FailOnTombstones and non-empty OriginTimestamp are incompatible") + } } return nil } diff --git a/pkg/kv/kvpb/api.proto b/pkg/kv/kvpb/api.proto index 87a225e1cec1..c65ecf94b9b3 100644 --- a/pkg/kv/kvpb/api.proto +++ b/pkg/kv/kvpb/api.proto @@ -333,6 +333,8 @@ message ConditionalPutRequest { // exist with that value. Passing this indicates that it is also OK if the key // does not exist. This is useful when a given value is expected but it is // possible it has not yet been written. + // + // Cannot be combined with FailOnTombstones nor with OriginTimestamp. bool allow_if_does_not_exist = 5; // Specify as true to put the value without a corresponding // timestamp. This option should be used with care as it precludes @@ -371,6 +373,11 @@ message ConditionalPutRequest { // // This must only be used in conjunction with OriginTimestamp. bool should_win_origin_timestamp_tie = 9; + + // If true, tombstones cause ConditionFailedErrors. + // + // Cannot be combined with AllowIfDoesNotExit nor with OriginTimestamp. + bool failOnTombstones = 10; } // A ConditionalPutResponse is the return value from the @@ -385,6 +392,9 @@ message ConditionalPutResponse { // - If key exists, returns a ConditionFailedError if value != existing value // If failOnTombstones is set to true, tombstone values count as mismatched // values and will cause a ConditionFailedError. +// TODO(yuzefovich): this can be removed once the compatibility with the version +// in which we stopped issuing InitPuts (some time after 25.3) is no longer +// needed. message InitPutRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; Value value = 2 [(gogoproto.nullable) = false]; diff --git a/pkg/kv/kvserver/batcheval/cmd_conditional_put.go b/pkg/kv/kvserver/batcheval/cmd_conditional_put.go index ad60521030fd..9d806c43b240 100644 --- a/pkg/kv/kvserver/batcheval/cmd_conditional_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_conditional_put.go @@ -66,6 +66,7 @@ func ConditionalPut( return result.Result{}, errors.AssertionFailedf("OriginTimestamp cannot be passed via CPut arg and in request header") } + failOnTombstones := args.FailOnTombstones && !cArgs.EvalCtx.EvalKnobs().DisableInitPutCPutFailOnTombstones opts := storage.ConditionalPutWriteOptions{ MVCCWriteOptions: storage.MVCCWriteOptions{ Txn: h.Txn, @@ -80,6 +81,7 @@ func ConditionalPut( Category: fs.BatchEvalReadCategory, }, AllowIfDoesNotExist: storage.CPutMissingBehavior(args.AllowIfDoesNotExist), + FailOnTombstones: storage.CPutTombstoneBehavior(failOnTombstones), OriginTimestamp: args.OriginTimestamp, ShouldWinOriginTimestampTie: args.ShouldWinOriginTimestampTie, } diff --git a/pkg/kv/kvserver/batcheval/cmd_init_put.go b/pkg/kv/kvserver/batcheval/cmd_init_put.go index 3fbd70236528..a251e29bc5da 100644 --- a/pkg/kv/kvserver/batcheval/cmd_init_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_init_put.go @@ -29,7 +29,7 @@ func InitPut( args := cArgs.Args.(*kvpb.InitPutRequest) h := cArgs.Header - failOnTombstones := args.FailOnTombstones && !cArgs.EvalCtx.EvalKnobs().DisableInitPutFailOnTombstones + failOnTombstones := args.FailOnTombstones && !cArgs.EvalCtx.EvalKnobs().DisableInitPutCPutFailOnTombstones opts := storage.MVCCWriteOptions{ Txn: h.Txn, LocalTimestamp: cArgs.Now, diff --git a/pkg/kv/kvserver/kvserverbase/knobs.go b/pkg/kv/kvserver/kvserverbase/knobs.go index 84536ad425db..566ac20fbaa6 100644 --- a/pkg/kv/kvserver/kvserverbase/knobs.go +++ b/pkg/kv/kvserver/kvserverbase/knobs.go @@ -36,10 +36,11 @@ type BatchEvalTestingKnobs struct { // explanation of why. AllowGCWithNewThresholdAndKeys bool - // DisableInitPutFailOnTombstones disables FailOnTombstones for InitPut. This - // is useful together with e.g. StoreTestingKnobs.GlobalMVCCRangeTombstone, - // where we still want InitPut to succeed on top of the range tombstone. - DisableInitPutFailOnTombstones bool + // DisableInitPutCPutFailOnTombstones disables FailOnTombstones for InitPut + // and CPut. This is useful together with e.g. + // StoreTestingKnobs.GlobalMVCCRangeTombstone, where we still want InitPut + // and CPut to succeed on top of the range tombstone. + DisableInitPutCPutFailOnTombstones bool // UseRangeTombstonesForPointDeletes will use point-sized MVCC range // tombstones when deleting point keys, to increase test coverage. These diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 45f9f6222bf3..8b4ab89e2551 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -1372,7 +1372,7 @@ func cPutArgs(key roachpb.Key, value, expValue []byte) kvpb.ConditionalPutReques if expValue != nil { expValue = roachpb.MakeValueFromBytes(expValue).TagAndDataBytes() } - req := kvpb.NewConditionalPut(key, roachpb.MakeValueFromBytes(value), expValue, false /* allowNotExist */) + req := kvpb.NewConditionalPut(key, roachpb.MakeValueFromBytes(value), expValue, false /* allowNotExist */, false /* failOnTombstones */) return *req.(*kvpb.ConditionalPutRequest) } diff --git a/pkg/server/config.go b/pkg/server/config.go index b0699a061a7e..3b6a58561efa 100644 --- a/pkg/server/config.go +++ b/pkg/server/config.go @@ -348,7 +348,7 @@ func (cfg *BaseConfig) InitTestingKnobs() { } storeKnobs := cfg.TestingKnobs.Store.(*kvserver.StoreTestingKnobs) storeKnobs.GlobalMVCCRangeTombstone = true - storeKnobs.EvalKnobs.DisableInitPutFailOnTombstones = true + storeKnobs.EvalKnobs.DisableInitPutCPutFailOnTombstones = true cfg.TestingKnobs.RangeFeed.(*rangefeed.TestingKnobs).IgnoreOnDeleteRangeError = true } diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index afc1d1825b10..cb85228bcee3 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -1536,8 +1536,8 @@ func (t *logicTest) newCluster( DisableConsistencyQueue: true, GlobalMVCCRangeTombstone: globalMVCCRangeTombstone, EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ - DisableInitPutFailOnTombstones: ignoreMVCCRangeTombstoneErrors, - UseRangeTombstonesForPointDeletes: shouldUseMVCCRangeTombstonesForPointDeletes, + DisableInitPutCPutFailOnTombstones: ignoreMVCCRangeTombstoneErrors, + UseRangeTombstonesForPointDeletes: shouldUseMVCCRangeTombstonesForPointDeletes, }, }, RangeFeed: &rangefeed.TestingKnobs{ diff --git a/pkg/sql/sqlliveness/slstorage/BUILD.bazel b/pkg/sql/sqlliveness/slstorage/BUILD.bazel index 5122e74e8a8c..526e62d3c2c9 100644 --- a/pkg/sql/sqlliveness/slstorage/BUILD.bazel +++ b/pkg/sql/sqlliveness/slstorage/BUILD.bazel @@ -13,6 +13,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slstorage", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvpb", diff --git a/pkg/sql/sqlliveness/slstorage/slstorage.go b/pkg/sql/sqlliveness/slstorage/slstorage.go index 245860c12575..df121c2e99a9 100644 --- a/pkg/sql/sqlliveness/slstorage/slstorage.go +++ b/pkg/sql/sqlliveness/slstorage/slstorage.go @@ -10,6 +10,7 @@ import ( "math/rand" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" @@ -529,7 +530,11 @@ func (s *Storage) Insert( } v := encodeValue(expiration) - batch.InitPut(k, &v, true) + if s.settings.Version.IsActive(ctx, clusterversion.V25_1) { + batch.CPutFailOnTombstones(k, &v, nil /* expValue */) + } else { + batch.InitPut(k, &v, true) + } return txn.CommitInBatch(ctx, batch) }); err != nil { diff --git a/pkg/storage/metamorphic/operations.go b/pkg/storage/metamorphic/operations.go index c898ce308a5c..2a3dd6728a96 100644 --- a/pkg/storage/metamorphic/operations.go +++ b/pkg/storage/metamorphic/operations.go @@ -235,7 +235,9 @@ func (m mvccCPutOp) run(ctx context.Context) string { _, err := storage.MVCCConditionalPut(ctx, writer, m.key, txn.ReadTimestamp, m.value, m.expVal, storage.ConditionalPutWriteOptions{ - MVCCWriteOptions: storage.MVCCWriteOptions{Txn: txn}, + MVCCWriteOptions: storage.MVCCWriteOptions{Txn: txn}, + // TODO: why we unconditionally set this flag? + // TODO: should we add FailOnTombstones? AllowIfDoesNotExist: true, }) if err != nil { diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 0287966d1f56..a7ac53dfb5cd 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -2883,12 +2883,25 @@ const ( CPutFailIfMissing CPutMissingBehavior = false ) +// CPutTombstoneBehavior describes the handling of a tombstone. +type CPutTombstoneBehavior bool + +const ( + // CPutIgnoreTombstones is used to indicate that CPut should ignore the + // tombstones and treat them as key non-existing. + CPutIgnoreTombstones CPutTombstoneBehavior = false + // CPutFailOnTombstones is used to indicate that CPut should fail if it + // finds a tombstone. + CPutFailOnTombstones CPutTombstoneBehavior = true +) + // ConditionalPutWriteOptions bundles options for the // MVCCConditionalPut and MVCCBlindConditionalPut functions. type ConditionalPutWriteOptions struct { MVCCWriteOptions AllowIfDoesNotExist CPutMissingBehavior + FailOnTombstones CPutTombstoneBehavior // OriginTimestamp, if set, indicates that the caller wants to put the // value only if any existing key is older than this timestamp. // @@ -3011,10 +3024,16 @@ func mvccConditionalPutUsingIter( expBytes []byte, opts ConditionalPutWriteOptions, ) (roachpb.LockAcquisition, error) { + if bool(opts.AllowIfDoesNotExist) && bool(opts.FailOnTombstones) { + return roachpb.LockAcquisition{}, errors.AssertionFailedf("AllowIfDoesNotExist and FailOnTombstones are incompatible") + } if !opts.OriginTimestamp.IsEmpty() { if bool(opts.AllowIfDoesNotExist) { return roachpb.LockAcquisition{}, errors.AssertionFailedf("AllowIfDoesNotExist and non-zero OriginTimestamp are incompatible") } + if bool(opts.FailOnTombstones) { + return roachpb.LockAcquisition{}, errors.AssertionFailedf("FailOnTombstones and non-zero OriginTimestamp are incompatible") + } putIsInline := timestamp.IsEmpty() if putIsInline { return roachpb.LockAcquisition{}, errors.AssertionFailedf("inline put and non-zero OriginTimestamp are incompatible") @@ -3023,11 +3042,26 @@ func mvccConditionalPutUsingIter( var valueFn func(existVal optionalValue) (roachpb.Value, error) if opts.OriginTimestamp.IsEmpty() { - valueFn = func(actualValue optionalValue) (roachpb.Value, error) { - if err := maybeConditionFailedError(expBytes, actualValue, bool(opts.AllowIfDoesNotExist)); err != nil { - return roachpb.Value{}, err + if opts.FailOnTombstones { + valueFn = func(existVal optionalValue) (roachpb.Value, error) { + if existVal.IsTombstone() { + // We found a tombstone and FailOnTombstones is true: fail. + return roachpb.Value{}, &kvpb.ConditionFailedError{ + ActualValue: existVal.ToPointer(), + } + } + if err := maybeConditionFailedError(expBytes, existVal, false /* allowNoExisting */); err != nil { + return roachpb.Value{}, err + } + return value, nil + } + } else { + valueFn = func(existVal optionalValue) (roachpb.Value, error) { + if err := maybeConditionFailedError(expBytes, existVal, bool(opts.AllowIfDoesNotExist)); err != nil { + return roachpb.Value{}, err + } + return value, nil } - return value, nil } } else { valueFn = func(existVal optionalValue) (roachpb.Value, error) { diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index e1272d49f4dc..faf40d3d8cf1 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -86,7 +86,7 @@ var ( // check_for_acquire_lock t= k= str= [maxLockConflicts=] [targetLockConflictBytes=] // acquire_lock t= k= str= [maxLockConflicts=] [targetLockConflictBytes=] // -// cput [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] [targetLockConflictBytes=] k= v= [raw] [cond=] +// cput [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] [targetLockConflictBytes=] k= v= [raw] [cond=] [allow_missing] [failOnTombstones] // del [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] [targetLockConflictBytes=] k= // del_range [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] [ambiguousReplay] [maxLockConflicts=] [targetLockConflictBytes=] k= end= [max=] [returnKeys] // del_range_ts [ts=[,]] [localTs=[,]] [maxLockConflicts=] [targetLockConflictBytes=] k= end= [idempotent] [noCoveredStats] @@ -1293,9 +1293,13 @@ func cmdCPut(e *evalCtx) error { rexpVal := e.getValInternal("cond") expVal = rexpVal.TagAndDataBytes() } - behavior := storage.CPutFailIfMissing + missingBehavior := storage.CPutFailIfMissing if e.hasArg("allow_missing") { - behavior = storage.CPutAllowIfMissing + missingBehavior = storage.CPutAllowIfMissing + } + tombstoneBehavior := storage.CPutIgnoreTombstones + if e.hasArg("failOnTombstones") { + tombstoneBehavior = storage.CPutFailOnTombstones } originTimestamp := hlc.Timestamp{} @@ -1314,7 +1318,8 @@ func cmdCPut(e *evalCtx) error { ReplayWriteTimestampProtection: e.getAmbiguousReplay(), MaxLockConflicts: e.getMaxLockConflicts(), }, - AllowIfDoesNotExist: behavior, + AllowIfDoesNotExist: missingBehavior, + FailOnTombstones: tombstoneBehavior, OriginTimestamp: originTimestamp, } acq, err := storage.MVCCConditionalPut(e.ctx, rw, key, ts, val, expVal, opts) diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put b/pkg/storage/testdata/mvcc_histories/conditional_put index 0c6e5213922d..90a552493e3f 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put +++ b/pkg/storage/testdata/mvcc_histories/conditional_put @@ -55,7 +55,7 @@ data: "k"/123.000000000,5 -> /BYTES/ data: "k"/123.000000000,2 -> /BYTES/v stats: key_count=1 key_bytes=26 val_count=2 val_bytes=11 live_count=1 live_bytes=19 gc_bytes_age=-414 -# Move key2 (which does not exist) to from value1 to value2. +# Move key2 (which does not exist) from value1 to value2. # Expect it to fail since it does not exist with value1. run error @@ -66,7 +66,7 @@ data: "k"/123.000000000,5 -> /BYTES/ data: "k"/123.000000000,2 -> /BYTES/v error: (*kvpb.ConditionFailedError:) unexpected value: -# Move key2 (which does not yet exist) to from value1 to value2, but +# Move key2 (which does not yet exist) from value1 to value2, but # allowing for it not existing. run stats ok @@ -126,3 +126,50 @@ run ok get k=k ts=123,11 ---- get: "k" -> /BYTES/v2 @123.000000000,10 + +# Verify that FailOnTombstones flag is respected. First create a point +# tombstone. + +run ok +del k=k ts=123,12 +---- +del: "k": found key true +>> at end: +data: "k"/123.000000000,12 -> / +data: "k"/123.000000000,10 -> /BYTES/v2 +data: "k"/123.000000000,5 -> /BYTES/ +data: "k"/123.000000000,2 -> /BYTES/v +data: "k2"/123.000000000,9 -> /BYTES/ +data: "k2"/123.000000000,7 -> /BYTES/v2 + +# Try overwriting it with FailOnTombstones set - expect an error. +run stats error +cput k=k v=v ts=123,13 failOnTombstones +---- +>> cput k=k v=v ts=123,13 failOnTombstones +stats: no change +>> at end: +data: "k"/123.000000000,12 -> / +data: "k"/123.000000000,10 -> /BYTES/v2 +data: "k"/123.000000000,5 -> /BYTES/ +data: "k"/123.000000000,2 -> /BYTES/v +data: "k2"/123.000000000,9 -> /BYTES/ +data: "k2"/123.000000000,7 -> /BYTES/v2 +stats: key_count=2 key_bytes=77 val_count=6 val_bytes=30 live_count=1 live_bytes=20 gc_bytes_age=-2001 +error: (*kvpb.ConditionFailedError:) unexpected value: timestamp: + +# Try overwriting it without FailOnTombstones set - expect success. +run stats ok +cput k=k v=v ts=123,13 +---- +>> cput k=k v=v ts=123,13 +stats: key_bytes=+12 val_count=+1 val_bytes=+6 live_count=+1 live_bytes=+20 gc_bytes_age=+46 +>> at end: +data: "k"/123.000000000,13 -> /BYTES/v +data: "k"/123.000000000,12 -> / +data: "k"/123.000000000,10 -> /BYTES/v2 +data: "k"/123.000000000,5 -> /BYTES/ +data: "k"/123.000000000,2 -> /BYTES/v +data: "k2"/123.000000000,9 -> /BYTES/ +data: "k2"/123.000000000,7 -> /BYTES/v2 +stats: key_count=2 key_bytes=89 val_count=7 val_bytes=36 live_count=2 live_bytes=40 gc_bytes_age=-1955 diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts b/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts index 6db9216adfe9..53e72cf8e262 100644 --- a/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_conflicts @@ -206,6 +206,49 @@ data: "j"/1.000000000,0 -> /INT/1 stats: key_count=6 key_bytes=108 val_count=8 val_bytes=94 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=1 live_bytes=68 gc_bytes_age=16400 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 error: (*kvpb.ConditionFailedError:) unexpected value: timestamp: +# CPuts expecting no value covered by a range tombstone should error with +# failOnTombstones. +run stats error +cput k=d ts=7 v=d7 failOnTombstones +---- +>> cput k=d ts=7 v=d7 failOnTombstones +stats: no change +>> 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=00000001 key=/Min iso=Serializable 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 +stats: key_count=6 key_bytes=108 val_count=8 val_bytes=94 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=1 live_bytes=68 gc_bytes_age=16400 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 +error: (*kvpb.ConditionFailedError:) unexpected value: timestamp: + +# CPuts expecting no value covered by a range tombstone should succeed. +run stats ok +cput k=d ts=7 v=d7 +---- +>> cput k=d ts=7 v=d7 +stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_bytes_age=-194 +>> at end: +rangekey: {a-c}/[3.000000000,0=/] +rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "d"/7.000000000,0 -> /BYTES/d7 +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=00000001 key=/Min iso=Serializable 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 +stats: key_count=6 key_bytes=120 val_count=9 val_bytes=101 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=2 live_bytes=89 gc_bytes_age=16206 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 + # A CPut replay of an intent expecting a value covered by a range tombstone # should error because of the range tombstone covering it. run stats error @@ -217,6 +260,7 @@ stats: no change >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/1.000000000,0 -> /BYTES/d1 data: "e"/2.000000000,0 -> / data: "e"/1.000000000,0 -> /BYTES/e1 @@ -226,7 +270,7 @@ 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 -stats: key_count=6 key_bytes=108 val_count=8 val_bytes=94 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=1 live_bytes=68 gc_bytes_age=16400 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 +stats: key_count=6 key_bytes=120 val_count=9 val_bytes=101 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=2 live_bytes=89 gc_bytes_age=16206 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 error: (*kvpb.ConditionFailedError:) unexpected value: timestamp: # A CPut replacing an existing but ignored intent expecting a value covered @@ -243,6 +287,7 @@ stats: no change txn: "A" meta={id=00000001 key=/Min iso=Serializable 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: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/1.000000000,0 -> /BYTES/d1 data: "e"/2.000000000,0 -> / data: "e"/1.000000000,0 -> /BYTES/e1 @@ -252,7 +297,7 @@ 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 -stats: key_count=6 key_bytes=108 val_count=8 val_bytes=94 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=1 live_bytes=68 gc_bytes_age=16400 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 +stats: key_count=6 key_bytes=120 val_count=9 val_bytes=101 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=2 live_bytes=89 gc_bytes_age=16206 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 error: (*kvpb.ConditionFailedError:) unexpected value: timestamp: # An InitPut with failOnTombstones above a range tombstone should error. @@ -264,6 +309,7 @@ stats: no change >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/1.000000000,0 -> /BYTES/d1 data: "e"/2.000000000,0 -> / data: "e"/1.000000000,0 -> /BYTES/e1 @@ -273,7 +319,7 @@ 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 -stats: key_count=6 key_bytes=108 val_count=8 val_bytes=94 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=1 live_bytes=68 gc_bytes_age=16400 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 +stats: key_count=6 key_bytes=120 val_count=9 val_bytes=101 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=2 live_bytes=89 gc_bytes_age=16206 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 error: (*kvpb.ConditionFailedError:) unexpected value: timestamp: # An InitPut with a different value as an existing key should succeed when there's @@ -286,6 +332,7 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+7 live_count=+1 live_bytes=+21 gc_b >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/1.000000000,0 -> /BYTES/d1 data: "e"/2.000000000,0 -> / data: "e"/1.000000000,0 -> /BYTES/e1 @@ -296,7 +343,7 @@ 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 -stats: key_count=6 key_bytes=120 val_count=9 val_bytes=101 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=2 live_bytes=89 gc_bytes_age=16206 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 +stats: key_count=6 key_bytes=132 val_count=10 val_bytes=108 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=3 live_bytes=110 gc_bytes_age=16012 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 # An increment below a range tombstone throw a WriteTooOldError. run stats error @@ -307,6 +354,7 @@ stats: no change >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/1.000000000,0 -> /BYTES/d1 data: "e"/2.000000000,0 -> / data: "e"/1.000000000,0 -> /BYTES/e1 @@ -317,7 +365,7 @@ 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 -stats: key_count=6 key_bytes=120 val_count=9 val_bytes=101 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=2 live_bytes=89 gc_bytes_age=16206 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 +stats: key_count=6 key_bytes=132 val_count=10 val_bytes=108 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=3 live_bytes=110 gc_bytes_age=16012 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 error: (*kvpb.WriteTooOldError:) WriteTooOldError: write for key "i" at timestamp 2.000000000,0 too old; must write at or above 5.000000000,1 # An increment above a range tombstone should reset to 1. @@ -330,6 +378,7 @@ stats: key_bytes=+12 val_count=+1 val_bytes=+6 live_count=+1 live_bytes=+20 gc_b >> at end: rangekey: {a-c}/[3.000000000,0=/] rangekey: {c-k}/[5.000000000,0=/ 3.000000000,0=/] +data: "d"/7.000000000,0 -> /BYTES/d7 data: "d"/1.000000000,0 -> /BYTES/d1 data: "e"/2.000000000,0 -> / data: "e"/1.000000000,0 -> /BYTES/e1 @@ -341,4 +390,4 @@ data: "g"/1.000000000,0 -> /BYTES/g1 data: "i"/1.000000000,0 -> /INT/1 data: "j"/7.000000000,0 -> /INT/1 data: "j"/1.000000000,0 -> /INT/1 -stats: key_count=6 key_bytes=132 val_count=10 val_bytes=107 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=3 live_bytes=109 gc_bytes_age=16012 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93 +stats: key_count=6 key_bytes=144 val_count=11 val_bytes=114 range_key_count=2 range_key_bytes=35 range_val_count=3 live_count=4 live_bytes=130 gc_bytes_age=15818 intent_count=1 intent_bytes=18 lock_count=1 lock_age=93