diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index e7f032e9d7e3..500bf74e59c5 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -180,4 +180,4 @@ trace.debug.enable boolean false if set, traces for recent requests can be seen trace.jaeger.agent string the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. 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.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 21.2-76 set the active cluster version in the format '.' +version version 21.2-78 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 627804581fa5..76945ba695f4 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -193,6 +193,6 @@ trace.jaeger.agentstringthe address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. 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.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion21.2-76set the active cluster version in the format '.' +versionversion21.2-78set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index ec76552e72fd..5cd81f5c356f 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -303,6 +303,10 @@ const ( // BackupResolutionInJob defaults to resolving backup destinations during the // execution of a backup job rather than during planning. BackupResolutionInJob + // ExperimentalMVCCRangeTombstones enables the use of highly experimental MVCC + // range tombstones. + ExperimentalMVCCRangeTombstones + // ************************************************* // Step (1): Add new versions here. // Do not add new versions to a patch release. @@ -486,11 +490,15 @@ var versionsSingleton = keyedVersions{ Key: EnablePebbleFormatVersionRangeKeys, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 74}, }, - { Key: BackupResolutionInJob, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 76}, }, + { + Key: ExperimentalMVCCRangeTombstones, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 78}, + }, + // ************************************************* // Step (2): Add new versions here. // Do not add new versions to a patch release. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index c67d8de9a255..4d86e6f3432b 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -47,11 +47,12 @@ func _() { _ = x[EnsurePebbleFormatVersionRangeKeys-36] _ = x[EnablePebbleFormatVersionRangeKeys-37] _ = x[BackupResolutionInJob-38] + _ = x[ExperimentalMVCCRangeTombstones-39] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysBackupResolutionInJob" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysBackupResolutionInJobExperimentalMVCCRangeTombstones" -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, 988, 1022, 1043} +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, 988, 1022, 1043, 1074} 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..5becc4a62d32 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 +// storage.CanUseExperimentalMVCCRangeTombstones() 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 05e834a1a220..e616d227622e 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -546,6 +546,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 +// storage.CanUseExperimentalMVCCRangeTombstones() 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/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 60554d27ad4b..ec3d1e4455cd 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,27 @@ func DeleteRange( h := cArgs.Header reply := resp.(*roachpb.DeleteRangeResponse) + // Use experimental MVCC range tombstone if requested. The caller is expected + // to have checked storage.CanUseExperimentalMVCCRangeTombstones() first. + // + // TODO(erikgrinaker): Add integration tests for this. + 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/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index b44c26d3dee4..30a0c42a197b 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.MVCCRangeKeyValue { - panic("not implemented") + return i.i.RangeKeys() } // ComputeStats is part of the storage.MVCCIterator interface. @@ -617,11 +617,17 @@ func (s spanSetWriter) ClearIterRange(iter storage.MVCCIterator, start, end roac func (s spanSetWriter) ExperimentalPutMVCCRangeKey( rangeKey storage.MVCCRangeKey, value []byte, ) error { - panic("not implemented") + if err := s.checkAllowedRange(rangeKey.StartKey, rangeKey.EndKey); err != nil { + return err + } + return s.w.ExperimentalPutMVCCRangeKey(rangeKey, value) } 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) Merge(key storage.MVCCKey, value []byte) error { diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 1feab34d8fbf..1e80c4ccf582 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1233,6 +1233,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 378d9c33b0c6..2fb632af53f9 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -348,6 +348,17 @@ 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 + // cheap constant-time operation. This option cannot be used in a transaction, + // and it cannot be combined with Inline or ReturnKeys. + // + // The caller must check storage.CanUseExperimentalMVCCRangeTombstones() + // before enabling this parameter. + // + // 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 5b45ea34d975..dea42892987a 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/roachpb/api_test.go @@ -318,6 +318,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 39b215766e61..4fddb07b7515 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -20,10 +20,12 @@ import ( "sync" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -84,6 +86,19 @@ var rocksdbConcurrency = envutil.EnvOrDefaultInt( return max }()) +// CanUseExperimentalMVCCRangeTombstones returns true if MVCC range tombstones +// are enabled. Callers must check this before using range tombstones. +// +// These are 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 CanUseExperimentalMVCCRangeTombstones(ctx context.Context, st *cluster.Settings) bool { + // TODO(erikgrinaker): Consider using a cluster setting rather than an + // environment variable once range tombstones are fully implemented. + return st.Version.IsActive(ctx, clusterversion.ExperimentalMVCCRangeTombstones) && + envutil.EnvOrDefaultBool("COCKROACH_EXPERIMENTAL_MVCC_RANGE_TOMBSTONES", false) +} + // MakeValue returns the inline value. func MakeValue(meta enginepb.MVCCMetadata) roachpb.Value { return roachpb.Value{RawBytes: meta.RawBytes}