Skip to content

Commit 2097e1e

Browse files
craig[bot]spilchen
andcommitted
Merge #155774
155774: sql/inspect: add DETACHED option to run INSPECT without waiting r=spilchen a=spilchen Adds support for running `INSPECT ... WITH OPTIONS (DETACHED)`, which submits the inspection as a background job and returns immediately with the job ID in a notice. This allows INSPECT to run inside a transaction. **Note to reviewers:** This was added to support running INSPECT within the random schema workload, which executes statements inside transactions. Previously, INSPECT could not run inside a transaction. It also gives us async behaviour. While expanding INSPECT test coverage, I have wanted to kick off an async run of INSPECT. Until now, I worked around it by using `statement_timeout` to force the job to run in the background so we could later wait on it. The detached option provides a cleaner approach. It is similar to how commands like BACKUP support an async mode. Informs: #155483 Epic: CRDB-55075 Release note (sql change): INSPECT supports a DETACHED option to run the operation without waiting for it. Co-authored-by: Matt Spilchen <[email protected]>
2 parents 629a290 + 4ae44b7 commit 2097e1e

File tree

7 files changed

+238
-6
lines changed

7 files changed

+238
-6
lines changed

docs/generated/sql/bnf/stmt_block.bnf

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3643,6 +3643,9 @@ only_signed_fconst ::=
36433643
inspect_option ::=
36443644
'INDEX' 'ALL'
36453645
| 'INDEX' '(' table_index_name_list ')'
3646+
| 'DETACHED'
3647+
| 'DETACHED' '=' 'TRUE'
3648+
| 'DETACHED' '=' 'FALSE'
36463649

36473650
virtual_cluster_name ::=
36483651
'VIRTUAL_CLUSTER_NAME'

