From 0ad96469af1bcd645c0b4b2327f2186b61d5e5da Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Mon, 7 Feb 2022 21:07:01 +0000 Subject: [PATCH] batcheval: add range tombstone support for `DeleteRange` This patch adds a parameter `UseExperimentalRangeTombstone` for `DeleteRange`, which deletes the span using an MVCC range tombstone. This must only be called after checking `storage.CanUseExperimentalMVCCRangeTombstones()`, which ensures the `ExperimentalMVCCRangeTombstones` version gate and `COCKROACH_EXPERIMENTAL_MVCC_RANGE_TOMBSTONES` environment variable are set. Did I mention that MVCC range tombstones are experimental? They are currently under active development, and are not respected by the KV or MVCC APIs, nor are they persisted. This patch simply sets up the plumbing for it. Release note: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 7 +++++ pkg/clusterversion/key_string.go | 5 ++-- pkg/kv/batch.go | 28 +++++++++++++++++++ pkg/kv/db.go | 16 +++++++++++ pkg/kv/kvserver/batcheval/cmd_delete_range.go | 22 +++++++++++++++ pkg/kv/kvserver/spanset/batch.go | 16 +++++++---- pkg/roachpb/api.go | 4 +++ pkg/roachpb/api.proto | 11 ++++++++ pkg/roachpb/api_test.go | 1 + pkg/storage/mvcc.go | 15 ++++++++++ 12 files changed, 120 insertions(+), 9 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 471fd2159831..75e2e0139ae4 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -181,4 +181,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-74 set the active cluster version in the format '.' +version version 21.2-76 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index be0133cc1187..51a4a4c78486 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -194,6 +194,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-74set the active cluster version in the format '.' +versionversion21.2-76set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index a719965fd039..43f094daf35b 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -300,6 +300,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. @@ -484,6 +487,10 @@ var versionsSingleton = keyedVersions{ Key: EnablePebbleFormatVersionRangeKeys, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 74}, }, + { + Key: ExperimentalMVCCRangeTombstones, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 76}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 5d5fc454792b..e67d6b559141 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -46,11 +46,12 @@ func _() { _ = x[EnableLeaseHolderRemoval-35] _ = x[EnsurePebbleFormatVersionRangeKeys-36] _ = x[EnablePebbleFormatVersionRangeKeys-37] + _ = x[ExperimentalMVCCRangeTombstones-38] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeys" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysExperimentalMVCCRangeTombstones" -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} +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, 1053} 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}