Skip to content

Commit

Permalink
batcheval: add range tombstone support for DeleteRange
Browse files Browse the repository at this point in the history
This patch adds a parameter `UseExperimentalRangeTombstone` for
`DeleteRange`, which deletes the span using an MVCC range tombstone.
This must only be called after checking
`storage.CanUseExperimentalMVCCRangeTombstones()`, which ensures the
`ExperimentalMVCCRangeTombstones` version gate and
`COCKROACH_EXPERIMENTAL_MVCC_RANGE_TOMBSTONES` environment variable are
set.

Did I mention that MVCC range tombstones are experimental? They are
currently under active development, and are not respected by the KV or
MVCC APIs, nor are they persisted. This patch simply sets up the
plumbing for it.

Release note: None
  • Loading branch information
erikgrinaker committed Feb 12, 2022
1 parent 1d961ad commit dac98d4
Show file tree
Hide file tree
Showing 12 changed files with 119 additions and 9 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -174,4 +174,4 @@ trace.debug.enable boolean false if set, traces for recent requests can be seen
trace.jaeger.agent string the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.
trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.
version version 21.2-62 set the active cluster version in the format '<major>.<minor>'
version version 21.2-64 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,6 @@
<tr><td><code>trace.jaeger.agent</code></td><td>string</td><td><code></code></td><td>the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.</td></tr>
<tr><td><code>trace.opentelemetry.collector</code></td><td>string</td><td><code></code></td><td>address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-62</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-64</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
7 changes: 7 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,9 @@ const (
DontProposeWriteTimestampForLeaseTransfers
// TenantSettingsTable adds the system table for tracking tenant usage.
TenantSettingsTable
// ExperimentalMVCCRangeTombstones enables the use of highly experimental MVCC
// range tombstones.
ExperimentalMVCCRangeTombstones

// *************************************************
// Step (1): Add new versions here.
Expand Down Expand Up @@ -429,6 +432,10 @@ var versionsSingleton = keyedVersions{
Key: TenantSettingsTable,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 62},
},
{
Key: ExperimentalMVCCRangeTombstones,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 64},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
5 changes: 3 additions & 2 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

28 changes: 28 additions & 0 deletions pkg/kv/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -649,6 +649,34 @@ func (b *Batch) DelRange(s, e interface{}, returnKeys bool) {
b.initResult(1, 0, notRaw, nil)
}

// ExperimentalDelRangeUsingTombstone deletes the rows between begin (inclusive)
// and end (exclusive) using an MVCC range tombstone. The caller must check
// storage.CanUseExperimentalMVCCRangeTombstones() before using this.
//
// This method is EXPERIMENTAL: range tombstones are under active development,
// and have severe limitations including being ignored by all KV and MVCC APIs
// and only being stored in memory.
func (b *Batch) ExperimentalDelRangeUsingTombstone(s, e interface{}) {
start, err := marshalKey(s)
if err != nil {
b.initResult(0, 0, notRaw, err)
return
}
end, err := marshalKey(e)
if err != nil {
b.initResult(0, 0, notRaw, err)
return
}
b.appendReqs(&roachpb.DeleteRangeRequest{
RequestHeader: roachpb.RequestHeader{
Key: start,
EndKey: end,
},
UseExperimentalRangeTombstone: true,
})
b.initResult(1, 0, notRaw, nil)
}

// adminMerge is only exported on DB. It is here for symmetry with the
// other operations.
func (b *Batch) adminMerge(key interface{}) {
Expand Down
16 changes: 16 additions & 0 deletions pkg/kv/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -546,6 +546,22 @@ func (db *DB) DelRange(
return r.Keys, err
}

// ExperimentalDelRangeUsingTombstone deletes the rows between begin (inclusive)
// and end (exclusive) using an MVCC range tombstone. The caller must check
// storage.CanUseExperimentalMVCCRangeTombstones() before using this.
//
// This method is EXPERIMENTAL: range tombstones are under active development,
// and have severe limitations including being ignored by all KV and MVCC APIs
// and only being stored in memory.
func (db *DB) ExperimentalDelRangeUsingTombstone(
ctx context.Context, begin, end interface{},
) error {
b := &Batch{}
b.ExperimentalDelRangeUsingTombstone(begin, end)
_, err := getOneResult(db.Run(ctx, b), b)
return err
}

// AdminMerge merges the range containing key and the subsequent range. After
// the merge operation is complete, the range containing key will contain all of
// the key/value pairs of the subsequent range and the subsequent range will no
Expand Down
20 changes: 20 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/errors"
)

