Skip to content

refactor: migrate Druid HTTP client from Netty 3 to 4#19567

Open
clintropolis wants to merge 6 commits into
apache:masterfrom
clintropolis:remove-netty-3-attempt-3
Open

refactor: migrate Druid HTTP client from Netty 3 to 4#19567
clintropolis wants to merge 6 commits into
apache:masterfrom
clintropolis:remove-netty-3-attempt-3

Conversation

@clintropolis

Copy link
Copy Markdown
Member

Description

Third time's a charm? Previous attempts #12032 and #14479 afaict both got stuck trying to resolve failures in the integration tests (the old docker based integration tests). I started fresh for this attempt, but the embedded-tests that have replaced our old frameworks made it a lot easier to dig in and determine the problem (and credit to prior attempts which were used as reference for comparison)

The root problem with prior attempts that resulted in the observed hanging is that Netty 4's setAutoRead(true) is just a config flag. Netty 3's setReadable(true) also kicked an immediate read. For Druid's chunked responses where the server flushes the status line in one TCP write and the body in subsequent writes (e.g. QueryResource), the body bytes sit on the wire for ~5 minutes until connection close. Fix is a one-line ctx.read() after handleResponse in NettyHttpClient with a comment explaining the semantic difference.

Allocator and direct memory

The HTTP client previously used Netty 3's HeapChannelBufferFactory which is unpooled heap buffers. Netty 4.2's default is AdaptiveByteBufAllocator (pooled internally via AdaptivePoolingAllocator, direct by default). This is a substantive change in memory characteristics:

  • Direct memory: the HTTP client now consumes some direct memory. Operators
    running close to their -XX:MaxDirectMemorySize budget may need to bump it.
  • Pooled vs unpooled: buffers are now pooled. Resident memory may show
    higher than under Netty 3 even when idle, normal pooled-allocator behavior.

This PR is using Netty 4.2's default allocator (adaptive) rather than pinning to the PooledByteBufAllocator that was the Netty 4 default until 4.2 (or the unpooled heap behavior of Netty 3). Since Druid operators are coming from heap-unpooled (Netty 3), neither Netty 4 choice represents a "known good baseline"; both are new characteristics for this workload. However, using pooled off-heap buffers of some kind should offer a performance improvement so it seems worth the risk.

The allocator can be swapped without a code change:

-Dio.netty.allocator.type=… Effect
adaptive (default in Netty 4.2) What this PR ships with
pooled The classic PooledByteBufAllocator
unpooled Closest to Netty 3 behavior; combine with -Dio.netty.noPreferDirect=true to also stay on-heap

Other useful knobs:

Property Default Purpose
io.netty.noPreferDirect false true ⇒ pooled but heap rather than direct
io.netty.allocator.maxOrder 9 Smaller value ⇒ smaller chunk size, lower memory floor
io.netty.maxDirectMemory derived from -XX:MaxDirectMemorySize Netty's direct-memory cap
io.netty.leakDetection.level simple disabled / simple / advanced / paranoid

Other stuff

InputStreamHolder.fromChannelBuffer(ByteBuf, long) now copies bytes synchronously instead of wrapping a ByteBufInputStream over a pooled buffer that SimpleChannelInboundHandler releases after channelRead0 returns. The prior PRs hit CodeQL leak warnings on SequenceInputStreamResponseHandler.java; that handler is now leak-safe too, and so are DirectDruidClient and DataServerResponseHandler which all funnel through the same helper.

changes:
* `NettyHttpClient` now uses netty 4 (along with all associated configs, handlers, etc)
* allocator: uses Netty 4.2's default `AdaptiveByteBufAllocator` (pooled internally, direct by default), replacing Netty 3's unpooled heap allocator. Operators running close to `-XX:MaxDirectMemorySize` may need to bump it. Allocator type is swappable at runtime via `-Dio.netty.allocator.type={adaptive,pooled,unpooled}` (operators might wish to experiment on upgrade).
* fix a leak in `SequenceInputStreamResponseHandler`, copy chunk bytes to `byte[]` before queueing (was wrapping a pooled `ByteBuf` that `SimpleChannelInboundHandler` releases after `channelRead0` returns)
* lots of import changes to swap to netty 4
* cleanup pom, licenses.yaml, OWASP suppressions
* added 'paranoid' leak detection to embedded-tests
Assert.assertEquals(
json,
StringUtils.fromUtf8(ByteStreams.toByteArray(new ChannelBufferInputStream(request.getContent())))
StringUtils.fromUtf8(ByteStreams.toByteArray(new ByteBufInputStream(request.getContent())))
Assert.assertEquals(
"{\"foo\":3}",
StringUtils.fromUtf8(ByteStreams.toByteArray(new ChannelBufferInputStream(request.getContent())))
StringUtils.fromUtf8(ByteStreams.toByteArray(new ByteBufInputStream(request.getContent())))
Comment thread server/src/main/java/org/apache/druid/rpc/StandardRetryPolicy.java Fixed

@FrankChen021 FrankChen021 left a comment

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Severity Findings
P0 0
P1 1
P2 0
P3 0
Total 1

Reviewed 164 of 164 changed files.


This is an automated review by Codex GPT-5.5

method,
urlFile.isEmpty() ? "/" : urlFile
urlFile.isEmpty() ? "/" : urlFile,
request.hasContent() ? request.getContent() : Unpooled.EMPTY_BUFFER

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

[P1] Do not hand the request-owned ByteBuf to Netty

This passes the Request's stored ByteBuf directly into a Netty 4 FullHttpRequest. Netty's HTTP encoder releases FullHttpMessages after encoding, and may also advance the content reader index, so after the first send request.getContent() can be released or consumed. Kerberos' 401 retry path calls request.copy() after the first response, which can then fail or resend an empty body for authenticated POSTs. Please build the outbound request from a retained duplicate/copy, or make Request store bytes and create a fresh ByteBuf per send.

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.

3 participants