Skip to content

Fix broker write failure handling in ServerChannels#17861

Merged
xiangfu0 merged 1 commit intoapache:masterfrom
suvodeep-pyne:spyne/broker-write-failure-handling
Mar 11, 2026
Merged

Fix broker write failure handling in ServerChannels#17861
xiangfu0 merged 1 commit intoapache:masterfrom
suvodeep-pyne:spyne/broker-write-failure-handling

Conversation

@suvodeep-pyne
Copy link
Contributor

Summary

  • Fix unguarded writeAndFlush() in ServerChannels.sendRequestWithoutLocking(): The write listener never checked f.isSuccess(), so on write failure markRequestSent() was called anyway (false positive), no channel close occurred (zombie channel), no logging/metrics, and queries waited for full timeout instead of failing fast.
  • Add f.isSuccess() check: On failure, logs the error, increments NETTY_CONNECTION_SEND_REQUEST_FAILURES metric, calls markServerDown() for fast query failure, and closes the channel to prevent zombies. On success, existing behavior is preserved.
  • Add NETTY_CONNECTION_SEND_REQUEST_FAILURES broker metric for observability of write failures.
  • This is the broker-side counterpart to the server-side fix in PR Fix server outbound write failure creating zombie channels #17845 / Fix QueryServer _allChannels cleanup on channel close #17854.

Design Decisions

  • markServerDown() over markQueryFailed(): Race-safe — if server already responded, it's a no-op. The _channel.close() also triggers channelInactive()markServerDown() for all in-flight queries. Double invocation is safe (idempotent).
  • Wrap f.cause() in RuntimeException: markServerDown expects Exception but f.cause() returns Throwable (could be OutOfMemoryError).
  • REQUESTS_SENT/BYTES_SENT kept outside listener: They're incremented synchronously before write completes. Moving them inside the success path would be a separate behavioral change.

Test plan

  • Added testWriteFailureClosesChannelAndFailsQuery — verifies channel close, markServerDown called, markRequestSent never called
  • Added testWriteSuccessMarksRequestSent — verifies markRequestSent called, markServerDown never called, close never called
  • Point-compiled pinot-common and pinot-core
  • All ServerChannelsTest tests pass
  • license:format and spotless:apply pass

ServerChannels.sendRequestWithoutLocking() never checked writeAndFlush()
success, causing silent failures: markRequestSent() called on failed
writes (false positive), no channel close (zombie channels), no metrics,
and queries waiting for full timeout instead of failing fast.

Add f.isSuccess() check in the writeAndFlush listener:
- On success: existing behavior (latency timer + markRequestSent)
- On failure: log error, increment NETTY_CONNECTION_SEND_REQUEST_FAILURES
  metric, call markServerDown() for fast query failure, close channel

Uses markServerDown() over markQueryFailed() because it is race-safe
(no-op if server already responded) and the channel close also triggers
channelInactive() for all in-flight queries. Wraps f.cause() in
RuntimeException since markServerDown expects Exception but cause() can
return Throwable (e.g. OutOfMemoryError).
@codecov-commenter
Copy link

codecov-commenter commented Mar 11, 2026

Codecov Report

✅ All modified and coverable lines are covered by tests.
✅ Project coverage is 63.30%. Comparing base (e05a8a6) to head (8c81241).

Additional details and impacted files
@@             Coverage Diff              @@
##             master   #17861      +/-   ##
============================================
+ Coverage     63.26%   63.30%   +0.04%     
  Complexity     1460     1460              
============================================
  Files          3190     3190              
  Lines        192011   192022      +11     
  Branches      29412    29413       +1     
============================================
+ Hits         121469   121569     +100     
+ Misses        61026    60924     -102     
- Partials       9516     9529      +13     
Flag Coverage Δ
integration 0.00% <ø> (ø)
integration2 0.00% <ø> (ø)
java-11 63.28% <100.00%> (+0.06%) ⬆️
java-21 63.22% <100.00%> (-0.01%) ⬇️
temurin 63.30% <100.00%> (+0.04%) ⬆️
unittests 63.30% <100.00%> (+0.04%) ⬆️
unittests1 55.58% <100.00%> (+0.01%) ⬆️
unittests2 34.30% <7.14%> (+0.03%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

🚀 New features to boost your workflow:
  • 📦 JS Bundle Analysis: Save yourself from yourself by tracking and limiting bundle sizes in JS merges.

Copy link
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 broker-to-server Netty reliability by handling outbound writeAndFlush() failures in ServerChannels, ensuring failed sends don’t get incorrectly treated as “request sent” and that broken channels are closed and observed via metrics.

Changes:

  • Add ChannelFuture.isSuccess() handling in ServerChannels.ServerChannel.sendRequestWithoutLocking() to log, meter, mark the server down, and close the channel on write failure.
  • Introduce a new broker meter NETTY_CONNECTION_SEND_REQUEST_FAILURES for observability of outbound write failures.
  • Add unit tests covering both write-failure and write-success listener behavior (plus small test hooks in ServerChannels).

Reviewed changes

Copilot reviewed 3 out of 3 changed files in this pull request and generated 1 comment.

File Description
pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java Adds write-failure handling in the Netty write listener; exposes minimal test hooks.
pinot-core/src/test/java/org/apache/pinot/core/transport/ServerChannelsTest.java Adds tests validating correct behavior on write success vs failure (close channel / mark server down).
pinot-common/src/main/java/org/apache/pinot/common/metrics/BrokerMeter.java Adds a new broker metric for outbound request send failures.

You can also share your feedback on Copilot code review. Take the survey.

Comment on lines +263 to +267
_brokerMetrics.addMeteredGlobalValue(BrokerMeter.NETTY_CONNECTION_SEND_REQUEST_FAILURES, 1);
asyncQueryResponse.markServerDown(serverRoutingInstance,
new RuntimeException("Failed to send request to server: " + serverRoutingInstance, f.cause()));
_channel.close();
}
Copy link

Copilot AI Mar 11, 2026

Choose a reason for hiding this comment

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

In the write failure listener, calling _channel.close() uses the mutable ServerChannel._channel field, which may have been reconnected/updated by another request by the time the listener runs. That can accidentally close a newly-established healthy channel instead of the channel associated with the failed write. Capture the current channel in a local variable before writeAndFlush() (or use the channel from the future) and close that specific channel on failure.

Copilot uses AI. Check for mistakes.
@xiangfu0 xiangfu0 merged commit 111cbf7 into apache:master Mar 11, 2026
18 of 20 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants