diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 099353749197..d60f5c1dd817 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -194,4 +194,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 22.1-1002 set the active cluster version in the format '.' +version version 22.1-1004 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index e030fd7c3e98..fedc70d31a53 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -211,6 +211,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion22.1-1002set the active cluster version in the format '.' +versionversion22.1-1004set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 8b68163fb921..fbc161a47805 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -358,6 +358,9 @@ const ( // version is guaranteed to reside in a cluster where all nodes support range // keys at the Pebble layer. EnablePebbleFormatVersionRangeKeys + // ExperimentalMVCCRangeTombstones enables the use of highly experimental MVCC + // range tombstones. + ExperimentalMVCCRangeTombstones // ************************************************* // Step (1): Add new versions here. @@ -627,6 +630,10 @@ var versionsSingleton = keyedVersions{ Key: EnablePebbleFormatVersionRangeKeys, Version: roachpb.Version{Major: 22, Minor: 1, Internal: 1002}, }, + { + Key: ExperimentalMVCCRangeTombstones, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 1004}, + }, } // TODO(irfansharif): clusterversion.binary{,MinimumSupported}Version diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index d8ed7546907f..4a2ccc6d86e7 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -65,11 +65,12 @@ func _() { _ = x[V22_1-54] _ = x[EnsurePebbleFormatVersionRangeKeys-55] _ = x[EnablePebbleFormatVersionRangeKeys-56] + _ = x[ExperimentalMVCCRangeTombstones-57] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1EnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeys" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1EnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysExperimentalMVCCRangeTombstones" -var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 455, 483, 504, 517, 536, 570, 608, 642, 674, 710, 742, 778, 820, 839, 879, 911, 930, 954, 975, 1006, 1024, 1065, 1095, 1106, 1137, 1160, 1193, 1217, 1241, 1263, 1276, 1288, 1314, 1328, 1349, 1367, 1372, 1406, 1440} +var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 455, 483, 504, 517, 536, 570, 608, 642, 674, 710, 742, 778, 820, 839, 879, 911, 930, 954, 975, 1006, 1024, 1065, 1095, 1106, 1137, 1160, 1193, 1217, 1241, 1263, 1276, 1288, 1314, 1328, 1349, 1367, 1372, 1406, 1440, 1471} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index 3c248d9ab3a8..4f2bbd1298be 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -649,6 +649,34 @@ func (b *Batch) DelRange(s, e interface{}, returnKeys bool) { b.initResult(1, 0, notRaw, nil) } +// ExperimentalDelRangeUsingTombstone deletes the rows between begin (inclusive) +// and end (exclusive) using an MVCC range tombstone. The caller must check +// the ExperimentalMVCCRangeTombstones version gate before using this. +// +// This method is EXPERIMENTAL: range tombstones are under active development, +// and have severe limitations including being ignored by all KV and MVCC APIs +// and only being stored in memory. +func (b *Batch) ExperimentalDelRangeUsingTombstone(s, e interface{}) { + start, err := marshalKey(s) + if err != nil { + b.initResult(0, 0, notRaw, err) + return + } + end, err := marshalKey(e) + if err != nil { + b.initResult(0, 0, notRaw, err) + return + } + b.appendReqs(&roachpb.DeleteRangeRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: start, + EndKey: end, + }, + UseExperimentalRangeTombstone: true, + }) + b.initResult(1, 0, notRaw, nil) +} + // adminMerge is only exported on DB. It is here for symmetry with the // other operations. func (b *Batch) adminMerge(key interface{}) { diff --git a/pkg/kv/db.go b/pkg/kv/db.go index c2bce4e020b0..6f58db2182ee 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -547,6 +547,22 @@ func (db *DB) DelRange( return r.Keys, err } +// ExperimentalDelRangeUsingTombstone deletes the rows between begin (inclusive) +// and end (exclusive) using an MVCC range tombstone. The caller must check +// the ExperimentalMVCCRangeTombstones version gate before using this. +// +// This method is EXPERIMENTAL: range tombstones are under active development, +// and have severe limitations including being ignored by all KV and MVCC APIs +// and only being stored in memory. +func (db *DB) ExperimentalDelRangeUsingTombstone( + ctx context.Context, begin, end interface{}, +) error { + b := &Batch{} + b.ExperimentalDelRangeUsingTombstone(begin, end) + _, err := getOneResult(db.Run(ctx, b), b) + return err +} + // AdminMerge merges the range containing key and the subsequent range. After // the merge operation is complete, the range containing key will contain all of // the key/value pairs of the subsequent range and the subsequent range will no diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 9939a60c4074..12cca1f348c7 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -100,6 +100,7 @@ go_test( srcs = [ "cmd_add_sstable_test.go", "cmd_clear_range_test.go", + "cmd_delete_range_test.go", "cmd_end_transaction_test.go", "cmd_export_test.go", "cmd_get_test.go", diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 60554d27ad4b..8110ed488768 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" ) func init() { @@ -49,6 +50,24 @@ func DeleteRange( h := cArgs.Header reply := resp.(*roachpb.DeleteRangeResponse) + // Use experimental MVCC range tombstone if requested. + if args.UseExperimentalRangeTombstone { + if cArgs.Header.Txn != nil { + return result.Result{}, ErrTransactionUnsupported + } + if args.Inline { + return result.Result{}, errors.AssertionFailedf("Inline can't be used with range tombstones") + } + if args.ReturnKeys { + return result.Result{}, errors.AssertionFailedf( + "ReturnKeys can't be used with range tombstones") + } + maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV) + err := storage.ExperimentalMVCCDeleteRangeUsingTombstone( + ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, h.Timestamp, maxIntents) + return result.Result{}, err + } + var timestamp hlc.Timestamp if !args.Inline { timestamp = h.Timestamp diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go new file mode 100644 index 000000000000..e11e28635f3f --- /dev/null +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go @@ -0,0 +1,207 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package batcheval_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +// TestDeleteRangeTombstone tests DeleteRange range tombstones directly, using +// only a Pebble engine. +// +// Most MVCC range tombstone logic is tested exhaustively in the MVCC history +// tests, this just tests the RPC plumbing. +func TestDeleteRangeTombstone(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Initial data for each test. x is point tombstone, [] is intent, + // o---o is range tombstone. + // + // 5 [i5] + // 4 c4 + // 3 x + // 2 b2 d2 o-------o + // 1 + // a b c d e f g h i + writeInitialData := func(t *testing.T, ctx context.Context, rw storage.ReadWriter) { + t.Helper() + txn := roachpb.MakeTransaction("test", nil /* baseKey */, roachpb.NormalUserPriority, hlc.Timestamp{WallTime: 5}, 0, 0) + require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("b"), hlc.Timestamp{WallTime: 2}, roachpb.MakeValueFromString("b2"), nil)) + require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("c"), hlc.Timestamp{WallTime: 4}, roachpb.MakeValueFromString("c4"), nil)) + require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 2}, roachpb.MakeValueFromString("d2"), nil)) + require.NoError(t, storage.MVCCDelete(ctx, rw, nil, roachpb.Key("d"), hlc.Timestamp{WallTime: 3}, nil)) + require.NoError(t, storage.MVCCPut(ctx, rw, nil, roachpb.Key("i"), hlc.Timestamp{WallTime: 5}, roachpb.MakeValueFromString("i5"), &txn)) + require.NoError(t, storage.ExperimentalMVCCDeleteRangeUsingTombstone(ctx, rw, nil, roachpb.Key("f"), roachpb.Key("h"), hlc.Timestamp{WallTime: 3}, 0)) + } + + testcases := map[string]struct { + start string + end string + ts int64 + txn bool + inline bool + returnKeys bool + expectErr interface{} // error type, substring, or true (any) + }{ + "above points succeed": { + start: "a", + end: "f", + ts: 10, + expectErr: nil, + }, + "above range tombstone succeed": { + start: "f", + end: "h", + ts: 10, + expectErr: nil, + }, + "transaction errors": { + start: "a", + end: "f", + ts: 10, + txn: true, + expectErr: batcheval.ErrTransactionUnsupported, + }, + "inline errors": { + start: "a", + end: "f", + ts: 10, + inline: true, + expectErr: "Inline can't be used with range tombstones", + }, + "returnKeys errors": { + start: "a", + end: "f", + ts: 10, + returnKeys: true, + expectErr: "ReturnKeys can't be used with range tombstones", + }, + "intent errors with WriteIntentError": { + start: "i", + end: "j", + ts: 10, + expectErr: &roachpb.WriteIntentError{}, + }, + "below point errors with WriteTooOldError": { + start: "a", + end: "d", + ts: 1, + expectErr: &roachpb.WriteTooOldError{}, + }, + "below range tombstone errors with WriteTooOldError": { + start: "f", + end: "h", + ts: 1, + expectErr: &roachpb.WriteTooOldError{}, + }, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + engine := storage.NewDefaultInMemForTesting() + defer engine.Close() + + writeInitialData(t, ctx, engine) + + rangeKey := storage.MVCCRangeKey{ + StartKey: roachpb.Key(tc.start), + EndKey: roachpb.Key(tc.end), + Timestamp: hlc.Timestamp{WallTime: tc.ts}, + } + + var txn *roachpb.Transaction + if tc.txn { + tx := roachpb.MakeTransaction("txn", nil /* baseKey */, roachpb.NormalUserPriority, rangeKey.Timestamp, 0, 0) + txn = &tx + } + + // Run the request. + var ms enginepb.MVCCStats + resp := &roachpb.DeleteRangeResponse{} + _, err := batcheval.DeleteRange(ctx, engine, batcheval.CommandArgs{ + EvalCtx: (&batcheval.MockEvalCtx{ClusterSettings: st}).EvalContext(), + Stats: &ms, + Header: roachpb.Header{ + Timestamp: rangeKey.Timestamp, + Txn: txn, + }, + Args: &roachpb.DeleteRangeRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: rangeKey.StartKey, + EndKey: rangeKey.EndKey, + }, + UseExperimentalRangeTombstone: true, + Inline: tc.inline, + ReturnKeys: tc.returnKeys, + }, + }, resp) + + // Check the error. + if tc.expectErr != nil { + require.Error(t, err) + if b, ok := tc.expectErr.(bool); ok && b { + // any error is fine + } else if expectMsg, ok := tc.expectErr.(string); ok { + require.Contains(t, err.Error(), expectMsg) + } else if e, ok := tc.expectErr.(error); ok { + require.True(t, errors.HasType(err, e), "expected %T, got %v", e, err) + } else { + require.Fail(t, "invalid expectErr", "expectErr=%v", tc.expectErr) + } + return + } + require.NoError(t, err) + + // Check that the range tombstone was written successfully. + iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypeRangesOnly, + LowerBound: rangeKey.StartKey, + UpperBound: rangeKey.EndKey, + }) + defer iter.Close() + iter.SeekGE(storage.MVCCKey{Key: rangeKey.StartKey}) + + var endSeen roachpb.Key + for { + ok, err := iter.Valid() + require.NoError(t, err) + if !ok { + break + } + require.True(t, ok) + for _, rk := range iter.RangeKeys() { + if rk.Timestamp.Equal(rangeKey.Timestamp) { + endSeen = rk.EndKey.Clone() + break + } + } + iter.Next() + } + require.Equal(t, rangeKey.EndKey, endSeen) + + // TODO(erikgrinaker): This should test MVCC stats when implemented. + }) + } +} diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 5f26a98abcda..b33d99d11672 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -178,17 +178,17 @@ func (i *MVCCIterator) UnsafeValue() []byte { // HasPointAndRange implements SimpleMVCCIterator. func (i *MVCCIterator) HasPointAndRange() (bool, bool) { - panic("not implemented") + return i.i.HasPointAndRange() } // RangeBounds implements SimpleMVCCIterator. func (i *MVCCIterator) RangeBounds() (roachpb.Key, roachpb.Key) { - panic("not implemented") + return i.i.RangeBounds() } // RangeKeys implements SimpleMVCCIterator. func (i *MVCCIterator) RangeKeys() []storage.MVCCRangeKey { - panic("not implemented") + return i.i.RangeKeys() } // ComputeStats is part of the storage.MVCCIterator interface. @@ -610,11 +610,17 @@ func (s spanSetWriter) ClearIterRange(start, end roachpb.Key) error { } func (s spanSetWriter) ExperimentalPutMVCCRangeKey(rangeKey storage.MVCCRangeKey) error { - panic("not implemented") + if err := s.checkAllowedRange(rangeKey.StartKey, rangeKey.EndKey); err != nil { + return err + } + return s.w.ExperimentalPutMVCCRangeKey(rangeKey) } func (s spanSetWriter) ExperimentalClearMVCCRangeKey(rangeKey storage.MVCCRangeKey) error { - panic("not implemented") + if err := s.checkAllowedRange(rangeKey.StartKey, rangeKey.EndKey); err != nil { + return err + } + return s.w.ExperimentalClearMVCCRangeKey(rangeKey) } func (s spanSetWriter) ExperimentalClearAllMVCCRangeKeys(start, end roachpb.Key) error { diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index ca0109b9f066..8b64c71b767b 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1260,6 +1260,10 @@ func (*DeleteRequest) flags() flag { } func (drr *DeleteRangeRequest) flags() flag { + // DeleteRangeRequest using MVCC range tombstones cannot be transactional. + if drr.UseExperimentalRangeTombstone { + return isWrite | isRange | isAlone | appliesTSCache + } // DeleteRangeRequest has different properties if the "inline" flag is set. // This flag indicates that the request is deleting inline MVCC values, // which cannot be deleted transactionally - inline DeleteRange will thus diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index cbe3678e1d4f..2302e03684a9 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -348,6 +348,18 @@ message DeleteRangeRequest { // Inline values cannot be deleted transactionally; a DeleteRange with // "inline" set to true will fail if it is executed within a transaction. bool inline = 4; + // If enabled, the range is deleted using an MVCC range tombstone, which is a + // cheaper constant-time write operation, but still requires a scan to check + // for conflicts. This option cannot be used in a transaction, and it cannot + // be combined with Inline or ReturnKeys. + // + // The caller must check the ExperimentalMVCCRangeTombstones version gate + // before using this parameter, as it is new in 22.2. + // + // This parameter is EXPERIMENTAL: range tombstones are under active + // development, and have severe limitations including being ignored by all + // KV and MVCC APIs and only being stored in memory. + bool use_experimental_range_tombstone = 5; } // A DeleteRangeResponse is the return value from the DeleteRange() diff --git a/pkg/roachpb/api_test.go b/pkg/roachpb/api_test.go index ef1eca1ab746..6169dce15fac 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/roachpb/api_test.go @@ -321,6 +321,7 @@ func TestFlagCombinations(t *testing.T) { reqVariants := []Request{ &AddSSTableRequest{SSTTimestampToRequestTimestamp: hlc.Timestamp{Logical: 1}}, &DeleteRangeRequest{Inline: true}, + &DeleteRangeRequest{UseExperimentalRangeTombstone: true}, &GetRequest{KeyLocking: lock.Exclusive}, &ReverseScanRequest{KeyLocking: lock.Exclusive}, &ScanRequest{KeyLocking: lock.Exclusive}, diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 55f9fe3074a0..7bb683d03188 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -2202,6 +2202,86 @@ func MVCCDeleteRange( return keys, res.ResumeSpan, res.NumKeys, nil } +// ExperimentalMVCCDeleteRangeUsingTombstone deletes the given MVCC keyspan at +// the given timestamp using a range tombstone (rather than point tombstones). +// This operation is non-transactional, but will check for existing intents and +// return a WriteIntentError containing up to maxIntents intents. +// +// This method is EXPERIMENTAL: range keys are under active development, and +// have severe limitations including being ignored by all KV and MVCC APIs and +// only being stored in memory. +// +// TODO(erikgrinaker): This needs handling of stats. +func ExperimentalMVCCDeleteRangeUsingTombstone( + ctx context.Context, + rw ReadWriter, + ms *enginepb.MVCCStats, + startKey, endKey roachpb.Key, + timestamp hlc.Timestamp, + maxIntents int64, +) error { + // Validate the range key. We must do this first, to catch e.g. any bound violations. + rangeKey := MVCCRangeKey{StartKey: startKey, EndKey: endKey, Timestamp: timestamp} + if err := rangeKey.Validate(); err != nil { + return err + } + + // Check for any overlapping intents, and return them to be resolved. + if intents, err := ScanIntents(ctx, rw, startKey, endKey, maxIntents, 0); err != nil { + return err + } else if len(intents) > 0 { + return &roachpb.WriteIntentError{Intents: intents} + } + + // Check for any conflicts, i.e. newer values. + // + // TODO(erikgrinaker): This introduces an O(n) read penalty. We should + // consider either optimizing it or making the check optional somehow. + iter := rw.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: startKey, + UpperBound: endKey, + MinTimestampHint: timestamp, + MaxTimestampHint: hlc.MaxTimestamp, + }) + defer iter.Close() + + iter.SeekGE(MVCCKey{Key: startKey}) + var prevRangeStart roachpb.Key + for { + if ok, err := iter.Valid(); err != nil { + return err + } else if !ok { + break + } + + hasPoint, hasRange := iter.HasPointAndRange() + if hasPoint { + key := iter.UnsafeKey() + if timestamp.LessEq(key.Timestamp) { + return roachpb.NewWriteTooOldError(timestamp, key.Timestamp.Next(), key.Key.Clone()) + } + if key.Timestamp.IsEmpty() { + return errors.Errorf("can't write range tombstone across inline key %s", key) + } + } + if hasRange { + if rangeStart, _ := iter.RangeBounds(); !rangeStart.Equal(prevRangeStart) { + latest := iter.RangeKeys()[0] + if timestamp.LessEq(latest.Timestamp) { + return roachpb.NewWriteTooOldError( + timestamp, latest.Timestamp.Next(), latest.StartKey.Clone()) + } + prevRangeStart = append(prevRangeStart[:0], rangeStart...) + } + } + iter.NextKey() + } + + // Write the tombstone. + return rw.ExperimentalPutMVCCRangeKey(rangeKey) +} + func recordIteratorStats(traceSpan *tracing.Span, iteratorStats IteratorStats) { stats := iteratorStats.Stats if traceSpan != nil { diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 39664e26db66..f8f2f31631eb 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -60,6 +60,7 @@ import ( // cput [t=] [ts=[,]] [resolve [status=]] k= v= [raw] [cond=] // del [t=] [ts=[,]] [resolve [status=]] k= // del_range [t=] [ts=[,]] [resolve [status=]] k= [end=] [max=] [returnKeys] +// del_range_ts [ts=[,]] k= end= // get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] // increment [t=] [ts=[,]] [resolve [status=]] k= [inc=] // put [t=] [ts=[,]] [resolve [status=]] k= v= [raw] @@ -440,6 +441,7 @@ var commands = map[string]cmd{ "cput": {typDataUpdate, cmdCPut}, "del": {typDataUpdate, cmdDelete}, "del_range": {typDataUpdate, cmdDeleteRange}, + "del_range_ts": {typDataUpdate, cmdDeleteRangeTombstone}, "get": {typReadOnly, cmdGet}, "increment": {typDataUpdate, cmdIncrement}, "merge": {typDataUpdate, cmdMerge}, @@ -710,6 +712,15 @@ func cmdDeleteRange(e *evalCtx) error { }) } +func cmdDeleteRangeTombstone(e *evalCtx) error { + key, endKey := e.getKeyRange() + ts := e.getTs(nil) + + return e.withWriter("del_range_ts", func(rw ReadWriter) error { + return ExperimentalMVCCDeleteRangeUsingTombstone(e.ctx, rw, nil, key, endKey, ts, 0) + }) +} + func cmdGet(e *evalCtx) error { txn := e.getTxn(optional) key := e.getKey() diff --git a/pkg/storage/testdata/mvcc_histories/range_tombstone_mutations b/pkg/storage/testdata/mvcc_histories/range_tombstone_mutations new file mode 100644 index 000000000000..8320cb96d3b9 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/range_tombstone_mutations @@ -0,0 +1,438 @@ +# Set up some point keys, point tombstones x, range tombstones o--o, +# and intents []. +# +# 7 [d7] [i7] +# 6 +# 5 +# 4 x d4 f4 x o-------------------o +# 3 +# 2 a2 g2 +# 1 +# 0 h0 +# a b c d e f g h i j k l m n o p +run ok +put k=a ts=2 v=a2 +del k=a ts=4 +put k=d ts=4 v=d4 +put k=f ts=4 v=f4 +put k=g ts=2 v=g2 +del k=g ts=4 +put k=h ts=0 v=inline +del_range_ts k=k end=p ts=4 +with t=A + txn_begin ts=7 + put k=d v=d7 + put k=i v=i7 +---- +>> 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: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 + +# Writing invalid range tombstones should error. +run error +del_range_ts k=z end=x ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*withstack.withStack:) invalid range key {z-x}/3.000000000,0: start key "z" is at or after end key "x" + +run error +del_range_ts k=x end=z ts=0 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*withstack.withStack:) invalid range key {x-z}: no timestamp + +run error +del_range_ts k=x end=x ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*withstack.withStack:) invalid range key x{-}/3.000000000,0: start key "x" is at or after end key "x" + +# Writing at or below existing point keys should return a WriteTooOldError, +# both at the start key and in the middle of the range key. +run error +del_range_ts k=a end=b ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 + +run error +del_range_ts k=a end=b ts=4 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "a" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 + +run error +del_range_ts k=e end=g ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "f" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 + +# Writing at or below existing range tombstones should return a WriteTooOldError, +# regardless of how they overlap. +run error +del_range_ts k=k end=p ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 + +run error +del_range_ts k=k end=p ts=4 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 4.000000000,0 too old; wrote at 4.000000000,1 + +run error +del_range_ts k=j end=m ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 + +run error +del_range_ts k=o end=q ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "o" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 + +run error +del_range_ts k=j end=q ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 + +run error +del_range_ts k=k end=n ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 3.000000000,0 too old; wrote at 4.000000000,1 + +# Writing below intents should return a WriteIntentError, both when above and +# below the intent timestamp and any existing values. +run error +del_range_ts k=d end=e ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteIntentError:) conflicting intents on "d" + +run error +del_range_ts k=d end=e ts=5 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteIntentError:) conflicting intents on "d" + +run error +del_range_ts k=i end=j ts=5 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteIntentError:) conflicting intents on "i" + +run error +del_range_ts k=i end=j ts=7 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteIntentError:) conflicting intents on "i" + +run error +del_range_ts k=i end=j ts=10 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*roachpb.WriteIntentError:) conflicting intents on "i" + +# Writing above an inline value should error. +run error +del_range_ts k=h end=i ts=3 +---- +>> at end: +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 +error: (*withstack.withStack:) can't write range tombstone across inline key "h"/0,0 + +# Writing next to or above point keys and tombstones should work. +run ok +del_range_ts k=a end=b ts=10 +del_range_ts k=b end=d ts=4 +---- +>> at end: +rangekey: {a-b}/[10.000000000,0] +rangekey: {b-d}/[4.000000000,0] +rangekey: {k-p}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 + +# Writing range tombstones next to other range tombstones will merge them, but +# only at same timestamp. +run ok +del_range_ts k=s end=x ts=4 +del_range_ts k=p end=s ts=3 +---- +>> at end: +rangekey: {a-b}/[10.000000000,0] +rangekey: {b-d}/[4.000000000,0] +rangekey: {k-p}/[4.000000000,0] +rangekey: {p-s}/[3.000000000,0] +rangekey: {s-x}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7 + +run ok +del_range_ts k=p end=s ts=4 +---- +>> at end: +rangekey: {a-b}/[10.000000000,0] +rangekey: {b-d}/[4.000000000,0] +rangekey: {k-p}/[4.000000000,0] +rangekey: {p-s}/[4.000000000,0 3.000000000,0] +rangekey: {s-x}/[4.000000000,0] +data: "a"/4.000000000,0 -> / +data: "a"/2.000000000,0 -> /BYTES/a2 +meta: "d"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "d"/7.000000000,0 -> /BYTES/d7 +data: "d"/4.000000000,0 -> /BYTES/d4 +data: "f"/4.000000000,0 -> /BYTES/f4 +data: "g"/4.000000000,0 -> / +data: "g"/2.000000000,0 -> /BYTES/g2 +meta: "h"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/inline mergeTs= txnDidNotUpdateMeta=false +meta: "i"/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=7 mergeTs= txnDidNotUpdateMeta=true +data: "i"/7.000000000,0 -> /BYTES/i7