func init() {
Expand Down Expand Up @@ -49,6 +50,25 @@ 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.
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
Expand Down
16 changes: 11 additions & 5 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,17 +178,17 @@ func (i *MVCCIterator) UnsafeValue() []byte {

// HasPointAndRange implements SimpleMVCCIterator.
func (i *MVCCIterator) HasPointAndRange() (bool, bool) {
panic("not implemented")
return i.i.HasPointAndRange()
}

// RangeBounds implements SimpleMVCCIterator.
func (i *MVCCIterator) RangeBounds() (roachpb.Key, roachpb.Key) {
panic("not implemented")
return i.i.RangeBounds()
}

// RangeKeys implements SimpleMVCCIterator.
func (i *MVCCIterator) RangeKeys() []storage.MVCCRangeKeyValue {
panic("not implemented")
return i.i.RangeKeys()
}

// ComputeStats is part of the storage.MVCCIterator interface.
Expand Down Expand Up @@ -617,11 +617,17 @@ func (s spanSetWriter) ClearIterRange(iter storage.MVCCIterator, start, end roac
func (s spanSetWriter) ExperimentalPutMVCCRangeKey(
rangeKey storage.MVCCRangeKey, value []byte,
) error {
panic("not implemented")
if err := s.checkAllowedRange(rangeKey.StartKey, rangeKey.EndKey); err != nil {
return err
}
return s.w.ExperimentalPutMVCCRangeKey(rangeKey, value)
}

func (s spanSetWriter) ExperimentalClearMVCCRangeKey(rangeKey storage.MVCCRangeKey) error {
panic("not implemented")
if err := s.checkAllowedRange(rangeKey.StartKey, rangeKey.EndKey); err != nil {
return err
}
return s.w.ExperimentalClearMVCCRangeKey(rangeKey)
}

func (s spanSetWriter) Merge(key storage.MVCCKey, value []byte) error {
Expand Down
5 changes: 5 additions & 0 deletions pkg/roachpb/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1225,6 +1225,11 @@ func (*DeleteRequest) flags() flag {
}

func (drr *DeleteRangeRequest) flags() flag {
// DeleteRangeRequest using MVCC range tombstones cannot be transactional,
// and thus cannot span ranges.
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
Expand Down
11 changes: 11 additions & 0 deletions pkg/roachpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -348,6 +348,17 @@ message DeleteRangeRequest {
// Inline values cannot be deleted transactionally; a DeleteRange with
// "inline" set to true will fail if it is executed within a transaction.
bool inline = 4;
// If enabled, the range is deleted using an MVCC range tombstone, which is a
// cheap constant-time operation. This option cannot be used in a transaction,
// and it cannot be combined with Inline or ReturnKeys.
//
// The caller must check storage.CanUseExperimentalMVCCRangeTombstones()
// before enabling this parameter.
//
// This parameter is EXPERIMENTAL: range tombstones are under active
// development, and have severe limitations including being ignored by all
// KV and MVCC APIs and only being stored in memory.
bool use_experimental_range_tombstone = 5;
}

// A DeleteRangeResponse is the return value from the DeleteRange()
Expand Down
1 change: 1 addition & 0 deletions pkg/roachpb/api_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -317,6 +317,7 @@ func TestFlagCombinations(t *testing.T) {
reqVariants := []Request{
&AddSSTableRequest{WriteAtRequestTimestamp: true},
&DeleteRangeRequest{Inline: true},
&DeleteRangeRequest{UseExperimentalRangeTombstone: true},
&GetRequest{KeyLocking: lock.Exclusive},
&ReverseScanRequest{KeyLocking: lock.Exclusive},
&ScanRequest{KeyLocking: lock.Exclusive},
Expand Down
15 changes: 15 additions & 0 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,12 @@ import (
"sync"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -84,6 +86,19 @@ var rocksdbConcurrency = envutil.EnvOrDefaultInt(
return max
}())

// CanUseExperimentalMVCCRangeTombstones returns true if MVCC range tombstones
// are enabled. Callers must check this before using range tombstones.
//
// These are EXPERIMENTAL: range tombstones are under active
// development, and have severe limitations including being ignored by all
// KV and MVCC APIs and only being stored in memory.
func CanUseExperimentalMVCCRangeTombstones(ctx context.Context, st *cluster.Settings) bool {
// TODO(erikgrinaker): Consider using a cluster setting rather than an
// environment variable once range tombstones are fully implemented.
return st.Version.IsActive(ctx, clusterversion.ExperimentalMVCCRangeTombstones) &&
envutil.EnvOrDefaultBool("COCKROACH_EXPERIMENTAL_MVCC_RANGE_TOMBSTONES", false)
}

// MakeValue returns the inline value.
func MakeValue(meta enginepb.MVCCMetadata) roachpb.Value {
return roachpb.Value{RawBytes: meta.RawBytes}
Expand Down

0 comments on commit dac98d4

Please sign in to comment.