Skip to content

Commit d433b29

Browse files
committed
sql/inspect: support checking multiple indexes in one INSPECT job
Previously, there was an assert that made sure when building the INSPECT job that we only had one check. This removes that assert so that we can have multiple checks. When you run INSPECT against many indexes on a table or against a database, we will have multiple checks, so it was necessary for that case. Closes #148300 Release note: none Epic: none
1 parent d995e25 commit d433b29

File tree

4 files changed

+106
-22
lines changed

4 files changed

+106
-22
lines changed

pkg/sql/inspect/index_consistency_check_test.go

Lines changed: 77 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -128,6 +128,9 @@ func TestDetectIndexConsistencyErrors(t *testing.T) {
128128
splitRangeDDL string
129129
// indexDDL is the DDL to create the indexes on the table.
130130
indexDDL []string
131+
// corruptionTargetIndex specifies which secondary index to corrupt (0-based position).
132+
// If not specified, defaults to 0 (first index).
133+
corruptionTargetIndex int
131134
// missingIndexEntrySelector defines a SQL predicate that selects rows
132135
// whose secondary index entries will be manually deleted to simulate
133136
// missing index entries (i.e., present in the primary index but not in the
@@ -247,6 +250,57 @@ func TestDetectIndexConsistencyErrors(t *testing.T) {
247250
danglingIndexEntryInsertQuery: "SELECT 15, 30, 300, 'corrupt', 'e_3', 300.5", // Add dangling entry after timestamp
248251
useTimestampBeforeCorruption: true, // Use timestamp from before corruption
249252
},
253+
{
254+
desc: "2 indexes, corrupt second index, missing entry",
255+
splitRangeDDL: "ALTER TABLE test.t SPLIT AT VALUES (500)",
256+
indexDDL: []string{
257+
"CREATE INDEX idx_t_a ON test.t (a)",
258+
"CREATE INDEX idx_t_b ON test.t (b) STORING (e)",
259+
},
260+
corruptionTargetIndex: 1, // Target second index (idx_t_b)
261+
missingIndexEntrySelector: "a = 7",
262+
expectedIssues: []inspectIssue{
263+
{
264+
ErrorType: "missing_secondary_index_entry",
265+
PrimaryKey: "e'(7, \\'d_7\\')'",
266+
Details: map[redact.RedactableString]interface{}{
267+
"index_name": "idx_t_b",
268+
},
269+
},
270+
},
271+
expectedErrRegex: expectedInspectFoundInconsistencies,
272+
},
273+
{
274+
desc: "3 indexes, corrupt middle index, dangling entry",
275+
splitRangeDDL: "ALTER TABLE test.t SPLIT AT VALUES (333),(666)",
276+
indexDDL: []string{
277+
"CREATE INDEX idx_t_a ON test.t (a)",
278+
"CREATE INDEX idx_t_b ON test.t (b) STORING (c)",
279+
"CREATE INDEX idx_t_c ON test.t (c) STORING (f)",
280+
},
281+
corruptionTargetIndex: 1, // Target second index (middle one)
282+
danglingIndexEntryInsertQuery: "SELECT 25, 50, 500, 'corrupt_middle', 'e_25', 125.5",
283+
expectedIssues: []inspectIssue{
284+
{
285+
ErrorType: "dangling_secondary_index_entry",
286+
PrimaryKey: "e'(25, \\'corrupt_middle\\')'",
287+
Details: map[redact.RedactableString]interface{}{
288+
"index_name": "idx_t_b",
289+
},
290+
},
291+
},
292+
expectedErrRegex: expectedInspectFoundInconsistencies,
293+
},
294+
{
295+
desc: "multiple indexes, no corruption - all should be checked",
296+
indexDDL: []string{
297+
"CREATE INDEX idx_t_a ON test.t (a)",
298+
"CREATE INDEX idx_t_b ON test.t (b)",
299+
"CREATE INDEX idx_t_c ON test.t (c)",
300+
},
301+
// No corruptionTargetIndex specified, no corruption
302+
missingIndexEntrySelector: "", // No corruption
303+
},
250304
} {
251305
t.Run(tc.desc, func(t *testing.T) {
252306
issueLogger.reset()
@@ -313,7 +367,14 @@ func TestDetectIndexConsistencyErrors(t *testing.T) {
313367
}
314368

315369
tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, codec, "test", "t")
316-
secIndex := tableDesc.PublicNonPrimaryIndexes()[0]
370+
371+
// Select target index based on corruptionTargetIndex with bounds checking
372+
indexes := tableDesc.PublicNonPrimaryIndexes()
373+
targetIndexPos := tc.corruptionTargetIndex
374+
if targetIndexPos < 0 || targetIndexPos >= len(indexes) {
375+
targetIndexPos = 0 // Default to first index for safety/backward compatibility
376+
}
377+
secIndex := indexes[targetIndexPos]
317378

318379
// Apply test-specific corruption based on configured selectors:
319380
// - If missingIndexEntrySelector is set, we delete the secondary index entries
@@ -438,6 +499,21 @@ func TestDetectIndexConsistencyErrors(t *testing.T) {
438499
}
439500
}
440501
}
502+
503+
// Validate Details if provided in expected issue
504+
if expectedIssue.Details != nil {
505+
require.NotNil(t, foundIssue.Details, "issue should have details when expected")
506+
507+
// Check that all expected detail keys and values match
508+
for expectedKey, expectedValue := range expectedIssue.Details {
509+
require.Contains(t, foundIssue.Details, expectedKey,
510+
"issue should contain detail key: %s", expectedKey)
511+
512+
actualValue := foundIssue.Details[expectedKey]
513+
require.Equal(t, expectedValue, actualValue,
514+
"detail %s should be %v, got %v", expectedKey, expectedValue, actualValue)
515+
}
516+
}
441517
}
442518

