Skip to content

fix(): Fix query failure logging race#27479

Merged
spershin merged 1 commit intoprestodb:masterfrom
spershin:export-D98954601
Apr 1, 2026
Merged

fix(): Fix query failure logging race#27479
spershin merged 1 commit intoprestodb:masterfrom
spershin:export-D98954601

Conversation

@spershin
Copy link
Copy Markdown
Contributor

@spershin spershin commented Mar 31, 2026

Summary:
When a query failed, there is a race between dispatching and executing code to log the query failure event. That can result in double-logging in the systems that allow it or logging without any stats/metrics in the systems which allow logging only once. We observe both behaviors in our logging systems.
The fix is to set a flag that the query went for execution and not log from the dispatching side in this case.

Differential Revision: D98954601

== NO RELEASE NOTE ==

Summary by Sourcery

Prevent duplicate zero-stats query failure events for dispatched queries by tracking execution submission and guarding immediate-failure logging.

Bug Fixes:

  • Avoid emitting queryImmediateFailureEvent after a query has been submitted for execution, preventing duplicate or zeroed-stat completion events on failures and cancellations.

Tests:

  • Add unit coverage to ensure fail() and cancel() do not emit immediate failure events after dispatch, while pre-dispatch failures still emit them.

Summary:
When a query failed, there is a race between dispatching and executing code to log the query failure event. That can result in double-logging in the systems that allow it or logging without any stats/metrics in the systems which allow logging only once. We observe both behaviors in our logging systems.
The fix is to set a flag that the query went for execution and not log from the dispatching side in this case.

Differential Revision: D98954601
@spershin spershin requested a review from a team as a code owner March 31, 2026 19:03
@prestodb-ci prestodb-ci added the from:Meta PR from Meta label Mar 31, 2026
@linux-foundation-easycla
Copy link
Copy Markdown

CLA Missing ID CLA Not Signed

@sourcery-ai
Copy link
Copy Markdown
Contributor

sourcery-ai bot commented Mar 31, 2026

Reviewer's Guide

Adds a guard flag in LocalDispatchQuery to distinguish pre-dispatch from post-dispatch failures/cancellations so that immediate failure events with zeroed stats are only logged if the query was never submitted for execution, and introduces unit tests to cover the race conditions and expected logging behavior before and after dispatch.

Sequence diagram for query failure before dispatch (immediate failure event emitted)

sequenceDiagram
    actor Client
    participant LocalDispatchQuery
    participant QueryStateMachine
    participant QueryMonitor

    Client->>LocalDispatchQuery: fail(throwable)
    LocalDispatchQuery->>QueryStateMachine: transitionToFailed(throwable)
    QueryStateMachine-->>LocalDispatchQuery: true
    Note over LocalDispatchQuery,QueryStateMachine: sentForExecution is false (query not submitted)
    LocalDispatchQuery->>QueryStateMachine: getBasicQueryInfo(Optional.empty)
    QueryStateMachine-->>LocalDispatchQuery: BasicQueryInfo
    LocalDispatchQuery->>LocalDispatchQuery: toFailure(throwable)
    LocalDispatchQuery->>QueryMonitor: queryImmediateFailureEvent(BasicQueryInfo, ExecutionFailureInfo)
    QueryMonitor-->>QueryMonitor: Log failure with zeroed stats
Loading

Sequence diagram for query failure after dispatch (completion event only)

sequenceDiagram
    actor Client
    participant LocalDispatchQuery
    participant QueryStateMachine
    participant SqlQueryManager
    participant QueryMonitor

    Client->>LocalDispatchQuery: startExecution(queryExecution, isDispatching=true)
    LocalDispatchQuery->>QueryExecution: setResourceGroupQueryLimits(...)
    LocalDispatchQuery->>SqlQueryManager: querySubmitter.accept(queryExecution)
    SqlQueryManager-->>SqlQueryManager: Register query for execution
    LocalDispatchQuery->>LocalDispatchQuery: sentForExecution = true

    Client->>LocalDispatchQuery: fail(throwable)
    LocalDispatchQuery->>QueryStateMachine: transitionToFailed(throwable)
    QueryStateMachine-->>LocalDispatchQuery: true
    Note over LocalDispatchQuery: sentForExecution is true
    LocalDispatchQuery-->>QueryMonitor: (no call to queryImmediateFailureEvent)

    SqlQueryManager-->>QueryMonitor: queryCompletedEvent(QueryInfoWithStats)
    QueryMonitor-->>QueryMonitor: Log failure with real execution stats
Loading

Class diagram for LocalDispatchQuery logging guard changes

