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 15, 2023
1 parent 62faa26 commit 68d4daf
Show file tree
Hide file tree
Showing 16 changed files with 46 additions and 52 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 @@ -299,4 +299,4 @@ trace.opentelemetry.collector string address of an OpenTelemetry trace collecto
trace.snapshot.rate duration 0s if non-zero, interval at which background trace snapshots are captured
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-80 set the active cluster version in the format '<major>.<minor>'
version version 1000022.2-84 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 @@ -240,6 +240,6 @@
<tr><td><div id="setting-trace-snapshot-rate" class="anchored"><code>trace.snapshot.rate</code></div></td><td>duration</td><td><code>0s</code></td><td>if non-zero, interval at which background trace snapshots are captured</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-80</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-84</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td></tr>
</tbody>
</table>
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-80 dep
debug declarative-print-rules 1000022.2-84 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-80 op
debug declarative-print-rules 1000022.2-84 op
rules
----
[]
38 changes: 20 additions & 18 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -252,16 +252,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 @@ -484,6 +474,18 @@ const (
// progress of each job in the system.jobs table.
V23_1JobInfoTableIsBackfilled

// 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 @@ -637,14 +639,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 @@ -837,6 +831,14 @@ var rawVersionsSingleton = keyedVersions{
Key: V23_1JobInfoTableIsBackfilled,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 80},
},
{
Key: V23_1_UseDelRangeInGCJob,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 82},
},
{
Key: V23_1WaitedForDelRangeInGCJob,
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 84},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/gcjob/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/gcjob",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/config",
"//pkg/config/zonepb",
"//pkg/jobs",
Expand Down
2 changes: 0 additions & 2 deletions pkg/sql/gcjob/gc_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
Expand Down Expand Up @@ -508,7 +507,6 @@ func shouldUseDelRange(
) bool {
// TODO(ajwerner): Adopt the DeleteRange protocol for tenant GC.
return details.Tenant == nil &&
s.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob) &&
(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
4 changes: 1 addition & 3 deletions pkg/sql/repair.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
Expand Down Expand Up @@ -742,8 +741,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) &&
storage.CanUseMVCCRangeTombstones(ctx, p.execCfg.Settings) {
if storage.CanUseMVCCRangeTombstones(ctx, p.execCfg.Settings) {
b.AddRawRequest(&kvpb.DeleteRangeRequest{
RequestHeader: requestHeader,
UseRangeTombstone: true,
Expand Down
11 changes: 5 additions & 6 deletions pkg/sql/schema_changer.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -738,7 +739,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),
!storage.CanUseMVCCRangeTombstones(ctx, sc.settings),
); err != nil {
return err
}
Expand Down Expand Up @@ -1074,7 +1075,7 @@ func (sc *SchemaChanger) rollbackSchemaChange(ctx context.Context, err error) er
},
},
},
!sc.settings.Version.IsActive(ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob),
!storage.CanUseMVCCRangeTombstones(ctx, sc.settings),
)
if _, err := sc.jobRegistry.CreateJobWithTxn(ctx, jobRecord, gcJobID, txn); err != nil {
return err
Expand Down Expand Up @@ -1284,7 +1285,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,9 +2762,7 @@ func (r schemaChangeResumer) Resume(ctx context.Context, execCtx interface{}) er
r.job.Payload().UsernameProto.Decode(),
r.job.Payload().Description,
multiTableGCDetails,
!p.ExecCfg().Settings.Version.IsActive(
ctx, clusterversion.TODODelete_V22_2UseDelRangeInGCJob,
),
!storage.CanUseMVCCRangeTombstones(ctx, p.ExecCfg().Settings),
); err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/schemachanger/scdeps/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdeps",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/jobs",
"//pkg/jobs/jobspb",
"//pkg/keys",
Expand Down Expand Up @@ -42,6 +41,7 @@ go_library(
"//pkg/sql/sqlerrors",
"//pkg/sql/sqltelemetry",
"//pkg/sql/types",
"//pkg/storage",
"//pkg/util/timeutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/schemachanger/scdeps/exec_deps.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
Expand All @@ -34,6 +33,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/errors"
)

Expand Down 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 !storage.CanUseMVCCRangeTombstones(ctx, d.settings)
}

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
6 changes: 2 additions & 4 deletions pkg/sql/truncate.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"
"math/rand"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
Expand All @@ -28,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
Expand Down Expand Up @@ -257,9 +257,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,
),
!storage.CanUseMVCCRangeTombstones(ctx, p.execCfg.Settings),
)
if _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn(
ctx, record, p.ExecCfg().JobRegistry.MakeJobID(), p.InternalSQLTxn(),
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 @@ -117,11 +117,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 @@ -282,6 +277,11 @@ var upgrades = []upgradebase.Upgrade{
upgrade.NoPrecondition,
backfillJobInfoTable,
),
upgrade.NewTenantUpgrade("ensure all GC jobs send DeleteRange requests",
toCV(clusterversion.V23_1_UseDelRangeInGCJob),
checkForPausedGCJobs,
waitForDelRangeInGCJob,
),
}

func init() {
Expand Down
8 changes: 4 additions & 4 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,14 +33,14 @@ 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()
settings := cluster.MakeTestingClusterSettingsWithVersions(v1, v0, false /* initializeVersion */)
require.NoError(t, clusterversion.Initialize(ctx, v0, &settings.SV))
storage.MVCCRangeTombstonesEnabledInMixedClusters.Override(ctx, &settings.SV, true)
storage.MVCCRangeTombstonesEnabledInMixedClusters.Override(ctx, &settings.SV, false)
testServer, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{
Settings: settings,
Knobs: base.TestingKnobs{
Expand Down Expand Up @@ -95,7 +95,7 @@ SELECT count(*)
WHERE job_type = 'SCHEMA CHANGE GC'
AND status = 'paused'`,
[][]string{{"2"}})
tdb.ExpectErr(t, `verifying precondition for version \d*22.1-\d+: `+
tdb.ExpectErr(t, `verifying precondition for version \d*22.2-\d+: `+
`paused GC jobs prevent upgrading GC job behavior: \[\d+ \d+]`,
"SET CLUSTER SETTING version = crdb_internal.node_executable_version()")

Expand Down

0 comments on commit 68d4daf

Please sign in to comment.