pkg/sql/inspect/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@ go_library(
4141
"//pkg/sql/lexbase",
4242
"//pkg/sql/pgwire/pgcode",
4343
"//pkg/sql/pgwire/pgerror",
44+
"//pkg/sql/pgwire/pgnotice",
4445
"//pkg/sql/physicalplan",
4546
"//pkg/sql/privilege",
4647
"//pkg/sql/rowexec",

pkg/sql/inspect/inspect_plan.go

Lines changed: 71 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -11,11 +11,13 @@ import (
1111
"github.com/cockroachdb/cockroach/pkg/clusterversion"
1212
"github.com/cockroachdb/cockroach/pkg/jobs"
1313
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
14+
"github.com/cockroachdb/cockroach/pkg/security/username"
1415
"github.com/cockroachdb/cockroach/pkg/sql"
1516
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
1617
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
1718
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
1819
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
20+
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
1921
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
2022
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
2123
"github.com/cockroachdb/cockroach/pkg/util/hlc"
@@ -25,12 +27,21 @@ import (
2527
func inspectTypeCheck(
2628
ctx context.Context, stmt tree.Statement, p sql.PlanHookState,
2729
) (matched bool, header colinfo.ResultColumns, _ error) {
28-
_, ok := stmt.(*tree.Inspect)
30+
inspectStmt, ok := stmt.(*tree.Inspect)
2931
if !ok {
3032
return false, nil, nil
3133
}
3234

33-
return true, jobs.InspectJobExecutionResultHeader, nil
35+
// Determine header based on DETACHED option to ensure consistency in
36+
// local-prepared mode where plans are cached.
37+
detached := inspectStmt.Options.IsDetached()
38+
if detached {
39+
header = jobs.DetachedJobExecutionResultHeader
40+
} else {
41+
header = jobs.InspectJobExecutionResultHeader
42+
}
43+
44+
return true, header, nil
3445
}
3546

3647
// inspectRun represents the runtime state of an execution of INSPECT.
@@ -72,8 +83,8 @@ func newInspectRun(
7283
return inspectRun{}, errors.AssertionFailedf("unexpected INSPECT type received, got: %v", stmt.Typ)
7384
}
7485

75-
if len(stmt.Options) == 0 || stmt.Options.HasIndexAll() {
76-
// No options or INDEX ALL specified - inspect all indexes.
86+
if !stmt.Options.HasIndexOption() || stmt.Options.HasIndexAll() {
87+
// No INDEX options or INDEX ALL specified - inspect all indexes.
7788
switch stmt.Typ {
7889
case tree.InspectTable:
7990
checks, err := sql.InspectChecksForTable(ctx, p, run.table)
@@ -166,7 +177,9 @@ func inspectPlanHook(
166177
return nil, nil, false, err
167178
}
168179

169-
if !p.ExtendedEvalContext().TxnIsSingleStmt {
180+
detached := inspectStmt.Options.IsDetached()
181+
182+
if !detached && !p.ExtendedEvalContext().TxnIsSingleStmt {
170183
return nil, nil, false, pgerror.Newf(pgcode.InvalidTransactionState,
171184
"cannot run within a multi-statement transaction")
172185
}
@@ -176,6 +189,30 @@ func inspectPlanHook(
176189
return nil, nil, false, err
177190
}
178191

192+
if detached {
193+
fn := func(ctx context.Context, resultsCh chan<- tree.Datums) error {
194+
jobID := p.ExecCfg().JobRegistry.MakeJobID()
195+
jr := makeInspectJobRecord(tree.AsString(stmt), jobID, run.checks, run.asOfTimestamp)
196+
if _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn(
197+
ctx, jr, jobID, p.InternalSQLTxn(),
198+
); err != nil {
199+
return err
200+
}
201+
var notice pgnotice.Notice
202+
if p.ExtendedEvalContext().TxnImplicit {
203+
notice = pgnotice.Newf("INSPECT job %d running in the background", jobID)
204+
} else {
205+
notice = pgnotice.Newf("INSPECT job %d queued; will start after the current transaction commits", jobID)
206+
}
207+
if err := p.SendClientNotice(ctx, notice, true /* immediateFlush */); err != nil {
208+
return err
209+
}
210+
resultsCh <- tree.Datums{tree.NewDInt(tree.DInt(jobID))}
211+
return nil
212+
}
213+
return fn, jobs.DetachedJobExecutionResultHeader, false, nil
214+
}
215+
179216
fn := func(ctx context.Context, resultsCh chan<- tree.Datums) error {
180217
// We create the job record in the planner's transaction to ensure that
181218
// the job record creation happens transactionally.
@@ -186,6 +223,14 @@ func inspectPlanHook(
186223
return err
187224
}
188225

226+
if err = p.SendClientNotice(
227+
ctx,
228+
pgnotice.Newf("waiting for INSPECT job to complete: %s\nIf the statement is canceled, the job will continue in the background.", sj.ID()),
229+
true, /* immediateFlush */
230+
); err != nil {
231+
return err
232+
}
233+
189234
if err := sj.AwaitCompletion(ctx); err != nil {
190235
return err
191236
}
@@ -195,6 +240,27 @@ func inspectPlanHook(
195240
return fn, jobs.InspectJobExecutionResultHeader, false, nil
196241
}
197242

243+
func makeInspectJobRecord(
244+
description string, jobID jobspb.JobID, checks []*jobspb.InspectDetails_Check, asOf hlc.Timestamp,
245+
) jobs.Record {
246+
descIDs := catalog.DescriptorIDSet{}
247+
for _, check := range checks {
248+
descIDs.Add(check.TableID)
249+
}
250+
return jobs.Record{
251+
JobID: jobID,
252+
Description: description,
253+
Details: jobspb.InspectDetails{
254+
Checks: checks,
255+
AsOf: asOf,
256+
},
257+
Progress: jobspb.InspectProgress{},
258+
CreatedBy: nil,
259+
Username: username.NodeUserName(),
260+
DescriptorIDs: descIDs.Ordered(),
261+
}
262+
}
263+
198264
func init() {
199265
sql.AddPlanHook("inspect", inspectPlanHook, inspectTypeCheck)
200266
}

pkg/sql/logictest/testdata/logic_test/inspect

Lines changed: 79 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,10 @@ WHERE job_type = 'INSPECT'
1515
ORDER BY created DESC
1616
LIMIT 1
1717

18+
# Make job adoption faster. This is needed to speed up DETACHED jobs.
19+
statement ok
20+
SET CLUSTER SETTING jobs.registry.interval.adopt = '500ms';
21+
1822
statement ok
1923
CREATE TABLE foo (c1 INT, c2 INT, INDEX idx_c1 (c1), INDEX idx_c2 (c2));
2024

@@ -27,6 +31,9 @@ INSPECT DATABASE test;
2731
statement ok
2832
SET enable_inspect_command = true;
2933

34+
subtest transaction_behavior
35+
36+
# Test that regular INSPECT cannot run within a transaction.
3037
statement ok
3138
BEGIN;
3239

@@ -36,7 +43,77 @@ INSPECT TABLE foo;
3643
statement ok
3744
ROLLBACK;
3845

46+
statement notice NOTICE: INSPECT job [0-9]+ running in the background
47+
INSPECT TABLE foo WITH OPTIONS DETACHED
48+
49+
# Wait for the detached job to eventually succeed.
50+
query T retry
51+
SELECT status FROM last_inspect_job
52+
----
53+
succeeded
54+
55+
# Test that DETACHED INSPECT can run inside a transaction.
56+
statement ok
57+
BEGIN;
58+
59+
statement notice NOTICE: INSPECT job [0-9]+ queued; will start after the current transaction commits
60+
INSPECT TABLE foo WITH OPTIONS DETACHED
61+
62+
# Verify the job isn't committed yet. It will stay in the running stage because we haven't committed it yet.
63+
query T
64+
SELECT status FROM last_inspect_job
65+
----
66+
running
67+
3968
statement ok
69+
COMMIT;
70+
71+
# Wait for the new job to eventually succeed.
72+
query T retry
73+
SELECT status FROM last_inspect_job
74+
----
75+
succeeded
76+
77+
subtest end
78+
79+
subtest inspect_options
80+
81+
# Test DETACHED option validation.
82+
83+
# Failure if both DETACHED and DETACHED = FALSE are specified.
84+
statement error pq: conflicting INSPECT options: DETACHED specified with different values
85+
INSPECT TABLE foo WITH OPTIONS DETACHED, DETACHED = FALSE
86+
87+
# DETACHED = TRUE is the same as DETACHED.
88+
# Both should check all indexes (2 total: idx_c1 and idx_c2).
89+
statement notice NOTICE: INSPECT job [0-9]+ running in the background
90+
INSPECT TABLE foo WITH OPTIONS DETACHED = TRUE
91+
92+
query TI retry
93+
SELECT * FROM last_inspect_job
94+
----
95+
succeeded 2
96+
97+
statement notice NOTICE: INSPECT job [0-9]+ running in the background
98+
INSPECT TABLE foo WITH OPTIONS DETACHED
99+
100+
query TI retry
101+
SELECT * FROM last_inspect_job
102+
----
103+
succeeded 2
104+
105+
# DETACHED = TRUE with a specific index (verify the number of checks).
106+
statement notice NOTICE: INSPECT job [0-9]+ running in the background
107+
INSPECT TABLE foo WITH OPTIONS DETACHED = TRUE, INDEX (idx_c1)
108+
109+
query TI retry
110+
SELECT * FROM last_inspect_job
111+
----
112+
succeeded 1
113+
114+
# Test regular (non-detached) inspect operations.
115+
116+
statement notice NOTICE: waiting for INSPECT job to complete: [0-9]+\nIf the statement is canceled, the job will continue in the background.
40117
INSPECT TABLE foo;
41118

42119
query TI
@@ -123,6 +200,8 @@ SELECT * FROM last_inspect_job;
123200
----
124201
succeeded 3
125202

203+
subtest end
204+
126205
subtest permissions
127206

128207
statement ok

pkg/sql/parser/sql.y

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -7939,6 +7939,18 @@ inspect_option:
79397939
{
79407940
$$.val = &tree.InspectOptionIndex{IndexNames: $3.newTableIndexNames()}
79417941
}
7942+
| DETACHED
7943+
{
7944+
$$.val = &tree.InspectOptionDetached{Detached: *tree.MakeDBool(true)}
7945+
}
7946+
| DETACHED '=' TRUE
7947+
{
7948+
$$.val = &tree.InspectOptionDetached{Detached: *tree.MakeDBool(true)}
7949+
}
7950+
| DETACHED '=' FALSE
7951+
{
7952+
$$.val = &tree.InspectOptionDetached{Detached: *tree.MakeDBool(false)}
7953+
}
79427954

79437955
// %Help: SHOW INSPECT ERRORS - list errors recorded by one INSPECT run
79447956
// %Category: Misc

pkg/sql/parser/testdata/inspect

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,14 @@ INSPECT DATABASE x AS OF SYSTEM TIME (1) WITH OPTIONS INDEX (index_name_1, index
3131
INSPECT DATABASE x AS OF SYSTEM TIME _ WITH OPTIONS INDEX (index_name_1, index_name_2) -- literals removed
3232
INSPECT DATABASE _ AS OF SYSTEM TIME 1 WITH OPTIONS INDEX (_, _) -- identifiers removed
3333

34+
parse
35+
INSPECT DATABASE x WITH OPTIONS DETACHED, INDEX (index_name)
36+
----
37+
INSPECT DATABASE x WITH OPTIONS DETACHED, INDEX (index_name)
38+
INSPECT DATABASE x WITH OPTIONS DETACHED, INDEX (index_name) -- fully parenthesized
39+
INSPECT DATABASE x WITH OPTIONS DETACHED, INDEX (index_name) -- literals removed
40+
INSPECT DATABASE _ WITH OPTIONS DETACHED, INDEX (_) -- identifiers removed
41+
3442
parse
3543
INSPECT TABLE x WITH OPTIONS INDEX (index_name)
3644
----
@@ -79,6 +87,14 @@ INSPECT TABLE x WITH OPTIONS INDEX (index_name_1, index_name_2) -- fully parenth
7987
INSPECT TABLE x WITH OPTIONS INDEX (index_name_1, index_name_2) -- literals removed
8088
INSPECT TABLE _ WITH OPTIONS INDEX (_, _) -- identifiers removed
8189

90+
parse
91+
INSPECT TABLE x WITH OPTIONS DETACHED
92+
----
93+
INSPECT TABLE x WITH OPTIONS DETACHED
94+
INSPECT TABLE x WITH OPTIONS DETACHED -- fully parenthesized
95+
INSPECT TABLE x WITH OPTIONS DETACHED -- literals removed
96+
INSPECT TABLE _ WITH OPTIONS DETACHED -- identifiers removed
97+
8298
parse
8399
INSPECT TABLE db.schema.t1 WITH OPTIONS INDEX (index_name)
84100
----

pkg/sql/sem/tree/inspect.go

Lines changed: 56 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -134,6 +134,19 @@ func indexMatchesTable(index *TableIndexName, table TableName) bool {
134134
return true
135135
}
136136

137+
// IsDetached returns the value of the DETACHED option, if set.
138+
// Returns false if DETACHED is not specified.
139+
// Conflicts are handled by validate(), so this simply returns the first
140+
// DETACHED value found.
141+
func (n *InspectOptions) IsDetached() bool {
142+
for _, option := range *n {
143+
if opt, ok := option.(*InspectOptionDetached); ok {
144+
return bool(opt.Detached)
145+
}
146+
}
147+
return false
148+
}
149+
137150
// HasIndexAll checks if the options include an INDEX ALL option.
138151
func (n *InspectOptions) HasIndexAll() bool {
139152
for _, option := range *n {
@@ -144,6 +157,18 @@ func (n *InspectOptions) HasIndexAll() bool {
144157
return false
145158
}
146159

160+
// HasIndexOption checks if the options include an INDEX option with
161+
// specific index names (i.e., INDEX (name1, name2, ...)).
162+
// This does NOT include INDEX ALL - use HasIndexAll() for that.
163+
func (n *InspectOptions) HasIndexOption() bool {
164+
for _, option := range *n {
165+
if _, ok := option.(*InspectOptionIndex); ok {
166+
return true
167+
}
168+
}
169+
return false
170+
}
171+
147172
// Validate checks for internal consistency of the INSPECT command.
148173
func (n *Inspect) Validate() error {
149174
if err := n.Options.validate(); err != nil {
@@ -156,13 +181,23 @@ func (n *Inspect) Validate() error {
156181
// validate checks for internal consistency of options on the INSPECT command.
157182
func (n *InspectOptions) validate() error {
158183
var hasOptionIndex, hasOptionIndexAll bool
184+
var detachedSeen bool
185+
var detachedVal bool
159186

160187
for _, option := range *n {
161-
switch option.(type) {
188+
switch opt := option.(type) {
162189
case *InspectOptionIndex:
163190
hasOptionIndex = true
164191
case *InspectOptionIndexAll:
165192
hasOptionIndexAll = true
193+
case *InspectOptionDetached:
194+
optVal := bool(opt.Detached)
195+
if detachedSeen && optVal != detachedVal {
196+
return pgerror.Newf(pgcode.Syntax,
197+
"conflicting INSPECT options: DETACHED specified with different values")
198+
}
199+
detachedSeen = true
200+
detachedVal = optVal
166201
default:
167202
return fmt.Errorf("unknown inspect option: %T", option)
168203
}
@@ -224,3 +259,23 @@ type InspectOptionIndexAll struct{}
224259
func (n *InspectOptionIndexAll) Format(ctx *FmtCtx) {
225260
ctx.WriteString("INDEX ALL")
226261
}
262+
263+
// InspectOptionDetached keeps track of state for the DETACHED option.
264+
type InspectOptionDetached struct {
265+
Detached DBool
266+
}
267+
268+
// inspectOptionType implements InspectOption.
269+
func (*InspectOptionDetached) inspectOptionType() {}
270+
271+
// Format implements the NodeFormatter interface.
272+
func (n *InspectOptionDetached) Format(ctx *FmtCtx) {
273+
if bool(n.Detached) {
274+
ctx.WriteString("DETACHED")
275+
return
276+
}
277+
ctx.WriteString("DETACHED = false")
278+
}
279+
280+
// String implements fmt.Stringer.
281+
func (n *InspectOptionDetached) String() string { return AsString(n) }

0 commit comments

Comments
 (0)