Skip to content

Commit

Permalink
sql,clusterversion,gc_job: hoist version gates for DelRange to 23.1
Browse files Browse the repository at this point in the history
We added version gates to unconditionally enable sending DelRange
tombstones in 22.2, but then we pre-empted that by disabling them with
a cluster setting. This PR hoists those gates and that invariant checking
up to 23.1.

Relates to #96763

Release note: None
  • Loading branch information
ajwerner committed Mar 8, 2023
1 parent 0b02d6f commit e0ed735
Show file tree
Hide file tree
Showing 12 changed files with 39 additions and 37 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-64 set the active cluster version in the format '<major>.<minor>'
version version 1000022.2-68 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-64</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-68</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td></tr>
</tbody>
</table>
38 changes: 20 additions & 18 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -257,16 +257,6 @@ const (
// options table id column cannot be null. This is the final step
// of the system.role_options table migration.
TODODelete_V22_2SetRoleOptionsUserIDColumnNotNull
// TODODelete_V22_2UseDelRangeInGCJob enables the use of the DelRange operation in the
// GC job. Before it is enabled, the GC job uses ClearRange operations
// after the job waits out the GC TTL. After it has been enabled, the
// job instead issues DelRange operations at the beginning of the job
// and then waits for the data to be removed automatically before removing
// the descriptor and zone configurations.
TODODelete_V22_2UseDelRangeInGCJob
// TODODelete_V22_2WaitedForDelRangeInGCJob corresponds to the migration which waits for
// the GC jobs to adopt the use of DelRange with tombstones.
TODODelete_V22_2WaitedForDelRangeInGCJob
// TODODelete_V22_2RangefeedUseOneStreamPerNode changes rangefeed implementation to use 1 RPC stream per node.
TODODelete_V22_2RangefeedUseOneStreamPerNode
// TODODelete_V22_2NoNonMVCCAddSSTable adds a migration which waits for all
Expand Down Expand Up @@ -454,6 +444,18 @@ const (
// on Pebble stores that have shared storage configured.
V23_1SetPebbleCreatorID

// V23_1_UseDelRangeInGCJob enables the use of the DelRange operation in the
// GC job. Before it is enabled, the GC job uses ClearRange operations
// after the job waits out the GC TTL. After it has been enabled, the
// job instead issues DelRange operations at the beginning of the job
// and then waits for the data to be removed automatically before removing
// the descriptor and zone configurations.
V23_1_UseDelRangeInGCJob

// V23_1WaitedForDelRangeInGCJob corresponds to the migration which waits for
// the GC jobs to adopt the use of DelRange with tombstones.
V23_1WaitedForDelRangeInGCJob

// *************************************************
// Step (1): Add new versions here.
// Do not add new versions to a patch release.
Expand Down Expand Up @@ -611,14 +613,6 @@ var rawVersionsSingleton = keyedVersions{
Key: TODODelete_V22_2SetRoleOptionsUserIDColumnNotNull,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 54},
},
{
Key: TODODelete_V22_2UseDelRangeInGCJob,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 56},
},
{
Key: TODODelete_V22_2WaitedForDelRangeInGCJob,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 58},
},
{
Key: TODODelete_V22_2RangefeedUseOneStreamPerNode,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 60},
Expand Down Expand Up @@ -783,6 +777,14 @@ var rawVersionsSingleton = keyedVersions{
Key: V23_1SetPebbleCreatorID,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 64},
},
{
Key: V23_1_UseDelRangeInGCJob,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 66},
},
{
Key: V23_1WaitedForDelRangeInGCJob,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 68},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/gcjob/gc_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -508,7 +508,7 @@ func shouldUseDelRange(
) bool {
// TODO(ajwerner): Adopt the DeleteRange protocol for tenant GC.
return details.Tenant == nil &&
s.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob) &&
s.Version.IsActive(ctx, clusterversion.V23_1_UseDelRangeInGCJob) &&
(storage.CanUseMVCCRangeTombstones(ctx, s) ||
// Allow this testing knob to override the storage setting, for convenience.
knobs.SkipWaitingForMVCCGC)
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 @@ -147,7 +147,7 @@ func (n *Notifier) run(_ context.Context) {
systemConfigUpdateCh, _ := n.provider.RegisterSystemConfigChannel()
var haveNotified syncutil.AtomicBool
versionSettingChanged := make(chan struct{}, 1)
versionBeingWaited := clusterversion.ByKey(clusterversion.TODODelete_V22_2UseDelRangeInGCJob)
versionBeingWaited := clusterversion.ByKey(clusterversion.V23_1_UseDelRangeInGCJob)
n.settings.Version.SetOnChange(func(ctx context.Context, newVersion clusterversion.ClusterVersion) {
if !haveNotified.Get() &&
versionBeingWaited.LessEq(newVersion.Version) &&
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/repair.go
Original file line number Diff line number Diff line change
Expand Up @@ -742,7 +742,7 @@ func (p *planner) ForceDeleteTableData(ctx context.Context, descID int64) error
Key: tableSpan.Key, EndKey: tableSpan.EndKey,
}
b := &kv.Batch{}
if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob) &&
if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.V23_1_UseDelRangeInGCJob) &&
storage.CanUseMVCCRangeTombstones(ctx, p.execCfg.Settings) {
b.AddRawRequest(&kvpb.DeleteRangeRequest{
RequestHeader: requestHeader,
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/schema_changer.go
Original file line number Diff line number Diff line change
Expand Up @@ -737,7 +737,7 @@ func (sc *SchemaChanger) exec(ctx context.Context) error {
sc.job.Payload().UsernameProto.Decode(),
sc.job.Payload().Description,
gcDetails,
!sc.settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob),
!sc.settings.Version.IsActive(ctx, clusterversion.V23_1_UseDelRangeInGCJob),
); err != nil {
return err
}
Expand Down Expand Up @@ -1073,7 +1073,7 @@ func (sc *SchemaChanger) rollbackSchemaChange(ctx context.Context, err error) er
},
},
},
!sc.settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob),
!sc.settings.Version.IsActive(ctx, clusterversion.V23_1_UseDelRangeInGCJob),
)
if _, err := sc.jobRegistry.CreateJobWithTxn(ctx, jobRecord, gcJobID, txn); err != nil {
return err
Expand Down Expand Up @@ -1283,7 +1283,7 @@ func (sc *SchemaChanger) createIndexGCJobWithDropTime(

gcJobRecord := CreateGCJobRecord(
jobDesc, sc.job.Payload().UsernameProto.Decode(), indexGCDetails,
!sc.settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob),
!sc.settings.Version.IsActive(ctx, clusterversion.V23_1_UseDelRangeInGCJob),
)
jobID := sc.jobRegistry.MakeJobID()
if _, err := sc.jobRegistry.CreateJobWithTxn(ctx, gcJobRecord, jobID, txn); err != nil {
Expand Down Expand Up @@ -2761,7 +2761,7 @@ func (r schemaChangeResumer) Resume(ctx context.Context, execCtx interface{}) er
r.job.Payload().Description,
multiTableGCDetails,
!p.ExecCfg().Settings.Version.IsActive(
ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob,
ctx, clusterversion.V23_1_UseDelRangeInGCJob,
),
); err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/schemachanger/scdeps/exec_deps.go
Original file line number Diff line number Diff line change
Expand Up @@ -253,7 +253,7 @@ func (d *txnDeps) CheckPausepoint(name string) error {
}

func (d *txnDeps) UseLegacyGCJob(ctx context.Context) bool {
return !d.settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob)
return !d.settings.Version.IsActive(ctx, clusterversion.V23_1_UseDelRangeInGCJob)
}