443519
// Validate job status matches expected outcome

pkg/sql/inspect/inspect_job.go

Lines changed: 11 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -13,14 +13,14 @@ import (
1313
"github.com/cockroachdb/cockroach/pkg/roachpb"
1414
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
1515
"github.com/cockroachdb/cockroach/pkg/sql"
16+
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
1617
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
1718
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
1819
"github.com/cockroachdb/cockroach/pkg/sql/isql"
1920
"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
2021
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
2122
"github.com/cockroachdb/cockroach/pkg/sql/types"
2223
"github.com/cockroachdb/cockroach/pkg/util/log"
23-
"github.com/cockroachdb/errors"
2424
)
2525

2626
type inspectResumer struct {
@@ -84,10 +84,17 @@ func (c *inspectResumer) getPrimaryIndexSpans(
8484
) ([]roachpb.Span, error) {
8585
details := c.job.Details().(jobspb.InspectDetails)
8686

87-
spans := make([]roachpb.Span, 0, len(details.Checks))
87+
// Deduplicate by table ID to avoid processing the same span multiple times
88+
// when there are multiple checks on the same table.
89+
uniqueTableIDs := make(map[descpb.ID]struct{})
90+
for i := range details.Checks {
91+
uniqueTableIDs[details.Checks[i].TableID] = struct{}{}
92+
}
93+
94+
spans := make([]roachpb.Span, 0, len(uniqueTableIDs))
8895
err := execCfg.InternalDB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
89-
for i := range details.Checks {
90-
desc, err := txn.Descriptors().ByIDWithLeased(txn.KV()).WithoutNonPublic().Get().Table(ctx, details.Checks[i].TableID)
96+
for tableID := range uniqueTableIDs {
97+
desc, err := txn.Descriptors().ByIDWithLeased(txn.KV()).WithoutNonPublic().Get().Table(ctx, tableID)
9198
if err != nil {
9299
return err
93100
}
@@ -104,9 +111,6 @@ func (c *inspectResumer) getPrimaryIndexSpans(
104111
func (c *inspectResumer) planInspectProcessors(
105112
ctx context.Context, jobExecCtx sql.JobExecContext, entirePKSpans []roachpb.Span,
106113
) (*sql.PhysicalPlan, *sql.PlanningCtx, error) {
107-
if len(entirePKSpans) > 1 {
108-
return nil, nil, errors.AssertionFailedf("we only support one check: %d", len(entirePKSpans))
109-
}
110114
distSQLPlanner := jobExecCtx.DistSQLPlanner()
111115
planCtx, _, err := distSQLPlanner.SetupAllNodesPlanning(ctx, jobExecCtx.ExtendedEvalContext(), jobExecCtx.ExecCfg())
112116
if err != nil {

pkg/sql/inspect/inspect_processor.go

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -257,13 +257,15 @@ func buildInspectCheckFactories(
257257
) ([]inspectCheckFactory, error) {
258258
checkFactories := make([]inspectCheckFactory, 0, len(spec.InspectDetails.Checks))
259259
for _, specCheck := range spec.InspectDetails.Checks {
260+
tableID := specCheck.TableID
261+
indexID := specCheck.IndexID
260262
switch specCheck.Type {
261263
case jobspb.InspectCheckIndexConsistency:
262264
checkFactories = append(checkFactories, func() inspectCheck {
263265
return &indexConsistencyCheck{
264266
flowCtx: flowCtx,
265-
tableID: specCheck.TableID,
266-
indexID: specCheck.IndexID,
267+
tableID: tableID,
268+
indexID: indexID,
267269
asOf: spec.InspectDetails.AsOf,
268270
}
269271
})

pkg/sql/scrub.go

Lines changed: 14 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -497,27 +497,29 @@ func TriggerInspectJob(
497497
// Consistency check is done async via a job.
498498
jobID := execCfg.JobRegistry.MakeJobID()
499499

500-
// TODO(148300): just grab the first secondary index and use that for the
501-
// consistency check.
502-
// TODO(148365): When INSPECT is added, we want to skip unsupported indexes
503-
// and return a NOTICE.
504500
secIndexes := tableDesc.PublicNonPrimaryIndexes()
505501
if len(secIndexes) == 0 {
506502
return jobID, errors.AssertionFailedf("must have at least one secondary index")
507503
}
508504

505+
// Create checks for all secondary indexes
506+
// TODO(148365): When INSPECT is added, we want to skip unsupported indexes
507+
// and return a NOTICE.
508+
checks := make([]*jobspb.InspectDetails_Check, 0, len(secIndexes))
509+
for _, index := range secIndexes {
510+
checks = append(checks, &jobspb.InspectDetails_Check{
511+
Type: jobspb.InspectCheckIndexConsistency,
512+
TableID: tableDesc.GetID(),
513+
IndexID: index.GetID(),
514+
})
515+
}
516+
509517
// TODO(sql-queries): add row count check when that is implemented.
510518
jr := jobs.Record{
511519
Description: jobRecordDescription,
512520
Details: jobspb.InspectDetails{
513-
Checks: []*jobspb.InspectDetails_Check{
514-
{
515-
Type: jobspb.InspectCheckIndexConsistency,
516-
TableID: tableDesc.GetID(),
517-
IndexID: secIndexes[0].GetID(),
518-
},
519-
},
520-
AsOf: asOf,
521+
Checks: checks,
522+
AsOf: asOf,
521523
},
522524
Progress: jobspb.InspectProgress{},
523525
CreatedBy: nil,

0 commit comments

Comments
 (0)