Skip to content

Fix server outbound write failure creating zombie channels#17845

Merged
xiangfu0 merged 2 commits intoapache:masterfrom
suvodeep-pyne:spyne/fix-server-outbound-write-failure-zombie-channels
Mar 10, 2026
Merged

Fix server outbound write failure creating zombie channels#17845
xiangfu0 merged 2 commits intoapache:masterfrom
suvodeep-pyne:spyne/fix-server-outbound-write-failure-zombie-channels

Conversation

@suvodeep-pyne
Copy link
Contributor

Summary

  • When writeAndFlush() fails on the server (e.g. direct memory OOM), Netty half-closes the channel via shutdownOutput() but channelInactive() never fires, creating a zombie channel that accepts queries but never sends responses
  • The broker sees silent timeouts and keeps routing queries to the broken channel
  • Add f.isSuccess() check in the writeAndFlush listener in InstanceRequestHandler.sendResponse(). On failure: log the error, increment NETTY_CONNECTION_SEND_RESPONSE_EXCEPTIONS metric, and close the channel via ctx.close() to trigger proper cleanup
  • Since sendErrorResponse() delegates to sendResponse(), this single change covers all outbound writes

Test plan

  • Added testWriteFailureClosesChannel unit test that captures the write listener, invokes it with a failed future, and verifies ctx.close() is called
  • Existing testCancelQuery test passes (no regression)
  • InstanceRequestHandlerTest: 2 tests, 0 failures

When writeAndFlush() fails (e.g. direct memory OOM), the channel is
half-closed by Netty but channelInactive() never fires, creating a
zombie channel that accepts queries but never sends responses. The
broker sees silent timeouts and keeps routing to the broken channel.

Add f.isSuccess() check in the writeAndFlush listener in
InstanceRequestHandler.sendResponse(). On failure: log the error,
increment NETTY_CONNECTION_SEND_RESPONSE_EXCEPTIONS metric, and close
the channel via ctx.close() to trigger proper cleanup. Since
sendErrorResponse() delegates to sendResponse(), this covers all
outbound writes.
@codecov-commenter
Copy link

codecov-commenter commented Mar 9, 2026

❌ 1 Tests Failed:

Tests completed Failed Passed Skipped
10979 1 10978 55
View the top 2 failed test(s) by shortest run time
org.apache.pinot.controller.helix.core.minion.PinotTaskManagerDistributedLockingTest::testCreateTaskBlocksScheduleTaskForSpecificTable
Stack Traces | 11.6s run time
expected [Could not acquire table level distributed lock for scheduled task type: TestDistributedLockTaskType, table: testTable1_OFFLINE. Another controller is likely generating tasks for this table. Please try again later.] but found [Could not acquire table level distributed lock for scheduled task type: TestDistributedLockTaskType, table: testTable2_OFFLINE. Another controller is likely generating tasks for this table. Please try again later.]
org.apache.pinot.controller.helix.core.minion.PinotTaskManagerDistributedLockingTest::testCreateTaskBlocksScheduleTaskForSpecificTable
Stack Traces | 14.5s run time
expected [Could not acquire table level distributed lock for scheduled task type: TestDistributedLockTaskType, table: testTable1_OFFLINE. Another controller is likely generating tasks for this table. Please try again later.] but found [Could not acquire table level distributed lock for scheduled task type: TestDistributedLockTaskType, table: testTable2_OFFLINE. Another controller is likely generating tasks for this table. Please try again later.]
View the full list of 1 ❄️ flaky test(s)
org.apache.pinot.integration.tests.PauselessRealtimeIngestionWithDedupIntegrationTest::setUp

Flake rate in main: 100.00% (Passed 0 times, Failed 68 times)

Stack Traces | 16.9s run time
Failed to load 5 documents; current count=4 for table=DedupTableWithReplicas_REALTIME expected [5] but found [4]

To view more test analytics, go to the Test Analytics Dashboard
📋 Got 3 mins? Take this short survey to help us improve Test Analytics.

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

