Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
107294: ttl: show table name for jobs in SHOW SCHEDULES r=chrisseto a=chrisseto

Previously, `SHOW SCHEDULES` would only show a table's ID for row level TTL jobs. For the convenience of debugging, this commit upgrades the `label` column to contain the table's name instead of its ID.

Rather than performing table name resolution at query time, the job name is set at creation time and updated by the schema change during table renames. This minimizes the likelihood of `SHOW SCHEDULES` being rendered unusable when a cluster is in an unstable state.

Epic: CRDB-18322
Fixes: #93774

107316: sqlproxyccl: simplify NewSubStopper r=JeffSwenson a=JeffSwenson

The lock in NewSubStopper caused lock ordering warnings when run under deadlock detection. The justification given for the lock's existence is wrong. If a closer is added to an already stopped stopper, the closer is called immediately.

Release note: None
Part of: #106571

107941: storage: don't import `testing` for `DisableMetamorphicSimpleValueEncoding` r=erikgrinaker a=erikgrinaker

We shouldn't link `testing` in binaries.

Epic: none
Release note: None

107950: testutils: fix determinism in random predecessor history tests r=jayshrivastava a=renatolabs

Previously, the `rng` instance used by the tests was initialized once in the `var` block. That meant that every assertion that relied on that rng was order-dependent: running a single test in that file in isolation would lead to failed assertions.

This commit ensures that every test resets the rng before making any assertions, guaranteeing we get the same values regardless of execution order.

Epic: none

Release note: None

Co-authored-by: Chris Seto <chriskseto@gmail.com>
Co-authored-by: Jeff <swenson@cockroachlabs.com>
Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
Co-authored-by: Renato Costa <renato@cockroachlabs.com>
  • Loading branch information
