diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 255925356a61..a713de26faf1 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -303,9 +303,6 @@ 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 // LooselyCoupledRaftLogTruncation allows the cluster to reduce the coupling // for raft log truncation, by allowing each replica to treat a truncation // proposal as an upper bound on what should be truncated. @@ -494,14 +491,12 @@ 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}, - }, + // Internal: 78 was reverted (ExperimentalMVCCRangeTombstones) { Key: LooselyCoupledRaftLogTruncation, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 80}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index e53e4338fa70..a212758d79d3 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -47,13 +47,12 @@ func _() { _ = x[EnsurePebbleFormatVersionRangeKeys-36] _ = x[EnablePebbleFormatVersionRangeKeys-37] _ = x[BackupResolutionInJob-38] - _ = x[ExperimentalMVCCRangeTombstones-39] - _ = x[LooselyCoupledRaftLogTruncation-40] + _ = x[LooselyCoupledRaftLogTruncation-39] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysBackupResolutionInJobExperimentalMVCCRangeTombstonesLooselyCoupledRaftLogTruncation" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalEnsurePebbleFormatVersionRangeKeysEnablePebbleFormatVersionRangeKeysBackupResolutionInJobLooselyCoupledRaftLogTruncation" -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, 1105} +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 5becc4a62d32..3c248d9ab3a8 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -649,34 +649,6 @@ 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 e616d227622e..05e834a1a220 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -546,22 +546,6 @@ 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 ec3d1e4455cd..60554d27ad4b 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -19,7 +19,6 @@ 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() { @@ -50,27 +49,6 @@ 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/batcheval/cmd_revert_range.go b/pkg/kv/kvserver/batcheval/cmd_revert_range.go index 1afcfbd39f87..7e7fe6e05a58 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range.go @@ -51,12 +51,8 @@ func isEmptyKeyTimeRange( // that there is *a* key in the SST that is in the time range. Thus we should // proceed to iteration that actually checks timestamps on each key. iter := readWriter.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{ - // TODO(erikgrinaker): Make sure TBIs respect range keys too. - KeyTypes: storage.IterKeyTypePointsAndRanges, // revert any range keys as well - LowerBound: from, - UpperBound: to, - MinTimestampHint: since.Next(), // exclusive - MaxTimestampHint: until, + LowerBound: from, UpperBound: to, + MinTimestampHint: since.Next() /* make exclusive */, MaxTimestampHint: until, }) defer iter.Close() iter.SeekGE(storage.MVCCKey{Key: from}) @@ -82,39 +78,29 @@ func RevertRange( args := cArgs.Args.(*roachpb.RevertRangeRequest) reply := resp.(*roachpb.RevertRangeResponse) + pd := result.Result{ + Replicated: kvserverpb.ReplicatedEvalResult{ + MVCCHistoryMutation: &kvserverpb.ReplicatedEvalResult_MVCCHistoryMutation{ + Spans: []roachpb.Span{{Key: args.Key, EndKey: args.EndKey}}, + }, + }, + } if empty, err := isEmptyKeyTimeRange( readWriter, args.Key, args.EndKey, args.TargetTime, cArgs.Header.Timestamp, ); err != nil { return result.Result{}, err } else if empty { - log.VEventf(ctx, 2, "no keys to revert in specified time range") + log.VEventf(ctx, 2, "no keys to clear in specified time range") return result.Result{}, nil } - log.VEventf(ctx, 2, "reverting keys with timestamp (%v, %v]", - args.TargetTime, cArgs.Header.Timestamp) + log.VEventf(ctx, 2, "clearing keys with timestamp (%v, %v]", args.TargetTime, cArgs.Header.Timestamp) - var pd result.Result - var resume *roachpb.Span - var err error - if args.ExperimentalPreserveHistory { - const deleteRangeThreshold = 100 - maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV) - // TODO(erikgrinaker): Write a test for this once MVCC range tombstones are - // properly written to batches and replicated. - // TODO(erikgrinaker): Test that this records MVCC logical ops correctly. - resume, err = storage.ExperimentalMVCCRevertRange(ctx, readWriter, cArgs.Stats, - args.Key, args.EndKey, cArgs.Header.Timestamp, args.TargetTime, deleteRangeThreshold, - cArgs.Header.MaxSpanRequestKeys, maxRevertRangeBatchBytes, maxIntents) - } else { - resume, err = storage.MVCCClearTimeRange(ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, - args.TargetTime, cArgs.Header.Timestamp, cArgs.Header.MaxSpanRequestKeys, - maxRevertRangeBatchBytes, args.EnableTimeBoundIteratorOptimization) - pd.Replicated.MVCCHistoryMutation = &kvserverpb.ReplicatedEvalResult_MVCCHistoryMutation{ - Spans: []roachpb.Span{{Key: args.Key, EndKey: args.EndKey}}, - } - } + resume, err := storage.MVCCClearTimeRange(ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, + args.TargetTime, cArgs.Header.Timestamp, cArgs.Header.MaxSpanRequestKeys, + maxRevertRangeBatchBytes, + args.EnableTimeBoundIteratorOptimization) if err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/rangefeed/task_test.go b/pkg/kv/kvserver/rangefeed/task_test.go index edf833d47904..87b695a7a228 100644 --- a/pkg/kv/kvserver/rangefeed/task_test.go +++ b/pkg/kv/kvserver/rangefeed/task_test.go @@ -190,21 +190,6 @@ func (s *testIterator) curKV() storage.MVCCKeyValue { return s.kvs[s.cur] } -// HasPointAndRange implements SimpleMVCCIterator. -func (s *testIterator) HasPointAndRange() (bool, bool) { - panic("not implemented") -} - -// RangeBounds implements SimpleMVCCIterator. -func (s *testIterator) RangeBounds() (roachpb.Key, roachpb.Key) { - panic("not implemented") -} - -// RangeTombstones implements SimpleMVCCIterator. -func (s *testIterator) RangeKeys() []storage.MVCCRangeKeyValue { - panic("not implemented") -} - func TestInitResolvedTSScan(t *testing.T) { defer leaktest.AfterTest(t)() startKey := roachpb.RKey("d") diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 30a0c42a197b..ae012cb87a87 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -176,21 +176,6 @@ func (i *MVCCIterator) UnsafeValue() []byte { return i.i.UnsafeValue() } -// HasPointAndRange implements SimpleMVCCIterator. -func (i *MVCCIterator) HasPointAndRange() (bool, bool) { - return i.i.HasPointAndRange() -} - -// RangeBounds implements SimpleMVCCIterator. -func (i *MVCCIterator) RangeBounds() (roachpb.Key, roachpb.Key) { - return i.i.RangeBounds() -} - -// RangeKeys implements SimpleMVCCIterator. -func (i *MVCCIterator) RangeKeys() []storage.MVCCRangeKeyValue { - return i.i.RangeKeys() -} - // ComputeStats is part of the storage.MVCCIterator interface. func (i *MVCCIterator) ComputeStats( start, end roachpb.Key, nowNanos int64, @@ -614,22 +599,6 @@ func (s spanSetWriter) ClearIterRange(iter storage.MVCCIterator, start, end roac return s.w.ClearIterRange(iter, start, end) } -func (s spanSetWriter) ExperimentalPutMVCCRangeKey( - rangeKey storage.MVCCRangeKey, value []byte, -) error { - 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 { - 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 { if s.spansOnly { if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil { diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 2514b5ef09e3..2dd836e61cdb 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -778,8 +778,8 @@ func (crr *ClearRangeRequest) ShallowCopy() Request { } // ShallowCopy implements the Request interface. -func (rrr *RevertRangeRequest) ShallowCopy() Request { - shallowCopy := *rrr +func (crr *RevertRangeRequest) ShallowCopy() Request { + shallowCopy := *crr return &shallowCopy } @@ -1233,10 +1233,6 @@ 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 @@ -1270,14 +1266,9 @@ func (*ClearRangeRequest) flags() flag { return isWrite | isRange | isAlone | bypassesReplicaCircuitBreaker } -// Note that RevertRange commands cannot be part of a transaction, as they -// either clear MVCC versions or write MVCC range tombstones, neither of which -// is supported within transactions. -func (rrr *RevertRangeRequest) flags() flag { - if rrr.ExperimentalPreserveHistory { - return isRead | isWrite | isRange | isAlone | updatesTSCache | appliesTSCache | - bypassesReplicaCircuitBreaker - } +// Note that RevertRange commands cannot be part of a transaction as +// they clear all MVCC versions above their target time. +func (*RevertRangeRequest) flags() flag { return isWrite | isRange | isAlone | bypassesReplicaCircuitBreaker } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index a8ca0c9de814..5cb44aafea36 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -348,17 +348,6 @@ 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() @@ -402,34 +391,18 @@ message ClearRangeResponse { } -// A RevertRangeRequest specifies a range of keys to revert to some past time. -// By default, it will clear all revision more recent that TargetTime from the -// underlying engine. However, this violates several guarantees including MVCC -// immutability, the closed timestamp, timestamp cache, and others. See the -// ExperimentalPreserveHistory parameter which will uphold these guarantees. +// A RevertRangeRequest specifies a range of keys in which to clear all MVCC +// revisions more recent than some TargetTime from the underlying engine, thus +// reverting the range (from the perspective of an MVCC scan) to that time. message RevertRangeRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - // TargetTime specifies a the time to which to "revert" the range to. Any - // versions later than TargetTime will be undone. TargetTime must be higher + // TargetTime specifies a the time to which to "revert" the range by clearing + // any MVCC key with a strictly higher timestamp. TargetTime must be higher // than the GC Threshold for the replica - so that it is assured that the keys // for that time are still there — or the request will fail. util.hlc.Timestamp target_time = 2 [(gogoproto.nullable) = false]; - // ExperimentalPreserveHistory will preserve MVCC history by, rather than - // clearing newer versions, deleting them using tombstones or updating them - // back to their original value as of the target time. Long runs of key - // deletions will use an MVCC range tombstone instead. This respects the - // closed timestamp and timestamp cache. - // - // 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 experimental_preserve_history = 5; - bool enable_time_bound_iterator_optimization = 3; // IgnoreGcThreshold can be set by a caller to ignore the target-time when diff --git a/pkg/roachpb/api_test.go b/pkg/roachpb/api_test.go index 9abfa2cace00..5b45ea34d975 100644 --- a/pkg/roachpb/api_test.go +++ b/pkg/roachpb/api_test.go @@ -318,10 +318,8 @@ 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}, - &RevertRangeRequest{ExperimentalPreserveHistory: true}, &ScanRequest{KeyLocking: lock.Exclusive}, } diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 7aa7649bf288..a92b50a04867 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -21,7 +21,6 @@ go_library( "mvcc_incremental_iterator.go", "mvcc_key.go", "mvcc_logical_ops.go", - "mvcc_range_key_iterator.go", "open.go", "pebble.go", "pebble_batch.go", @@ -109,7 +108,6 @@ go_test( "mvcc_incremental_iterator_test.go", "mvcc_key_test.go", "mvcc_logical_ops_test.go", - "mvcc_range_key_iterator_test.go", "mvcc_stats_test.go", "mvcc_test.go", "pebble_file_registry_test.go", diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index ad7f1b9ac138..1ad4bfeea5b8 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -67,46 +67,11 @@ type SimpleMVCCIterator interface { // reverse iteration to forward iteration. NextKey() // UnsafeKey returns the same value as Key, but the memory is invalidated on - // the next call to {Next,NextKey,Prev,SeekGE,SeekLT,Close}. If the iterator - // is on a range key only, this returns the start bound of the range key. + // the next call to {Next,NextKey,Prev,SeekGE,SeekLT,Close}. UnsafeKey() MVCCKey // UnsafeValue returns the same value as Value, but the memory is // invalidated on the next call to {Next,NextKey,Prev,SeekGE,SeekLT,Close}. UnsafeValue() []byte - // HasPointAndRange returns whether the current iterator position has a point - // key and/or a range key. If Valid() returns true, one of these will be true. - // Range keys are only emitted when requested via IterOptions.KeyTypes. - HasPointAndRange() (bool, bool) - // RangeBounds returns the range bounds for the current range key fragment, if - // any. See RangeKeys() for more info on range key fragments. - RangeBounds() (roachpb.Key, roachpb.Key) - // RangeKeys returns all range key fragments (at different timestamps) at the - // current key position. If we are at a point key, it will return all range - // keys that overlap that point key at any timestamp. - // - // For defragmented iteration, use MVCCRangeKeyIterator instead. Fragmented - // iteration is primarily useful in two cases: - // - // - To iterate over point keys while accessing overlapping range keys - // (e.g. to determine if it is hidden by a range tombstone). - // - // - For partial iteration with later resumption, e.g. Export requests with - // byte limits that have to return point and range key data for a partial - // key span and then resume from that point in a later request. - // - // Range keys are fragmented by Pebble such that all overlapping range keys - // between two fragment bounds form a "stack" of range key fragments at - // different timestamps. Fragmentation is desirable at the storage layer to - // store range keys across SSTs and CRDB ranges without incurring - // cross-SST/range access costs. Stacking is desirable to easily see all range - // keys that overlap with a given point, and to translate range keys from the - // 2D MVCC keyspan to the 1D Pebble keyspan. - // - // Range keys will also split and merge along with CRDB ranges, can be - // partially removed by GC, and may be truncated by iterator bounds. - // - // TODO(erikgrinaker): Write a tech note on range keys and link it here. - RangeKeys() []MVCCRangeKeyValue } // IteratorStats is returned from {MVCCIterator,EngineIterator}.Stats. @@ -344,27 +309,8 @@ type IterOptions struct { // use such an iterator is to use it in concert with an iterator without // timestamp hints, as done by MVCCIncrementalIterator. MinTimestampHint, MaxTimestampHint hlc.Timestamp - // KeyTypes specifies the types of keys to surface: point and/or range keys. - // Use HasPointAndRange() to determine which key type is present at a given - // iterator position, and RangeBounds() and RangeKeys() to access range keys. - // Defaults to IterKeyTypePointsOnly. For more info, see RangeKeys(). - KeyTypes IterKeyType } -// IterKeyType configures which types of keys an iterator should surface. -// -// TODO(erikgrinaker): Combine this with MVCCIterKind somehow. -type IterKeyType = pebble.IterKeyType - -const ( - // IterKeyTypePointsOnly iterates over point keys only. - IterKeyTypePointsOnly = pebble.IterKeyTypePointsOnly - // IterKeyTypePointsAndRanges iterates over both point and range keys. - IterKeyTypePointsAndRanges = pebble.IterKeyTypePointsAndRanges - // IterKeyTypeRangesOnly iterates over only range keys. - IterKeyTypeRangesOnly = pebble.IterKeyTypeRangesOnly -) - // MVCCIterKind is used to inform Reader about the kind of iteration desired // by the caller. type MVCCIterKind int @@ -629,42 +575,6 @@ type Writer interface { // returns. ClearIterRange(iter MVCCIterator, start, end roachpb.Key) error - // ExperimentalClearMVCCRangeKey deletes an MVCC range key from start - // (inclusive) to end (exclusive) at the given timestamp. For any range key - // that straddles the start and end boundaries, only the segments within the - // boundaries will be cleared. Clears are idempotent. - // - // This method is primarily intended for MVCC garbage collection and similar - // internal use. It mutates MVCC history, and does not check for intents or - // other conflicts. - // - // TODO(erikgrinaker): We'll likely need another method that calls through to - // Pebble's RangeKeyDelete(), which removes all range keys in a span. This - // will be used e.g. when removing replicas. - // - // 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. - ExperimentalClearMVCCRangeKey(rangeKey MVCCRangeKey) error - - // ExperimentalPutMVCCRangeKey writes a value to an MVCC range key. It is - // currently only used for range tombstones, which have a value of nil. Range - // keys exist separately from point keys in Pebble, and must be accessed via - // specialized iterator options and methods -- see e.g. IterOptions.KeyTypes, - // SimpleMVCCIterator.RangeKeys(), and MVCCRangeKeyIterator. - // - // A range key does not have a distinct identity, but should be considered a - // key continuum. They can be fragmented by Pebble as overlapping keys are - // written or removed, split/merged along with CRDB ranges, partially removed - // with ExperimentalClearMVCCRangeKey, and truncated during bounded iteration. - // - // TODO(erikgrinaker): Write a tech note on range keys and link it here. - // - // 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. - ExperimentalPutMVCCRangeKey(MVCCRangeKey, []byte) error - // Merge is a high-performance write operation used for values which are // accumulated over several writes. Multiple values can be merged // sequentially into a single key; a subsequent read will return a "merged" diff --git a/pkg/storage/intent_interleaving_iter.go b/pkg/storage/intent_interleaving_iter.go index e580380aa90c..20446b93ada8 100644 --- a/pkg/storage/intent_interleaving_iter.go +++ b/pkg/storage/intent_interleaving_iter.go @@ -715,21 +715,6 @@ func (i *intentInterleavingIter) Value() []byte { return i.iter.Value() } -// HasPointAndRange implements SimpleMVCCIterator. -func (i *intentInterleavingIter) HasPointAndRange() (bool, bool) { - panic("not implemented") -} - -// RangeBounds implements SimpleMVCCIterator. -func (i *intentInterleavingIter) RangeBounds() (roachpb.Key, roachpb.Key) { - panic("not implemented") -} - -// RangeKeys implements SimpleMVCCIterator. -func (i *intentInterleavingIter) RangeKeys() []MVCCRangeKeyValue { - panic("not implemented") -} - func (i *intentInterleavingIter) Close() { i.iter.Close() i.intentIter.Close() diff --git a/pkg/storage/multi_iterator.go b/pkg/storage/multi_iterator.go index 6d2558e10c21..9838adf60ec7 100644 --- a/pkg/storage/multi_iterator.go +++ b/pkg/storage/multi_iterator.go @@ -14,7 +14,6 @@ import ( "bytes" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/roachpb" ) const invalidIdxSentinel = -1 @@ -93,21 +92,6 @@ func (f *multiIterator) UnsafeValue() []byte { return f.iters[f.currentIdx].UnsafeValue() } -// HasPointAndRange implements SimpleMVCCIterator. -func (f *multiIterator) HasPointAndRange() (bool, bool) { - panic("not implemented") -} - -// RangeBounds implements SimpleMVCCIterator. -func (f *multiIterator) RangeBounds() (roachpb.Key, roachpb.Key) { - panic("not implemented") -} - -// RangeKeys implements SimpleMVCCIterator. -func (f *multiIterator) RangeKeys() []MVCCRangeKeyValue { - panic("not implemented") -} - // Next advances the iterator to the next key/value in the iteration. After this // call, Valid() will be true if the iterator was not positioned at the last // key. diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 3bd3548e1e3e..64b0d1abaef2 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -20,12 +20,10 @@ 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" @@ -86,19 +84,6 @@ 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} @@ -114,12 +99,6 @@ type MVCCKeyValue struct { Value []byte } -// MVCCRangeKeyValue represents a ranged key/value pair. -type MVCCRangeKeyValue struct { - Key MVCCRangeKey - Value []byte -} - // optionalValue represents an optional roachpb.Value. It is preferred // over a *roachpb.Value to avoid the forced heap allocation. type optionalValue struct { @@ -1967,8 +1946,6 @@ func MVCCMerge( // // If the underlying iterator encounters an intent with a timestamp in the span // (startTime, endTime], or any inline meta, this method will return an error. -// -// TODO(erikgrinaker): This should clear any range keys as well. func MVCCClearTimeRange( _ context.Context, rw ReadWriter, @@ -2225,39 +2202,6 @@ 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. -func ExperimentalMVCCDeleteRangeUsingTombstone( - ctx context.Context, - rw ReadWriter, - ms *enginepb.MVCCStats, - startKey, endKey roachpb.Key, - timestamp hlc.Timestamp, - maxIntents int64, -) error { - if intents, err := ScanIntents(ctx, rw, startKey, endKey, maxIntents, 0); err != nil { - return err - } else if len(intents) > 0 { - return &roachpb.WriteIntentError{Intents: intents} - } - return experimentalMVCCDeleteRangeUsingTombstoneInternal(ctx, rw, ms, MVCCRangeKey{ - StartKey: startKey, EndKey: endKey, Timestamp: timestamp}) -} - -// TODO(erikgrinaker): Needs handling of conflicts (e.g. WriteTooOldError), -// MVCCStats, and tests. -func experimentalMVCCDeleteRangeUsingTombstoneInternal( - ctx context.Context, rw ReadWriter, ms *enginepb.MVCCStats, rangeKey MVCCRangeKey, -) error { - return rw.ExperimentalPutMVCCRangeKey(rangeKey, nil) -} - func recordIteratorStats(traceSpan *tracing.Span, iteratorStats IteratorStats) { stats := iteratorStats.Stats if traceSpan != nil { @@ -2274,222 +2218,6 @@ func recordIteratorStats(traceSpan *tracing.Span, iteratorStats IteratorStats) { } } -// ExperimentalMVCCRevertRange will revert a range back to its state as of some -// past timestamp, writing tombstones or key updates as appropriate at the given -// write timestamp. Long runs of key deletions will be written using MVCC range -// tombstones. -// -// This function cannot be used in a transaction. However, it will scan for -// existing intents and return a WriteIntentError, and scan for newer writes -// and return WriteTooOldError. -// -// This function is EXPERIMENTAL. Range tombstones are not supported throughout -// the MVCC API, and the on-disk format is unstable. -// -// TODO(erikgrinaker): Handle range keys. -func ExperimentalMVCCRevertRange( - ctx context.Context, - rw ReadWriter, - ms *enginepb.MVCCStats, - startKey, endKey roachpb.Key, - writeTimestamp hlc.Timestamp, - revertTimestamp hlc.Timestamp, - deleteRangeThreshold int, - maxBatchSize int64, - maxBatchBytes int64, - maxIntents int64, -) (*roachpb.Span, error) { - // We must resolve any intents within the span, so we may as well scan for - // separated intents before doing any work. - if intents, err := ScanIntents(ctx, rw, startKey, endKey, maxIntents, 0); err != nil { - return nil, err - } else if len(intents) > 0 { - return nil, &roachpb.WriteIntentError{Intents: intents} - } - - // We accumulate point deletes in deleteBuf until we either reach - // deleteRangeThreshold and switch to using a range deletion tombstone - // anchored at deleteRangeStart, or until we hit a visible key at which - // point we flush the deleteBuf as point deletes. - var deleteRangeStart roachpb.Key - var deleteBuf []roachpb.Key - var deleteBufIdx int - var deleteBufBytes int64 - if deleteRangeThreshold > 1 { - deleteBuf = make([]roachpb.Key, deleteRangeThreshold-1) - } - - putBuf := newPutBuffer() - defer putBuf.release() - - var batchSize, batchBytes int64 - - flushDeletes := func(nonMatch roachpb.Key) error { - if len(deleteRangeStart) > 0 { - err := experimentalMVCCDeleteRangeUsingTombstoneInternal(ctx, rw, ms, MVCCRangeKey{ - StartKey: deleteRangeStart, EndKey: nonMatch, Timestamp: writeTimestamp}) - deleteRangeStart = nil - batchBytes += int64(encodedMVCCKeyLength(MVCCKey{Key: nonMatch})) // account for end key - return err - } - - if deleteBufIdx > 0 { - iter := newMVCCIterator(rw, false, IterOptions{Prefix: true}) - defer iter.Close() - for i := 0; i < deleteBufIdx; i++ { - err := mvccPutInternal( - ctx, rw, iter, ms, deleteBuf[i], writeTimestamp, nil, nil, putBuf, nil) - if err != nil { - return err - } - } - deleteBufIdx = 0 - deleteBufBytes = 0 - } - return nil - } - - revert := func(k roachpb.Key, v []byte) (*roachpb.Key, error) { - // For non-deletions, we have to flush any pending deletes first. This may also - // flush a range tombstone, which will add to batchBytes. - if len(v) > 0 { - if err := flushDeletes(k); err != nil { - return nil, err - } - } - - // If the batch is full, return a resume key after flushing any deletes. - if batchSize >= maxBatchSize || batchBytes >= maxBatchBytes { - err := flushDeletes(k) - return &k, err - } - bytes := int64(encodedMVCCKeyLength(MVCCKey{Key: k, Timestamp: writeTimestamp}) + len(v)) - - if len(v) > 0 || len(deleteBuf) == 0 { - batchSize++ - batchBytes += bytes - iter := newMVCCIterator(rw, false, IterOptions{Prefix: true}) - defer iter.Close() - return nil, mvccPutInternal(ctx, rw, iter, ms, k, writeTimestamp, v, nil, putBuf, nil) - - } else if len(deleteRangeStart) == 0 { - // We're currently buffering point deletions. - if deleteBufIdx < len(deleteBuf) { - deleteBuf[deleteBufIdx] = append(deleteBuf[deleteBufIdx][:0], k...) - deleteBufIdx++ - deleteBufBytes += bytes - batchSize++ - batchBytes += bytes - } else { - // Buffer is full -- switch to tracking the start of the range delete. We - // remove the buffered keys from the batch size, and instead only track - // the range key. - batchSize -= int64(deleteBufIdx) - 1 // -1 accounts for the range key - batchBytes -= deleteBufBytes - - int64(encodedMVCCKeyLength(MVCCKey{Key: deleteBuf[0], Timestamp: writeTimestamp})) - deleteRangeStart = deleteBuf[0] - deleteBufIdx = 0 - deleteBufBytes = 0 - } - } - return nil, nil - } - - // We set up an incremental iterator from the revert time to look for any - // changes that need to be reverted. However, we also need to inspect older - // values to e.g. find the value to revert to or make sure we don't drop range - // tombstones across them -- we do this by using the IgnoringTime() methods on - // the MVCCIncrementalIterator. - iter := NewMVCCIncrementalIterator(rw, MVCCIncrementalIterOptions{ - EnableTimeBoundIteratorOptimization: true, - EndKey: endKey, - StartTime: revertTimestamp, - EndTime: writeTimestamp, // puts will error on any newer versions - }) - defer iter.Close() - - // TODO(erikgrinaker): Consider rewriting the below to iterate over keys and - // versions separately, which might make the logic clearer. Also consider - // using a struct to manage the deletion state. - var revertKey roachpb.Key - var revertValue, revertValueFrom []byte - iter.SeekGE(MVCCKey{Key: startKey}) - for { - if ok, err := iter.Valid(); err != nil { - return nil, err - } else if !ok { - break - } - - key := iter.UnsafeKey() - - if key.Timestamp.IsEmpty() { - return nil, errors.Errorf("encountered inline key %s", key) - } - - // If a key was scheduled for reversion, revert it when the key changes, - // but only if the original value differs from the latest value. - if len(revertKey) > 0 && !revertKey.Equal(key.Key) { - if !bytes.Equal(revertValue, revertValueFrom) { - if resumeKey, err := revert(revertKey, revertValue); err != nil || resumeKey != nil { - return &roachpb.Span{Key: *resumeKey, EndKey: endKey}, err - } - } - revertKey, revertValue, revertValueFrom = nil, nil, nil // TODO(erikgrinaker): reuse slices - } - - if revertTimestamp.Less(key.Timestamp) { - // Schedule this key for reversion. - if len(revertKey) == 0 { - // TODO(erikgrinaker): reuse byte slices - revertKey = key.Key.Clone() - revertValueFrom = append([]byte(nil), iter.Value()...) - } - - // Move the iterator to the next key, even if <= resumeTimestamp. If it - // finds an old version of this key, it will set the value to revert to. - iter.NextIgnoringTime() - - } else if bytes.Equal(revertKey, key.Key) { - // This is the version of revertKey that we should revert back to. If it - // is visible we can move to the next TBI key, because it's going to flush - // any pending deletes. However, if it's a tombstone we have to move to - // the next key even if it's outside of the TBI bounds to see if it's - // visible and avoid dropping a range tombstone across it. - revertValue = append([]byte(nil), iter.Value()...) - if len(revertValue) > 0 { - iter.Next() - } else { - iter.NextKeyIgnoringTime() - } - - } else if len(iter.Value()) > 0 { - // This is a different visible key at or below the revert timestamp. We - // have to flush any deletes up to here to avoid dropping a range - // tombstone across it. - if err := flushDeletes(key.Key); err != nil { - return nil, err - } - iter.Next() - - } else { - // This is a tombstone for a different key. We have to move to the next - // key (ignoring TBI) to check whether it could be a visible key outside - // of the time bounds -- if it is, we don't want to drop a range tombstone - // across it. - iter.NextKeyIgnoringTime() - } - } - - // Handle a revert at the very end of the iteration. - if len(revertKey) > 0 && !bytes.Equal(revertValue, revertValueFrom) { - if resumeKey, err := revert(revertKey, revertValue); err != nil || resumeKey != nil { - return &roachpb.Span{Key: *resumeKey, EndKey: endKey}, err - } - } - return nil, flushDeletes(endKey) -} - func mvccScanToBytes( ctx context.Context, iter MVCCIterator, diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 9451f0beb91d..cd3f571bf4e0 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -55,21 +55,17 @@ import ( // resolve_intent t= k= [status=] // check_intent k= [none] // -// 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=] -// iter_range_keys k= end= [minTS=[,]] [maxTS=[,]] [fragmented] -// put [t=] [ts=[,]] [resolve [status=]] k= v= [raw] -// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] +// 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] +// 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] +// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] // // merge [ts=[,]] k= v= [raw] // -// clear_range k= end= -// clear_range_key k= end= [ts=[,]] -// revert_range [ts=[,]] k= end= revertTS=[,int] [deleteRangeThreshold=] [maxBatchSize=] [maxBatchBytes=] +// clear_range k= end= // // Where `` can be a simple string, or a string // prefixed by the following characters: @@ -116,25 +112,8 @@ func TestMVCCHistories(t *testing.T) { defer engine.Close() reportDataEntries := func(buf *redact.StringBuilder) error { - var hasData bool - - iter := NewMVCCRangeKeyIterator(engine, MVCCRangeKeyIterOptions{ - LowerBound: span.Key, - UpperBound: span.EndKey, - }) - defer iter.Close() - for { - if ok, err := iter.Valid(); err != nil { - return err - } else if !ok { - break - } - hasData = true - buf.Printf("range key: %s -> %+v\n", iter.Key(), iter.Value()) - iter.Next() - } - - err = engine.MVCCIterate(span.Key, span.EndKey, MVCCKeyAndIntentsIterKind, func(r MVCCKeyValue) error { + hasData := false + err := engine.MVCCIterate(span.Key, span.EndKey, MVCCKeyAndIntentsIterKind, func(r MVCCKeyValue) error { hasData = true if r.Key.Timestamp.IsEmpty() { // Meta is at timestamp zero. @@ -417,19 +396,15 @@ var commands = map[string]cmd{ // TODO(nvanbenschoten): test "resolve_intent_range". "check_intent": {typReadOnly, cmdCheckIntent}, - "clear_range": {typDataUpdate, cmdClearRange}, - "clear_range_key": {typDataUpdate, cmdClearRangeKey}, - "cput": {typDataUpdate, cmdCPut}, - "del": {typDataUpdate, cmdDelete}, - "del_range": {typDataUpdate, cmdDeleteRange}, - "del_range_ts": {typDataUpdate, cmdDeleteRangeTombstone}, - "get": {typReadOnly, cmdGet}, - "increment": {typDataUpdate, cmdIncrement}, - "iter_range_keys": {typReadOnly, cmdIterRangeKeys}, - "merge": {typDataUpdate, cmdMerge}, - "put": {typDataUpdate, cmdPut}, - "revert_range": {typDataUpdate, cmdRevertRange}, - "scan": {typReadOnly, cmdScan}, + "clear_range": {typDataUpdate, cmdClearRange}, + "cput": {typDataUpdate, cmdCPut}, + "del": {typDataUpdate, cmdDelete}, + "del_range": {typDataUpdate, cmdDeleteRange}, + "get": {typReadOnly, cmdGet}, + "increment": {typDataUpdate, cmdIncrement}, + "merge": {typDataUpdate, cmdMerge}, + "put": {typDataUpdate, cmdPut}, + "scan": {typReadOnly, cmdScan}, } func cmdTxnAdvance(e *evalCtx) error { @@ -606,37 +581,7 @@ func cmdCheckIntent(e *evalCtx) error { func cmdClearRange(e *evalCtx) error { key, endKey := e.getKeyRange() - if err := e.engine.ClearMVCCRangeAndIntents(key, endKey); err != nil { - return err - } - // TODO(erikgrinaker): Consider removing range keys in ClearMVCCRangeAndIntents. - iter := NewMVCCRangeKeyIterator(e.engine, MVCCRangeKeyIterOptions{ - LowerBound: key, - UpperBound: endKey, - }) - defer iter.Close() - for { - if ok, err := iter.Valid(); err != nil { - return err - } else if !ok { - break - } - if err := e.engine.ExperimentalClearMVCCRangeKey(iter.Key()); err != nil { - return err - } - iter.Next() - } - return nil -} - -func cmdClearRangeKey(e *evalCtx) error { - key, endKey := e.getKeyRange() - ts := e.getTs(nil) - return e.engine.ExperimentalClearMVCCRangeKey(MVCCRangeKey{ - StartKey: key, - EndKey: endKey, - Timestamp: ts, - }) + return e.engine.ClearMVCCRangeAndIntents(key, endKey) } func cmdCPut(e *evalCtx) error { @@ -715,24 +660,6 @@ func cmdDeleteRange(e *evalCtx) error { }) } -func cmdDeleteRangeTombstone(e *evalCtx) error { - key, endKey := e.getKeyRange() - ts := e.getTs(nil) - - return e.withWriter("del_range", func(rw ReadWriter) error { - err := ExperimentalMVCCDeleteRangeUsingTombstone(e.ctx, rw, nil, key, endKey, ts, 0) - if err != nil { - return err - } - e.results.buf.Printf("del_range_ts: %s\n", MVCCRangeKey{ - StartKey: key, - EndKey: endKey, - Timestamp: ts, - }) - return nil - }) -} - func cmdGet(e *evalCtx) error { txn := e.getTxn(optional) key := e.getKey() @@ -836,39 +763,6 @@ func cmdPut(e *evalCtx) error { }) } -func cmdRevertRange(e *evalCtx) error { - ts := e.getTs(nil) - revertTS := e.getTsWithName(nil, "revertTS") - key, endKey := e.getKeyRange() - - deleteRangeThreshold := 100 - if e.hasArg("deleteRangeThreshold") { - e.scanArg("deleteRangeThreshold", &deleteRangeThreshold) - } - - maxBatchSize := 1000 - if e.hasArg("maxBatchSize") { - e.scanArg("maxBatchSize", &maxBatchSize) - } - - maxBatchBytes := int(1e6) - if e.hasArg("maxBatchBytes") { - e.scanArg("maxBatchBytes", &maxBatchBytes) - } - - return e.withWriter("revertRange", func(rw ReadWriter) error { - resumeSpan, err := ExperimentalMVCCRevertRange(e.ctx, rw, nil, key, endKey, ts, revertTS, - deleteRangeThreshold, int64(maxBatchSize), int64(maxBatchBytes), 1000) - if err != nil { - return err - } - if resumeSpan != nil { - e.results.buf.Printf("revert_range: resume span [%s,%s)\n", resumeSpan.Key, resumeSpan.EndKey) - } - return err - }) -} - func cmdScan(e *evalCtx) error { txn := e.getTxn(optional) key, endKey := e.getKeyRange() @@ -938,33 +832,6 @@ func cmdScan(e *evalCtx) error { return err } -func cmdIterRangeKeys(e *evalCtx) error { - opts := MVCCRangeKeyIterOptions{} - opts.LowerBound, opts.UpperBound = e.getKeyRange() - opts.MinTimestamp = e.getTsWithName(nil, "minTS") - opts.MaxTimestamp = e.getTsWithName(nil, "maxTS") - opts.Fragmented = e.hasArg("fragmented") - - iter := NewMVCCRangeKeyIterator(e.engine, opts) - ok, err := iter.Valid() - if err != nil { - return err - } - if !ok { - e.results.buf.Printf("iter_range_keys: %v-%v -> \n", opts.LowerBound, opts.UpperBound) - } - for { - if ok, err = iter.Valid(); err != nil { - return err - } else if !ok { - break - } - e.results.buf.Printf("iter_range_keys: %s -> %v\n", iter.Key(), iter.Value()) - iter.Next() - } - return nil -} - // evalCtx stored the current state of the environment of a running // script. type evalCtx struct { diff --git a/pkg/storage/mvcc_incremental_iterator.go b/pkg/storage/mvcc_incremental_iterator.go index 2aa1a2a0d07b..db4a5e397073 100644 --- a/pkg/storage/mvcc_incremental_iterator.go +++ b/pkg/storage/mvcc_incremental_iterator.go @@ -485,21 +485,6 @@ func (i *MVCCIncrementalIterator) UnsafeKey() MVCCKey { return i.iter.UnsafeKey() } -// HasPointAndRange implements SimpleMVCCIterator. -func (i *MVCCIncrementalIterator) HasPointAndRange() (bool, bool) { - panic("not implemented") -} - -// RangeBounds implements SimpleMVCCIterator. -func (i *MVCCIncrementalIterator) RangeBounds() (roachpb.Key, roachpb.Key) { - panic("not implemented") -} - -// RangeKeys implements SimpleMVCCIterator. -func (i *MVCCIncrementalIterator) RangeKeys() []MVCCRangeKeyValue { - panic("not implemented") -} - // UnsafeValue returns the same value as Value, but the memory is invalidated on // the next call to {Next,Reset,Close}. func (i *MVCCIncrementalIterator) UnsafeValue() []byte { diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 58d68de08695..02cb4016ba92 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -173,12 +173,6 @@ func encodeMVCCKeyToBuf(buf []byte, key MVCCKey, keyLen int) { } } -// encodeMVCCKeyPrefix encodes an MVCC user key (without timestamp) into its -// Pebble prefix representation. -func encodeMVCCKeyPrefix(key roachpb.Key) []byte { - return EncodeMVCCKey(MVCCKey{Key: key}) -} - // encodeMVCCTimestamp encodes an MVCC timestamp into its Pebble // representation, excluding length suffix and sentinel byte. func encodeMVCCTimestamp(ts hlc.Timestamp) []byte { @@ -293,68 +287,3 @@ func decodeMVCCTimestampSuffix(encodedTS []byte) (hlc.Timestamp, error) { } return decodeMVCCTimestamp(encodedTS[:encodedLen-1]) } - -// MVCCRangeKey is a versioned key span. -type MVCCRangeKey struct { - StartKey roachpb.Key - EndKey roachpb.Key - Timestamp hlc.Timestamp -} - -// Clone returns a copy of the range key. -func (k MVCCRangeKey) Clone() MVCCRangeKey { - // k is already a copy, but byte slices must be cloned. - k.StartKey = k.StartKey.Clone() - k.EndKey = k.EndKey.Clone() - return k -} - -// Compare returns -1 if this key is less than the given key, 0 if they're -// equal, or 1 if this is greater. Comparison is by start,timestamp,end, where -// larger timestamps sort before smaller ones except empty ones which sort first -// (like elsewhere in MVCC). -func (k MVCCRangeKey) Compare(o MVCCRangeKey) int { - if c := k.StartKey.Compare(o.StartKey); c != 0 { - return c - } - if k.Timestamp.IsEmpty() && !o.Timestamp.IsEmpty() { - return -1 - } else if !k.Timestamp.IsEmpty() && o.Timestamp.IsEmpty() { - return 1 - } else if c := k.Timestamp.Compare(o.Timestamp); c != 0 { - return -c // timestamps sort in reverse - } - return k.EndKey.Compare(o.EndKey) -} - -// String formats the range key. -func (k MVCCRangeKey) String() string { - s := roachpb.Span{Key: k.StartKey, EndKey: k.EndKey}.String() - if !k.Timestamp.IsEmpty() { - s += fmt.Sprintf("/%s", k.Timestamp) - } - return s -} - -// Validate returns an error if the range key is invalid. -func (k MVCCRangeKey) Validate() (err error) { - defer func() { - err = errors.Wrapf(err, "invalid range key %s", k) - }() - - if k.StartKey == nil { - return errors.Errorf("no start key") - } - if k.EndKey == nil { - return errors.Errorf("no end key") - } - if k.Timestamp.IsEmpty() { - return errors.Errorf("no timestamp") - } - // We allow equal start and end key, since we allow empty spans in many MVCC - // APIs (e.g. scans). - if k.StartKey.Compare(k.EndKey) > 0 { - return errors.Errorf("start key %s is after end key %s", k.StartKey, k.EndKey) - } - return nil -} diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index 0b651f2838c7..63af9beda842 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -238,94 +238,3 @@ func BenchmarkDecodeMVCCKey(b *testing.B) { } benchmarkDecodeMVCCKeyResult = mvccKey // avoid compiler optimizing away function call } - -func TestMVCCRangeKeyString(t *testing.T) { - defer leaktest.AfterTest(t)() - - testcases := map[string]struct { - rk MVCCRangeKey - expect string - }{ - "empty": {MVCCRangeKey{}, "/Min"}, - "only start": {MVCCRangeKey{StartKey: roachpb.Key("foo")}, "foo"}, - "only end": {MVCCRangeKey{EndKey: roachpb.Key("foo")}, "{/Min-foo}"}, - "only timestamp": {MVCCRangeKey{Timestamp: hlc.Timestamp{Logical: 1}}, "/Min/0,1"}, - "only span": {MVCCRangeKey{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("z")}, "{a-z}"}, - "all": {MVCCRangeKey{StartKey: roachpb.Key("a"), EndKey: roachpb.Key("z"), Timestamp: hlc.Timestamp{Logical: 1}}, "{a-z}/0,1"}, - "all overlapping": {MVCCRangeKey{StartKey: roachpb.Key("ab"), EndKey: roachpb.Key("af"), Timestamp: hlc.Timestamp{Logical: 1}}, "a{b-f}/0,1"}, - } - for name, tc := range testcases { - t.Run(name, func(t *testing.T) { - require.Equal(t, tc.expect, tc.rk.String()) - }) - } -} - -func TestMVCCRangeKeyCompare(t *testing.T) { - defer leaktest.AfterTest(t)() - - ab1 := MVCCRangeKey{roachpb.Key("a"), roachpb.Key("b"), hlc.Timestamp{Logical: 1}} - ac1 := MVCCRangeKey{roachpb.Key("a"), roachpb.Key("c"), hlc.Timestamp{Logical: 1}} - ac2 := MVCCRangeKey{roachpb.Key("a"), roachpb.Key("c"), hlc.Timestamp{Logical: 2}} - bc0 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("c"), hlc.Timestamp{Logical: 0}} - bc1 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("c"), hlc.Timestamp{Logical: 1}} - bc3 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("c"), hlc.Timestamp{Logical: 3}} - bd4 := MVCCRangeKey{roachpb.Key("b"), roachpb.Key("d"), hlc.Timestamp{Logical: 4}} - - testcases := map[string]struct { - a MVCCRangeKey - b MVCCRangeKey - expect int - }{ - "equal": {ac1, ac1, 0}, - "start lt": {ac1, bc1, -1}, - "start gt": {bc1, ac1, 1}, - "end lt": {ab1, ac1, -1}, - "end gt": {ac1, ab1, 1}, - "time lt": {ac2, ac1, -1}, // MVCC timestamps sort in reverse order - "time gt": {ac1, ac2, 1}, // MVCC timestamps sort in reverse order - "empty time lt set": {bc0, bc1, -1}, // empty MVCC timestamps sort before non-empty - "set time gt empty": {bc1, bc0, 1}, // empty MVCC timestamps sort before non-empty - "start time precedence": {ac2, bc3, -1}, // a before b, but 3 before 2; key takes precedence - "time end precedence": {bd4, bc3, -1}, // c before d, but 4 before 3; time takes precedence - } - for name, tc := range testcases { - t.Run(name, func(t *testing.T) { - require.Equal(t, tc.expect, tc.a.Compare(tc.b)) - }) - } -} - -func TestMVCCRangeKeyValidate(t *testing.T) { - defer leaktest.AfterTest(t)() - - a := roachpb.Key("a") - b := roachpb.Key("b") - blank := roachpb.Key("") - ts1 := hlc.Timestamp{Logical: 1} - - testcases := map[string]struct { - rangeKey MVCCRangeKey - expectErr string // empty if no error - }{ - "valid": {MVCCRangeKey{StartKey: a, EndKey: b, Timestamp: ts1}, ""}, - "start at end": {MVCCRangeKey{StartKey: a, EndKey: a, Timestamp: ts1}, ""}, - "blank keys": {MVCCRangeKey{StartKey: blank, EndKey: blank, Timestamp: ts1}, ""}, // equivalent to MinKey - "no start": {MVCCRangeKey{EndKey: b, Timestamp: ts1}, "{/Min-b}/0,1: no start key"}, - "no end": {MVCCRangeKey{StartKey: a, Timestamp: ts1}, "a/0,1: no end key"}, - "no timestamp": {MVCCRangeKey{StartKey: a, EndKey: b}, "{a-b}: no timestamp"}, - "empty": {MVCCRangeKey{}, "/Min: no start key"}, - "end before start": {MVCCRangeKey{StartKey: b, EndKey: a, Timestamp: ts1}, `{b-a}/0,1: start key "b" is after end key "a"`}, - } - for name, tc := range testcases { - t.Run(name, func(t *testing.T) { - err := tc.rangeKey.Validate() - if tc.expectErr == "" { - require.NoError(t, err) - } else { - require.Error(t, err) - require.Contains(t, err.Error(), tc.expectErr) - } - }) - } -} diff --git a/pkg/storage/mvcc_range_key_iterator.go b/pkg/storage/mvcc_range_key_iterator.go deleted file mode 100644 index cd343ec3fb46..000000000000 --- a/pkg/storage/mvcc_range_key_iterator.go +++ /dev/null @@ -1,267 +0,0 @@ -// 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 storage - -import ( - "bytes" - - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/errors" -) - -// MVCCRangeKeyIterOptions are options for an MVCCRangeKeyIterator. -type MVCCRangeKeyIterOptions struct { - // LowerBound sets the inclusive lower bound of the iterator. Range keys that - // straddle the bound will have their start key truncated to it. - LowerBound roachpb.Key - // UpperBound sets the exclusive upper bound of the iterator. Range keys that - // straddle the upper bound will have their end key truncated to it. - UpperBound roachpb.Key - // MinTimestamp sets the inclusive lower timestamp bound for the iterator. - MinTimestamp hlc.Timestamp - // MaxTimestamp sets the inclusive upper timestamp bound for the iterator. - MaxTimestamp hlc.Timestamp - // Fragmented disables defragmentation, emitting non-deterministic fragments - // like SimpleMVCCIterator does. When enabled, this results in an iteration - // order of StartKey,Timestamp rather than EndKey,Timestamp. - Fragmented bool -} - -// MVCCRangeKeyIterator is an iterator over range keys in an engine. Unlike -// SimpleMVCCIterator, range keys are defragmented into contiguous deterministic -// range keys. It does not support seeking or backtracking, see -// MVCCRangeKeyIterOptions for lower/upper bounds and other options. -// -// Iteration is in EndKey,Timestamp order rather than StartKey,Timestamp. For -// example: [c-e)@2, [a-z)@3, [x-z)@1. This is a memory optimization when -// defragmenting, which allows emitting completed range keys as soon as -// possible, only buffering incomplete ones in memory. To emit in -// StartKey,Timestamp order, we would additionally need to buffer all complete -// range keys that start after the current incomplete ones -- in the worst case, -// a range key across the entire key span would require all other range keys to -// be buffered in memory. But see the Fragmented option to emit -// non-deterministic range key fragments in StartKey,Timestamp order. -type MVCCRangeKeyIterator struct { - iter MVCCIterator - opts MVCCRangeKeyIterOptions - incomplete []*MVCCRangeKeyValue // defragmentation buffer - complete []MVCCRangeKeyValue // queued for emission - completeIdx int // current Key() - err error -} - -// NewMVCCRangeKeyIterator sets up a new MVCCRangeKeyIterator and seeks to the -// first range key. The caller must call Close() when done. -func NewMVCCRangeKeyIterator(r Reader, opts MVCCRangeKeyIterOptions) *MVCCRangeKeyIterator { - iter := &MVCCRangeKeyIterator{ - iter: r.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ - KeyTypes: IterKeyTypeRangesOnly, - LowerBound: opts.LowerBound, - UpperBound: opts.UpperBound, - // TODO(erikgrinaker): We do not set Min/MaxTimestampHint here, because - // both are required and it's apparently not always safe to use. - }), - opts: opts, - incomplete: make([]*MVCCRangeKeyValue, 0), - complete: make([]MVCCRangeKeyValue, 0), - } - - // Seek the iterator to the lower bound and iterate until we've collected - // the first complete range key (if any). - iter.iter.SeekGE(MVCCKey{Key: opts.LowerBound}) - iter.findCompleteRangeKeys() - - return iter -} - -// findCompleteRangeKeys defragments range keys at the current iterator position -// and any subsequent iterator positions until it completes one or more range -// keys, populating p.complete. Current p.complete is discarded. -func (p *MVCCRangeKeyIterator) findCompleteRangeKeys() { - p.complete = p.complete[:0] - p.completeIdx = 0 - p.updateRangeKeys() - - for len(p.complete) == 0 { - if ok, err := p.iter.Valid(); err != nil { - p.err = err - return - } else if !ok { - break - } - p.iter.Next() - // NB: We update range keys even if Next() invalidates the iterator, because - // there may be incomplete range keys that become complete when the iterator - // is exhausted. - p.updateRangeKeys() - } -} - -// updateRangeKeys inspects the range keys at the current Pebble iterator -// position, defragments them in p.incomplete, and moves any completed -// range keys into p.complete. -func (p *MVCCRangeKeyIterator) updateRangeKeys() { - var startKey, endKey roachpb.Key - var rangeKeys []MVCCRangeKeyValue - - // If the iterator is exhausted, we still want to complete any remaining - // incomplete range keys. - if ok, err := p.iter.Valid(); err != nil { - p.err = err - return - } else if ok { - startKey, endKey = p.iter.RangeBounds() - rangeKeys = p.iter.RangeKeys() - } - - // Both rangeKeys and p.incomplete are sorted in descending timestamp order, - // so we iterate over them in lockstep and insert/update/delete p.incomplete - // as appropriate. - var tsIdx, rkIdx int - - for rkIdx < len(rangeKeys) { - rangeKey := rangeKeys[rkIdx] - - // Filter rangekeys by timestamp. - // - // TODO(erikgrinaker): This can be optimized to skip unnecessary comparisons - // since rangeKeys is sorted by timestamp. Maybe later. - if !p.opts.MinTimestamp.IsEmpty() && rangeKey.Key.Timestamp.Less(p.opts.MinTimestamp) { - rkIdx++ - continue - } - if !p.opts.MaxTimestamp.IsEmpty() && p.opts.MaxTimestamp.Less(rangeKey.Key.Timestamp) { - rkIdx++ - continue - } - - // If we're at the end of p.incomplete, this range key must be new. - if tsIdx >= len(p.incomplete) { - p.incomplete = append(p.incomplete, &MVCCRangeKeyValue{ - Key: MVCCRangeKey{ - StartKey: startKey.Clone(), - EndKey: endKey.Clone(), - Timestamp: rangeKey.Key.Timestamp, - }, - Value: append([]byte{}, rangeKey.Value...), - }) - rkIdx++ - tsIdx++ - continue - } - - incomplete := p.incomplete[tsIdx] - cmp := incomplete.Key.Timestamp.Compare(rangeKey.Key.Timestamp) - switch { - // If the timestamps match, the key spans are adjacent or overlapping, and - // the values match then this range key extends the incomplete one. - case cmp == 0 && bytes.Compare(startKey, incomplete.Key.EndKey) <= 0 && - bytes.Equal(rangeKey.Value, incomplete.Value): - incomplete.Key.EndKey = append(incomplete.Key.EndKey[:0], endKey...) - tsIdx++ - rkIdx++ - - // This is a different range key at the same timestamp: complete the - // existing one and start a new one. - case cmp == 0: - p.complete = append(p.complete, *incomplete) - // NB: can't reuse slices, since they were placed in the completed range key. - incomplete.Key.StartKey = startKey.Clone() - incomplete.Key.EndKey = endKey.Clone() - incomplete.Value = append([]byte{}, rangeKey.Value...) - tsIdx++ - rkIdx++ - - // This incomplete range key is not present at this range key: complete it - // and remove it from the list, then try the current range key again. - case cmp == 1: - p.complete = append(p.complete, *incomplete) - p.incomplete = append(p.incomplete[:tsIdx], p.incomplete[tsIdx+1:]...) - - // This range key is a new incomplete range key: start defragmenting it. - case cmp == -1: - p.incomplete = append(p.incomplete[:tsIdx+1], p.incomplete[tsIdx:]...) - p.incomplete[tsIdx] = &MVCCRangeKeyValue{ - Key: MVCCRangeKey{ - StartKey: startKey.Clone(), - EndKey: endKey.Clone(), - Timestamp: rangeKey.Key.Timestamp, - }, - Value: append([]byte{}, rangeKey.Value...), - } - tsIdx++ - rkIdx++ - - default: - p.err = errors.AssertionFailedf("unexpected comparison result %d", cmp) - return - } - } - - // If the caller has requested fragments, we complete all range keys we found - // this iteration by resetting tsIdx to 0. The loop below handles the rest. - if p.opts.Fragmented { - tsIdx = 0 - } - - // If there are any remaining incomplete range keys, they must be complete: - // make them so. - for _, ts := range p.incomplete[tsIdx:] { - p.complete = append(p.complete, *ts) - } - p.incomplete = p.incomplete[:tsIdx] -} - -// Close frees up resources held by the iterator. -func (p *MVCCRangeKeyIterator) Close() { - p.iter.Close() - p.complete = nil - p.completeIdx = 0 -} - -// Next iterates to the next defragmented range key. Note the unusual iteration -// order of EndKey,Timestamp -- see MVCCRangeKeyIterator comment for details. -func (p *MVCCRangeKeyIterator) Next() { - p.completeIdx++ - if p.completeIdx >= len(p.complete) { - p.iter.Next() - // NB: Called even if iterator is now invalid, because we may have - // incomplete range keys that become complete when the iterator is - // exhausted. - p.findCompleteRangeKeys() - } -} - -// Key returns the current range key. It will not be modified by the iterator, -// but it will be shared by all callers. -func (p *MVCCRangeKeyIterator) Key() MVCCRangeKey { - return p.complete[p.completeIdx].Key -} - -// Value returns the value of the current range key. It will not be modified -// by the iterator, but it will be shared by all callers. -func (p *MVCCRangeKeyIterator) Value() []byte { - return p.complete[p.completeIdx].Value -} - -// Valid returns (true, nil) if the iterator points to a valid key, (false, nil) -// if the iterator is exhausted, or (false, error) if an error occurred during -// iteration. -func (p *MVCCRangeKeyIterator) Valid() (bool, error) { - if p.err != nil { - return false, p.err - } - if _, err := p.iter.Valid(); err != nil { - return false, err - } - return p.completeIdx < len(p.complete), nil -} diff --git a/pkg/storage/mvcc_range_key_iterator_test.go b/pkg/storage/mvcc_range_key_iterator_test.go deleted file mode 100644 index 90267ad2a456..000000000000 --- a/pkg/storage/mvcc_range_key_iterator_test.go +++ /dev/null @@ -1,292 +0,0 @@ -// 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 storage - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/stretchr/testify/require" -) - -func TestMVCCRangeKeyIterator(t *testing.T) { - defer leaktest.AfterTest(t)() - - eng := NewDefaultInMemForTesting() - defer eng.Close() - - rangeKeys := []MVCCRangeKeyValue{ - rangeKV("b", "c", 3, "bc3"), - rangeKV("e", "g", 3, "eg3"), - rangeKV("d", "f", 5, "df5"), - rangeKV("f", "g", 5, "fg5"), - rangeKV("d", "f", 2, "df2"), - rangeKV("a", "m", 4, "az4"), // same value as below so these should merge into one - rangeKV("m", "z", 4, "az4"), - rangeKV("x", "z", 1, ""), // range tombstone - } - for _, rk := range rangeKeys { - require.NoError(t, eng.ExperimentalPutMVCCRangeKey(rk.Key, rk.Value)) - } - - testcases := map[string]struct { - opts MVCCRangeKeyIterOptions - expect []MVCCRangeKeyValue - }{ - "all range keys": { - MVCCRangeKeyIterOptions{}, - []MVCCRangeKeyValue{ - rangeKV("b", "c", 3, "bc3"), - rangeKV("d", "f", 5, "df5"), - rangeKV("d", "f", 2, "df2"), - rangeKV("f", "g", 5, "fg5"), - rangeKV("e", "g", 3, "eg3"), - rangeKV("a", "z", 4, "az4"), - rangeKV("x", "z", 1, ""), - }}, - "truncated range keys": { - MVCCRangeKeyIterOptions{ - LowerBound: roachpb.Key("c"), - UpperBound: roachpb.Key("e"), - }, - []MVCCRangeKeyValue{ - rangeKV("d", "e", 5, "df5"), - rangeKV("c", "e", 4, "az4"), - rangeKV("d", "e", 2, "df2"), - }}, - "truncation between range key bounds": { - MVCCRangeKeyIterOptions{ - LowerBound: roachpb.Key("ccc"), - UpperBound: roachpb.Key("eee"), - }, - []MVCCRangeKeyValue{ - rangeKV("d", "eee", 5, "df5"), - rangeKV("ccc", "eee", 4, "az4"), - rangeKV("e", "eee", 3, "eg3"), - rangeKV("d", "eee", 2, "df2"), - }}, - "fragmented range keys": { - MVCCRangeKeyIterOptions{ - Fragmented: true, - }, - []MVCCRangeKeyValue{ - rangeKV("a", "b", 4, "az4"), - rangeKV("b", "c", 4, "az4"), - rangeKV("b", "c", 3, "bc3"), - rangeKV("c", "d", 4, "az4"), - rangeKV("d", "e", 5, "df5"), - rangeKV("d", "e", 4, "az4"), - rangeKV("d", "e", 2, "df2"), - rangeKV("e", "f", 5, "df5"), - rangeKV("e", "f", 4, "az4"), - rangeKV("e", "f", 3, "eg3"), - rangeKV("e", "f", 2, "df2"), - rangeKV("f", "g", 5, "fg5"), - rangeKV("f", "g", 4, "az4"), - rangeKV("f", "g", 3, "eg3"), - rangeKV("g", "x", 4, "az4"), - rangeKV("x", "z", 4, "az4"), - rangeKV("x", "z", 1, ""), - }}, - "empty interval": { - MVCCRangeKeyIterOptions{ - LowerBound: roachpb.Key("A"), - UpperBound: roachpb.Key("Z"), - }, - nil}, - "zero-length interval": { - MVCCRangeKeyIterOptions{ - LowerBound: roachpb.Key("c"), - UpperBound: roachpb.Key("c"), - }, - nil}, - "end after start": { - MVCCRangeKeyIterOptions{ - LowerBound: roachpb.Key("e"), - UpperBound: roachpb.Key("d"), - }, - nil}, - "min timestamp": { - MVCCRangeKeyIterOptions{ - MinTimestamp: hlc.Timestamp{Logical: 3}, - }, - []MVCCRangeKeyValue{ - rangeKV("b", "c", 3, "bc3"), - rangeKV("d", "f", 5, "df5"), - rangeKV("f", "g", 5, "fg5"), - rangeKV("e", "g", 3, "eg3"), - rangeKV("a", "z", 4, "az4"), - }}, - "max timestamp": { - MVCCRangeKeyIterOptions{ - MaxTimestamp: hlc.Timestamp{Logical: 3}, - }, - []MVCCRangeKeyValue{ - rangeKV("b", "c", 3, "bc3"), - rangeKV("d", "f", 2, "df2"), - rangeKV("e", "g", 3, "eg3"), - rangeKV("x", "z", 1, ""), - }}, - "both timestamps": { - MVCCRangeKeyIterOptions{ - MinTimestamp: hlc.Timestamp{Logical: 3}, - MaxTimestamp: hlc.Timestamp{Logical: 3}, - }, - []MVCCRangeKeyValue{ - rangeKV("b", "c", 3, "bc3"), - rangeKV("e", "g", 3, "eg3"), - }}, - } - for name, tc := range testcases { - t.Run(name, func(t *testing.T) { - opts := tc.opts - if opts.UpperBound == nil { - opts.UpperBound = keys.MaxKey // appease pebbleIterator - } - iter := NewMVCCRangeKeyIterator(eng, opts) - defer iter.Close() - - var rangeKVs []MVCCRangeKeyValue - for { - ok, err := iter.Valid() - require.NoError(t, err) - if !ok { - break - } - rangeKVs = append(rangeKVs, MVCCRangeKeyValue{ - Key: iter.Key(), - Value: iter.Value(), - }) - iter.Next() - } - require.Equal(t, tc.expect, rangeKVs) - }) - } -} - -// TestMVCCRangeKeyIteratorPebbleTimestampBounds tests that MVCCRangeKeyIterator -// and pebbleIterator returns appropriate range bounds, even in corner cases -// where Pebble has range keys split in the middle of timestamps. Pebble should -// defragment these range keys internally. -func TestMVCCIRangeKeyteratorPebbleTimestampBounds(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - eng := NewDefaultInMemForTesting() - defer eng.Close() - db := eng.(*Pebble).db - - // First, just set up some regular old range keys. - require.NoError(t, eng.ExperimentalPutMVCCRangeKey(rangeKey("a", "z", 1), []byte("az1"))) - require.NoError(t, eng.ExperimentalPutMVCCRangeKey(rangeKey("b", "d", 4), []byte("bd4"))) - require.NoError(t, eng.ExperimentalPutMVCCRangeKey(rangeKey("e", "g", 3), []byte("eg3"))) - - // Then, write a range key with suffix bounds. The range key will be - // [b-e)@5=be5, but we'll write it with artificial fragment bounds - // [b-b@5), [b@5-d@2), [d@2-e) - require.NoError(t, db.Experimental().RangeKeySet( // [b-b@5) - EncodeMVCCKey(pointKey("b", 0)), - EncodeMVCCKey(pointKey("b", 5)), - encodeMVCCTimestampSuffix(hlc.Timestamp{Logical: 5}), - []byte("be5"), - nil, - )) - require.NoError(t, db.Experimental().RangeKeySet( // [b@5-d@2) - EncodeMVCCKey(pointKey("b", 5)), - EncodeMVCCKey(pointKey("d", 2)), - encodeMVCCTimestampSuffix(hlc.Timestamp{Logical: 5}), - []byte("be5"), - nil, - )) - require.NoError(t, db.Experimental().RangeKeySet( // [d@2-e) - EncodeMVCCKey(pointKey("d", 2)), - EncodeMVCCKey(pointKey("e", 0)), - encodeMVCCTimestampSuffix(hlc.Timestamp{Logical: 5}), - []byte("be5"), - nil, - )) - - // Scan the fragmented range keys. Pebble should return fragments - // without timestamp bounds. - iter := NewMVCCRangeKeyIterator(eng, MVCCRangeKeyIterOptions{ - Fragmented: true, - UpperBound: keys.MaxKey, - }) - defer iter.Close() - - var actual []MVCCRangeKeyValue - for { - ok, err := iter.Valid() - require.NoError(t, err) - if !ok { - break - } - actual = append(actual, MVCCRangeKeyValue{Key: iter.Key(), Value: iter.Value()}) - iter.Next() - } - require.Equal(t, []MVCCRangeKeyValue{ - rangeKV("a", "b", 1, "az1"), - rangeKV("b", "d", 5, "be5"), - rangeKV("b", "d", 4, "bd4"), - rangeKV("b", "d", 1, "az1"), - rangeKV("d", "e", 5, "be5"), - rangeKV("d", "e", 1, "az1"), - rangeKV("e", "g", 3, "eg3"), - rangeKV("e", "g", 1, "az1"), - rangeKV("g", "z", 1, "az1"), - }, actual) - - // Scan the defragmented range keys. - iter = NewMVCCRangeKeyIterator(eng, MVCCRangeKeyIterOptions{ - UpperBound: keys.MaxKey, - }) - defer iter.Close() - - actual = nil - for { - ok, err := iter.Valid() - require.NoError(t, err) - if !ok { - break - } - actual = append(actual, MVCCRangeKeyValue{Key: iter.Key(), Value: iter.Value()}) - iter.Next() - } - require.Equal(t, []MVCCRangeKeyValue{ - rangeKV("b", "d", 4, "bd4"), - rangeKV("b", "e", 5, "be5"), - rangeKV("e", "g", 3, "eg3"), - rangeKV("a", "z", 1, "az1"), - }, actual) -} - -func rangeKey(start, end string, ts int) MVCCRangeKey { - return MVCCRangeKey{ - StartKey: roachpb.Key(start), - EndKey: roachpb.Key(end), - Timestamp: hlc.Timestamp{Logical: int32(ts)}, - } -} - -func rangeKV(start, end string, ts int, value string) MVCCRangeKeyValue { - return MVCCRangeKeyValue{ - Key: rangeKey(start, end, ts), - Value: []byte(value), - } -} - -func pointKey(key string, ts int) MVCCKey { - return MVCCKey{Key: roachpb.Key(key), Timestamp: hlc.Timestamp{Logical: int32(ts)}} -} diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index acdaa7104107..0c1c87b4e17e 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -498,8 +498,6 @@ func DefaultPebbleOptions() *pebble.Options { TablePropertyCollectors: PebbleTablePropertyCollectors, BlockPropertyCollectors: PebbleBlockPropertyCollectors, } - // Used for experimental MVCC range tombstones. - opts.Experimental.RangeKeys = new(pebble.RangeKeysArena) // Automatically flush 10s after the first range tombstone is added to a // memtable. This ensures that we can reclaim space even when there's no // activity on the database generating flushes. @@ -1147,31 +1145,6 @@ func (p *Pebble) ClearIterRange(iter MVCCIterator, start, end roachpb.Key) error return batch.Commit(true) } -// ExperimentalClearMVCCRangeKey implements the Engine interface. -func (p *Pebble) ExperimentalClearMVCCRangeKey(rangeKey MVCCRangeKey) error { - if err := rangeKey.Validate(); err != nil { - return err - } - return p.db.Experimental().RangeKeyUnset( - encodeMVCCKeyPrefix(rangeKey.StartKey), - encodeMVCCKeyPrefix(rangeKey.EndKey), - encodeMVCCTimestampSuffix(rangeKey.Timestamp), - pebble.Sync) -} - -// ExperimentalPutMVCCRangeKey implements the Engine interface. -func (p *Pebble) ExperimentalPutMVCCRangeKey(rangeKey MVCCRangeKey, value []byte) error { - if err := rangeKey.Validate(); err != nil { - return err - } - return p.db.Experimental().RangeKeySet( - encodeMVCCKeyPrefix(rangeKey.StartKey), - encodeMVCCKeyPrefix(rangeKey.EndKey), - encodeMVCCTimestampSuffix(rangeKey.Timestamp), - value, - pebble.Sync) -} - // Merge implements the Engine interface. func (p *Pebble) Merge(key MVCCKey, value []byte) error { if len(key.Key) == 0 { @@ -1987,14 +1960,6 @@ func (p *pebbleReadOnly) ClearIterRange(iter MVCCIterator, start, end roachpb.Ke panic("not implemented") } -func (p *pebbleReadOnly) ExperimentalPutMVCCRangeKey(_ MVCCRangeKey, _ []byte) error { - panic("not implemented") -} - -func (p *pebbleReadOnly) ExperimentalClearMVCCRangeKey(_ MVCCRangeKey) error { - panic("not implemented") -} - func (p *pebbleReadOnly) Merge(key MVCCKey, value []byte) error { panic("not implemented") } diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index 708043c79893..371f430dd590 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -413,31 +413,6 @@ func (p *pebbleBatch) ClearIterRange(iter MVCCIterator, start, end roachpb.Key) return nil } -// ExperimentalClearMVCCRangeKey implements the Engine interface. -func (p *pebbleBatch) ExperimentalClearMVCCRangeKey(rangeKey MVCCRangeKey) error { - if err := rangeKey.Validate(); err != nil { - return err - } - return p.batch.Experimental().RangeKeyUnset( - encodeMVCCKeyPrefix(rangeKey.StartKey), - encodeMVCCKeyPrefix(rangeKey.EndKey), - encodeMVCCTimestampSuffix(rangeKey.Timestamp), - nil) -} - -// ExperimentalPutMVCCRangeKey implements the Batch interface. -func (p *pebbleBatch) ExperimentalPutMVCCRangeKey(rangeKey MVCCRangeKey, value []byte) error { - if err := rangeKey.Validate(); err != nil { - return err - } - return p.batch.Experimental().RangeKeySet( - encodeMVCCKeyPrefix(rangeKey.StartKey), - encodeMVCCKeyPrefix(rangeKey.EndKey), - encodeMVCCTimestampSuffix(rangeKey.Timestamp), - value, - nil) -} - // Merge implements the Batch interface. func (p *pebbleBatch) Merge(key MVCCKey, value []byte) error { if len(key.Key) == 0 { diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index 75c0ff66b36f..ea7a7b77a0d9 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -181,8 +181,6 @@ func (p *pebbleIterator) init( panic("min timestamp hint set without max timestamp hint") } - p.options.KeyTypes = opts.KeyTypes - if doClone { var err error if p.iter, err = iterToClone.Clone(); err != nil { @@ -592,56 +590,6 @@ func (p *pebbleIterator) ValueProto(msg protoutil.Message) error { return protoutil.Unmarshal(value, msg) } -// HasPointAndRange implements the MVCCIterator interface. -func (p *pebbleIterator) HasPointAndRange() (bool, bool) { - return p.iter.HasPointAndRange() -} - -// RangeBounds implements the MVCCIterator interface. -func (p *pebbleIterator) RangeBounds() (roachpb.Key, roachpb.Key) { - start, end := p.iter.RangeBounds() - - // TODO(erikgrinaker): We should surface this error somehow, but for now - // we follow UnsafeKey()'s example and silently return empty bounds. - startKey, err := DecodeMVCCKey(start) - if err != nil { - return nil, nil - } - endKey, err := DecodeMVCCKey(end) - if err != nil { - return nil, nil - } - - return startKey.Key, endKey.Key -} - -// RangeKeys implements the MVCCIterator interface. -// -// TODO(erikgrinaker): Add unit tests for the range key methods. -func (p *pebbleIterator) RangeKeys() []MVCCRangeKeyValue { - startKey, endKey := p.RangeBounds() - rangeKeys := p.iter.RangeKeys() - rangeValues := make([]MVCCRangeKeyValue, 0, len(rangeKeys)) - - for _, rangeKey := range rangeKeys { - timestamp, err := decodeMVCCTimestampSuffix(rangeKey.Suffix) - if err != nil { - // TODO(erikgrinaker): We should surface this error somehow, but for now - // we follow UnsafeKey()'s example and silently skip them. - continue - } - rangeValues = append(rangeValues, MVCCRangeKeyValue{ - Key: MVCCRangeKey{ - StartKey: startKey, - EndKey: endKey, - Timestamp: timestamp, - }, - Value: rangeKey.Value, - }) - } - return rangeValues -} - // ComputeStats implements the MVCCIterator interface. func (p *pebbleIterator) ComputeStats( start, end roachpb.Key, nowNanos int64, diff --git a/pkg/storage/sst_iterator.go b/pkg/storage/sst_iterator.go index c5c81b756c37..9bd8b49b1b39 100644 --- a/pkg/storage/sst_iterator.go +++ b/pkg/storage/sst_iterator.go @@ -158,18 +158,3 @@ func (r *sstIterator) UnsafeKey() MVCCKey { func (r *sstIterator) UnsafeValue() []byte { return r.value } - -// HasPointAndRange implements SimpleMVCCIterator. -func (r *sstIterator) HasPointAndRange() (bool, bool) { - panic("not implemented") -} - -// RangeBounds implements SimpleMVCCIterator. -func (r *sstIterator) RangeBounds() (roachpb.Key, roachpb.Key) { - panic("not implemented") -} - -// RangeKeys implements SimpleMVCCIterator. -func (r *sstIterator) RangeKeys() []MVCCRangeKeyValue { - panic("not implemented") -} diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 88e4d996d266..779767b26c1a 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -142,16 +142,6 @@ func (fw *SSTWriter) ClearMVCCRange(start, end MVCCKey) error { return fw.clearRange(start, end) } -// ExperimentalPutMVCCRangeKey implements the Writer interface. -func (fw *SSTWriter) ExperimentalPutMVCCRangeKey(rangeKey MVCCRangeKey, value []byte) error { - panic("not implemented") -} - -// ExperimentalClearMVCCRangeKey implements the Writer interface. -func (fw *SSTWriter) ExperimentalClearMVCCRangeKey(rangeKey MVCCRangeKey) error { - panic("not implemented") -} - func (fw *SSTWriter) clearRange(start, end MVCCKey) error { if fw.fw == nil { return errors.New("cannot call ClearRange on a closed writer") diff --git a/pkg/storage/testdata/mvcc_histories/delete_range_tombstone b/pkg/storage/testdata/mvcc_histories/delete_range_tombstone deleted file mode 100644 index b9c2a9cb0b5b..000000000000 --- a/pkg/storage/testdata/mvcc_histories/delete_range_tombstone +++ /dev/null @@ -1,138 +0,0 @@ -# TODO(erikgrinaker): The MVCC API does not respect range tombstones yet, so -# we don't test point keys because they remain unaffected. -# TODO(erikgrinaker): This needs conflict tests, implement later. - -# Write some range tombstones. Some will abut and merge. -run ok -del_range_ts k=b end=c ts=3 -del_range_ts k=e end=g ts=3 -del_range_ts k=d end=f ts=5 -del_range_ts k=d end=f ts=2 -del_range_ts k=m end=z ts=1 -del_range_ts k=a end=m ts=4 -del_range_ts k=m end=z ts=4 ----- -del_range_ts: {b-c}/3.000000000,0 -del_range_ts: {e-g}/3.000000000,0 -del_range_ts: {d-f}/5.000000000,0 -del_range_ts: {d-f}/2.000000000,0 -del_range_ts: {m-z}/1.000000000,0 -del_range_ts: {a-m}/4.000000000,0 -del_range_ts: {m-z}/4.000000000,0 ->> at end: -range key: {b-c}/3.000000000,0 -> [] -range key: {d-f}/5.000000000,0 -> [] -range key: {d-f}/2.000000000,0 -> [] -range key: {e-g}/3.000000000,0 -> [] -range key: {a-z}/4.000000000,0 -> [] -range key: {m-z}/1.000000000,0 -> [] - -# Iterate over all tombstones. -run ok -iter_range_keys k=a end=z ----- -iter_range_keys: {b-c}/3.000000000,0 -> [] -iter_range_keys: {d-f}/5.000000000,0 -> [] -iter_range_keys: {d-f}/2.000000000,0 -> [] -iter_range_keys: {e-g}/3.000000000,0 -> [] -iter_range_keys: {a-z}/4.000000000,0 -> [] -iter_range_keys: {m-z}/1.000000000,0 -> [] - -# Iterator truncates to range bounds. -run ok -iter_range_keys k=c end=e ----- -iter_range_keys: {d-e}/5.000000000,0 -> [] -iter_range_keys: {c-e}/4.000000000,0 -> [] -iter_range_keys: {d-e}/2.000000000,0 -> [] - -# Iterator truncates to bounds between range key bounds. -run ok -iter_range_keys k=ccc end=eee ----- -iter_range_keys: {d-eee}/5.000000000,0 -> [] -iter_range_keys: {ccc-eee}/4.000000000,0 -> [] -iter_range_keys: e{-ee}/3.000000000,0 -> [] -iter_range_keys: {d-eee}/2.000000000,0 -> [] - -# Iterator with constrained timestamps. -run ok -iter_range_keys k=a end=z minTS=2 maxTS=3 ----- -iter_range_keys: {b-c}/3.000000000,0 -> [] -iter_range_keys: {d-f}/2.000000000,0 -> [] -iter_range_keys: {e-g}/3.000000000,0 -> [] - -# Fragmented iteration. -run ok -iter_range_keys k=a end=z fragmented ----- -iter_range_keys: {a-b}/4.000000000,0 -> [] -iter_range_keys: {b-c}/4.000000000,0 -> [] -iter_range_keys: {b-c}/3.000000000,0 -> [] -iter_range_keys: {c-d}/4.000000000,0 -> [] -iter_range_keys: {d-e}/5.000000000,0 -> [] -iter_range_keys: {d-e}/4.000000000,0 -> [] -iter_range_keys: {d-e}/2.000000000,0 -> [] -iter_range_keys: {e-f}/5.000000000,0 -> [] -iter_range_keys: {e-f}/4.000000000,0 -> [] -iter_range_keys: {e-f}/3.000000000,0 -> [] -iter_range_keys: {e-f}/2.000000000,0 -> [] -iter_range_keys: {f-g}/4.000000000,0 -> [] -iter_range_keys: {f-g}/3.000000000,0 -> [] -iter_range_keys: {g-m}/4.000000000,0 -> [] -iter_range_keys: {m-z}/4.000000000,0 -> [] -iter_range_keys: {m-z}/1.000000000,0 -> [] - -# Fragmented iteration with key and time bounds. -run ok -iter_range_keys k=ccc end=eee fragmented minTS=3 maxTS=4 ----- -iter_range_keys: {ccc-d}/4.000000000,0 -> [] -iter_range_keys: {d-e}/4.000000000,0 -> [] -iter_range_keys: e{-ee}/4.000000000,0 -> [] -iter_range_keys: e{-ee}/3.000000000,0 -> [] - -# Empty iterations. -run ok -iter_range_keys k=A end=Z -iter_range_keys k=c end=c -iter_range_keys k=z end=a ----- -iter_range_keys: "A"-"Z" -> -iter_range_keys: "c"-"c" -> -iter_range_keys: "z"-"a" -> - -# Remove some range keys, both a non-existant one and a span across two. -run ok -clear_range_key k=a end=z ts=10 -clear_range_key k=b end=g ts=3 ----- ->> at end: -range key: {d-f}/5.000000000,0 -> [] -range key: {d-f}/2.000000000,0 -> [] -range key: {a-z}/4.000000000,0 -> [] -range key: {m-z}/1.000000000,0 -> [] - -# Remove the middle section of [a-z)@4, twice for idempotency. -run ok -clear_range_key k=k end=n ts=4 -clear_range_key k=k end=n ts=4 ----- ->> at end: -range key: {d-f}/5.000000000,0 -> [] -range key: {d-f}/2.000000000,0 -> [] -range key: {a-k}/4.000000000,0 -> [] -range key: {n-z}/4.000000000,0 -> [] -range key: {m-z}/1.000000000,0 -> [] - -# Remove portions of the [a-k)@4 and [n-z)@4 range keys in one operation. -run ok -clear_range_key k=eee end=ttt ts=4 ----- ->> at end: -range key: {a-eee}/4.000000000,0 -> [] -range key: {d-f}/5.000000000,0 -> [] -range key: {d-f}/2.000000000,0 -> [] -range key: {ttt-z}/4.000000000,0 -> [] -range key: {m-z}/1.000000000,0 -> [] diff --git a/pkg/storage/testdata/mvcc_histories/revert_range b/pkg/storage/testdata/mvcc_histories/revert_range deleted file mode 100644 index 9b39bbadf828..000000000000 --- a/pkg/storage/testdata/mvcc_histories/revert_range +++ /dev/null @@ -1,275 +0,0 @@ -# TODO(erikgrinaker): Test this with existing range keys too. -# TODO(erikgrinaker): Conflict and intent tests. -# TODO(erikgrinaker): Instead of duplicating the data set, we can use -# clear_range to clear the revert_range write timestamp. However, this requires -# support for clearing range keys in MVCCClearTimeRange. - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=a end=z ts=5 revertTS=1 deleteRangeThreshold=2 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -scan: "a" -> /BYTES/1 @5.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/1 @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> /BYTES/2 @2.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -range key: {e-z}/5.000000000,0 -> [] -data: "a"/5.000000000,0 -> /BYTES/1 -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> /BYTES/1 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=a end=z ts=5 revertTS=2 deleteRangeThreshold=2 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -scan: "a" -> /BYTES/2 @2.000000000,0 -scan: "b" -> /BYTES/2 @5.000000000,0 -scan: "c" -> / @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> /BYTES/2 @2.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> / @5.000000000,0 ->> at end: -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/5.000000000,0 -> /BYTES/2 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> / -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/5.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=a end=z ts=5 revertTS=3 deleteRangeThreshold=2 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -scan: "a" -> /BYTES/2 @2.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/3 @3.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> /BYTES/2 @2.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=c end=z ts=5 revertTS=1 deleteRangeThreshold=2 maxBatchSize=1 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -revert_range: resume span ["e","z") -scan: "a" -> /BYTES/2 @2.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/1 @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> /BYTES/2 @2.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> /BYTES/1 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=a end=z ts=5 revertTS=1 deleteRangeThreshold=10 maxBatchSize=3 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -revert_range: resume span ["g","z") -scan: "a" -> /BYTES/1 @5.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/1 @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> / @5.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -data: "a"/5.000000000,0 -> /BYTES/1 -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> /BYTES/1 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/5.000000000,0 -> / -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=a end=z ts=5 revertTS=1 deleteRangeThreshold=2 maxBatchSize=3 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -revert_range: resume span ["g","z") -scan: "a" -> /BYTES/1 @5.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/1 @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> / @5.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -data: "a"/5.000000000,0 -> /BYTES/1 -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> /BYTES/1 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/5.000000000,0 -> / -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 - -run ok -clear_range k=a end=z -put k=a v=1 ts=1 -put k=a v=2 ts=2 -put k=b v=2 ts=2 -del k=b ts=3 -put k=c v=1 ts=1 -del k=c ts=2 -put k=c v=3 ts=3 -put k=d v=1 ts=1 -put k=e v=2 ts=2 -del k=f ts=1 -put k=g v=3 ts=3 -revert_range k=c end=z ts=5 revertTS=1 deleteRangeThreshold=3 maxBatchBytes=1 -scan ts=5 k=a end=z # NB: does not respect range tombstones yet ----- -revert_range: resume span ["e","z") -scan: "a" -> /BYTES/2 @2.000000000,0 -scan: "b" -> / @3.000000000,0 -scan: "c" -> /BYTES/1 @5.000000000,0 -scan: "d" -> /BYTES/1 @1.000000000,0 -scan: "e" -> /BYTES/2 @2.000000000,0 -scan: "f" -> / @1.000000000,0 -scan: "g" -> /BYTES/3 @3.000000000,0 ->> at end: -data: "a"/2.000000000,0 -> /BYTES/2 -data: "a"/1.000000000,0 -> /BYTES/1 -data: "b"/3.000000000,0 -> / -data: "b"/2.000000000,0 -> /BYTES/2 -data: "c"/5.000000000,0 -> /BYTES/1 -data: "c"/3.000000000,0 -> /BYTES/3 -data: "c"/2.000000000,0 -> / -data: "c"/1.000000000,0 -> /BYTES/1 -data: "d"/1.000000000,0 -> /BYTES/1 -data: "e"/2.000000000,0 -> /BYTES/2 -data: "f"/1.000000000,0 -> / -data: "g"/3.000000000,0 -> /BYTES/3 diff --git a/pkg/util/hlc/timestamp.go b/pkg/util/hlc/timestamp.go index 9d026be6fc71..b66fdbbb377e 100644 --- a/pkg/util/hlc/timestamp.go +++ b/pkg/util/hlc/timestamp.go @@ -54,22 +54,6 @@ func (t Timestamp) LessEq(s Timestamp) bool { return t.WallTime < s.WallTime || (t.WallTime == s.WallTime && t.Logical <= s.Logical) } -// Compare returns -1 if this timestamp is lesser than the given timestamp, 1 if -// it is greater, and 0 if they are equal. -func (t Timestamp) Compare(s Timestamp) int { - if t.WallTime > s.WallTime { - return 1 - } else if t.WallTime < s.WallTime { - return -1 - } else if t.Logical > s.Logical { - return 1 - } else if t.Logical < s.Logical { - return -1 - } else { - return 0 - } -} - // String implements the fmt.Stringer interface. func (t Timestamp) String() string { // The following code was originally written as diff --git a/pkg/util/hlc/timestamp_test.go b/pkg/util/hlc/timestamp_test.go index 9ba6ffe81764..59e45dde2a87 100644 --- a/pkg/util/hlc/timestamp_test.go +++ b/pkg/util/hlc/timestamp_test.go @@ -15,7 +15,6 @@ import ( "testing" "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func makeTS(walltime int64, logical int32) Timestamp { @@ -86,38 +85,6 @@ func TestLessEq(t *testing.T) { } } -func TestCompare(t *testing.T) { - w0l0 := Timestamp{} - w1l1 := Timestamp{WallTime: 1, Logical: 1} - w1l2 := Timestamp{WallTime: 1, Logical: 2} - w2l1 := Timestamp{WallTime: 2, Logical: 1} - w2l2 := Timestamp{WallTime: 2, Logical: 2} - - testcases := map[string]struct { - a Timestamp - b Timestamp - expect int - }{ - "empty eq empty": {w0l0, w0l0, 0}, - "empty lt set": {w0l0, w1l1, -1}, - "set gt empty": {w1l1, w0l0, 1}, - "set eq set": {w1l1, w1l1, 0}, - - "wall lt": {w1l1, w2l1, -1}, - "wall gt": {w2l1, w1l1, 1}, - "logical lt": {w1l1, w1l2, -1}, - "logical gt": {w1l2, w1l1, 1}, - "both lt": {w1l1, w2l2, -1}, - "both gt": {w2l2, w1l1, 1}, - "wall precedence": {w2l1, w1l2, 1}, - } - for name, tc := range testcases { - t.Run(name, func(t *testing.T) { - require.Equal(t, tc.expect, tc.a.Compare(tc.b)) - }) - } -} - func TestIsEmpty(t *testing.T) { a := makeTS(0, 0) assert.True(t, a.IsEmpty())