@@ -10,10 +10,17 @@ import (
10
10
11
11
"github.com/cockroachdb/cockroach/pkg/jobs"
12
12
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
13
+ "github.com/cockroachdb/cockroach/pkg/roachpb"
13
14
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
14
15
"github.com/cockroachdb/cockroach/pkg/sql"
16
+ "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
17
+ "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
15
18
"github.com/cockroachdb/cockroach/pkg/sql/isql"
19
+ "github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
20
+ "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
21
+ "github.com/cockroachdb/cockroach/pkg/sql/types"
16
22
"github.com/cockroachdb/cockroach/pkg/util/log"
23
+ "github.com/cockroachdb/errors"
17
24
)
18
25
19
26
type inspectResumer struct {
@@ -34,25 +41,26 @@ func (c *inspectResumer) Resume(ctx context.Context, execCtx interface{}) error
34
41
knobs = * inspectKnobs
35
42
}
36
43
37
- if knobs .OnInspectJobStart != nil {
38
- if err := knobs .OnInspectJobStart (); err != nil {
39
- return err
40
- }
44
+ if err := maybeRunOnJobStartHook (knobs ); err != nil {
45
+ return err
41
46
}
42
47
43
- if err := c .job .NoTxn ().Update (ctx ,
44
- func (_ isql.Txn , md jobs.JobMetadata , ju * jobs.JobUpdater ) error {
45
- progress := md .Progress
46
- progress .Progress = & jobspb.Progress_FractionCompleted {
47
- FractionCompleted : 1 ,
48
- }
49
- ju .UpdateProgress (progress )
50
- return nil
51
- },
52
- ); err != nil {
48
+ pkSpans , err := c .getPrimaryIndexSpans (ctx , execCfg )
49
+ if err != nil {
53
50
return err
54
51
}
55
- return nil
52
+
53
+ // TODO(149460): add a goroutine that will replan the job on topology changes
54
+ plan , planCtx , err := c .planInspectProcessors (ctx , jobExecCtx , pkSpans )
55
+ if err != nil {
56
+ return err
57
+ }
58
+
59
+ if err := c .runInspectPlan (ctx , jobExecCtx , planCtx , plan ); err != nil {
60
+ return err
61
+ }
62
+
63
+ return c .markJobComplete (ctx )
56
64
}
57
65
58
66
// OnFailOrCancel implements the Resumer interface
@@ -67,6 +75,132 @@ func (c *inspectResumer) CollectProfile(ctx context.Context, execCtx interface{}
67
75
return nil
68
76
}
69
77
78
+ func maybeRunOnJobStartHook (knobs sql.InspectTestingKnobs ) error {
79
+ if knobs .OnInspectJobStart != nil {
80
+ return knobs .OnInspectJobStart ()
81
+ }
82
+ return nil
83
+ }
84
+
85
+ // getPrimaryIndexSpans returns the primary index spans for all tables involved in
86
+ // the INSPECT job's checks.
87
+ func (c * inspectResumer ) getPrimaryIndexSpans (
88
+ ctx context.Context , execCfg * sql.ExecutorConfig ,
89
+ ) ([]roachpb.Span , error ) {
90
+ details := c .job .Details ().(jobspb.InspectDetails )
91
+
92
+ spans := make ([]roachpb.Span , 0 , len (details .Checks ))
93
+ err := execCfg .InternalDB .DescsTxn (ctx , func (ctx context.Context , txn descs.Txn ) error {
94
+ for i := range details .Checks {
95
+ desc , err := txn .Descriptors ().ByIDWithLeased (txn .KV ()).WithoutNonPublic ().Get ().Table (ctx , details .Checks [i ].TableID )
96
+ if err != nil {
97
+ return err
98
+ }
99
+ spans = append (spans , desc .PrimaryIndexSpan (execCfg .Codec ))
100
+ }
101
+ return nil
102
+ })
103
+ return spans , err
104
+ }
105
+
106
+ // planInspectProcessors constructs the physical plan for the INSPECT job by
107
+ // partitioning the given primary index spans across all nodes in the cluster.
108
+ // Each processor will be assigned one or more spans to run their checks on.
109
+ func (c * inspectResumer ) planInspectProcessors (
110
+ ctx context.Context , jobExecCtx sql.JobExecContext , entirePKSpans []roachpb.Span ,
111
+ ) (* sql.PhysicalPlan , * sql.PlanningCtx , error ) {
112
+ if len (entirePKSpans ) > 1 {
113
+ return nil , nil , errors .AssertionFailedf ("we only support one check: %d" , len (entirePKSpans ))
114
+ }
115
+ distSQLPlanner := jobExecCtx .DistSQLPlanner ()
116
+ planCtx , _ , err := distSQLPlanner .SetupAllNodesPlanning (ctx , jobExecCtx .ExtendedEvalContext (), jobExecCtx .ExecCfg ())
117
+ if err != nil {
118
+ return nil , nil , err
119
+ }
120
+
121
+ spanPartitions , err := distSQLPlanner .PartitionSpans (ctx , planCtx , entirePKSpans , sql .PartitionSpansBoundDefault )
122
+ if err != nil {
123
+ return nil , nil , err
124
+ }
125
+
126
+ jobID := c .job .ID ()
127
+ newProcessorSpec := func (spans []roachpb.Span ) * execinfrapb.InspectSpec {
128
+ return & execinfrapb.InspectSpec {
129
+ JobID : jobID ,
130
+ Spans : spans ,
131
+ }
132
+ }
133
+
134
+ // Set up a one-stage plan with one proc per input spec.
135
+ processorCorePlacements := make ([]physicalplan.ProcessorCorePlacement , len (spanPartitions ))
136
+ for i , spanPartition := range spanPartitions {
137
+ processorCorePlacements [i ].SQLInstanceID = spanPartition .SQLInstanceID
138
+ processorCorePlacements [i ].Core .Inspect = newProcessorSpec (spanPartition .Spans )
139
+ }
140
+
141
+ physicalPlan := planCtx .NewPhysicalPlan ()
142
+ physicalPlan .AddNoInputStage (
143
+ processorCorePlacements ,
144
+ execinfrapb.PostProcessSpec {},
145
+ []* types.T {},
146
+ execinfrapb.Ordering {},
147
+ nil , /* finalizeLastStageCb */
148
+ )
149
+ physicalPlan .PlanToStreamColMap = []int {}
150
+
151
+ sql .FinalizePlan (ctx , planCtx , physicalPlan )
152
+ return physicalPlan , planCtx , nil
153
+ }
154
+
155
+ // runInspectPlan executes the distributed physical plan for the INSPECT job.
156
+ // It sets up a metadata-only DistSQL receiver to collect any execution errors,
157
+ // then runs the plan using the provided planning context and evaluation context.
158
+ // This function returns any error surfaced via metadata from the processors.
159
+ func (c * inspectResumer ) runInspectPlan (
160
+ ctx context.Context ,
161
+ jobExecCtx sql.JobExecContext ,
162
+ planCtx * sql.PlanningCtx ,
163
+ plan * sql.PhysicalPlan ,
164
+ ) error {
165
+ execCfg := jobExecCtx .ExecCfg ()
166
+
167
+ metadataCallbackWriter := sql .NewMetadataOnlyMetadataCallbackWriter ()
168
+
169
+ distSQLReceiver := sql .MakeDistSQLReceiver (
170
+ ctx ,
171
+ metadataCallbackWriter ,
172
+ tree .Rows ,
173
+ execCfg .RangeDescriptorCache ,
174
+ nil , /* txn */
175
+ nil , /* clockUpdater */
176
+ jobExecCtx .ExtendedEvalContext ().Tracing ,
177
+ )
178
+ defer distSQLReceiver .Release ()
179
+
180
+ distSQLPlanner := jobExecCtx .DistSQLPlanner ()
181
+
182
+ // Copy the eval.Context, as dsp.Run() might change it.
183
+ evalCtxCopy := jobExecCtx .ExtendedEvalContext ().Context .Copy ()
184
+
185
+ distSQLPlanner .Run (ctx , planCtx , nil /* txn */ , plan ,
186
+ distSQLReceiver , evalCtxCopy , nil /* finishedSetupFn */ )
187
+ return metadataCallbackWriter .Err ()
188
+ }
189
+
190
+ func (c * inspectResumer ) markJobComplete (ctx context.Context ) error {
191
+ // TODO(148297): add fine-grained progress reporting
192
+ return c .job .NoTxn ().Update (ctx ,
193
+ func (_ isql.Txn , md jobs.JobMetadata , ju * jobs.JobUpdater ) error {
194
+ progress := md .Progress
195
+ progress .Progress = & jobspb.Progress_FractionCompleted {
196
+ FractionCompleted : 1 ,
197
+ }
198
+ ju .UpdateProgress (progress )
199
+ return nil
200
+ },
201
+ )
202
+ }
203
+
70
204
func init () {
71
205
createResumerFn := func (job * jobs.Job , settings * cluster.Settings ) jobs.Resumer {
72
206
return & inspectResumer {job : job }
0 commit comments