5 people committed Aug 1, 2023
5 parents f6c4d9d + fc59204 + 6d94042 + b67f136 + 1395ba8 commit 509f8c8
Show file tree
Hide file tree
Showing 22 changed files with 260 additions and 111 deletions.
3 changes: 1 addition & 2 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -2239,8 +2239,7 @@ func (r *restoreResumer) publishDescriptors(
jobsKnobs,
jobs.ScheduledJobTxn(txn),
user,
mutTable.GetID(),
mutTable.GetRowLevelTTL(),
mutTable,
)
if err != nil {
return err
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ job cancel=a
----

query-sql
SELECT count(1) FROM [SHOW SCHEDULES] WHERE label LIKE 'row-level-ttl-%';
SELECT count(1) FROM [SHOW SCHEDULES] WHERE label LIKE 'row-level-ttl%';
----
0

Expand Down Expand Up @@ -75,7 +75,7 @@ job cancel=b
----

query-sql
SELECT count(1) FROM [SHOW SCHEDULES] WHERE label LIKE 'row-level-ttl-%';
SELECT count(1) FROM [SHOW SCHEDULES] WHERE label LIKE 'row-level-ttl%';
----
0

Expand Down
14 changes: 7 additions & 7 deletions pkg/ccl/backupccl/testdata/backup-restore/row_level_ttl
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ new-cluster name=s2 share-io-dir=s1 allow-implicit-access

query-sql
SELECT count(1) FROM [SHOW SCHEDULES]
WHERE label LIKE 'row-level-ttl-%'
WHERE label LIKE 'row-level-ttl%'
----
0

Expand All @@ -43,7 +43,7 @@ CREATE TABLE public.t (

query-sql
SELECT count(1) FROM [SHOW SCHEDULES]
WHERE label LIKE 'row-level-ttl-%'
WHERE label LIKE 'row-level-ttl%'
----
1

Expand All @@ -53,7 +53,7 @@ DROP DATABASE d

query-sql
SELECT count(1) FROM [SHOW SCHEDULES]
WHERE label LIKE 'row-level-ttl-%'
WHERE label LIKE 'row-level-ttl%'
----
0

Expand All @@ -76,7 +76,7 @@ CREATE TABLE public.t (

query-sql
SELECT count(1) FROM [SHOW SCHEDULES]
WHERE label LIKE 'row-level-ttl-%'
WHERE label LIKE 'row-level-ttl%'
----
1

Expand All @@ -86,7 +86,7 @@ DROP DATABASE d

query-sql
SELECT count(1) FROM [SHOW SCHEDULES]
WHERE label LIKE 'row-level-ttl-%'
WHERE label LIKE 'row-level-ttl%'
----
0

Expand All @@ -113,7 +113,7 @@ CREATE TABLE public.t (

query-sql
SELECT count(1) FROM [SHOW SCHEDULES]
WHERE label LIKE 'row-level-ttl-%'
WHERE label LIKE 'row-level-ttl%'
----
1

Expand All @@ -123,6 +123,6 @@ DROP TABLE d.public.t

query-sql
SELECT count(1) FROM [SHOW SCHEDULES]
WHERE label LIKE 'row-level-ttl-%'
WHERE label LIKE 'row-level-ttl%'
----
0
18 changes: 2 additions & 16 deletions pkg/ccl/sqlproxyccl/tenantdirsvr/test_directory_svr.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,26 +29,12 @@ import (
var _ tenant.DirectoryServer = (*TestDirectoryServer)(nil)

// NewSubStopper creates a new stopper that will be stopped when either the
// parent is stopped or its own Stop is called. The code is slightly more
// complicated that simply calling NewStopper followed by AddCloser since there
// is a possibility that between the two calls, the parent stopper completes a
// stop and then the leak detection may find a leaked stopper.
// parent is stopped or its own Stop is called.
func NewSubStopper(parentStopper *stop.Stopper) *stop.Stopper {
var mu syncutil.Mutex
var subStopper *stop.Stopper
subStopper := stop.NewStopper(stop.WithTracer(parentStopper.Tracer()))
parentStopper.AddCloser(stop.CloserFn(func() {
mu.Lock()
defer mu.Unlock()
if subStopper == nil {
subStopper = stop.NewStopper(stop.WithTracer(parentStopper.Tracer()))
}
subStopper.Stop(context.Background())
}))
mu.Lock()
defer mu.Unlock()
if subStopper == nil {
subStopper = stop.NewStopper(stop.WithTracer(parentStopper.Tracer()))
}
return subStopper
}

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -508,6 +508,7 @@ go_library(
"//pkg/sql/storageparam/tablestorageparam",
"//pkg/sql/syntheticprivilege",
"//pkg/sql/syntheticprivilegecache",
"//pkg/sql/ttl/ttlbase",
"//pkg/sql/types",
"//pkg/sql/vtable",
"//pkg/storage",
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/check.go
Original file line number Diff line number Diff line change
Expand Up @@ -804,8 +804,7 @@ func (p *planner) RepairTTLScheduledJobForTable(ctx context.Context, tableID int
p.ExecCfg().JobsKnobs(),
jobs.ScheduledJobTxn(p.InternalSQLTxn()),
p.User(),
tableDesc.GetID(),
tableDesc.GetRowLevelTTL(),
tableDesc,
)
if err != nil {
return err
Expand Down
27 changes: 14 additions & 13 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/storageparam"
"github.com/cockroachdb/cockroach/pkg/sql/storageparam/indexstorageparam"
"github.com/cockroachdb/cockroach/pkg/sql/storageparam/tablestorageparam"
"github.com/cockroachdb/cockroach/pkg/sql/ttl/ttlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -2416,8 +2417,7 @@ func newTableDesc(
params.ExecCfg().JobsKnobs(),
jobs.ScheduledJobTxn(params.p.InternalSQLTxn()),
params.p.User(),
ret.GetID(),
ttl,
ret,
)
if err != nil {
return nil, err
Expand All @@ -2428,27 +2428,25 @@ func newTableDesc(
}

// newRowLevelTTLScheduledJob returns a *jobs.ScheduledJob for row level TTL
// for a given table.
// for a given table. newRowLevelTTLScheduledJob assumes that
// tblDesc.RowLevelTTL is not nil.
func newRowLevelTTLScheduledJob(
env scheduledjobs.JobSchedulerEnv,
owner username.SQLUsername,
tblID descpb.ID,
ttl *catpb.RowLevelTTL,
env scheduledjobs.JobSchedulerEnv, owner username.SQLUsername, tblDesc *tabledesc.Mutable,
) (*jobs.ScheduledJob, error) {
sj := jobs.NewScheduledJob(env)
sj.SetScheduleLabel(fmt.Sprintf("row-level-ttl-%d", tblID))
sj.SetScheduleLabel(ttlbase.BuildScheduleLabel(tblDesc))
sj.SetOwner(owner)
sj.SetScheduleDetails(jobspb.ScheduleDetails{
Wait: jobspb.ScheduleDetails_WAIT,
// If a job fails, try again at the allocated cron time.
OnError: jobspb.ScheduleDetails_RETRY_SCHED,
})

if err := sj.SetSchedule(ttl.DeletionCronOrDefault()); err != nil {
if err := sj.SetSchedule(tblDesc.RowLevelTTL.DeletionCronOrDefault()); err != nil {
return nil, err
}
args := &catpb.ScheduledRowLevelTTLArgs{
TableID: tblID,
TableID: tblDesc.GetID(),
}
any, err := pbtypes.MarshalAny(args)
if err != nil {
Expand All @@ -2467,12 +2465,15 @@ func CreateRowLevelTTLScheduledJob(
knobs *jobs.TestingKnobs,
s jobs.ScheduledJobStorage,
owner username.SQLUsername,
tblID descpb.ID,
ttl *catpb.RowLevelTTL,
tblDesc *tabledesc.Mutable,
) (*jobs.ScheduledJob, error) {
if !tblDesc.HasRowLevelTTL() {
return nil, errors.AssertionFailedf("CreateRowLevelTTLScheduledJob called with no .RowLevelTTL: %#v", tblDesc)
}

telemetry.Inc(sqltelemetry.RowLevelTTLCreated)
env := JobSchedulerEnv(knobs)
j, err := newRowLevelTTLScheduledJob(env, owner, tblID, ttl)
j, err := newRowLevelTTLScheduledJob(env, owner, tblDesc)
if err != nil {
return nil, err
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/descmetadata/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,10 +10,12 @@ go_library(
"//pkg/settings",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/descs",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/isql",
"//pkg/sql/schemachanger/scexec",
"//pkg/sql/sessiondata",
"//pkg/sql/sessioninit",
"//pkg/sql/ttl/ttlbase",
],
)

Expand Down
22 changes: 22 additions & 0 deletions pkg/sql/descmetadata/metadata_updater.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sessioninit"
"github.com/cockroachdb/cockroach/pkg/sql/ttl/ttlbase"
)

// metadataUpdater which implements scexec.MetaDataUpdater that is used to update
Expand Down Expand Up @@ -94,3 +96,23 @@ func (mu metadataUpdater) DeleteSchedule(ctx context.Context, scheduleID int64)
)
return err
}

// UpdateTTLScheduleLabel implement scexec.DescriptorMetadataUpdater.
func (mu metadataUpdater) UpdateTTLScheduleLabel(
ctx context.Context, tbl *tabledesc.Mutable,
) error {
if !tbl.HasRowLevelTTL() {
return nil
}

_, err := mu.txn.ExecEx(
ctx,
"update-ttl-schedule-label",
mu.txn.KV(),
sessiondata.RootUserSessionDataOverride,
"UPDATE system.scheduled_jobs SET schedule_name = $1 WHERE schedule_id = $2",
ttlbase.BuildScheduleLabel(tbl),
tbl.RowLevelTTL.ScheduleID,
)
return err
}
Loading

0 comments on commit 509f8c8

Please sign in to comment.