Skip to content

Commit 84ea39c

Browse files
committed
sql/inspect: clarify error message for internal failures
Previously, when INSPECT encountered internal errors, it always reported "INSPECT found inconsistencies," even if the issue was caused by something other than actual data corruption. This made it hard to tell whether the problem was in user data or within INSPECT itself (for example, a bad query generated internally). This commit refines that behavior. If all observed errors are internal, INSPECT now reports: "INSPECT encountered internal errors." This makes it clear that the problem might stem from an internal failure in INSPECT or from data corruption detected during internal queries, rather than always implying user data inconsistencies. Additionally, a hint is now included in the error message, regardless of error type, guiding users to run SHOW INSPECT ERRORS to retrieve more information. Informs: #155676 Epic: CRDB-55075 Release note: none
1 parent 5fdc01e commit 84ea39c

File tree

7 files changed

+79
-27
lines changed

7 files changed

+79
-27
lines changed

pkg/sql/inspect/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -120,6 +120,7 @@ go_test(
120120
"@com_github_cockroachdb_errors//:errors",
121121
"@com_github_cockroachdb_redact//:redact",
122122
"@com_github_gogo_protobuf//types",
123+
"@com_github_lib_pq//:pq",
123124
"@com_github_stretchr_testify//require",
124125
],
125126
)

pkg/sql/inspect/index_consistency_check_test.go

Lines changed: 10 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -26,10 +26,14 @@ import (
2626
"github.com/cockroachdb/cockroach/pkg/util/log"
2727
"github.com/cockroachdb/errors"
2828
"github.com/cockroachdb/redact"
29+
"github.com/lib/pq"
2930
"github.com/stretchr/testify/require"
3031
)
3132

32-
const expectedInspectFoundInconsistencies = "INSPECT found inconsistencies"
33+
const (
34+
expectedInspectFoundInconsistencies = "INSPECT found inconsistencies"
35+
expectedInspectInternalErrors = "INSPECT encountered internal errors"
36+
)
3337

