Skip to content

[improve](streaming-job) Cap auto-resume attempts and expose structured FailureReason#62345

Merged
JNSimba merged 1 commit intoapache:masterfrom
JNSimba:fix_retry
Apr 24, 2026
Merged

[improve](streaming-job) Cap auto-resume attempts and expose structured FailureReason#62345
JNSimba merged 1 commit intoapache:masterfrom
JNSimba:fix_retry

Conversation

@JNSimba
Copy link
Copy Markdown
Member

@JNSimba JNSimba commented Apr 10, 2026

What problem does this PR solve?

StreamingInsertJob has two gaps:

  1. No retry budget. autoResumeHandler retries forever — a permanently broken job keeps hammering at the 5-min backoff cap indefinitely.
  2. Unstructured ErrorMsg. The jobs() TVF only returns FailureReason.msg, so the frontend cannot tell manual pause from auto-recovery from non-recoverable failure without grepping the error message.

Changes

  • New config streaming_job_max_auto_resume_count (default 10). Once exhausted, autoResumeHandler rewrites the failure reason to CANNOT_RESUME_ERR and stops retrying — no new JobStatus value needed.
  • ErrorMsg column now returns a FailureReason JSON ({"code":"...","msg":"..."}). Existing .contains(...) test assertions still work because the message is embedded.
  • New LastTaskSuccessTime TVF column for observability.
  • Fix latent backoff bug: handlePendingState used to reset autoResumeCount on every PENDING → RUNNING, so the backoff interval was stuck at the base value. Reset is now inside resetFailureInfo(null) (fires on task success and RESUME JOB), so exponential backoff and the retry budget accumulate correctly.

Release note

Add streaming_job_max_auto_resume_count config (default 10) to cap auto-resume attempts for streaming insert jobs. The ErrorMsg column in jobs("type"="insert") now returns FailureReason as JSON. New LastTaskSuccessTime column shows the time of the last successful task commit.

Check List (For Author)

  • Test

    • Regression test (test_streaming_job_max_retry; test_streaming_insert_job extended with LastTaskSuccessTime assertion)
    • Unit Test
    • Manual test
    • No need to test
  • Behavior changed:

    • Yes.
      • ErrorMsg column: plain message → FailureReason JSON (.contains(...) unaffected).
      • New LastTaskSuccessTime column appended to InsertJob schema.
      • Persistently failing streaming jobs now hit the circuit breaker at streaming_job_max_auto_resume_count attempts.
  • Does this need documentation?

    • Yes.

Check List (For Reviewer who merge this PR)

  • Confirm the release note
  • Confirm test cases
  • Confirm document
  • Add branch pick label

@Thearas
Copy link
Copy Markdown
Contributor

Thearas commented Apr 10, 2026

Thank you for your contribution to Apache Doris.
Don't know what should be done next? See How to process your PR.

Please clearly describe your PR:

  1. What problem was fixed (it's best to include specific error reporting information). How it was fixed.
  2. Which behaviors were modified. What was the previous behavior, what is it now, why was it modified, and what possible impacts might there be.
  3. What features were added. Why was this function added?
  4. Which code was refactored and why was this part of the code refactored?
  5. Which functions were optimized and what is the difference before and after the optimization?

@JNSimba
Copy link
Copy Markdown
Member Author

JNSimba commented Apr 10, 2026

/review

@JNSimba
Copy link
Copy Markdown
Member Author

JNSimba commented Apr 10, 2026

run buildall

Copy link
Copy Markdown
Contributor

@github-actions github-actions Bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Found 2 issues worth fixing before merging.

  1. StreamingInsertJob.resetFailureInfo() does not reset latestAutoResumeTimestamp, so the next failure after a manual resume or a successful run can reuse the previous retry window and auto-resume earlier than the base backoff.
  2. lastTaskSuccessTime is written and exposed, but streaming replay only applies a partial update in replayOnUpdated(). The new field is not copied there, so jobs() loses LastTaskSuccessTime after FE replay/restart.

Critical checkpoints

  • Goal of task: Partially met. The retry cap/backoff accumulation and TVF additions are implemented, but the two issues above leave the fresh-backoff behavior and persisted TVF state incomplete.
  • Minimality/focus: Yes. The patch stays focused on streaming job retry policy and job TVF output.
  • Concurrency: Reviewed. The changed paths run under the existing job lock/scheduler flow; I did not find a new lock-order or deadlock issue. The fresh-backoff bug is logical rather than synchronization-related.
  • Lifecycle/static init: No special lifecycle or static-init issue introduced.
  • Config changes: streaming_job_max_auto_resume_count is master-only and mutable, and the scheduler reads it directly so runtime changes take effect without restart.
  • Compatibility/incompatible change: Applicable. jobs("type"="insert") gains a new column; no FE/BE protocol issue seen. User-visible TVF behavior changed, so replay correctness for the new field matters.
  • Parallel paths: The new LastTaskSuccessTime column was added for both insert and streaming job TVF paths.
  • Special conditionals: The new auto-resume guards are understandable, but the reset path is incomplete because it leaves the old retry timestamp behind.
  • Test coverage: Added regression tests cover happy-path LastTaskSuccessTime and max-retry exhaustion, but they miss FE restart/edit-log replay and the fresh-backoff-after-resume/success timing case.
  • Observability: Adequate for this scope.
  • Transaction/persistence: Applicable. The new serialized field is not fully replayed on the streaming partial-update path.
  • Data writes/modifications: Not applicable beyond job metadata persistence.
  • FE/BE variable passing: Not applicable.
  • Performance: No material new performance issue found.
  • Other issues: None beyond the two findings above.

Overall opinion: not ready yet; the retry reset and replay path should be fixed first.

Copy link
Copy Markdown
Contributor

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

This PR improves operational safety and observability for streaming insert jobs by (1) capping auto-resume retries to avoid infinite retry loops, and (2) exposing richer, structured failure/progress information via the jobs("type"="insert") TVF.

Changes:

  • Add a new FE config streaming_job_max_auto_resume_count to cap auto-resume attempts; once exhausted, the job rewrites its failure reason to CANNOT_RESUME_ERR to stop further retries.
  • Change streaming insert job TVF ErrorMsg to return a JSON-serialized FailureReason (code + msg), and add a new TVF column LastTaskSuccessTime.
  • Fix retry/backoff accounting by no longer resetting autoResumeCount on PENDING -> RUNNING, and add regression coverage for the retry cap + new TVF column.

Reviewed changes

Copilot reviewed 7 out of 7 changed files in this pull request and generated 2 comments.

Show a summary per file
File Description
regression-test/suites/job_p0/streaming_job/test_streaming_job_max_retry.groovy New regression to verify retry budget exhaustion rewrites failure reason to CANNOT_RESUME_ERR.
regression-test/suites/job_p0/streaming_job/test_streaming_insert_job.groovy Adds assertion that LastTaskSuccessTime is populated after successful commits.
fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java Enforces max auto-resume count and stops retrying by rewriting failure reason.
fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java Resets retry budget only on true “success/resume”; emits FailureReason JSON and LastTaskSuccessTime in TVF.
fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/InsertJob.java Extends insert-job TVF schema/row with LastTaskSuccessTime.
fe/fe-core/src/main/java/org/apache/doris/job/base/AbstractJob.java Tracks lastTaskSuccessTime on task success for jobs that use onTaskSuccess.
fe/fe-common/src/main/java/org/apache/doris/common/Config.java Adds streaming_job_max_auto_resume_count config definition.

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

@JNSimba JNSimba force-pushed the fix_retry branch 2 times, most recently from e23f742 to d7bc886 Compare April 10, 2026 10:53
@JNSimba
Copy link
Copy Markdown
Member Author

JNSimba commented Apr 10, 2026

run buildall

…ed FailureReason

Add a max auto-resume retry budget and make the jobs() TVF expose the full
FailureReason as JSON so the frontend can discriminate failure codes without
grepping the message string. Also fix a latent exponential backoff bug so
the new budget actually accumulates across failed retries.

- New master-only mutable config streaming_job_max_auto_resume_count (default
  10). Once exhausted, autoResumeHandler rewrites the failure reason to
  CANNOT_RESUME_ERR and stops retrying, without introducing a new JobStatus.
- ErrorMsg TVF column now returns GsonUtils.GSON.toJson(failureReason) so
  frontends can switch on the structured code instead of pattern-matching msg.
- New LastTaskSuccessTime TVF column shows the time of the last successful
  task commit, so operators can see whether a job is making real progress.
  Also copied in replayOnUpdated() so it survives FE restart.
- Fix latent backoff bug: handlePendingState used to reset autoResumeCount on
  every PENDING->RUNNING transition, so the exponential backoff interval was
  stuck at the base value across repeated failures. The counter (and the
  paired latestAutoResumeTimestamp) is now reset only inside
  resetFailureInfo(null), which fires on a successful task or on RESUME JOB,
  so exponential backoff and the max retry budget actually accumulate across
  failed retries.
- autoResumeCount and latestAutoResumeTimestamp are intentionally not
  serialized: an FE restart resets the retry budget, which gives paused jobs
  a fresh retry chance rather than freezing them at the burn-out boundary.

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
@JNSimba
Copy link
Copy Markdown
Member Author

JNSimba commented Apr 24, 2026

run buildall

@github-actions github-actions Bot added the approved Indicates a PR has been approved by one committer. label Apr 24, 2026
@github-actions
Copy link
Copy Markdown
Contributor

PR approved by at least one committer and no changes requested.

@github-actions
Copy link
Copy Markdown
Contributor

PR approved by anyone and no changes requested.

@JNSimba
Copy link
Copy Markdown
Member Author

JNSimba commented Apr 24, 2026

run cloud_p0

@hello-stephen
Copy link
Copy Markdown
Contributor

FE Regression Coverage Report

Increment line coverage 88.89% (32/36) 🎉
Increment coverage report
Complete coverage report

@JNSimba JNSimba merged commit ff563c2 into apache:master Apr 24, 2026
30 of 31 checks passed
github-actions Bot pushed a commit that referenced this pull request Apr 27, 2026
…ed FailureReason (#62345)

### What problem does this PR solve?

`StreamingInsertJob` has two gaps:

1. **No retry budget.** `autoResumeHandler` retries forever — a
permanently broken job keeps hammering at the 5-min backoff cap
indefinitely.
2. **Unstructured `ErrorMsg`.** The `jobs()` TVF only returns
`FailureReason.msg`, so the frontend cannot tell manual pause from
auto-recovery from non-recoverable failure without grepping the error
message.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

approved Indicates a PR has been approved by one committer. dev/4.1.x reviewed

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants