Skip to content

Commit 2738f04

Browse files
authored
Merge pull request #154137 from spilchen/blathers/backport-release-25.4-153965
release-25.4: sql/inspect: support checking multiple indexes in one INSPECT job
2 parents 7c19a98 + b253ab2 commit 2738f04

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)