Fixes a server-side Netty edge case where outbound writeAndFlush() failures can leave channels half-closed (“zombie” channels) that still accept requests but never return responses, by explicitly closing the channel on write failure and tracking the failure via a new metric.

Changes:

  • Add a ChannelFuture listener success/failure branch in InstanceRequestHandler.sendResponse(); on failure, log, meter, and ctx.close().
  • Introduce a new server meter NETTY_CONNECTION_SEND_RESPONSE_FAILURES.
  • Add a unit test asserting the channel is closed when the write future completes unsuccessfully.

Reviewed changes

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

File Description
pinot-core/src/main/java/org/apache/pinot/core/transport/InstanceRequestHandler.java Close the Netty channel and record a metric when response writes fail, preventing “zombie” channels.
pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerMeter.java Add a new server meter for outbound send-response failures.
pinot-core/src/test/java/org/apache/pinot/core/transport/InstanceRequestHandlerTest.java Add unit coverage for the failed-write listener behavior (verifies ctx.close()).

Comment on lines +325 to +328
Throwable cause = f.cause();
LOGGER.error("Failed to send response for request: {} table: {}", requestId, tableNameWithType, cause);
_serverMetrics.addMeteredGlobalValue(ServerMeter.NETTY_CONNECTION_SEND_RESPONSE_FAILURES, 1);
ctx.close();
Copy link

Copilot AI Mar 10, 2026

Choose a reason for hiding this comment

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

The PR description mentions incrementing NETTY_CONNECTION_SEND_RESPONSE_EXCEPTIONS, but the implementation adds/increments NETTY_CONNECTION_SEND_RESPONSE_FAILURES. Please align the metric name (either rename the meter for consistency with existing *_EXCEPTIONS meters, or update the PR description and any dashboards/alerts that expect the old name).

Copilot uses AI. Check for mistakes.
Comment on lines +59 to +64
@BeforeClass
public void setUp() {
PinotMetricUtils.init(new PinotConfiguration());
PinotMetricsRegistry registry = PinotMetricUtils.getPinotMetricsRegistry();
ServerMetrics.register(new ServerMetrics(registry));
}
Copy link

Copilot AI Mar 10, 2026

Choose a reason for hiding this comment

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

PinotMetricUtils.init(...) performs global metrics initialization (including default registration listeners such as the JMX reporter) and ServerMetrics.register(...) mutates a global singleton. This test doesn’t assert on metrics, and InstanceRequestHandler works with the NOOP ServerMetrics by default, so this setup appears unnecessary and can leak state across the test suite. Consider removing this setup, or adding an @AfterClass cleanup (e.g., ServerMetrics.deregister() and PinotMetricUtils.cleanUp()) to avoid cross-test interference.

Copilot uses AI. Check for mistakes.
ChannelFuture writeFuture = mock(ChannelFuture.class);
when(ctx.writeAndFlush(any())).thenReturn(writeFuture);

ArgumentCaptor<GenericFutureListener> listenerCaptor = ArgumentCaptor.forClass(GenericFutureListener.class);
Copy link

Copilot AI Mar 10, 2026

Choose a reason for hiding this comment

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

In testWriteFailureClosesChannel, the test uses a raw GenericFutureListener captor and suppresses unchecked warnings. It would be safer and clearer to parameterize the captor with the expected Netty listener type (matching ChannelFuture.addListener(...)) so the compiler can enforce the correct future type and the suppression can be removed.

Suggested change
ArgumentCaptor<GenericFutureListener> listenerCaptor = ArgumentCaptor.forClass(GenericFutureListener.class);
ArgumentCaptor<GenericFutureListener<Future<Void>>> listenerCaptor =
ArgumentCaptor.forClass((Class<GenericFutureListener<Future<Void>>>) (Class<?>) GenericFutureListener.class);

Copilot uses AI. Check for mistakes.
@xiangfu0 xiangfu0 merged commit 58daa99 into apache:master Mar 10, 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

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants