Skip to content

Commit

Permalink
Merge #97770
Browse files Browse the repository at this point in the history
97770: clusterversion,storage: use range tombstones unconditionally in 23.1 r=nicktrav a=nicktrav

Currently, MVCC range tombstones are only enabled when the `storage.mvcc.range_tombstones.enabled` cluster setting is `true`. Originally, the plan for 23.1 was to remove this cluster setting entirely, which would enable the feature unconditionally. However, it was decided that for 23.1, the setting would remain, along with the utility methods that determine whether or not the feature is enabled.

As there is still an expectation that the feature is enabled in 23.1, add an internal cluster version that is used as a feature gate. If the cluster is at least of version
`V23_1_MVCCRangeTombstonesUnconditionallyEnabled`, MVCC range tombstones are _unconditionally_ enabled, irrespective of whether the cluster setting is enabled or not.

Fix #91147.

Release note: None.

Epic: CRDB-20465.

Co-authored-by: Nick Travers <[email protected]>
  • Loading branch information
craig[bot] and nicktrav committed Mar 1, 2023
2 parents 8f3aeef + 1c3e46d commit 21786aa
Show file tree
Hide file tree
Showing 12 changed files with 42 additions and 19 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 @@ -297,4 +297,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using
trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https://<ui>/#/debug/tracez
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 1000022.2-56 set the active cluster version in the format '<major>.<minor>'
version version 1000022.2-58 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 @@ -238,6 +238,6 @@
<tr><td><div id="setting-trace-opentelemetry-collector" class="anchored"><code>trace.opentelemetry.collector</code></div></td><td>string</td><td><code></code></td><td>address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as &lt;host&gt;:&lt;port&gt;. If no port is specified, 4317 will be used.</td></tr>
<tr><td><div id="setting-trace-span-registry-enabled" class="anchored"><code>trace.span_registry.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if set, ongoing traces can be seen at https://&lt;ui&gt;/#/debug/tracez</td></tr>
<tr><td><div id="setting-trace-zipkin-collector" class="anchored"><code>trace.zipkin.collector</code></div></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as &lt;host&gt;:&lt;port&gt;. If no port is specified, 9411 will be used.</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000022.2-56</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000022.2-58</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td></tr>
</tbody>
</table>
3 changes: 2 additions & 1 deletion pkg/ccl/backupccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,8 @@ var localityCfgs = map[string]roachpb.Locality{
}

var clusterVersionKeys = map[string]clusterversion.Key{
"Start22_2": clusterversion.TODODelete_V22_2Start,
"Start22_2": clusterversion.TODODelete_V22_2Start,
"23_1_MVCCTombstones": clusterversion.V23_1_MVCCRangeTombstonesUnconditionallyEnabled,
}