classDiagram
    class LocalDispatchQuery {
        // Fields
        - SettableFuture submitted
        - AtomicReference resourceGroupQueryLimits
        - boolean retry
        - QueryPrerequisites queryPrerequisites
        - QueryMonitor queryMonitor
        - Consumer querySubmitter
        - volatile boolean sentForExecution

        // Methods (subset related to this change)
        - void startExecution(QueryExecution queryExecution, boolean isDispatching)
        - void fail(Throwable throwable)
        - void cancel()
    }

    class QueryExecution {
        + void setResourceGroupQueryLimits(ResourceGroupQueryLimits limits)
    }

    class QueryMonitor {
        + void queryImmediateFailureEvent(BasicQueryInfo queryInfo, ExecutionFailureInfo failureInfo)
        + void queryCompletedEvent(QueryInfo queryInfo)
    }

    class QueryStateMachine {
        + boolean transitionToFailed(Throwable throwable)
        + boolean transitionToCanceled()
        + BasicQueryInfo getBasicQueryInfo(Optional optionalToken)
    }

    class SqlQueryManager {
        + void accept(QueryExecution queryExecution)
        + void finalQueryInfoListener(QueryInfo queryInfo)
    }

    class BasicQueryInfo {
        + ExecutionFailureInfo getFailureInfo()
    }

    class ExecutionFailureInfo
    class QueryInfo
    class ResourceGroupQueryLimits
    class QueryPrerequisites
    class Optional
    class Throwable

    LocalDispatchQuery --> QueryExecution : uses
    LocalDispatchQuery --> QueryMonitor : logs_via
    LocalDispatchQuery --> QueryStateMachine : manages_state_via
    LocalDispatchQuery --> SqlQueryManager : submits_via_querySubmitter
    LocalDispatchQuery --> QueryPrerequisites : uses
    LocalDispatchQuery --> Optional : uses
    QueryMonitor --> BasicQueryInfo : parameter
    QueryMonitor --> ExecutionFailureInfo : parameter
    SqlQueryManager --> QueryMonitor : notifies_via_finalQueryInfoListener
    BasicQueryInfo --> ExecutionFailureInfo : contains
    QueryStateMachine --> BasicQueryInfo : returns
    QueryStateMachine --> Throwable : parameter
Loading

File-Level Changes

Change Details Files
Prevent LocalDispatchQuery from emitting zeroed-stat immediate failure events once a query has been successfully submitted for execution.
  • Introduce a volatile sentForExecution flag on LocalDispatchQuery to track whether the query has been submitted to SqlQueryManager.
  • Set sentForExecution to true only after querySubmitter.accept(queryExecution) completes successfully in startExecution.
  • Gate queryMonitor.queryImmediateFailureEvent calls in fail() on sentForExecution being false, so post-dispatch failures no longer emit immediate failure events.
  • Gate queryMonitor.queryImmediateFailureEvent calls in cancel() on sentForExecution being false, so post-dispatch cancellations no longer emit immediate failure events.
presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQuery.java
Add tests validating immediate failure event emission behavior before and after dispatch.
  • Add a test that fail() after dispatch does not emit queryImmediateFailureEvent and relies on SqlQueryManager to emit the final completion event.
  • Add a test that cancel() after dispatch does not emit queryImmediateFailureEvent.
  • Add a test that fail() before dispatch still emits queryImmediateFailureEvent to ensure the pre-dispatch path remains functional.
presto-main/src/test/java/com/facebook/presto/dispatcher/TestLocalDispatchQuery.java

Tips and commands

Interacting with Sourcery

  • Trigger a new review: Comment @sourcery-ai review on the pull request.
  • Continue discussions: Reply directly to Sourcery's review comments.
  • Generate a GitHub issue from a review comment: Ask Sourcery to create an
    issue from a review comment by replying to it. You can also reply to a
    review comment with @sourcery-ai issue to create an issue from it.
  • Generate a pull request title: Write @sourcery-ai anywhere in the pull
    request title to generate a title at any time. You can also comment
    @sourcery-ai title on the pull request to (re-)generate the title at any time.
  • Generate a pull request summary: Write @sourcery-ai summary anywhere in
    the pull request body to generate a PR summary at any time exactly where you
    want it. You can also comment @sourcery-ai summary on the pull request to
    (re-)generate the summary at any time.
  • Generate reviewer's guide: Comment @sourcery-ai guide on the pull
    request to (re-)generate the reviewer's guide at any time.
  • Resolve all Sourcery comments: Comment @sourcery-ai resolve on the
    pull request to resolve all Sourcery comments. Useful if you've already
    addressed all the comments and don't want to see them anymore.
  • Dismiss all Sourcery reviews: Comment @sourcery-ai dismiss on the pull
    request to dismiss all existing Sourcery reviews. Especially useful if you
    want to start fresh with a new review - don't forget to comment
    @sourcery-ai review to trigger a new review!