3438
// requireCheckCountsMatch verifies that the job's total check count equals its completed check count.
3539
// This is used to verify that progress tracking correctly counted all checks.
@@ -229,7 +233,7 @@ func TestDetectIndexConsistencyErrors(t *testing.T) {
229233
expectedIssues: []inspectIssue{
230234
{ErrorType: "internal_error"},
231235
},
232-
expectedErrRegex: expectedInspectFoundInconsistencies,
236+
expectedErrRegex: expectedInspectInternalErrors,
233237
expectedInternalErrorPatterns: []map[string]string{
234238
{
235239
"error_message": "error decoding.*float64",
@@ -463,6 +467,10 @@ func TestDetectIndexConsistencyErrors(t *testing.T) {
463467

464468
require.Error(t, err)
465469
require.Regexp(t, tc.expectedErrRegex, err.Error())
470+
var pqErr *pq.Error
471+
require.True(t, errors.As(err, &pqErr), "expected pq.Error, got %T", err)
472+
require.NotEmpty(t, pqErr.Hint, "expected error to have a hint")
473+
require.Regexp(t, "SHOW INSPECT ERRORS FOR JOB [0-9]+ WITH DETAILS", pqErr.Hint)
466474

467475
numExpected := len(tc.expectedIssues)
468476
numFound := issueLogger.numIssuesFound()

pkg/sql/inspect/inspect_job.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -105,8 +105,8 @@ func (c *inspectResumer) OnFailOrCancel(
105105
execCfg := jobExecCtx.ExecCfg()
106106
c.maybeCleanupProtectedTimestamp(ctx, execCfg)
107107

108-
// Record RunsWithIssues metric if the job failed due to finding inconsistencies.
109-
if jobErr != nil && errors.Is(jobErr, errInspectFoundInconsistencies) {
108+
// Record RunsWithIssues metric if the job failed due to finding issues (including internal errors).
109+
if errors.Is(jobErr, errInspectFoundInconsistencies) || errors.Is(jobErr, errInspectInternalErrors) {
110110
execCfg.JobRegistry.MetricsStruct().Inspect.(*InspectMetrics).RunsWithIssues.Inc(1)
111111
}
112112
return nil

pkg/sql/inspect/inspect_logger.go

Lines changed: 38 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -23,29 +23,55 @@ type inspectLogger interface {
2323
hasIssues() bool
2424
}
2525

26-
// inspectLoggers manages a collection of inspectLogger instances.
27-
type inspectLoggers []inspectLogger
26+
// inspectLoggerBundle fan-outs logged inspect issues to a slice of loggers while
27+
// tracking aggregate state about the issues encountered.
28+
type inspectLoggerBundle struct {
29+
loggers []inspectLogger
30+
sawInternalIssue atomic.Bool
31+
sawNonInternalIssue atomic.Bool
32+
}
2833

29-
var _ inspectLogger = inspectLoggers{}
34+
var _ inspectLogger = (*inspectLoggerBundle)(nil)
3035

31-
func (l inspectLoggers) logIssue(ctx context.Context, issue *inspectIssue) error {
32-
var retErr error
36+
// newInspectLoggerBundle constructs a bundle that delegates to the provided loggers.
37+
func newInspectLoggerBundle(loggers ...inspectLogger) *inspectLoggerBundle {
38+
return &inspectLoggerBundle{loggers: loggers}
39+
}
40+
41+
// logIssue implements the inspectLogger interface.
42+
func (l *inspectLoggerBundle) logIssue(ctx context.Context, issue *inspectIssue) error {
43+
if issue == nil {
44+
return errors.AssertionFailedf("issue is nil")
45+
}
46+
if issue.ErrorType == InternalError {
47+
l.sawInternalIssue.Store(true)
48+
} else {
49+
l.sawNonInternalIssue.Store(true)
50+
}
3351

34-
for _, logger := range l {
52+
var retErr error
53+
for _, logger := range l.loggers {
3554
if err := logger.logIssue(ctx, issue); err != nil {
3655
retErr = errors.CombineErrors(retErr, err)
3756
}
3857
}
3958
return retErr
4059
}
4160

42-
func (l inspectLoggers) hasIssues() bool {
43-
for _, logger := range l {
44-
if logger.hasIssues() {
45-
return true
46-
}
61+
// hasIssues implements the inspectLogger interface.
62+
func (l *inspectLoggerBundle) hasIssues() bool {
63+
if l == nil {
64+
return false
65+
}
66+
return l.sawInternalIssue.Load() || l.sawNonInternalIssue.Load()
67+
}
68+
69+
// sawOnlyInternalIssues reports whether every recorded issue was an internal error.
70+
func (l *inspectLoggerBundle) sawOnlyInternalIssues() bool {
71+
if l == nil {
72+
return false
4773
}
48-
return false
74+
return l.sawInternalIssue.Load() && !l.sawNonInternalIssue.Load()
4975
}
5076

5177
// metricsLogger increments metrics when issues are logged.

pkg/sql/inspect/inspect_metrics_test.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@ import (
1717
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
1818
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
1919
"github.com/cockroachdb/cockroach/pkg/util/log"
20+
"github.com/cockroachdb/errors"
21+
"github.com/lib/pq"
2022
"github.com/stretchr/testify/require"
2123
)
2224

@@ -75,6 +77,10 @@ func TestInspectMetrics(t *testing.T) {
7577
_, err = db.Exec("INSPECT TABLE db.t")
7678
require.Error(t, err, "INSPECT should fail when corruption is detected")
7779
require.Contains(t, err.Error(), "INSPECT found inconsistencies")
80+
var pqErr *pq.Error
81+
require.True(t, errors.As(err, &pqErr), "expected pq.Error, got %T", err)
82+
require.NotEmpty(t, pqErr.Hint, "expected error to have a hint")
83+
require.Regexp(t, "SHOW INSPECT ERRORS FOR JOB [0-9]+ WITH DETAILS", pqErr.Hint)
7884
require.Equal(t, initialRuns+2, metrics.Runs.Count(),
7985
"Runs counter should increment for each job execution")
8086
require.Equal(t, initialRunsWithIssues+1, metrics.RunsWithIssues.Count(),

pkg/sql/inspect/inspect_processor.go

Lines changed: 20 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,8 @@ var (
3434
// errInspectFoundInconsistencies is a sentinel error used to mark errors
3535
// returned when INSPECT jobs find data inconsistencies.
3636
errInspectFoundInconsistencies = pgerror.New(pgcode.DataException, "INSPECT found inconsistencies")
37+
// errInspectInternalErrors marks jobs that only encountered internal errors while running.
38+
errInspectInternalErrors = pgerror.New(pgcode.Internal, "INSPECT encountered internal errors")
3739

3840
processorConcurrencyOverride = settings.RegisterIntSetting(
3941
settings.ApplicationLevel,
@@ -87,7 +89,7 @@ type inspectProcessor struct {
8789
cfg *execinfra.ServerConfig
8890
checkFactories []inspectCheckFactory
8991
spanSrc spanSource
90-
logger inspectLogger
92+
loggerBundle *inspectLoggerBundle
9193
concurrency int
9294
clock *hlc.Clock
9395
mu struct {
@@ -203,9 +205,17 @@ func (p *inspectProcessor) runInspect(ctx context.Context, output execinfra.RowR
203205
return err
204206
}
205207

206-
log.Dev.Infof(ctx, "INSPECT processor completed processorID=%d issuesFound=%t", p.processorID, p.logger.hasIssues())
207-
if p.logger.hasIssues() {
208-
return errInspectFoundInconsistencies
208+
log.Dev.Infof(ctx, "INSPECT processor completed processorID=%d issuesFound=%t", p.processorID, p.loggerBundle.hasIssues())
209+
if p.loggerBundle.hasIssues() {
210+
errToReturn := errInspectFoundInconsistencies
211+
if p.loggerBundle.sawOnlyInternalIssues() {
212+
errToReturn = errInspectInternalErrors
213+
}
214+
return errors.WithHintf(
215+
errToReturn,
216+
"Run 'SHOW INSPECT ERRORS FOR JOB %d WITH DETAILS' for more information.",
217+
p.spec.JobID,
218+
)
209219
}
210220
return nil
211221
}
@@ -221,12 +231,12 @@ func getProcessorConcurrency(flowCtx *execinfra.FlowCtx) int {
221231
return runtime.GOMAXPROCS(0)
222232
}
223233

224-
// getInspectLogger returns a logger for the inspect processor.
225-
func getInspectLogger(flowCtx *execinfra.FlowCtx, jobID jobspb.JobID) inspectLogger {
234+
// getInspectLogger returns a logger bundle for the inspect processor.
235+
func getInspectLogger(flowCtx *execinfra.FlowCtx, jobID jobspb.JobID) *inspectLoggerBundle {
226236
execCfg := flowCtx.Cfg.ExecutorConfig.(*sql.ExecutorConfig)
227237
metrics := execCfg.JobRegistry.MetricsStruct().Inspect.(*InspectMetrics)
228238

229-
loggers := inspectLoggers{
239+
loggers := []inspectLogger{
230240
&logSink{},
231241
&tableSink{
232242
db: flowCtx.Cfg.DB,
@@ -243,7 +253,7 @@ func getInspectLogger(flowCtx *execinfra.FlowCtx, jobID jobspb.JobID) inspectLog
243253
loggers = append(loggers, knobs.InspectIssueLogger.(inspectLogger))
244254
}
245255

246-
return loggers
256+
return newInspectLoggerBundle(loggers...)
247257
}
248258

249259
// processSpan executes all configured inspect checks against a single span.
@@ -270,7 +280,7 @@ func (p *inspectProcessor) processSpan(
270280

271281
runner := inspectRunner{
272282
checks: checks,
273-
logger: p.logger,
283+
logger: p.loggerBundle,
274284
}
275285
defer func() {
276286
if closeErr := runner.Close(ctx); closeErr != nil {
@@ -412,7 +422,7 @@ func newInspectProcessor(
412422
checkFactories: checkFactories,
413423
cfg: flowCtx.Cfg,
414424
spanSrc: newSliceSpanSource(spec.Spans),
415-
logger: getInspectLogger(flowCtx, spec.JobID),
425+
loggerBundle: getInspectLogger(flowCtx, spec.JobID),
416426
concurrency: getProcessorConcurrency(flowCtx),
417427
clock: flowCtx.Cfg.DB.KV().Clock(),
418428
}, nil

pkg/sql/inspect/inspect_processor_test.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -257,6 +257,7 @@ func makeProcessor(
257257
t.Helper()
258258
clock := hlc.NewClockForTesting(nil)
259259
logger := &testIssueCollector{}
260+
loggerBundle := newInspectLoggerBundle(logger)
260261

261262
// Mock a FlowCtx for test purposes.
262263
var c base.NodeIDContainer
@@ -277,7 +278,7 @@ func makeProcessor(
277278
cfg: flowCtx.Cfg,
278279
flowCtx: flowCtx,
279280
spanSrc: src,
280-
logger: logger,
281+
loggerBundle: loggerBundle,
281282
concurrency: concurrency,
282283
clock: clock,
283284
}

0 commit comments

Comments
 (0)