Skip to content

Commit

Permalink
jobs: only store 23.1 debugging info after 23.1 upgrade
Browse files Browse the repository at this point in the history
Release note: none.
Epic: none.
  • Loading branch information
dt committed Nov 6, 2023
1 parent 80415cb commit c284442
Show file tree
Hide file tree
Showing 7 changed files with 17 additions and 6 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_processor_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,7 @@ func distBackup(

defer close(progCh)
execCfg := execCtx.ExecCfg()
jobsprofiler.StorePlanDiagram(ctx, execCfg.DistSQLSrv.Stopper, p, execCfg.InternalDB, jobID)
jobsprofiler.StorePlanDiagram(ctx, execCfg.DistSQLSrv.Stopper, p, execCfg.InternalDB, jobID, execCfg.Settings.Version)

// Copy the evalCtx, as dsp.Run() might change it.
evalCtxCopy := *evalCtx
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/restore_processor_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -299,7 +299,7 @@ func distRestore(
defer recv.Release()

execCfg := execCtx.ExecCfg()
jobsprofiler.StorePlanDiagram(ctx, execCfg.DistSQLSrv.Stopper, p, execCfg.InternalDB, jobID)
jobsprofiler.StorePlanDiagram(ctx, execCfg.DistSQLSrv.Stopper, p, execCfg.InternalDB, jobID, execCfg.Settings.Version)

// Copy the evalCtx, as dsp.Run() might change it.
evalCtxCopy := *evalCtx
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed_dist.go
Original file line number Diff line number Diff line change
Expand Up @@ -320,7 +320,7 @@ func startDistChangefeed(
finishedSetupFn = func(flowinfra.Flow) { resultsCh <- tree.Datums(nil) }
}

jobsprofiler.StorePlanDiagram(ctx, execCfg.DistSQLSrv.Stopper, p, execCfg.InternalDB, jobID)
jobsprofiler.StorePlanDiagram(ctx, execCfg.DistSQLSrv.Stopper, p, execCfg.InternalDB, jobID, execCfg.Settings.Version)

// Copy the evalCtx, as dsp.Run() might change it.
evalCtxCopy := *evalCtx
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/streamingccl/streamingest/stream_ingestion_dist.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ func startDistIngestion(
defer recv.Release()

jobsprofiler.StorePlanDiagram(ctx, execCtx.ExecCfg().DistSQLSrv.Stopper, p, execCtx.ExecCfg().InternalDB,
ingestionJob.ID())
ingestionJob.ID(), execCtx.ExecCfg().Settings.Version)

// Copy the evalCtx, as dsp.Run() might change it.
evalCtxCopy := *execCtx.ExtendedEvalContext()
Expand Down
1 change: 1 addition & 0 deletions pkg/jobs/jobsprofiler/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/jobs/jobsprofiler",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/jobs",
"//pkg/jobs/jobspb",
"//pkg/sql",
Expand Down
12 changes: 11 additions & 1 deletion pkg/jobs/jobsprofiler/profiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/sql"
Expand All @@ -28,8 +29,17 @@ import (
// table. The generation of the plan diagram and persistence to the info table
// are done asynchronously and this method does not block on their completion.
func StorePlanDiagram(
ctx context.Context, stopper *stop.Stopper, p *sql.PhysicalPlan, db isql.DB, jobID jobspb.JobID,
ctx context.Context,
stopper *stop.Stopper,
p *sql.PhysicalPlan,
db isql.DB,
jobID jobspb.JobID,
cv clusterversion.Handle,
) {
if !cv.IsActive(ctx, clusterversion.V23_1) {
return
}

if err := stopper.RunAsyncTask(ctx, "jobs-store-plan-diagram", func(ctx context.Context) {
var cancel func()
ctx, cancel = stopper.WithCancelOnQuiesce(ctx)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/importer/import_processor_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -270,7 +270,7 @@ func distImport(
}

execCfg := execCtx.ExecCfg()
jobsprofiler.StorePlanDiagram(ctx, execCfg.DistSQLSrv.Stopper, p, execCfg.InternalDB, job.ID())
jobsprofiler.StorePlanDiagram(ctx, execCfg.DistSQLSrv.Stopper, p, execCfg.InternalDB, job.ID(), execCtx.ExecCfg().Settings.Version)

// Copy the evalCtx, as dsp.Run() might change it.
evalCtxCopy := *evalCtx
Expand Down

0 comments on commit c284442

Please sign in to comment.