Customizing Your Experience

Access your dashboard to:

  • Enable or disable review features such as the Sourcery-generated pull request
    summary, the reviewer's guide, and others.
  • Change the review language.
  • Add, remove or edit custom review instructions.
  • Adjust other review settings.

Getting Help

Copy link
Copy Markdown
Contributor

@sourcery-ai sourcery-ai bot left a comment

Choose a reason for hiding this comment

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

Hey - I've found 1 issue, and left some high level feedback:

  • There is still a small race window between querySubmitter.accept(queryExecution) completing and sentForExecution being set to true, during which fail()/cancel() can still emit a duplicate immediate-failure event; consider setting sentForExecution before calling the submitter and clearing it on failure, or otherwise tying this guard to a state already tracked by the QueryStateMachine.
  • Since sentForExecution is guarding correctness against races, you might want to make the lifecycle more explicit (e.g., using an AtomicBoolean with compare-and-set or documenting all threads that can write it) to avoid future changes accidentally introducing additional racy writes.
Prompt for AI Agents
Please address the comments from this code review:

## Overall Comments
- There is still a small race window between `querySubmitter.accept(queryExecution)` completing and `sentForExecution` being set to `true`, during which `fail()`/`cancel()` can still emit a duplicate immediate-failure event; consider setting `sentForExecution` before calling the submitter and clearing it on failure, or otherwise tying this guard to a state already tracked by the `QueryStateMachine`.
- Since `sentForExecution` is guarding correctness against races, you might want to make the lifecycle more explicit (e.g., using an `AtomicBoolean` with compare-and-set or documenting all threads that can write it) to avoid future changes accidentally introducing additional racy writes.

## Individual Comments

### Comment 1
<location path="presto-main/src/main/java/com/facebook/presto/dispatcher/LocalDispatchQuery.java" line_range="218-221" />
<code_context>
                 try {
                     resourceGroupQueryLimits.get().ifPresent(queryExecution::setResourceGroupQueryLimits);
                     querySubmitter.accept(queryExecution);
+                    // Mark only after successful submission. If querySubmitter throws,
+                    // SqlQueryManager won't have the query, so we still need
+                    // queryImmediateFailureEvent from fail() below.
+                    sentForExecution = true;
                 }
                 catch (Throwable t) {
</code_context>
<issue_to_address>
**issue (bug_risk):** There is a race where `fail()`/`cancel()` can see `sentForExecution == false` after a successful `querySubmitter.accept`, causing duplicate events.

Because `sentForExecution` is set *after* `querySubmitter.accept`, `fail()`/`cancel()` can run in between and still see `sentForExecution == false` even though the query was successfully submitted. They will then emit `queryImmediateFailureEvent`, and later `SqlQueryManager` will emit its normal completion event, causing duplicates.

To avoid this race, set the flag before submission and clear it only if submission fails, e.g.:

```java
sentForExecution = true;
try {
    querySubmitter.accept(queryExecution);
}
catch (Throwable t) {
    sentForExecution = false; // submission didn’t complete successfully
    // existing catch handling
}
```
</issue_to_address>

Sourcery is free for open source - if you like our reviews please consider sharing them ✨
Help me be more useful! Please click 👍 or 👎 on each comment and I'll use the feedback to improve your reviews.

@steveburnett
Copy link
Copy Markdown
Contributor

  • Please sign the Presto CLA with the email address used for the commit.

  • Please edit the PR title to follow semantic commit style to pass the failing and required CI check. See the failure in the test for advice.

@spershin spershin changed the title Fix(): Fix query failure logging race fix(): Fix query failure logging race Mar 31, 2026
@spershin spershin merged commit dc6a369 into prestodb:master Apr 1, 2026
84 of 90 checks passed
bibith4 pushed a commit to bibith4/presto that referenced this pull request Apr 1, 2026
Summary:
When a query failed, there is a race between dispatching and executing
code to log the query failure event. That can result in double-logging
in the systems that allow it or logging without any stats/metrics in the
systems which allow logging only once. We observe both behaviors in our
logging systems.
The fix is to set a flag that the query went for execution and not log
from the dispatching side in this case.

Differential Revision: D98954601


```
== NO RELEASE NOTE ==
```

## Summary by Sourcery

Prevent duplicate zero-stats query failure events for dispatched queries
by tracking execution submission and guarding immediate-failure logging.

Bug Fixes:
- Avoid emitting queryImmediateFailureEvent after a query has been
submitted for execution, preventing duplicate or zeroed-stat completion
events on failures and cancellations.

Tests:
- Add unit coverage to ensure fail() and cancel() do not emit immediate
failure events after dispatch, while pre-dispatch failures still emit
them.

Co-authored-by: Sergey Pershin <spershin@meta.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants