Skip to content

Commit b7d1a56

Browse files
craig[bot]spilchen
andcommitted
Merge #158396
158396: sql: stop retrying temp schema cleanup on poisoned transactions r=spilchen a=spilchen Previously, the temporary schema cleanup job would retry up to five times on any error, including those that irreversibly poisoned the transaction. This caused unnecessary log noise (e.g., "txn already encountered an error; cannot be used anymore"). This change improves the retry logic by detecting for TxnAlreadyEncounteredErrorError and aborting immediately. Fixes #158152 Epic: none Release note (bug fix): Temporary schema cleanup no longer retries after poisoned transaction errors, reducing log noise. Co-authored-by: Matt Spilchen <matt.spilchen@cockroachlabs.com>
2 parents 5250b2f + 8eade9c commit b7d1a56

File tree

3 files changed

+126
-16
lines changed

3 files changed

+126
-16
lines changed

pkg/sql/exec_util.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2033,6 +2033,9 @@ type ExecutorTestingKnobs struct {
20332033
// OnTempObjectsCleanupDone will trigger when the temporary objects cleanup
20342034
// job is done.
20352035
OnTempObjectsCleanupDone func()
2036+
// TempObjectCleanupErrorInjection, if set, will be called during temp object
2037+
// cleanup and can return an error to inject into the cleanup process.
2038+
TempObjectCleanupErrorInjection func() error
20362039

20372040
// WithStatementTrace is called after the statement is executed in
20382041
// execStmtInOpenState.

pkg/sql/temporary_schema.go

Lines changed: 39 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@ import (
1313
"time"
1414

1515
"github.com/cockroachdb/cockroach/pkg/keys"
16+
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
1617
"github.com/cockroachdb/cockroach/pkg/security/username"
1718
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
1819
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
@@ -447,30 +448,43 @@ func makeTemporaryObjectCleanerMetrics() *temporaryObjectCleanerMetrics {
447448
}
448449
}
449450

451+
// shouldStopTempObjectCleanupRetry returns true if the error indicates that we
452+
// should not retry the operation.
453+
func shouldStopTempObjectCleanupRetry(err error) bool {
454+
if err == nil {
455+
return false
456+
}
457+
458+
// Check if transaction is already poisoned. Once this happens, the
459+
// transaction cannot be reused regardless of retry attempts.
460+
return errors.HasType(err, (*kvpb.TxnAlreadyEncounteredErrorError)(nil))
461+
}
462+
450463
// doTemporaryObjectCleanup performs the actual cleanup.
451464
func (c *TemporaryObjectCleaner) doTemporaryObjectCleanup(
452465
ctx context.Context, closerCh <-chan struct{},
453466
) error {
454467
defer log.Dev.Infof(ctx, "completed temporary object cleanup job")
455468
// Wrap the retry functionality with the default arguments.
456469
retryFunc := func(ctx context.Context, do func() error) error {
457-
return retry.WithMaxAttempts(
458-
ctx,
459-
retry.Options{
460-
InitialBackoff: 1 * time.Second,
461-
MaxBackoff: 1 * time.Minute,
462-
Multiplier: 2,
463-
Closer: closerCh,
464-
},
465-
5, // maxAttempts
466-
func() error {
467-
err := do()
468-
if err != nil {
469-
log.Dev.Warningf(ctx, "error during schema cleanup, retrying: %v", err)
470+
return retry.Options{
471+
InitialBackoff: 1 * time.Second,
472+
MaxBackoff: 1 * time.Minute,
473+
Multiplier: 2,
474+
MaxRetries: 4, // 5 total attempts (4 retries + 1 initial)
475+
Closer: closerCh,
476+
}.DoWithRetryable(ctx, func(ctx context.Context) (bool, error) {
477+
err := do()
478+
if err != nil {
479+
if shouldStopTempObjectCleanupRetry(err) {
480+
log.Dev.Warningf(ctx, "error during schema cleanup, not retryable: %v", err)
481+
return false, err // Don't retry
470482
}
471-
return err
472-
},
473-
)
483+
log.Dev.Warningf(ctx, "error during schema cleanup, retrying: %v", err)
484+
return true, err // Retry
485+
}
486+
return false, nil // Success
487+
})
474488
}
475489

476490
// For tenants, we will completely skip this logic since listing
@@ -507,6 +521,15 @@ func (c *TemporaryObjectCleaner) doTemporaryObjectCleanup(
507521
log.Dev.Infof(ctx, "running temporary object cleanup background job")
508522
var sessionIDs map[clusterunique.ID]struct{}
509523
if err := c.db.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
524+
// Testing knob to inject errors during cleanup.
525+
if c.testingKnobs.TempObjectCleanupErrorInjection != nil {
526+
if err := retryFunc(ctx, func() (err error) {
527+
return c.testingKnobs.TempObjectCleanupErrorInjection()
528+
}); err != nil {
529+
return err
530+
}
531+
}
532+
510533
sessionIDs = make(map[clusterunique.ID]struct{})
511534
// Only see temporary schemas after some delay as safety
512535
// mechanism.

pkg/sql/temporary_schema_test.go

Lines changed: 84 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@ import (
1414
"time"
1515

1616
"github.com/cockroachdb/cockroach/pkg/base"
17+
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
1718
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
1819
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
1920
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
@@ -22,8 +23,10 @@ import (
2223
"github.com/cockroachdb/cockroach/pkg/testutils"
2324
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
2425
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
26+
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2527
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2628
"github.com/cockroachdb/cockroach/pkg/util/log"
29+
"github.com/cockroachdb/cockroach/pkg/util/metric"
2730
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
2831
"github.com/cockroachdb/errors"
2932
"github.com/stretchr/testify/assert"
@@ -317,3 +320,84 @@ func constructNameToIDMapping(
317320
}
318321
return namesToID, tempSchemaNames
319322
}
323+
324+
// TestTemporaryObjectCleanupRetriesWithPoisonedTransaction tests that the
325+
// cleanup process doesn't retry when encountering a poisoned transaction error.
326+
func TestTemporaryObjectCleanupRetriesWithPoisonedTransaction(t *testing.T) {
327+
defer leaktest.AfterTest(t)()
328+
defer log.Scope(t).Close(t)
329+
330+
ctx := context.Background()
331+
332+
testCases := []struct {
333+
name string
334+
errorInjectionFunc func(attemptCount *int) error
335+
expectedAttempts int
336+
expectError bool
337+
errorSubstring string
338+
}{
339+
{
340+
name: "poisoned transaction stops retry",
341+
errorInjectionFunc: func(attemptCount *int) error {
342+
*attemptCount++
343+
return &kvpb.TxnAlreadyEncounteredErrorError{}
344+
},
345+
expectedAttempts: 1,
346+
expectError: true,
347+
errorSubstring: "txn already encountered an error",
348+
},
349+
{
350+
name: "regular error retries and succeeds",
351+
errorInjectionFunc: func(attemptCount *int) error {
352+
*attemptCount++
353+
// Fail on first attempt, succeed on second
354+
if *attemptCount == 1 {
355+
return errors.New("some retryable error")
356+
}
357+
return nil
358+
},
359+
expectedAttempts: 2,
360+
expectError: false,
361+
},
362+
}
363+
364+
for _, tc := range testCases {
365+
t.Run(tc.name, func(t *testing.T) {
366+
var attemptCount int
367+
368+
testingKnobs := ExecutorTestingKnobs{
369+
TempObjectCleanupErrorInjection: func() error {
370+
return tc.errorInjectionFunc(&attemptCount)
371+
},
372+
}
373+
374+
s := serverutils.StartServerOnly(t, base.TestServerArgs{})
375+
defer s.Stopper().Stop(ctx)
376+
377+
execCfg := s.ExecutorConfig().(ExecutorConfig)
378+
379+
// Create a temporary object cleaner with our testing knobs
380+
cleaner := NewTemporaryObjectCleaner(
381+
execCfg.Settings,
382+
execCfg.InternalDB,
383+
execCfg.Codec,
384+
metric.NewRegistry(),
385+
execCfg.SQLStatusServer,
386+
func(context.Context, hlc.ClockTimestamp) (bool, error) { return true, nil },
387+
testingKnobs,
388+
nil,
389+
)
390+
391+
// Run cleanup and verify behavior
392+
err := cleaner.doTemporaryObjectCleanup(ctx, nil)
393+
if tc.expectError {
394+
require.Error(t, err)
395+
require.ErrorContains(t, err, tc.errorSubstring)
396+
} else {
397+
require.NoError(t, err)
398+
}
399+
require.Equal(t, tc.expectedAttempts, attemptCount,
400+
"expected %d attempts, got %d", tc.expectedAttempts, attemptCount)
401+
})
402+
}
403+
}

0 commit comments

Comments
 (0)