type sqlDBKey struct {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@

# disabled to run within tenant as they don't have access to the
# storage.mvcc.range_tombstones.enabled cluster setting
new-cluster name=s1 disable-tenant
new-cluster name=s1 beforeVersion=23_1_MVCCTombstones disable-tenant
----

###########
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/declarative-rules/deprules
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
dep
----
debug declarative-print-rules 1000022.2-56 dep
debug declarative-print-rules 1000022.2-58 dep
deprules
----
- name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED'
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/declarative-rules/oprules
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
op
----
debug declarative-print-rules 1000022.2-56 op
debug declarative-print-rules 1000022.2-58 op
rules
----
[]
13 changes: 13 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -444,6 +444,15 @@ const (
// backfilled.
V23_1DatabaseRoleSettingsRoleIDColumnBackfilled

// V23_1_MVCCRangeTombstonesUnconditionallyEnabled is a version gate at and
// after which Cockroach will always write MVCC Range Tombstones, regardless
// of the value of the storage.mvcc.range_tombstones.enabled cluster setting.
// Prior to this version, it was possible for a cluster to be writing MVCC
// Range Tombstones, but only if the cluster had been opted in manually, under
// a specific set of circumstances (i.e. appropriate 22.2.x version, Cockroach
// Cloud cluster, etc.).
V23_1_MVCCRangeTombstonesUnconditionallyEnabled

// *************************************************
// Step (1): Add new versions here.
// Do not add new versions to a patch release.
Expand Down Expand Up @@ -769,6 +778,10 @@ var rawVersionsSingleton = keyedVersions{
Key: V23_1DatabaseRoleSettingsRoleIDColumnBackfilled,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 56},
},
{
Key: V23_1_MVCCRangeTombstonesUnconditionallyEnabled,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 58},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/gcjob/gcjobnotifier/notifier.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,7 @@ func (n *Notifier) run(_ context.Context) {
}
})
tombstonesEnableChanges := make(chan struct{}, 1)
storage.MVCCRangeTombstonesEnabled.SetOnChange(&n.settings.SV, func(ctx context.Context) {
storage.MVCCRangeTombstonesEnabledInMixedClusters.SetOnChange(&n.settings.SV, func(ctx context.Context) {
select {
case tombstonesEnableChanges <- struct{}{}:
default:
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/gcjob_test/gc_job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -271,7 +271,7 @@ func TestGCJobRetry(t *testing.T) {
failed.Store(false)
cs := cluster.MakeTestingClusterSettings()
gcjob.EmptySpanPollInterval.Override(ctx, &cs.SV, 100*time.Millisecond)
storage.MVCCRangeTombstonesEnabled.Override(ctx, &cs.SV, true)
storage.MVCCRangeTombstonesEnabledInMixedClusters.Override(ctx, &cs.SV, true)
params := base.TestServerArgs{Settings: cs}
params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals()
params.Knobs.Store = &kvserver.StoreTestingKnobs{
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/importer/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6262,7 +6262,7 @@ func TestImportPgDumpSchemas(t *testing.T) {
baseDir := datapathutils.TestDataPath(t, "pgdump")
mkArgs := func() base.TestServerArgs {
s := cluster.MakeTestingClusterSettings()
storage.MVCCRangeTombstonesEnabled.Override(ctx, &s.SV, true)
storage.MVCCRangeTombstonesEnabledInMixedClusters.Override(ctx, &s.SV, true)
return base.TestServerArgs{
Settings: s,
ExternalIODir: baseDir,
Expand Down
27 changes: 18 additions & 9 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"sync"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvnemesis/kvnemesisutil"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
Expand Down Expand Up @@ -74,27 +75,35 @@ var minWALSyncInterval = settings.RegisterDurationSetting(
settings.NonNegativeDurationWithMaximum(1*time.Second),
)

// MVCCRangeTombstonesEnabled enables writing of MVCC range tombstones.
// Currently, this is used for schema GC and import cancellation rollbacks.
// MVCCRangeTombstonesEnabledInMixedClusters enables writing of MVCC range
// tombstones. Currently, this is used for schema GC and import cancellation
// rollbacks.
//
// Note that any executing jobs may not pick up this change, so these need to be
// waited out before being certain that the setting has taken effect.
//
// If disabled after being enabled, this will prevent new range tombstones from
// being written, but already written tombstones will remain until GCed. The
// above note on jobs also applies in this case.
var MVCCRangeTombstonesEnabled = settings.RegisterBoolSetting(
//
// If the version of the cluster is at or beyond the version
// V23_1_MVCCRangeTombstonesUnconditionallyEnabled, the feature is
// unconditionally enabled.
var MVCCRangeTombstonesEnabledInMixedClusters = settings.RegisterBoolSetting(
settings.TenantReadOnly,
"storage.mvcc.range_tombstones.enabled",
"enables the use of MVCC range tombstones",
"controls the use of MVCC range tombstones in mixed version clusters; range tombstones are always on in 23.1 clusters",
true)

// CanUseMVCCRangeTombstones returns true if the caller can begin writing
// MVCC range tombstones, by setting DeleteRangeRequest.UseRangeTombstone.
// It requires the MVCCRangeTombstones version gate to be active, and the
// setting storage.mvcc.range_tombstones.enabled to be enabled.
// CanUseMVCCRangeTombstones returns true if the caller can begin writing MVCC
// range tombstones, by setting DeleteRangeRequest.UseRangeTombstone. It
// requires the storage.mvcc.range_tombstones.enabled cluster setting to be
// enabled, OR the cluster version is at or beyond the
// V23_1_MVCCRangeTombstonesUnconditionallyEnabled version (i.e. in 23.1, the
// feature is unconditionally enabled).
func CanUseMVCCRangeTombstones(ctx context.Context, st *cluster.Settings) bool {
return MVCCRangeTombstonesEnabled.Get(&st.SV)
return st.Version.IsActive(ctx, clusterversion.V23_1_MVCCRangeTombstonesUnconditionallyEnabled) ||
MVCCRangeTombstonesEnabledInMixedClusters.Get(&st.SV)
}

// MaxIntentsPerWriteIntentError sets maximum number of intents returned in
Expand Down
2 changes: 1 addition & 1 deletion pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ func TestWaitForDelRangeInGCJob(t *testing.T) {
ctx := context.Background()
settings := cluster.MakeTestingClusterSettingsWithVersions(v1, v0, false /* initializeVersion */)
require.NoError(t, clusterversion.Initialize(ctx, v0, &settings.SV))
storage.MVCCRangeTombstonesEnabled.Override(ctx, &settings.SV, true)
storage.MVCCRangeTombstonesEnabledInMixedClusters.Override(ctx, &settings.SV, true)
testServer, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{
Settings: settings,
Knobs: base.TestingKnobs{
Expand Down

0 comments on commit 21786aa

Please sign in to comment.