func (d *txnDeps) SchemaChangerJobID() jobspb.JobID {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/schemachanger/screl/scalars.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ func MinElementVersion(el scpb.Element) clusterversion.Key {
case *scpb.CompositeType, *scpb.CompositeTypeAttrType, *scpb.CompositeTypeAttrName:
return clusterversion.V23_1
case *scpb.IndexColumn, *scpb.EnumTypeValue, *scpb.TableZoneConfig:
return clusterversion.TODODelete_V22_2UseDelRangeInGCJob
return clusterversion.V22_2
case *scpb.DatabaseData, *scpb.TableData, *scpb.IndexData, *scpb.TablePartitioning,
*scpb.Function, *scpb.FunctionName, *scpb.FunctionVolatility, *scpb.FunctionLeakProof,
*scpb.FunctionNullInputBehavior, *scpb.FunctionBody, *scpb.FunctionParamDefaultExpression:
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/truncate.go
Original file line number Diff line number Diff line change
Expand Up @@ -258,7 +258,7 @@ func (p *planner) truncateTable(ctx context.Context, id descpb.ID, jobDesc strin
record := CreateGCJobRecord(
jobDesc, p.User(), details,
!p.execCfg.Settings.Version.IsActive(
ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob,
ctx, clusterversion.V23_1_UseDelRangeInGCJob,
),
)
if _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn(
Expand Down
10 changes: 5 additions & 5 deletions pkg/upgrade/upgrades/upgrades.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,11 +123,6 @@ var upgrades = []upgradebase.Upgrade{
upgrade.NoPrecondition,
ensureSQLSchemaTelemetrySchedule,
),
upgrade.NewTenantUpgrade("ensure all GC jobs send DeleteRange requests",
toCV(clusterversion.TODODelete_V22_2WaitedForDelRangeInGCJob),
checkForPausedGCJobs,
waitForDelRangeInGCJob,
),
upgrade.NewTenantUpgrade(
"wait for all in-flight schema changes",
toCV(clusterversion.TODODelete_V22_2NoNonMVCCAddSSTable),
Expand Down Expand Up @@ -263,6 +258,11 @@ var upgrades = []upgradebase.Upgrade{
upgrade.NoPrecondition,
backfillDatabaseRoleSettingsTableRoleIDColumn,
),
upgrade.NewTenantUpgrade("ensure all GC jobs send DeleteRange requests",
toCV(clusterversion.V23_1_UseDelRangeInGCJob),
checkForPausedGCJobs,
waitForDelRangeInGCJob,
),
}

func init() {
Expand Down
4 changes: 2 additions & 2 deletions pkg/upgrade/upgrades/wait_for_del_range_in_gc_job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,8 @@ func TestWaitForDelRangeInGCJob(t *testing.T) {
defer log.Scope(t).Close(t)

var (
v0 = clusterversion.ByKey(clusterversion.TODODelete_V22_2UseDelRangeInGCJob - 1)
v1 = clusterversion.ByKey(clusterversion.TODODelete_V22_2WaitedForDelRangeInGCJob)
v0 = clusterversion.ByKey(clusterversion.V23_1_UseDelRangeInGCJob - 1)
v1 = clusterversion.ByKey(clusterversion.V23_1WaitedForDelRangeInGCJob)
)

ctx := context.Background()
Expand Down

0 comments on commit e0ed735

Please sign in to comment.