From 1ddd05dfa4abfd40d3a922118f63fdc6f3b15e49 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sat, 28 Jun 2025 19:20:43 +0300 Subject: [PATCH 01/57] Http-netty4 connection pool implementation --- .../http-netty4/spotbugs-exclude.xml | 1 + .../http/netty4/NettyHttpClient.java | 291 +++++------ .../http/netty4/NettyHttpClientBuilder.java | 94 +++- .../implementation/Netty4AlpnHandler.java | 7 + .../Netty4ChannelBinaryData.java | 62 ++- .../Netty4ChannelInputStream.java | 25 +- .../implementation/Netty4ConnectionPool.java | 458 ++++++++++++++++++ .../Netty4EagerConsumeChannelHandler.java | 51 +- .../implementation/Netty4HandlerNames.java | 5 + .../Netty4InitiateOneReadHandler.java | 17 +- .../Netty4PipelineCleanupHandler.java | 103 ++++ .../Netty4ProgressAndTimeoutHandler.java | 5 +- .../HttpResponseDrainsBufferTests.java | 37 +- .../Netty4ConnectionPoolTests.java | 222 +++++++++ .../Netty4Http11ChannelInputStreamTests.java | 36 +- 15 files changed, 1151 insertions(+), 263 deletions(-) create mode 100644 sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java create mode 100644 sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java create mode 100644 sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java diff --git a/sdk/clientcore/http-netty4/spotbugs-exclude.xml b/sdk/clientcore/http-netty4/spotbugs-exclude.xml index 9b8d71f16243..e2139a1da9f8 100644 --- a/sdk/clientcore/http-netty4/spotbugs-exclude.xml +++ b/sdk/clientcore/http-netty4/spotbugs-exclude.xml @@ -9,6 +9,7 @@ + diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index 90fbafd96b16..c77538a6d852 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -4,7 +4,6 @@ package io.clientcore.http.netty4; import io.clientcore.core.http.client.HttpClient; -import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.http.models.HttpHeaderName; import io.clientcore.core.http.models.HttpRequest; import io.clientcore.core.http.models.Response; @@ -13,48 +12,40 @@ import io.clientcore.core.models.CoreException; import io.clientcore.core.models.ServerSentResult; import io.clientcore.core.models.binarydata.BinaryData; -import io.clientcore.core.utils.AuthenticateChallenge; import io.clientcore.core.utils.CoreUtils; import io.clientcore.core.utils.ProgressReporter; import io.clientcore.core.utils.ServerSentEventUtils; -import io.clientcore.http.netty4.implementation.ChannelInitializationProxyHandler; import io.clientcore.http.netty4.implementation.Netty4AlpnHandler; import io.clientcore.http.netty4.implementation.Netty4ChannelBinaryData; +import io.clientcore.http.netty4.implementation.Netty4ConnectionPool; import io.clientcore.http.netty4.implementation.Netty4EagerConsumeChannelHandler; import io.clientcore.http.netty4.implementation.Netty4HandlerNames; +import io.clientcore.http.netty4.implementation.Netty4PipelineCleanupHandler; import io.clientcore.http.netty4.implementation.Netty4ProgressAndTimeoutHandler; import io.clientcore.http.netty4.implementation.Netty4ResponseHandler; -import io.clientcore.http.netty4.implementation.Netty4SslInitializationHandler; import io.clientcore.http.netty4.implementation.ResponseBodyHandling; import io.clientcore.http.netty4.implementation.ResponseStateInfo; import io.netty.bootstrap.Bootstrap; -import io.netty.buffer.Unpooled; import io.netty.channel.Channel; -import io.netty.channel.ChannelFutureListener; -import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; -import io.netty.handler.codec.http2.Http2SecurityUtil; -import io.netty.handler.proxy.ProxyHandler; -import io.netty.handler.ssl.ApplicationProtocolConfig; -import io.netty.handler.ssl.ApplicationProtocolNames; -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslContextBuilder; -import io.netty.handler.ssl.SslHandler; -import io.netty.handler.ssl.SslProvider; -import io.netty.handler.ssl.SupportedCipherSuiteFilter; - -import javax.net.ssl.SSLException; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; + import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketAddress; import java.net.URI; -import java.util.List; +import java.nio.channels.ClosedChannelException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; import static io.clientcore.core.utils.ServerSentEventUtils.attemptRetry; import static io.clientcore.core.utils.ServerSentEventUtils.processTextEventStream; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_CODEC; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_RESPONSE; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PIPELINE_CLEANUP; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; import static io.clientcore.http.netty4.implementation.Netty4Utility.awaitLatch; import static io.clientcore.http.netty4.implementation.Netty4Utility.createCodec; @@ -73,188 +64,67 @@ class NettyHttpClient implements HttpClient { private static final String NO_LISTENER_ERROR_MESSAGE = "No ServerSentEventListener attached to HttpRequest to handle the text/event-stream response"; - private final Bootstrap bootstrap; - private final Consumer sslContextModifier; - private final ChannelInitializationProxyHandler channelInitializationProxyHandler; - private final AtomicReference> proxyChallenges; + private final EventLoopGroup eventLoopGroup; + private final Netty4ConnectionPool connectionPool; private final long readTimeoutMillis; private final long responseTimeoutMillis; private final long writeTimeoutMillis; - private final HttpProtocolVersion maximumHttpVersion; - - NettyHttpClient(Bootstrap bootstrap, Consumer sslContextModifier, - HttpProtocolVersion maximumHttpVersion, ChannelInitializationProxyHandler channelInitializationProxyHandler, - long readTimeoutMillis, long responseTimeoutMillis, long writeTimeoutMillis) { - this.bootstrap = bootstrap; - this.sslContextModifier = sslContextModifier; - this.maximumHttpVersion = maximumHttpVersion; - this.channelInitializationProxyHandler = channelInitializationProxyHandler; - this.proxyChallenges = new AtomicReference<>(); + + NettyHttpClient(EventLoopGroup eventLoopGroup, Netty4ConnectionPool connectionPool, long readTimeoutMillis, + long responseTimeoutMillis, long writeTimeoutMillis) { + this.eventLoopGroup = eventLoopGroup; + this.connectionPool = connectionPool; this.readTimeoutMillis = readTimeoutMillis; this.responseTimeoutMillis = responseTimeoutMillis; this.writeTimeoutMillis = writeTimeoutMillis; } Bootstrap getBootstrap() { - return bootstrap; + return connectionPool.getBootstrap(); } @Override public Response send(HttpRequest request) { - URI uri = request.getUri(); - String host = uri.getHost(); - int port = uri.getPort() == -1 ? ("https".equalsIgnoreCase(uri.getScheme()) ? 443 : 80) : uri.getPort(); - boolean isHttps = "https".equalsIgnoreCase(uri.getScheme()); - ProgressReporter progressReporter = (request.getContext() == null) - ? null - : (ProgressReporter) request.getContext().getMetadata("progressReporter"); - boolean addProgressAndTimeoutHandler - = progressReporter != null || writeTimeoutMillis > 0 || responseTimeoutMillis > 0 || readTimeoutMillis > 0; - - AtomicReference responseReference = new AtomicReference<>(); - AtomicReference errorReference = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - - // Configure an immutable ChannelInitializer in the builder with everything that can be added on a non-per - // request basis. - bootstrap.handler(new ChannelInitializer() { - @Override - protected void initChannel(Channel channel) throws SSLException { - // Test whether proxying should be applied to this Channel. If so, add it. - boolean hasProxy = channelInitializationProxyHandler.test(channel.remoteAddress()); - if (hasProxy) { - ProxyHandler proxyHandler = channelInitializationProxyHandler.createProxy(proxyChallenges); - proxyHandler.connectFuture().addListener(future -> { - if (!future.isSuccess()) { - setOrSuppressError(errorReference, future.cause()); - } - }); - - channel.pipeline().addFirst(Netty4HandlerNames.PROXY, proxyHandler); - } - - // Add SSL handling if the request is HTTPS. - if (isHttps) { - SslContextBuilder sslContextBuilder - = SslContextBuilder.forClient().endpointIdentificationAlgorithm("HTTPS"); - if (maximumHttpVersion == HttpProtocolVersion.HTTP_2) { - // If HTTP/2 is the maximum version, we need to ensure that ALPN is enabled. - SslProvider sslProvider = SslContext.defaultClientProvider(); - ApplicationProtocolConfig.SelectorFailureBehavior selectorBehavior; - ApplicationProtocolConfig.SelectedListenerFailureBehavior selectedBehavior; - if (sslProvider == SslProvider.JDK) { - selectorBehavior = ApplicationProtocolConfig.SelectorFailureBehavior.FATAL_ALERT; - selectedBehavior = ApplicationProtocolConfig.SelectedListenerFailureBehavior.FATAL_ALERT; - } else { - // Netty OpenSslContext doesn't support FATAL_ALERT, use NO_ADVERTISE and ACCEPT - // instead. - selectorBehavior = ApplicationProtocolConfig.SelectorFailureBehavior.NO_ADVERTISE; - selectedBehavior = ApplicationProtocolConfig.SelectedListenerFailureBehavior.ACCEPT; - } - - sslContextBuilder.ciphers(Http2SecurityUtil.CIPHERS, SupportedCipherSuiteFilter.INSTANCE) - .applicationProtocolConfig(new ApplicationProtocolConfig( - ApplicationProtocolConfig.Protocol.ALPN, selectorBehavior, selectedBehavior, - ApplicationProtocolNames.HTTP_2, ApplicationProtocolNames.HTTP_1_1)); - } - if (sslContextModifier != null) { - // Allow the caller to modify the SslContextBuilder before it is built. - sslContextModifier.accept(sslContextBuilder); - } + final URI uri = request.getUri(); + final boolean isHttps = "https".equalsIgnoreCase(uri.getScheme()); + final int port = uri.getPort() == -1 ? (isHttps ? 443 : 80) : uri.getPort(); + final SocketAddress remoteAddress = new InetSocketAddress(uri.getHost(), port); - SslContext ssl = sslContextBuilder.build(); - // SSL handling is added last here. This is done as proxying could require SSL handling too. - channel.pipeline().addLast(Netty4HandlerNames.SSL, ssl.newHandler(channel.alloc(), host, port)); - channel.pipeline() - .addLast(Netty4HandlerNames.SSL_INITIALIZER, new Netty4SslInitializationHandler()); + final CountDownLatch latch = new CountDownLatch(1); + final AtomicReference responseReference = new AtomicReference<>(); + final AtomicReference errorReference = new AtomicReference<>(); - channel.pipeline() - .addLast(Netty4HandlerNames.ALPN, - new Netty4AlpnHandler(request, responseReference, errorReference, latch)); - } - } - }); + Future channelFuture = connectionPool.acquire(remoteAddress, isHttps); - bootstrap.connect(host, port).addListener((ChannelFutureListener) connectListener -> { - if (!connectListener.isSuccess()) { - LOGGER.atError().setThrowable(connectListener.cause()).log("Failed connection."); - errorReference.set(connectListener.cause()); - connectListener.channel().close(); + channelFuture.addListener((GenericFutureListener>) future -> { + if (!future.isSuccess()) { + LOGGER.atError().setThrowable(future.cause()).log("Failed connection."); + errorReference.set(future.cause()); latch.countDown(); return; } - Channel channel = connectListener.channel(); - channel.closeFuture().addListener(closeListener -> { - if (!closeListener.isSuccess()) { - LOGGER.atError().setThrowable(closeListener.cause()).log("Channel closed with error"); - setOrSuppressError(errorReference, closeListener.cause()); + Channel channel = future.getNow(); + try { + configureRequestPipeline(channel, request, responseReference, errorReference, latch, isHttps); + } catch (Exception e) { + // An exception occurred during the pipeline setup. + // We fire the exception through the pipeline to trigger the cleanup handler, + // which will ensure the channel is properly closed and not returned to the pool. + setOrSuppressError(errorReference, e); + if (channel.isActive()) { + channel.pipeline().fireExceptionCaught(e); } - }); - - // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is - // applying timeouts. - // This is done to keep the ChannelPipeline shorter, therefore more performant, if this would - // effectively be a no-op. - if (addProgressAndTimeoutHandler) { - channel.pipeline() - .addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, - writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); - } - - Throwable earlyError = errorReference.get(); - if (earlyError != null) { - // If an error occurred between the connect and the request being sent, don't proceed with sending - // the request. latch.countDown(); - return; - } - - // What I basically want here is the following logic in Netty: - // 1. If a proxy exists, it should be added first. When the connection is activated, we should connect - // to the proxy (with or without SSL). If there is no proxy, skip this step. - // 2. Once step 1 is complete, we should wait until the SSL handshake is complete (if applicable). - // If SSL isn't being used, skip this step. - // 3. Once step 2 is complete, we should send the request. - // - // None of the steps should block the event loop, so we need to use listeners to ensure that the next - // step is only executed once the previous step is complete. - SslHandler sslHandler = channel.pipeline().get(SslHandler.class); - if (sslHandler != null) { - // If the SslHandler is present, trigger the SSL handshake to complete before sending the request. - sslHandler.handshakeFuture().addListener(handshakeListener -> { - if (!handshakeListener.isSuccess()) { - LOGGER.atError().setThrowable(handshakeListener.cause()).log("Failed SSL handshake."); - errorReference.set(handshakeListener.cause()); - latch.countDown(); - } - }); - channel.write(Unpooled.EMPTY_BUFFER); - } else { - // If there isn't an SslHandler, we can send the request immediately. - // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. - Netty4ResponseHandler responseHandler - = new Netty4ResponseHandler(request, responseReference, errorReference, latch); - channel.pipeline().addLast(HTTP_RESPONSE, responseHandler); - - String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; - channel.pipeline().addBefore(addBefore, Netty4HandlerNames.HTTP_CODEC, createCodec()); - - sendHttp11Request(request, channel, errorReference) - .addListener((ChannelFutureListener) sendListener -> { - if (!sendListener.isSuccess()) { - setOrSuppressError(errorReference, sendListener.cause()); - sendListener.channel().close(); - latch.countDown(); - } else { - sendListener.channel().read(); - } - }); } }); awaitLatch(latch); + if (errorReference.get() != null) { + throw LOGGER.throwableAtError().log(errorReference.get(), CoreException::from); + } + ResponseStateInfo info = responseReference.get(); if (info == null) { throw LOGGER.throwableAtError().log(errorReference.get(), CoreException::from); @@ -354,10 +224,75 @@ protected void initChannel(Channel channel) throws SSLException { return response; } + private void configureRequestPipeline(Channel channel, HttpRequest request, + AtomicReference responseReference, AtomicReference errorReference, + CountDownLatch latch, boolean isHttps) { + + // It's possible that the channel was closed between the time it was acquired and now. + // This check ensures that we don't try to add handlers to a closed channel. + if (!channel.isActive()) { + LOGGER.atWarning().log("Channel acquired from the pool is inactive, failing the request."); + setOrSuppressError(errorReference, new ClosedChannelException()); + latch.countDown(); + return; + } + + ProgressReporter progressReporter = (request.getContext() == null) + ? null + : (ProgressReporter) request.getContext().getMetadata("progressReporter"); + boolean addProgressAndTimeoutHandler + = progressReporter != null || writeTimeoutMillis > 0 || responseTimeoutMillis > 0 || readTimeoutMillis > 0; + + ChannelPipeline pipeline = channel.pipeline(); + + // The first handler added is the cleanup handler. It will be the last to execute + // in the outbound direction and the first in the inbound direction, but its main + // purpose is to clean up all other request-specific handlers and release the channel. + pipeline.addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool)); + + // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is + // applying timeouts. + // This is done to keep the ChannelPipeline shorter, therefore more performant, if this would + // effectively be a no-op. + if (addProgressAndTimeoutHandler) { + pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, + writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); + } + + pipeline.addLast(HTTP_RESPONSE, new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + + // The SslHandler is already in the pipeline if this is an HTTPS request, as it's added + // by the connection pool during the initial connection setup. The SSL handshake is also + // guaranteed to be complete by the time we get the channel because the Netty4AlpnHandler + // reacts to the result of the ALPN negotiation that happened during the SSL handshake. + if (isHttps) { + pipeline.addAfter(Netty4HandlerNames.SSL_INITIALIZER, Netty4HandlerNames.ALPN, + new Netty4AlpnHandler(request, responseReference, errorReference, latch)); + } else { + String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; + pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); + sendHttp11Request(request, channel, errorReference).addListener(f -> { + if (f.isSuccess()) { + channel.read(); + } else { + setOrSuppressError(errorReference, f.cause()); + pipeline.fireExceptionCaught(f.cause()); + latch.countDown(); + } + }); + } + } + public void close() { - EventLoopGroup group = bootstrap.config().group(); - if (group != null) { - group.shutdownGracefully(); + if (connectionPool != null) { + try { + connectionPool.close(); + } catch (IOException e) { + LOGGER.atWarning().setThrowable(e).log("Failed to close Netty4ConnectionPool."); + } + } + if (eventLoopGroup != null && !eventLoopGroup.isShuttingDown()) { + eventLoopGroup.shutdownGracefully(); } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index b74853327e3a..268149115ae6 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -9,6 +9,7 @@ import io.clientcore.core.instrumentation.logging.ClientLogger; import io.clientcore.core.utils.configuration.Configuration; import io.clientcore.http.netty4.implementation.ChannelInitializationProxyHandler; +import io.clientcore.http.netty4.implementation.Netty4ConnectionPool; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelOption; @@ -135,6 +136,13 @@ private static Class getChannelClass(String className) private Duration writeTimeout; private HttpProtocolVersion maximumHttpVersion = HttpProtocolVersion.HTTP_2; + // --- Connection Pool Configuration --- + private int connectionPoolSize = 500; + private Duration connectionIdleTimeout = Duration.ofSeconds(60); + private Duration maxConnectionLifetime; + private Duration pendingAcquireTimeout = Duration.ofSeconds(60); // Default wait time for a connection + private int maxPendingAcquires = 10_000; // Default pending queue size + /** * Creates a new instance of {@link NettyHttpClientBuilder}. */ @@ -281,6 +289,79 @@ public NettyHttpClientBuilder maximumHttpVersion(HttpProtocolVersion httpVersion return this; } + /** + * Sets the maximum number of connections allowed per remote address in the connection pool. + *

+ * If not set, a default value of 500 is used. + * + * @param connectionPoolSize The maximum number of connections. Must be greater than or equal to 0. + * @return The updated builder. + */ + public NettyHttpClientBuilder connectionPoolSize(int connectionPoolSize) { + if (connectionPoolSize < 0) { + throw LOGGER.throwableAtError() + .log("connectionPoolSize must be greater than or equal to 0", IllegalArgumentException::new); + } + this.connectionPoolSize = connectionPoolSize; + return this; + } + + /** + * Sets the maximum time a connection can remain idle in the pool before it is closed and removed. + *

+ * If not set, a default value of 60 seconds is used. + *

+ * A {@link Duration} of zero or less will make the connections never expire. Note: While this is + * provided as an option, it is not recommended for most use cases, as it can lead to + * request failures if network intermediaries (like load balancers or firewalls) silently drop idle + * connections. + * + * @param connectionIdleTimeout The idle timeout duration. + * @return The updated builder. + */ + public NettyHttpClientBuilder connectionIdleTimeout(Duration connectionIdleTimeout) { + this.connectionIdleTimeout = connectionIdleTimeout; + return this; + } + + /** + * Sets the maximum time a connection is allowed to exist. + * After this time, the connection will be closed upon release. A default of null means no lifetime limit. + * + * @param maxConnectionLifetime The maximum connection lifetime. + * @return The updated builder. + */ + public NettyHttpClientBuilder maxConnectionLifetime(Duration maxConnectionLifetime) { + this.maxConnectionLifetime = maxConnectionLifetime; + return this; + } + + /** + * Sets the maximum time to wait for a connection from the pool. + * + * @param pendingAcquireTimeout The timeout for pending acquires. + * @return The updated builder. + */ + public NettyHttpClientBuilder pendingAcquireTimeout(Duration pendingAcquireTimeout) { + this.pendingAcquireTimeout = pendingAcquireTimeout; + return this; + } + + /** + * Sets the maximum number of requests that can be queued waiting for a connection. + * + * @param maxPendingAcquires The maximum number of pending acquires. + * @return The updated builder. + */ + public NettyHttpClientBuilder maxPendingAcquires(int maxPendingAcquires) { + if (maxPendingAcquires <= 0) { + throw LOGGER.throwableAtError() + .log("maxPendingAcquires must be greater than 0", IllegalArgumentException::new); + } + this.maxPendingAcquires = maxPendingAcquires; + return this; + } + /** * Builds the NettyHttpClient. * @@ -298,6 +379,11 @@ public HttpClient build() { .addKeyValue("eventLoopGroupClass", group.getClass()) .addKeyValue("customChannelClass", this.channelClass != null) .addKeyValue("channelClass", channelClass) + .addKeyValue("connectionPoolSize", this.connectionPoolSize) + .addKeyValue("connectionIdleTimeout", this.connectionIdleTimeout) + .addKeyValue("maxConnectionLifetime", this.maxConnectionLifetime) + .addKeyValue("pendingAcquireTimeout", this.pendingAcquireTimeout) + .addKeyValue("maxPendingAcquires", this.maxPendingAcquires) .log("NettyHttpClient was built with these configurations."); Bootstrap bootstrap = new Bootstrap().group(group) @@ -312,8 +398,12 @@ public HttpClient build() { ProxyOptions buildProxyOptions = (proxyOptions == null) ? ProxyOptions.fromConfiguration(buildConfiguration, true) : proxyOptions; - return new NettyHttpClient(bootstrap, sslContextModifier, maximumHttpVersion, - new ChannelInitializationProxyHandler(buildProxyOptions), getTimeoutMillis(readTimeout), + Netty4ConnectionPool connectionPool + = new Netty4ConnectionPool(bootstrap, new ChannelInitializationProxyHandler(buildProxyOptions), + sslContextModifier, connectionPoolSize, connectionIdleTimeout, maxConnectionLifetime, + pendingAcquireTimeout, maxPendingAcquires, maximumHttpVersion); + + return new NettyHttpClient(group, connectionPool, getTimeoutMillis(readTimeout), getTimeoutMillis(responseTimeout), getTimeoutMillis(writeTimeout)); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index 024dc9f7686a..5b1dfc4e7428 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -131,4 +131,11 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { throw new IllegalStateException("unknown protocol: " + protocol); } } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + setOrSuppressError(errorReference, cause); + ctx.fireExceptionCaught(cause); + latch.countDown(); + } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 837f4d164b1b..654587d8b6ad 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets; import java.util.Objects; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; import static io.clientcore.http.netty4.implementation.Netty4Utility.awaitLatch; @@ -36,6 +37,7 @@ public final class Netty4ChannelBinaryData extends BinaryData { private final Channel channel; private final Long length; private final boolean isHttp2; + private final AtomicBoolean streamDrained = new AtomicBoolean(false); // Non-final to allow nulling out after use. private ByteArrayOutputStream eagerContent; @@ -64,24 +66,8 @@ public byte[] toBytes() { } if (bytes == null) { - CountDownLatch latch = new CountDownLatch(1); - Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, - buf -> buf.readBytes(eagerContent, buf.readableBytes()), isHttp2); - channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); - channel.config().setAutoRead(true); - - awaitLatch(latch); - - Throwable exception = handler.channelException(); - if (exception != null) { - if (exception instanceof Error) { - throw (Error) exception; - } else { - throw CoreException.from(exception); - } - } else { - bytes = eagerContent.toByteArray(); - } + drainStream(); + bytes = eagerContent.toByteArray(); eagerContent = null; } @@ -105,7 +91,7 @@ public T toObject(Type type, ObjectSerializer serializer) { @Override public InputStream toStream() { if (bytes == null) { - return new Netty4ChannelInputStream(eagerContent, channel, isHttp2); + return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, this::drainStream); } else { return new ByteArrayInputStream(bytes); } @@ -129,6 +115,7 @@ public void writeTo(OutputStream outputStream) { // Channel hasn't been read yet, don't buffer it, just write it to the OutputStream as it's being read. if (eagerContent.size() > 0) { outputStream.write(eagerContent.toByteArray()); + eagerContent.reset(); } CountDownLatch latch = new CountDownLatch(1); @@ -138,6 +125,7 @@ public void writeTo(OutputStream outputStream) { channel.config().setAutoRead(true); awaitLatch(latch); + streamDrained.set(true); Throwable exception = handler.channelException(); if (exception != null) { @@ -147,13 +135,14 @@ public void writeTo(OutputStream outputStream) { throw CoreException.from(exception); } } - eagerContent = null; } else { // Already converted the Channel to a byte[], use it. outputStream.write(bytes); } } catch (IOException ex) { throw LOGGER.throwableAtError().log(ex, CoreException::from); + } finally { + close(); } } @@ -182,10 +171,37 @@ public BinaryData toReplayableBinaryData() { return BinaryData.fromBytes(toBytes()); } + /** + * Ensures the underlying network stream is fully consumed but does not close the channel, + * allowing it to be reused by the connection pool. + */ @Override public void close() { - eagerContent = null; - channel.disconnect(); - channel.close(); + drainStream(); + } + + private void drainStream() { + if (streamDrained.compareAndSet(false, true)) { + if (channel.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { + return; + } + + CountDownLatch latch = new CountDownLatch(1); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, + buf -> buf.readBytes(eagerContent, buf.readableBytes()), isHttp2); + channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); + channel.config().setAutoRead(true); + + awaitLatch(latch); + + Throwable exception = handler.channelException(); + if (exception != null) { + if (exception instanceof Error) { + throw (Error) exception; + } else { + throw CoreException.from(exception); + } + } + } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java index 418df7945248..1d8882ffb9d8 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java @@ -16,6 +16,7 @@ public final class Netty4ChannelInputStream extends InputStream { private final Channel channel; private final boolean isHttp2; + private final Runnable onClose; // Indicator for the Channel being fully read. // This will become true before 'streamDone' becomes true, but both may become true in the same operation. @@ -46,10 +47,12 @@ public final class Netty4ChannelInputStream extends InputStream { * status line and response headers. * @param channel The {@link Channel} to read from. * @param isHttp2 Flag indicating whether the Channel is used for HTTP/2 or not. + * @param onClose A runnable to execute when the stream is closed. */ - Netty4ChannelInputStream(ByteArrayOutputStream eagerContent, Channel channel, boolean isHttp2) { + Netty4ChannelInputStream(ByteArrayOutputStream eagerContent, Channel channel, boolean isHttp2, Runnable onClose) { if (eagerContent != null && eagerContent.size() > 0) { this.currentBuffer = eagerContent.toByteArray(); + eagerContent.reset(); } else { this.currentBuffer = new byte[0]; } @@ -60,6 +63,7 @@ public final class Netty4ChannelInputStream extends InputStream { channel.pipeline().remove(Netty4InitiateOneReadHandler.class); } this.isHttp2 = isHttp2; + this.onClose = onClose; } byte[] getCurrentBuffer() { @@ -167,13 +171,20 @@ public long skip(long n) throws IOException { return n - toSkip; } + /** + * Closes this input stream and ensures the underlying connection can be returned to the pool. + * This method does not close the underlying channel. Instead, it triggers the onClose + * callback which is responsible for draining the rest of the stream content. + */ @Override - public void close() { - currentBuffer = null; - additionalBuffers.clear(); - if (channel.isOpen() || channel.isActive()) { - channel.disconnect(); - channel.close(); + public void close() throws IOException { + try { + if (onClose != null && !streamDone) { + onClose.run(); + } + } finally { + super.close(); + streamDone = true; } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java new file mode 100644 index 000000000000..71513939bccd --- /dev/null +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -0,0 +1,458 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package io.clientcore.http.netty4.implementation; + +import io.clientcore.core.http.client.HttpProtocolVersion; +import io.clientcore.core.instrumentation.logging.ClientLogger; +import io.clientcore.core.models.CoreException; +import io.clientcore.core.utils.AuthenticateChallenge; +import io.netty.bootstrap.Bootstrap; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.EventLoopGroup; +import io.netty.handler.codec.http2.Http2SecurityUtil; +import io.netty.handler.proxy.ProxyHandler; +import io.netty.handler.ssl.ApplicationProtocolConfig; +import io.netty.handler.ssl.ApplicationProtocolNames; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslContextBuilder; +import io.netty.handler.ssl.SslProvider; +import io.netty.handler.ssl.SupportedCipherSuiteFilter; +import io.netty.util.AttributeKey; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.Promise; + +import javax.net.ssl.SSLException; +import java.io.Closeable; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.time.Duration; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Deque; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; + +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL_INITIALIZER; + +/** + * A pool of Netty channels that can be reused for requests to the same remote address. + */ +public final class Netty4ConnectionPool implements Closeable { + + /** + * An attribute key to mark a channel as new, so that certain handlers (e.g. proxy, ssl) + * are only added once. + */ + public static final AttributeKey NEW_CHANNEL_KEY = AttributeKey.valueOf("new-channel"); + private static final AttributeKey CHANNEL_CREATION_TIME + = AttributeKey.valueOf("channel-creation-time"); + + private static final ClientLogger LOGGER = new ClientLogger(Netty4ConnectionPool.class); + private static final String CLOSED_POOL_ERROR_MESSAGE = "Connection pool has been closed."; + + private final ConcurrentMap pool = new ConcurrentHashMap<>(); + private final ConcurrentMap channelIdleSince = new ConcurrentHashMap<>(); + private final AtomicBoolean closed = new AtomicBoolean(false); + + private final Bootstrap bootstrap; + private final int maxConnectionsPerRoute; + private final long idleTimeoutNanos; + private final long maxLifetimeNanos; + private final Duration pendingAcquireTimeout; + private final int maxPendingAcquires; + private final Future cleanupTask; + + private final ChannelInitializationProxyHandler channelInitializationProxyHandler; + private final Consumer sslContextModifier; + private final AtomicReference> proxyChallenges; + private final HttpProtocolVersion maximumHttpVersion; + + public Netty4ConnectionPool(Bootstrap bootstrap, + ChannelInitializationProxyHandler channelInitializationProxyHandler, + Consumer sslContextModifier, int maxConnectionsPerRoute, Duration connectionIdleTimeout, + Duration maxConnectionLifetime, Duration pendingAcquireTimeout, int maxPendingAcquires, + HttpProtocolVersion maximumHttpVersion) { + this.bootstrap = bootstrap; + this.channelInitializationProxyHandler = channelInitializationProxyHandler; + this.sslContextModifier = sslContextModifier; + this.proxyChallenges = new AtomicReference<>(); + this.maxConnectionsPerRoute = maxConnectionsPerRoute; + this.idleTimeoutNanos = durationToNanos(connectionIdleTimeout); + this.maxLifetimeNanos = durationToNanos(maxConnectionLifetime); + this.pendingAcquireTimeout = pendingAcquireTimeout; + this.maxPendingAcquires = maxPendingAcquires; + this.maximumHttpVersion = maximumHttpVersion; + + if (this.idleTimeoutNanos > 0) { + EventLoopGroup eventLoopGroup = bootstrap.config().group(); + // This scheduled task cleans up idle connections periodically. + // The 30-second interval is a trade-off between precision and performance. + // Running it more frequently would be more precise but add more overhead. + // This means a connection may stay idle for up to (idleTimeout + 30s) before being closed, + // which is an acceptable behavior for preventing resource leaks. + this.cleanupTask + = eventLoopGroup.scheduleAtFixedRate(this::cleanupIdleConnections, 30, 30, TimeUnit.SECONDS); + } else { + this.cleanupTask = null; + } + } + + /** + * Acquires a channel for the given remote address from the pool. + * + * @param remoteAddress The remote address to connect to. + * @param isHttps Flag indicating whether connections for this route should be secured using TLS/SSL. + * @return A {@link Future} that will be notified when a channel is acquired. + * @throws IllegalStateException if the connection pool has been closed. + */ + public Future acquire(SocketAddress remoteAddress, boolean isHttps) { + if (closed.get()) { + throw LOGGER.throwableAtError().log(CLOSED_POOL_ERROR_MESSAGE, IllegalStateException::new); + } + + PerRoutePool perRoutePool = pool.computeIfAbsent(remoteAddress, k -> new PerRoutePool(k, isHttps)); + return perRoutePool.acquire(); + } + + /** + * Releases a channel back to the pool. + * The channel pipeline must be cleaned of request-specific handlers before releasing. + * + * @param channel The channel to release. + */ + public void release(Channel channel) { + if (channel == null) { + return; + } + if (closed.get()) { + channel.close(); + return; + } + + if (channel.remoteAddress() != null) { + PerRoutePool perRoutePool = pool.get(channel.remoteAddress()); + if (perRoutePool != null) { + perRoutePool.release(channel); + } else { + channel.close(); + } + } else { + channel.close(); + } + } + + /** + * Periodically cleans up connections that have been idle for too long. + */ + private void cleanupIdleConnections() { + if (idleTimeoutNanos <= 0 || channelIdleSince.isEmpty()) { + return; + } + + OffsetDateTime now = OffsetDateTime.now(ZoneOffset.UTC); + for (Iterator> it = channelIdleSince.entrySet().iterator(); it.hasNext();) { + Map.Entry entry = it.next(); + if (Duration.between(entry.getValue(), now).toNanos() >= idleTimeoutNanos) { + it.remove(); + if (entry.getKey().isActive()) { + entry.getKey().close(); + } + } + } + } + + @Override + public void close() throws IOException { + if (closed.compareAndSet(false, true)) { + if (cleanupTask != null) { + cleanupTask.cancel(false); + } + pool.values().forEach(PerRoutePool::close); + pool.clear(); + channelIdleSince.clear(); + } + } + + public Bootstrap getBootstrap() { + return bootstrap; + } + + private static long durationToNanos(Duration duration) { + return (duration == null || duration.isNegative() || duration.isZero()) ? -1 : duration.toNanos(); + } + + /** + * Manages connections and pending acquirers for a single route. + */ + private class PerRoutePool { + private final Deque idleConnections = new ConcurrentLinkedDeque<>(); + private final Deque> pendingAcquirers = new ConcurrentLinkedDeque<>(); + private final AtomicInteger activeConnections = new AtomicInteger(0); + private final SocketAddress route; + private final boolean isHttps; + + PerRoutePool(SocketAddress route, boolean isHttps) { + this.route = route; + this.isHttps = isHttps; + } + + //TODO: Maybe this should be using CAS atomics and loops instead of synchronized + // in case the http-netty4 is also used in async flows instead of just sync + synchronized Future acquire() { + if (closed.get()) { + return bootstrap.config() + .group() + .next() + .newFailedFuture(new IllegalStateException(CLOSED_POOL_ERROR_MESSAGE)); + } + + Channel channel; + while ((channel = idleConnections.poll()) != null) { + if (isHealthy(channel)) { + // Acquired an existing healthy connection. activeConnections count is not + // yet incremented for idle channels, so we do it here. + activeConnections.incrementAndGet(); + channel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(false)); + channelIdleSince.remove(channel); + return channel.eventLoop().newSucceededFuture(channel); + } + // Unhealthy idle connection was found and discarded. Don't decrement activeConnections + // as it was already decremented when the channel was released to the idle queue. + channel.close(); + } + + // No idle connections available, create a new one. + if (activeConnections.get() < maxConnectionsPerRoute) { + return createNewConnection(); + } + + // Pool is full, queue the request if there is space. + if (pendingAcquirers.size() >= maxPendingAcquires) { + return bootstrap.config() + .group() + .next() + .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); + } + + Promise promise = bootstrap.config().group().next().newPromise(); + promise.addListener(future -> { + if (future.isCancelled()) { + pendingAcquirers.remove(promise); + } + }); + pendingAcquirers.offer(promise); + + if (pendingAcquireTimeout != null) { + bootstrap.config().group().schedule(() -> { + if (!promise.isDone()) { + promise.tryFailure( + CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout)); + } + }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); + } + return promise; + } + + synchronized void release(Channel channel) { + Promise waiter; + while ((waiter = pendingAcquirers.poll()) != null) { + // A waiter exists, hand over this channel directly. + // The activeConnections count remains the same (one leaves, one joins). + if (waiter.trySuccess(channel)) { + return; + } + } + + // No waiters to hand off to. Check channel health before returning to pool + if (!isHealthy(channel)) { + activeConnections.decrementAndGet(); + channel.close(); + // A slot has freed up. Proactively try to satisfy a pending waiter. + satisfyWaiterWithNewConnection(); + return; + } + + // Channel is healthy and no waiters, return it to the idle queue. + activeConnections.decrementAndGet(); + idleConnections.offer(channel); + if (idleTimeoutNanos > 0) { + channelIdleSince.put(channel, OffsetDateTime.now(ZoneOffset.UTC)); + } + } + + private void satisfyWaiterWithNewConnection() { + // This method MUST be called from within a thread-safe block/method. + if (activeConnections.get() < maxConnectionsPerRoute) { + Promise waiter = pendingAcquirers.poll(); + if (waiter != null) { + // A waiter exists, and we have capacity, create a new connection for them. + Future newConnectionFuture = createNewConnection(); + newConnectionFuture.addListener(future -> { + if (future.isSuccess()) { + waiter.trySuccess((Channel) future.getNow()); + } else { + waiter.tryFailure(future.cause()); + } + }); + } + } + } + + private Future createNewConnection() { + // This method MUST be called from within a thread-safe block/method. + activeConnections.incrementAndGet(); + Bootstrap newConnectionBootstrap = bootstrap.clone(); + + newConnectionBootstrap.handler(new ChannelInitializer() { + @Override + public void initChannel(Channel channel) throws SSLException { + ChannelPipeline pipeline = channel.pipeline(); + // Test whether proxying should be applied to this Channel. If so, add it. + boolean hasProxy = channelInitializationProxyHandler.test(route); + if (hasProxy) { + ProxyHandler proxyHandler = channelInitializationProxyHandler.createProxy(proxyChallenges); + pipeline.addFirst(PROXY, proxyHandler); + } + + // Add SSL handling if the request is HTTPS. + if (isHttps) { + InetSocketAddress inetSocketAddress = (InetSocketAddress) route; + SslContext ssl = buildSslContext(); + // SSL handling is added last here. This is done as proxying could require SSL handling too. + channel.pipeline() + .addLast(SSL, ssl.newHandler(channel.alloc(), inetSocketAddress.getHostString(), + inetSocketAddress.getPort())); + channel.pipeline().addLast(SSL_INITIALIZER, new Netty4SslInitializationHandler()); + } + + //TODO: is this actually needed? + //WARNING: An exceptionCaught() event was fired, and it reached at the tail of the pipeline. + // It usually means the last handler in the pipeline did not handle the exception. + //io.netty.handler.proxy.HttpProxyHandler$HttpProxyConnectException: http, none, + // localhost/127.0.0.1:53330 => localhost/127.0.0.1:53329, Proxy Authentication Required; {"status":407} + // This handler acts as the final backstop for the connection setup pipeline. + // It catches any exceptions that occur during setup (like ProxyConnectException) + // and prevents them from logging the "unhandled" warning. + pipeline.addLast("connectionSetupExceptionHandler", new ChannelInboundHandlerAdapter() { + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + ctx.close(); + } + }); + } + }); + + Promise promise = newConnectionBootstrap.config().group().next().newPromise(); + newConnectionBootstrap.connect(route).addListener(future -> { + if (future.isSuccess()) { + Channel newChannel = ((ChannelFuture) future).channel(); + ProxyHandler proxyHandler = newChannel.pipeline().get(ProxyHandler.class); + + if (proxyHandler != null) { + // Wait for the proxy handshake to complete if proxy is being used. + proxyHandler.connectFuture().addListener(proxyFuture -> { + if (proxyFuture.isSuccess()) { + newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); + newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); + promise.setSuccess(newChannel); + } else { + promise.setFailure(proxyFuture.cause()); + newChannel.close(); + } + }); + } else { + newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); + newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); + promise.setSuccess(newChannel); + } + } else { + // Connection failed, decrement the counter and satisfy a waiter if possible. + // This listener can be called on a different thread, so we must synchronize + // to safely interact with the pool state. + synchronized (this) { + activeConnections.decrementAndGet(); + satisfyWaiterWithNewConnection(); + } + promise.setFailure(future.cause()); + } + }); + return promise; + } + + private SslContext buildSslContext() throws SSLException { + SslContextBuilder sslContextBuilder + = SslContextBuilder.forClient().endpointIdentificationAlgorithm("HTTPS"); + if (maximumHttpVersion == HttpProtocolVersion.HTTP_2) { + // If HTTP/2 is the maximum version, we need to ensure that ALPN is enabled. + SslProvider sslProvider = SslContext.defaultClientProvider(); + ApplicationProtocolConfig.SelectorFailureBehavior selectorBehavior; + ApplicationProtocolConfig.SelectedListenerFailureBehavior selectedBehavior; + if (sslProvider == SslProvider.JDK) { + selectorBehavior = ApplicationProtocolConfig.SelectorFailureBehavior.FATAL_ALERT; + selectedBehavior = ApplicationProtocolConfig.SelectedListenerFailureBehavior.FATAL_ALERT; + } else { + // Netty OpenSslContext doesn't support FATAL_ALERT, use NO_ADVERTISE and ACCEPT + // instead. + selectorBehavior = ApplicationProtocolConfig.SelectorFailureBehavior.NO_ADVERTISE; + selectedBehavior = ApplicationProtocolConfig.SelectedListenerFailureBehavior.ACCEPT; + } + + sslContextBuilder.ciphers(Http2SecurityUtil.CIPHERS, SupportedCipherSuiteFilter.INSTANCE) + .applicationProtocolConfig( + new ApplicationProtocolConfig(ApplicationProtocolConfig.Protocol.ALPN, selectorBehavior, + selectedBehavior, ApplicationProtocolNames.HTTP_2, ApplicationProtocolNames.HTTP_1_1)); + } + if (sslContextModifier != null) { + // Allow the caller to modify the SslContextBuilder before it is built. + sslContextModifier.accept(sslContextBuilder); + } + + return sslContextBuilder.build(); + } + + private boolean isHealthy(Channel channel) { + if (!channel.isActive() || !channel.isWritable()) { + return false; + } + + if (maxLifetimeNanos > 0) { + OffsetDateTime creationTime = channel.attr(CHANNEL_CREATION_TIME).get(); + OffsetDateTime now = OffsetDateTime.now(ZoneOffset.UTC); + if (creationTime != null && Duration.between(creationTime, now).toNanos() >= maxLifetimeNanos) { + return false; + } + } + return true; + } + + synchronized void close() { + Channel channel; + while ((channel = idleConnections.poll()) != null) { + channel.close(); + } + Promise waiter; + while ((waiter = pendingAcquirers.poll()) != null) { + waiter.tryFailure(new IOException(CLOSED_POOL_ERROR_MESSAGE)); + } + } + } + +} diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 74cf9911a145..cfeba45dc818 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -44,45 +44,40 @@ public Netty4EagerConsumeChannelHandler(CountDownLatch latch, IOExceptionChecked @Override public void channelRead(ChannelHandlerContext ctx, Object msg) { - ByteBuf buf = null; - if (msg instanceof ByteBufHolder) { - buf = ((ByteBufHolder) msg).content(); - } else if (msg instanceof ByteBuf) { - buf = (ByteBuf) msg; - } + try { + ByteBuf buf = (msg instanceof ByteBufHolder) ? ((ByteBufHolder) msg).content() : (ByteBuf) msg; - if (buf != null && buf.isReadable()) { - try { + if (buf != null && buf.isReadable()) { byteBufConsumer.accept(buf); - } catch (IOException | RuntimeException ex) { - ReferenceCountUtil.release(buf); - ctx.close(); - return; } - } - if (isHttp2) { - lastRead = msg instanceof Http2DataFrame && ((Http2DataFrame) msg).isEndStream(); - } else { - lastRead = msg instanceof LastHttpContent; + if (isHttp2) { + lastRead = msg instanceof Http2DataFrame && ((Http2DataFrame) msg).isEndStream(); + } else { + lastRead = msg instanceof LastHttpContent; + } + ctx.fireChannelRead(msg); + + } catch (IOException | RuntimeException ex) { + ReferenceCountUtil.release(msg); + ctx.fireExceptionCaught(ex); + cleanup(ctx); } - ctx.fireChannelRead(msg); } @Override public void channelReadComplete(ChannelHandlerContext ctx) { ctx.fireChannelReadComplete(); if (lastRead) { - latch.countDown(); - ctx.close(); + cleanup(ctx); } } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; - latch.countDown(); - ctx.close(); + ctx.fireExceptionCaught(cause); + cleanup(ctx); } Throwable channelException() { @@ -92,13 +87,21 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - latch.countDown(); + cleanup(ctx); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - latch.countDown(); + cleanup(ctx); ctx.fireChannelInactive(); } + + private void cleanup(ChannelHandlerContext ctx) { + if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { + ctx.pipeline().remove(this); + } + + latch.countDown(); + } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java index eee40c7c4fe7..8cae50f4a624 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java @@ -69,6 +69,11 @@ public final class Netty4HandlerNames { */ public static final String READ_ONE = "clientcore.readone"; + /** + * Name for the {@link Netty4PipelineCleanupHandler} + */ + public static final String PIPELINE_CLEANUP = "clientcore.pipelinecleanup"; + private Netty4HandlerNames() { } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java index 6a242ed03246..6f287b04f05e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java @@ -78,7 +78,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { byteBufConsumer.accept(buf); } catch (IOException | RuntimeException ex) { ReferenceCountUtil.release(buf); - ctx.close(); + exceptionCaught(ctx, ex); return; } } @@ -96,8 +96,8 @@ public void channelReadComplete(ChannelHandlerContext ctx) { latch.countDown(); if (lastRead) { ctx.pipeline().remove(this); - ctx.close(); } + ctx.fireChannelReadComplete(); } boolean isChannelConsumed() { @@ -108,7 +108,10 @@ boolean isChannelConsumed() { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; latch.countDown(); - ctx.close(); + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } + ctx.fireExceptionCaught(cause); } Throwable channelException() { @@ -119,14 +122,18 @@ Throwable channelException() { @Override public void channelUnregistered(ChannelHandlerContext ctx) { latch.countDown(); + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } ctx.fireChannelUnregistered(); - ctx.pipeline().remove(this); } @Override public void channelInactive(ChannelHandlerContext ctx) { latch.countDown(); + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } ctx.fireChannelInactive(); - ctx.pipeline().remove(this); } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java new file mode 100644 index 000000000000..2b72816cf271 --- /dev/null +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -0,0 +1,103 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package io.clientcore.http.netty4.implementation; + +import io.netty.channel.ChannelDuplexHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPipeline; +import io.netty.handler.codec.http.LastHttpContent; +import io.netty.handler.codec.http2.Http2DataFrame; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.ALPN; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.CHUNKED_WRITER; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.EAGER_CONSUME; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_CODEC; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_RESPONSE; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.READ_ONE; + +/** + * A handler that cleans up the pipeline after a request-response cycle and releases + * the channel back to the connection pool. + */ +public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { + + private final Netty4ConnectionPool connectionPool; + private final AtomicBoolean cleanedUp = new AtomicBoolean(false); + + private static final List HANDLERS_TO_REMOVE; + + static { + List handlers = new ArrayList<>(); + handlers.add(PROGRESS_AND_TIMEOUT); + handlers.add(HTTP_RESPONSE); + handlers.add(HTTP_CODEC); + handlers.add(ALPN); + handlers.add(CHUNKED_WRITER); + handlers.add(EAGER_CONSUME); + handlers.add(READ_ONE); + HANDLERS_TO_REMOVE = Collections.unmodifiableList(handlers); + } + + public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { + this.connectionPool = connectionPool; + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + // Inspect the message to see if it's the end of the response stream. + // We do this BEFORE propagating the message to avoid race conditions with + // downstream handlers that will release the message. + boolean lastRead = false; + if (msg instanceof LastHttpContent) { + lastRead = true; + } else if (msg instanceof Http2DataFrame) { + lastRead = ((Http2DataFrame) msg).isEndStream(); + } + + if (lastRead) { + ctx.channel().eventLoop().execute(() -> cleanup(ctx)); + } + + super.channelRead(ctx, msg); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + // An exception has occurred, which means the channel is likely in a bad state. + // We handle this by closing the channel. This prevents it from being + // returned to the connection pool. + ctx.close(); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + cleanup(ctx); + super.channelInactive(ctx); + } + + public void cleanup(ChannelHandlerContext ctx) { + if (!cleanedUp.compareAndSet(false, true)) { + return; + } + + ChannelPipeline pipeline = ctx.channel().pipeline(); + for (String handlerName : HANDLERS_TO_REMOVE) { + if (pipeline.get(handlerName) != null) { + pipeline.remove(handlerName); + } + } + + if (pipeline.get(Netty4PipelineCleanupHandler.class) != null) { + pipeline.remove(this); + } + + connectionPool.release(ctx.channel()); + } +} diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java index 90e07fea13d6..bc637e04fc45 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java @@ -158,9 +158,10 @@ void writeTimeoutRunnable(ChannelHandlerContext ctx, boolean trackingWriteTimeou // No progress has been made since the last timeout event, channel has timed out. if (!closed) { disposeWriteTimeoutWatcher(); + // Fire the exception up the pipeline. The PipelineCleanupHandler will catch this + // and release the channel. We do not close the channel here. ctx.fireExceptionCaught(new TimeoutException( "Channel write operation timed out after " + writeTimeoutMillis + " milliseconds.")); - ctx.close(); closed = true; } } @@ -204,7 +205,6 @@ void responseTimedOut(ChannelHandlerContext ctx, boolean trackingResponseTimeout disposeResponseTimeoutWatcher(); ctx.fireExceptionCaught( new TimeoutException("Channel response timed out after " + responseTimeoutMillis + " milliseconds.")); - ctx.close(); closed = true; } } @@ -277,7 +277,6 @@ void readTimeoutRunnable(ChannelHandlerContext ctx, boolean trackingReadTimeout) disposeReadTimeoutWatcher(); ctx.fireExceptionCaught( new TimeoutException("Channel read timed out after " + readTimeoutMillis + " milliseconds.")); - ctx.close(); closed = true; } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/HttpResponseDrainsBufferTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/HttpResponseDrainsBufferTests.java index 5f38223d84d6..06959e521553 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/HttpResponseDrainsBufferTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/HttpResponseDrainsBufferTests.java @@ -4,10 +4,12 @@ package io.clientcore.http.netty4.implementation; import io.clientcore.core.http.client.HttpClient; +import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.http.models.HttpMethod; import io.clientcore.core.http.models.HttpRequest; import io.clientcore.core.http.models.Response; import io.clientcore.core.models.binarydata.BinaryData; +import io.clientcore.core.shared.LocalTestServer; import io.clientcore.core.utils.IOExceptionCheckedConsumer; import io.clientcore.core.utils.SharedExecutorService; import io.clientcore.http.netty4.NettyHttpClientProvider; @@ -24,6 +26,7 @@ import org.junit.jupiter.api.parallel.ExecutionMode; import org.junit.jupiter.api.parallel.Isolated; +import javax.servlet.ServletException; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; @@ -56,7 +59,8 @@ @Execution(ExecutionMode.SAME_THREAD) public class HttpResponseDrainsBufferTests { private static ResourceLeakDetector.Level originalLevel; - private static final String URL = NettyHttpClientLocalTestServer.getServer().getUri() + LONG_BODY_PATH; + private static String url; + private static LocalTestServer server; private ResourceLeakDetectorFactory originalLeakDetectorFactory; private final TestResourceLeakDetectorFactory testResourceLeakDetectorFactory @@ -64,6 +68,25 @@ public class HttpResponseDrainsBufferTests { @BeforeAll public static void startTestServer() { + server = new LocalTestServer(HttpProtocolVersion.HTTP_1_1, false, (req, resp, requestBody) -> { + if ("GET".equalsIgnoreCase(req.getMethod()) && LONG_BODY_PATH.equals(req.getServletPath())) { + resp.setStatus(200); + resp.setContentType("application/octet-stream"); + resp.setContentLength(LONG_BODY.length); + try { + resp.getOutputStream().write(LONG_BODY); + resp.flushBuffer(); + } catch (IOException e) { + throw new ServletException(e); + } + } else { + resp.sendError(404, "Endpoint not found."); + } + }); + + server.start(); + url = server.getUri() + LONG_BODY_PATH; + originalLevel = ResourceLeakDetector.getLevel(); ResourceLeakDetector.setLevel(ResourceLeakDetector.Level.PARANOID); } @@ -81,7 +104,13 @@ public void resetLeakDetectorFactory() { @AfterAll public static void stopTestServer() { - ResourceLeakDetector.setLevel(originalLevel); + if (server != null) { + server.stop(); + } + + if (originalLevel != null) { + ResourceLeakDetector.setLevel(originalLevel); + } } @Test @@ -163,7 +192,7 @@ private void runScenario(IOExceptionCheckedConsumer> respon try { limiter.acquire(); responseConsumer - .accept(httpClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(URL))); + .accept(httpClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(url))); } finally { limiter.release(); } @@ -192,7 +221,7 @@ private void runScenario(IOExceptionCheckedConsumer> respon public void closingHttpResponseIsIdempotent() throws InterruptedException { HttpClient httpClient = new NettyHttpClientProvider().getSharedInstance(); - Response response = httpClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(URL)); + Response response = httpClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(url)); response.close(); Thread.sleep(1_000); response.close(); diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java new file mode 100644 index 000000000000..ccd4f04a9f48 --- /dev/null +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java @@ -0,0 +1,222 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package io.clientcore.http.netty4.implementation; + +import io.clientcore.core.http.client.HttpProtocolVersion; +import io.clientcore.core.models.CoreException; +import io.clientcore.core.shared.LocalTestServer; +import io.netty.bootstrap.Bootstrap; +import io.netty.channel.Channel; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.util.concurrent.Future; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests for {@link Netty4ConnectionPool}. + */ +@Timeout(value = 3, unit = TimeUnit.MINUTES) +public class Netty4ConnectionPoolTests { + + private static LocalTestServer server; + private static EventLoopGroup eventLoopGroup; + private static Bootstrap bootstrap; + private static SocketAddress serverAddress; + + @BeforeAll + public static void startTestServerAndEventLoopGroup() { + server = NettyHttpClientLocalTestServer.getServer(); + server.start(); + eventLoopGroup = new NioEventLoopGroup(2); + bootstrap = new Bootstrap().group(eventLoopGroup).channel(NioSocketChannel.class); + serverAddress = new InetSocketAddress("localhost", server.getPort()); + } + + @AfterAll + public static void stopTestServerAndEventLoopGroup() { + if (server != null) { + server.stop(); + } + if (eventLoopGroup != null && !eventLoopGroup.isShuttingDown()) { + eventLoopGroup.shutdownGracefully().awaitUninterruptibly(); + } + } + + private Netty4ConnectionPool createPool(int maxConnections, Duration idleTimeout, Duration maxLifetime, + Duration pendingAcquireTimeout, int maxPendingAcquires) { + return new Netty4ConnectionPool(bootstrap, new ChannelInitializationProxyHandler(null), null, // No SSL context modifier needed + maxConnections, idleTimeout, maxLifetime, pendingAcquireTimeout, maxPendingAcquires, + HttpProtocolVersion.HTTP_1_1); + } + + @Test + public void testAcquireAndRelease() throws IOException { + try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { + Future future = pool.acquire(serverAddress, false); + Channel channel = future.awaitUninterruptibly().getNow(); + assertNotNull(channel); + assertTrue(channel.isActive()); + pool.release(channel); + } + } + + @Test + public void testConnectionIsReusedForSameRemoteAddress() throws IOException { + try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { + Future future1 = pool.acquire(serverAddress, false); + Channel channel1 = future1.awaitUninterruptibly().getNow(); + pool.release(channel1); + + Future future2 = pool.acquire(serverAddress, false); + Channel channel2 = future2.awaitUninterruptibly().getNow(); + assertSame(channel1, channel2); + pool.release(channel2); + } + } + + @Test + public void testConnectionPoolSizeEnforced() throws IOException, InterruptedException { + final int maxConnections = 5; + try (Netty4ConnectionPool pool + = createPool(maxConnections, Duration.ofSeconds(10), null, Duration.ofSeconds(10), maxConnections)) { + List channels = new ArrayList<>(); + for (int i = 0; i < maxConnections; i++) { + channels.add(pool.acquire(serverAddress, false).awaitUninterruptibly().getNow()); + } + assertEquals(maxConnections, channels.size()); + + Future pendingFuture = pool.acquire(serverAddress, false); + Thread.sleep(100); + assertFalse(pendingFuture.isDone()); + + pool.release(channels.get(0)); + Channel pendingChannel = pendingFuture.awaitUninterruptibly().getNow(); + assertSame(channels.get(0), pendingChannel); + + for (int i = 1; i < channels.size(); i++) { + pool.release(channels.get(i)); + } + } + } + + @Test + public void testPendingAcquireTimeout() throws IOException, InterruptedException { + try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofMillis(100), 1)) { + Channel channel = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + + Future timeoutFuture = pool.acquire(serverAddress, false); + + assertTrue(timeoutFuture.await(500, TimeUnit.MILLISECONDS)); + + assertFalse(timeoutFuture.isSuccess()); + assertInstanceOf(CoreException.class, timeoutFuture.cause()); + assertTrue(timeoutFuture.cause().getMessage().contains("Connection acquisition timed out")); + + pool.release(channel); + } + } + + @Test + public void testIdleConnectionIsCleanedUp() throws IOException, InterruptedException { + try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { + Channel channel1 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + pool.release(channel1); + Thread.sleep(31000); // Wait for cleanup task to run (interval is 30s) + assertFalse(channel1.isActive()); + + Channel channel2 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + assertNotSame(channel1, channel2); + pool.release(channel2); + } + } + + @Test + public void testMaxConnectionLifetimeEnforced() throws IOException, InterruptedException { + try (Netty4ConnectionPool pool + = createPool(1, Duration.ofSeconds(10), Duration.ofMillis(500), Duration.ofSeconds(10), 1)) { + Channel channel1 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + Thread.sleep(600); + pool.release(channel1); + Thread.sleep(100); // Give a moment for close to propagate + assertFalse(channel1.isActive()); + + Channel channel2 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + assertNotSame(channel1, channel2); + pool.release(channel2); + } + } + + @Test + public void testUnhealthyConnectionIsDiscarded() throws IOException { + try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { + Channel channel1 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + pool.release(channel1); + channel1.close().awaitUninterruptibly(); + + Channel channel2 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + assertNotNull(channel2); + assertTrue(channel2.isActive()); + assertNotSame(channel1, channel2); + pool.release(channel2); + } + } + + @Test + public void testAcquireOnClosedPoolFails() throws IOException { + Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1); + pool.close(); + assertThrows(IllegalStateException.class, () -> pool.acquire(serverAddress, false)); + } + + @Test + public void testSeparatePoolsForSeparateRemoteAddresses() throws IOException { + LocalTestServer route1Server = new LocalTestServer(HttpProtocolVersion.HTTP_1_1, false, null); + LocalTestServer route2Server = new LocalTestServer(HttpProtocolVersion.HTTP_1_1, false, null); + + try { + route1Server.start(); + route2Server.start(); + + SocketAddress address1 = new InetSocketAddress("localhost", route1Server.getPort()); + SocketAddress address2 = new InetSocketAddress("localhost", route2Server.getPort()); + + try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { + Channel channel1 = pool.acquire(address1, false).awaitUninterruptibly().getNow(); + assertNotNull(channel1); + + Channel channel2 = pool.acquire(address2, false).awaitUninterruptibly().getNow(); + assertNotNull(channel2); + + assertNotSame(channel1, channel2); + + pool.release(channel1); + pool.release(channel2); + } + } finally { + route1Server.stop(); + route2Server.stop(); + } + } +} diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java index 67df3b7ac43e..3563d03c3813 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java @@ -23,7 +23,6 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Stream; import static io.clientcore.http.netty4.TestUtils.assertArraysEqual; @@ -31,7 +30,9 @@ import static io.netty.buffer.Unpooled.wrappedBuffer; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests {@link Netty4ChannelInputStream}. @@ -39,17 +40,17 @@ @Timeout(value = 3, unit = TimeUnit.MINUTES) public class Netty4Http11ChannelInputStreamTests { @Test - public void nullEagerContentResultsInEmptyInitialCurrentBuffer() { + public void nullEagerContentResultsInEmptyInitialCurrentBuffer() throws IOException { try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(null, createCloseableChannel(), false)) { + = new Netty4ChannelInputStream(null, createCloseableChannel(), false, null)) { assertEquals(0, channelInputStream.getCurrentBuffer().length); } } @Test - public void emptyEagerContentResultsInEmptyInitialCurrentBuffer() { + public void emptyEagerContentResultsInEmptyInitialCurrentBuffer() throws IOException { try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(new ByteArrayOutputStream(), createCloseableChannel(), false)) { + = new Netty4ChannelInputStream(new ByteArrayOutputStream(), createCloseableChannel(), false, null)) { assertEquals(0, channelInputStream.getCurrentBuffer().length); } } @@ -64,7 +65,7 @@ public void readConsumesCurrentBufferAndHasNoMoreData() throws IOException { // MockChannels aren't active by default, so once the eagerContent is consumed the stream will be done. Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(eagerContent, new MockChannel(), false); + = new Netty4ChannelInputStream(eagerContent, new MockChannel(), false, null); // Make sure the Netty4ChannelInputStream copied the eager content correctly. assertArraysEqual(expected, channelInputStream.getCurrentBuffer()); @@ -96,7 +97,7 @@ public void readConsumesCurrentBufferAndRequestsMoreData() throws IOException { handler.channelRead(ctx, wrappedBuffer(expected, 16, 16)); handler.channelRead(ctx, LastHttpContent.EMPTY_LAST_CONTENT); handler.channelReadComplete(ctx); - }), false); + }), false, null); int index = 0; byte[] actual = new byte[32]; @@ -118,7 +119,7 @@ public void multipleSmallerSkips() throws IOException { // MockChannels aren't active by default, so once the eagerContent is consumed the stream will be done. try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(eagerContent, createCloseableChannel(), false)) { + = new Netty4ChannelInputStream(eagerContent, createCloseableChannel(), false, null)) { long skipped = channelInputStream.skip(16); assertEquals(16, skipped); @@ -141,7 +142,7 @@ public void largeReadTriggersMultipleChannelReads() throws IOException { ThreadLocalRandom.current().nextBytes(expected); try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(null, createChannelThatReads8Kb(expected), false)) { + = new Netty4ChannelInputStream(null, createChannelThatReads8Kb(expected), false, null)) { byte[] actual = new byte[8192]; int read = channelInputStream.read(actual); @@ -162,7 +163,7 @@ public void largeSkipTriggersMultipleChannelReads() throws IOException { ThreadLocalRandom.current().nextBytes(expected); try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(null, createChannelThatReads8Kb(expected), false)) { + = new Netty4ChannelInputStream(null, createChannelThatReads8Kb(expected), false, null)) { long skipped = channelInputStream.skip(8192); assertEquals(8192, skipped); @@ -172,21 +173,22 @@ public void largeSkipTriggersMultipleChannelReads() throws IOException { } @Test - public void closingStreamClosesChannel() { - AtomicInteger closeCount = new AtomicInteger(); - AtomicInteger disconnectCount = new AtomicInteger(); + public void closingStreamTriggersOnCloseCallback() throws IOException { + AtomicBoolean onCloseCalled = new AtomicBoolean(false); - new Netty4ChannelInputStream(null, - createCloseableChannel(closeCount::incrementAndGet, disconnectCount::incrementAndGet), false).close(); + try (Netty4ChannelInputStream channelInputStream + = new Netty4ChannelInputStream(null, createCloseableChannel(), false, () -> onCloseCalled.set(true))) { + assertNotNull(channelInputStream); + } - assertEquals(1, closeCount.get()); + assertTrue(onCloseCalled.get()); } @ParameterizedTest @MethodSource("errorSupplier") public void streamPropagatesErrorFiredInChannel(Throwable expected) { InputStream inputStream - = new Netty4ChannelInputStream(null, createPartialReadThenErrorChannel(expected), false); + = new Netty4ChannelInputStream(null, createPartialReadThenErrorChannel(expected), false, null); Throwable actual = assertThrows(Throwable.class, () -> inputStream.read(new byte[8192])); From 784ade2d3f0a21ba0c57d422780f6b7305b83330 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sun, 29 Jun 2025 20:25:51 +0300 Subject: [PATCH 02/57] fix duplicate http response handler in the pipeline --- .../java/io/clientcore/http/netty4/NettyHttpClient.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index c77538a6d852..eacb56081df6 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -259,16 +259,18 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); } - pipeline.addLast(HTTP_RESPONSE, new Netty4ResponseHandler(request, responseReference, errorReference, latch)); - // The SslHandler is already in the pipeline if this is an HTTPS request, as it's added // by the connection pool during the initial connection setup. The SSL handshake is also // guaranteed to be complete by the time we get the channel because the Netty4AlpnHandler // reacts to the result of the ALPN negotiation that happened during the SSL handshake. if (isHttps) { + // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. pipeline.addAfter(Netty4HandlerNames.SSL_INITIALIZER, Netty4HandlerNames.ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, latch)); } else { + // If there isn't an SslHandler, we can send the request immediately. + // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. + pipeline.addLast(HTTP_RESPONSE, new Netty4ResponseHandler(request, responseReference, errorReference, latch)); String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); sendHttp11Request(request, channel, errorReference).addListener(f -> { From 7edd7a9026050f4f4c56ab947db475f57a2cb32d Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sun, 29 Jun 2025 20:50:36 +0300 Subject: [PATCH 03/57] apply ALPN after the correct handler --- .../main/java/io/clientcore/http/netty4/NettyHttpClient.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index eacb56081df6..c15d510d8e71 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -19,7 +19,6 @@ import io.clientcore.http.netty4.implementation.Netty4ChannelBinaryData; import io.clientcore.http.netty4.implementation.Netty4ConnectionPool; import io.clientcore.http.netty4.implementation.Netty4EagerConsumeChannelHandler; -import io.clientcore.http.netty4.implementation.Netty4HandlerNames; import io.clientcore.http.netty4.implementation.Netty4PipelineCleanupHandler; import io.clientcore.http.netty4.implementation.Netty4ProgressAndTimeoutHandler; import io.clientcore.http.netty4.implementation.Netty4ResponseHandler; @@ -43,10 +42,12 @@ import static io.clientcore.core.utils.ServerSentEventUtils.attemptRetry; import static io.clientcore.core.utils.ServerSentEventUtils.processTextEventStream; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.ALPN; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_CODEC; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_RESPONSE; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PIPELINE_CLEANUP; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; import static io.clientcore.http.netty4.implementation.Netty4Utility.awaitLatch; import static io.clientcore.http.netty4.implementation.Netty4Utility.createCodec; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; @@ -265,7 +266,7 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, // reacts to the result of the ALPN negotiation that happened during the SSL handshake. if (isHttps) { // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. - pipeline.addAfter(Netty4HandlerNames.SSL_INITIALIZER, Netty4HandlerNames.ALPN, + pipeline.addAfter(SSL, ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, latch)); } else { // If there isn't an SslHandler, we can send the request immediately. From ce50c9ebc43cab1101d364af26a3ae6da8d84635 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Mon, 30 Jun 2025 09:03:05 +0300 Subject: [PATCH 04/57] fix formatting --- .../java/io/clientcore/http/netty4/NettyHttpClient.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index c15d510d8e71..87e12074f190 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -266,12 +266,12 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, // reacts to the result of the ALPN negotiation that happened during the SSL handshake. if (isHttps) { // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. - pipeline.addAfter(SSL, ALPN, - new Netty4AlpnHandler(request, responseReference, errorReference, latch)); + pipeline.addAfter(SSL, ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, latch)); } else { // If there isn't an SslHandler, we can send the request immediately. // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. - pipeline.addLast(HTTP_RESPONSE, new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + pipeline.addLast(HTTP_RESPONSE, + new Netty4ResponseHandler(request, responseReference, errorReference, latch)); String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); sendHttp11Request(request, channel, errorReference).addListener(f -> { From 6b73bb3681b0796e947de2d8ba93a9191d25701e Mon Sep 17 00:00:00 2001 From: George Banasios Date: Mon, 30 Jun 2025 12:06:42 +0300 Subject: [PATCH 05/57] fix Netty4ChannelBinaryData --- .../http/netty4/implementation/Netty4ChannelBinaryData.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 654587d8b6ad..371570fb3507 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -124,6 +124,8 @@ public void writeTo(OutputStream outputStream) { channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); channel.config().setAutoRead(true); + channel.eventLoop().execute(channel::read); + awaitLatch(latch); streamDrained.set(true); @@ -192,6 +194,7 @@ private void drainStream() { channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); channel.config().setAutoRead(true); + channel.eventLoop().execute(channel::read); awaitLatch(latch); Throwable exception = handler.channelException(); From c8693a10dbf7d5d600ffad27d14170aca1060003 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Mon, 30 Jun 2025 13:11:35 +0300 Subject: [PATCH 06/57] fix connection pool release race condition --- .../ChannelInitializationProxyHandler.java | 2 +- .../Netty4ChannelBinaryData.java | 8 ++ .../implementation/Netty4ConnectionPool.java | 90 ++++++++----------- .../Netty4PipelineCleanupHandler.java | 9 +- .../Netty4ConnectionPoolTests.java | 2 +- 5 files changed, 52 insertions(+), 59 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/ChannelInitializationProxyHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/ChannelInitializationProxyHandler.java index 56b79d04d92c..9e0717730775 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/ChannelInitializationProxyHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/ChannelInitializationProxyHandler.java @@ -67,7 +67,7 @@ public boolean test(SocketAddress socketAddress) { InetSocketAddress inetSocketAddress = (InetSocketAddress) socketAddress; String hostString = inetSocketAddress.getHostString(); - return hostString != null && nonProxyHostsPattern.matcher(hostString).matches(); + return hostString != null && !nonProxyHostsPattern.matcher(hostString).matches(); } /** diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 371570fb3507..0f2351c63a69 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -118,6 +118,10 @@ public void writeTo(OutputStream outputStream) { eagerContent.reset(); } + if (!channel.isActive()) { + return; + } + CountDownLatch latch = new CountDownLatch(1); Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, buf -> buf.readBytes(outputStream, buf.readableBytes()), isHttp2); @@ -188,6 +192,10 @@ private void drainStream() { return; } + if (!channel.isActive()) { + return; + } + CountDownLatch latch = new CountDownLatch(1); Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, buf -> buf.readBytes(eagerContent, buf.readableBytes()), isHttp2); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 71513939bccd..1acfea0b96e9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -10,8 +10,6 @@ import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; @@ -272,6 +270,17 @@ synchronized Future acquire() { } synchronized void release(Channel channel) { + if (!isHealthy(channel)) { + activeConnections.decrementAndGet(); + channel.close(); + + // Since a connection slot has been freed, we should try to create a new, + // healthy connection for any request that might be waiting. + satisfyWaiterWithNewConnection(); + return; + } + + // The channel is healthy. Now, check if anyone is waiting for a connection. Promise waiter; while ((waiter = pendingAcquirers.poll()) != null) { // A waiter exists, hand over this channel directly. @@ -281,15 +290,6 @@ synchronized void release(Channel channel) { } } - // No waiters to hand off to. Check channel health before returning to pool - if (!isHealthy(channel)) { - activeConnections.decrementAndGet(); - channel.close(); - // A slot has freed up. Proactively try to satisfy a pending waiter. - satisfyWaiterWithNewConnection(); - return; - } - // Channel is healthy and no waiters, return it to the idle queue. activeConnections.decrementAndGet(); idleConnections.offer(channel); @@ -342,56 +342,44 @@ public void initChannel(Channel channel) throws SSLException { inetSocketAddress.getPort())); channel.pipeline().addLast(SSL_INITIALIZER, new Netty4SslInitializationHandler()); } - - //TODO: is this actually needed? - //WARNING: An exceptionCaught() event was fired, and it reached at the tail of the pipeline. - // It usually means the last handler in the pipeline did not handle the exception. - //io.netty.handler.proxy.HttpProxyHandler$HttpProxyConnectException: http, none, - // localhost/127.0.0.1:53330 => localhost/127.0.0.1:53329, Proxy Authentication Required; {"status":407} - // This handler acts as the final backstop for the connection setup pipeline. - // It catches any exceptions that occur during setup (like ProxyConnectException) - // and prevents them from logging the "unhandled" warning. - pipeline.addLast("connectionSetupExceptionHandler", new ChannelInboundHandlerAdapter() { - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - ctx.close(); - } - }); } }); Promise promise = newConnectionBootstrap.config().group().next().newPromise(); newConnectionBootstrap.connect(route).addListener(future -> { - if (future.isSuccess()) { - Channel newChannel = ((ChannelFuture) future).channel(); - ProxyHandler proxyHandler = newChannel.pipeline().get(ProxyHandler.class); - - if (proxyHandler != null) { - // Wait for the proxy handshake to complete if proxy is being used. - proxyHandler.connectFuture().addListener(proxyFuture -> { - if (proxyFuture.isSuccess()) { - newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); - newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); - promise.setSuccess(newChannel); - } else { - promise.setFailure(proxyFuture.cause()); - newChannel.close(); - } - }); - } else { - newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); - newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); - promise.setSuccess(newChannel); - } - } else { - // Connection failed, decrement the counter and satisfy a waiter if possible. - // This listener can be called on a different thread, so we must synchronize - // to safely interact with the pool state. + if (!future.isSuccess()) { synchronized (this) { activeConnections.decrementAndGet(); satisfyWaiterWithNewConnection(); } promise.setFailure(future.cause()); + return; + } + + Channel newChannel = ((ChannelFuture) future).channel(); + ProxyHandler proxyHandler = (ProxyHandler) newChannel.pipeline().get(PROXY); + + if (proxyHandler != null) { + // Wait for the proxy handshake to complete if proxy is being used. + proxyHandler.connectFuture().addListener(proxyFuture -> { + if (proxyFuture.isSuccess()) { + newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); + newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); + promise.setSuccess(newChannel); + } else { + promise.setFailure(proxyFuture.cause()); + newChannel.close(); + + synchronized (this) { + activeConnections.decrementAndGet(); + satisfyWaiterWithNewConnection(); + } + } + }); + } else { + newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); + newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); + promise.setSuccess(newChannel); } }); return promise; diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 2b72816cf271..29c71a7261af 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -51,9 +51,8 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - // Inspect the message to see if it's the end of the response stream. - // We do this BEFORE propagating the message to avoid race conditions with - // downstream handlers that will release the message. + ctx.fireChannelRead(msg); + boolean lastRead = false; if (msg instanceof LastHttpContent) { lastRead = true; @@ -62,10 +61,8 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } if (lastRead) { - ctx.channel().eventLoop().execute(() -> cleanup(ctx)); + cleanup(ctx); } - - super.channelRead(ctx, msg); } @Override diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java index ccd4f04a9f48..1eed9b83787b 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java @@ -37,7 +37,7 @@ /** * Tests for {@link Netty4ConnectionPool}. */ -@Timeout(value = 3, unit = TimeUnit.MINUTES) +@Timeout(value = 1, unit = TimeUnit.MINUTES) public class Netty4ConnectionPoolTests { private static LocalTestServer server; From e1525d87117fd993975b1476d9b29431bb30f7d1 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Mon, 30 Jun 2025 18:04:36 +0300 Subject: [PATCH 07/57] adjust chunked request --- .../netty4/implementation/Netty4Utility.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java index bc0daf8efe8c..8952c0748a3d 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java @@ -293,19 +293,21 @@ public static ChannelFuture sendHttp11Request(HttpRequest request, Channel chann private static ChannelFuture sendChunkedHttp11(Channel channel, ChunkedInput chunkedInput, io.netty.handler.codec.http.HttpRequest initialLineAndHeaders, AtomicReference errorReference) { - if (channel.pipeline().get(Netty4HandlerNames.CHUNKED_WRITER) == null) { - // Add the ChunkedWriteHandler which will handle sending the chunkedInput. - channel.pipeline() - .addAfter(Netty4HandlerNames.HTTP_CODEC, Netty4HandlerNames.CHUNKED_WRITER, new ChunkedWriteHandler()); - } - Throwable error = errorReference.get(); if (error != null) { return channel.newFailedFuture(error); } + if (channel.pipeline().get(Netty4HandlerNames.CHUNKED_WRITER) == null) { + channel.pipeline() + .addBefore(Netty4HandlerNames.HTTP_CODEC, Netty4HandlerNames.CHUNKED_WRITER, new ChunkedWriteHandler()); + } + + channel.config().setAutoRead(true); + channel.write(initialLineAndHeaders); - return channel.writeAndFlush(new HttpChunkedInput(chunkedInput)); + + return channel.writeAndFlush(chunkedInput); } /** From f26b6c373452300ada9c9d8b8401b99d41749903 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Mon, 30 Jun 2025 18:16:47 +0300 Subject: [PATCH 08/57] Revert "adjust chunked request" This reverts commit e1525d87117fd993975b1476d9b29431bb30f7d1. --- .../netty4/implementation/Netty4Utility.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java index 8952c0748a3d..bc0daf8efe8c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java @@ -293,21 +293,19 @@ public static ChannelFuture sendHttp11Request(HttpRequest request, Channel chann private static ChannelFuture sendChunkedHttp11(Channel channel, ChunkedInput chunkedInput, io.netty.handler.codec.http.HttpRequest initialLineAndHeaders, AtomicReference errorReference) { - Throwable error = errorReference.get(); - if (error != null) { - return channel.newFailedFuture(error); - } - if (channel.pipeline().get(Netty4HandlerNames.CHUNKED_WRITER) == null) { + // Add the ChunkedWriteHandler which will handle sending the chunkedInput. channel.pipeline() - .addBefore(Netty4HandlerNames.HTTP_CODEC, Netty4HandlerNames.CHUNKED_WRITER, new ChunkedWriteHandler()); + .addAfter(Netty4HandlerNames.HTTP_CODEC, Netty4HandlerNames.CHUNKED_WRITER, new ChunkedWriteHandler()); } - channel.config().setAutoRead(true); + Throwable error = errorReference.get(); + if (error != null) { + return channel.newFailedFuture(error); + } channel.write(initialLineAndHeaders); - - return channel.writeAndFlush(chunkedInput); + return channel.writeAndFlush(new HttpChunkedInput(chunkedInput)); } /** From 1602d4bc6875c0be7082313c17c498c7dce516c2 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Tue, 1 Jul 2025 12:07:28 +0300 Subject: [PATCH 09/57] fix proxy logic when acquiring a channel --- .../http/netty4/NettyHttpClient.java | 38 ++++++++++++-- .../http/netty4/NettyHttpClientBuilder.java | 10 +++- .../implementation/Netty4ConnectionPool.java | 30 +++++++---- .../Netty4ConnectionPoolKey.java | 51 +++++++++++++++++++ .../Netty4HttpProxyHandler.java | 2 +- .../Netty4ConnectionPoolTests.java | 39 +++++++------- 6 files changed, 136 insertions(+), 34 deletions(-) create mode 100644 sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolKey.java diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index 87e12074f190..e17670683fe2 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -6,6 +6,7 @@ import io.clientcore.core.http.client.HttpClient; import io.clientcore.core.http.models.HttpHeaderName; import io.clientcore.core.http.models.HttpRequest; +import io.clientcore.core.http.models.ProxyOptions; import io.clientcore.core.http.models.Response; import io.clientcore.core.http.models.ServerSentEventListener; import io.clientcore.core.instrumentation.logging.ClientLogger; @@ -15,9 +16,11 @@ import io.clientcore.core.utils.CoreUtils; import io.clientcore.core.utils.ProgressReporter; import io.clientcore.core.utils.ServerSentEventUtils; +import io.clientcore.http.netty4.implementation.ChannelInitializationProxyHandler; import io.clientcore.http.netty4.implementation.Netty4AlpnHandler; import io.clientcore.http.netty4.implementation.Netty4ChannelBinaryData; import io.clientcore.http.netty4.implementation.Netty4ConnectionPool; +import io.clientcore.http.netty4.implementation.Netty4ConnectionPoolKey; import io.clientcore.http.netty4.implementation.Netty4EagerConsumeChannelHandler; import io.clientcore.http.netty4.implementation.Netty4PipelineCleanupHandler; import io.clientcore.http.netty4.implementation.Netty4ProgressAndTimeoutHandler; @@ -67,14 +70,19 @@ class NettyHttpClient implements HttpClient { private final EventLoopGroup eventLoopGroup; private final Netty4ConnectionPool connectionPool; + private final ProxyOptions proxyOptions; + private final ChannelInitializationProxyHandler channelInitializationProxyHandler; private final long readTimeoutMillis; private final long responseTimeoutMillis; private final long writeTimeoutMillis; - NettyHttpClient(EventLoopGroup eventLoopGroup, Netty4ConnectionPool connectionPool, long readTimeoutMillis, + NettyHttpClient(EventLoopGroup eventLoopGroup, Netty4ConnectionPool connectionPool, ProxyOptions proxyOptions, + ChannelInitializationProxyHandler channelInitializationProxyHandler, long readTimeoutMillis, long responseTimeoutMillis, long writeTimeoutMillis) { this.eventLoopGroup = eventLoopGroup; this.connectionPool = connectionPool; + this.proxyOptions = proxyOptions; + this.channelInitializationProxyHandler = channelInitializationProxyHandler; this.readTimeoutMillis = readTimeoutMillis; this.responseTimeoutMillis = responseTimeoutMillis; this.writeTimeoutMillis = writeTimeoutMillis; @@ -89,13 +97,15 @@ public Response send(HttpRequest request) { final URI uri = request.getUri(); final boolean isHttps = "https".equalsIgnoreCase(uri.getScheme()); final int port = uri.getPort() == -1 ? (isHttps ? 443 : 80) : uri.getPort(); - final SocketAddress remoteAddress = new InetSocketAddress(uri.getHost(), port); + final SocketAddress finalDestination = new InetSocketAddress(uri.getHost(), port); + + final Netty4ConnectionPoolKey connectionPoolKey = constructConnectionPoolKey(finalDestination, isHttps); final CountDownLatch latch = new CountDownLatch(1); final AtomicReference responseReference = new AtomicReference<>(); final AtomicReference errorReference = new AtomicReference<>(); - Future channelFuture = connectionPool.acquire(remoteAddress, isHttps); + Future channelFuture = connectionPool.acquire(connectionPoolKey, isHttps); channelFuture.addListener((GenericFutureListener>) future -> { if (!future.isSuccess()) { @@ -305,4 +315,26 @@ private static BinaryData createBodyFromServerSentResult(ServerSentResult server : BinaryData.empty(); } + private Netty4ConnectionPoolKey constructConnectionPoolKey(SocketAddress finalDestination, boolean isHttps) { + final Netty4ConnectionPoolKey key; + + final boolean useProxy = channelInitializationProxyHandler.test(finalDestination); + if (useProxy) { + SocketAddress proxyAddress = proxyOptions.getAddress(); + if (isHttps) { + // For proxied HTTPS, the pool is keyed by the unique combination of the proxy + // and the final destination. This creates dedicated pools for each tunneled destination. + key = new Netty4ConnectionPoolKey(proxyAddress, finalDestination); + } else { + // For proxied plain HTTP, the pool is keyed only by the proxy address. + // This allows reusing the same connection to the proxy for different final destinations. + key = new Netty4ConnectionPoolKey(proxyAddress, proxyAddress); + } + } else { + key = new Netty4ConnectionPoolKey(finalDestination, finalDestination); + } + + return key; + } + } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 268149115ae6..0e70174ad134 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -326,7 +326,10 @@ public NettyHttpClientBuilder connectionIdleTimeout(Duration connectionIdleTimeo /** * Sets the maximum time a connection is allowed to exist. - * After this time, the connection will be closed upon release. A default of null means no lifetime limit. + *

+ * After this time, the connection will be closed upon release. + *

+ * A {@link Duration} of zero or less, or a null value, will result in connections having no lifetime limit. * * @param maxConnectionLifetime The maximum connection lifetime. * @return The updated builder. @@ -349,6 +352,8 @@ public NettyHttpClientBuilder pendingAcquireTimeout(Duration pendingAcquireTimeo /** * Sets the maximum number of requests that can be queued waiting for a connection. + *

+ * This limit is applied on a per-route (per-host) basis. * * @param maxPendingAcquires The maximum number of pending acquires. * @return The updated builder. @@ -403,7 +408,8 @@ public HttpClient build() { sslContextModifier, connectionPoolSize, connectionIdleTimeout, maxConnectionLifetime, pendingAcquireTimeout, maxPendingAcquires, maximumHttpVersion); - return new NettyHttpClient(group, connectionPool, getTimeoutMillis(readTimeout), + return new NettyHttpClient(group, connectionPool, buildProxyOptions, + new ChannelInitializationProxyHandler(buildProxyOptions), getTimeoutMillis(readTimeout), getTimeoutMillis(responseTimeout), getTimeoutMillis(writeTimeout)); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 1acfea0b96e9..cf9cd95680d8 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -62,11 +62,13 @@ public final class Netty4ConnectionPool implements Closeable { public static final AttributeKey NEW_CHANNEL_KEY = AttributeKey.valueOf("new-channel"); private static final AttributeKey CHANNEL_CREATION_TIME = AttributeKey.valueOf("channel-creation-time"); + private static final AttributeKey CONNECTION_POOL_KEY + = AttributeKey.valueOf("connection-pool-key"); private static final ClientLogger LOGGER = new ClientLogger(Netty4ConnectionPool.class); private static final String CLOSED_POOL_ERROR_MESSAGE = "Connection pool has been closed."; - private final ConcurrentMap pool = new ConcurrentHashMap<>(); + private final ConcurrentMap pool = new ConcurrentHashMap<>(); private final ConcurrentMap channelIdleSince = new ConcurrentHashMap<>(); private final AtomicBoolean closed = new AtomicBoolean(false); @@ -116,17 +118,17 @@ public Netty4ConnectionPool(Bootstrap bootstrap, /** * Acquires a channel for the given remote address from the pool. * - * @param remoteAddress The remote address to connect to. + * @param key The composite key representing the connection route. * @param isHttps Flag indicating whether connections for this route should be secured using TLS/SSL. * @return A {@link Future} that will be notified when a channel is acquired. * @throws IllegalStateException if the connection pool has been closed. */ - public Future acquire(SocketAddress remoteAddress, boolean isHttps) { + public Future acquire(Netty4ConnectionPoolKey key, boolean isHttps) { if (closed.get()) { throw LOGGER.throwableAtError().log(CLOSED_POOL_ERROR_MESSAGE, IllegalStateException::new); } - PerRoutePool perRoutePool = pool.computeIfAbsent(remoteAddress, k -> new PerRoutePool(k, isHttps)); + PerRoutePool perRoutePool = pool.computeIfAbsent(key, k -> new PerRoutePool(k, isHttps)); return perRoutePool.acquire(); } @@ -145,9 +147,11 @@ public void release(Channel channel) { return; } - if (channel.remoteAddress() != null) { - PerRoutePool perRoutePool = pool.get(channel.remoteAddress()); + Netty4ConnectionPoolKey key = channel.attr(CONNECTION_POOL_KEY).get(); + if (key != null) { + PerRoutePool perRoutePool = pool.get(key); if (perRoutePool != null) { + channel.attr(CONNECTION_POOL_KEY).set(null); perRoutePool.release(channel); } else { channel.close(); @@ -204,11 +208,13 @@ private class PerRoutePool { private final Deque idleConnections = new ConcurrentLinkedDeque<>(); private final Deque> pendingAcquirers = new ConcurrentLinkedDeque<>(); private final AtomicInteger activeConnections = new AtomicInteger(0); + private final Netty4ConnectionPoolKey key; private final SocketAddress route; private final boolean isHttps; - PerRoutePool(SocketAddress route, boolean isHttps) { - this.route = route; + PerRoutePool(Netty4ConnectionPoolKey key, boolean isHttps) { + this.key = key; + this.route = key.getConnectionTarget(); this.isHttps = isHttps; } @@ -230,6 +236,7 @@ synchronized Future acquire() { activeConnections.incrementAndGet(); channel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(false)); channelIdleSince.remove(channel); + channel.attr(CONNECTION_POOL_KEY).set(key); return channel.eventLoop().newSucceededFuture(channel); } // Unhealthy idle connection was found and discarded. Don't decrement activeConnections @@ -326,7 +333,8 @@ private Future createNewConnection() { public void initChannel(Channel channel) throws SSLException { ChannelPipeline pipeline = channel.pipeline(); // Test whether proxying should be applied to this Channel. If so, add it. - boolean hasProxy = channelInitializationProxyHandler.test(route); + // Proxy detection MUST use the final destination address from the key. + boolean hasProxy = channelInitializationProxyHandler.test(key.getFinalDestination()); if (hasProxy) { ProxyHandler proxyHandler = channelInitializationProxyHandler.createProxy(proxyChallenges); pipeline.addFirst(PROXY, proxyHandler); @@ -334,7 +342,7 @@ public void initChannel(Channel channel) throws SSLException { // Add SSL handling if the request is HTTPS. if (isHttps) { - InetSocketAddress inetSocketAddress = (InetSocketAddress) route; + InetSocketAddress inetSocketAddress = (InetSocketAddress) key.getFinalDestination(); SslContext ssl = buildSslContext(); // SSL handling is added last here. This is done as proxying could require SSL handling too. channel.pipeline() @@ -365,6 +373,7 @@ public void initChannel(Channel channel) throws SSLException { if (proxyFuture.isSuccess()) { newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); + newChannel.attr(CONNECTION_POOL_KEY).set(key); promise.setSuccess(newChannel); } else { promise.setFailure(proxyFuture.cause()); @@ -379,6 +388,7 @@ public void initChannel(Channel channel) throws SSLException { } else { newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); + newChannel.attr(CONNECTION_POOL_KEY).set(key); promise.setSuccess(newChannel); } }); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolKey.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolKey.java new file mode 100644 index 000000000000..1280ebcd7910 --- /dev/null +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolKey.java @@ -0,0 +1,51 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package io.clientcore.http.netty4.implementation; + +import java.net.SocketAddress; +import java.util.Objects; + +/** + * A composite key for the connection pool. + *

+ * For direct connections, connectionTarget and finalDestination are the same. + * For proxied connections, connectionTarget is the proxy's address. For plain HTTP through a proxy, + * finalDestination is also the proxy's address to allow connection reuse. For HTTPS through a proxy, + * finalDestination is the target server's address to create a dedicated pool for the tunnel. + */ +public final class Netty4ConnectionPoolKey { + private final SocketAddress connectionTarget; + private final SocketAddress finalDestination; + + public Netty4ConnectionPoolKey(SocketAddress connectionTarget, SocketAddress finalDestination) { + this.connectionTarget = connectionTarget; + this.finalDestination = finalDestination; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Netty4ConnectionPoolKey poolKey = (Netty4ConnectionPoolKey) o; + return Objects.equals(connectionTarget, poolKey.connectionTarget) + && Objects.equals(finalDestination, poolKey.finalDestination); + } + + @Override + public int hashCode() { + return Objects.hash(connectionTarget, finalDestination); + } + + public SocketAddress getConnectionTarget() { + return this.connectionTarget; + } + + public SocketAddress getFinalDestination() { + return this.finalDestination; + } +} diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HttpProxyHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HttpProxyHandler.java index 5b8883b98261..9907983c8bb6 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HttpProxyHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HttpProxyHandler.java @@ -101,7 +101,7 @@ public String authScheme() { @Override protected void addCodec(ChannelHandlerContext ctx) { - // TODO (alzimmer): Need to support HTTP/2 proxying. Check if Netty itself even supports this. + // TODO (alzimmer): Need to support HTTP/2 proxying. Check (issue 12088) if Netty itself even supports this. ctx.pipeline().addBefore(ctx.name(), Netty4HandlerNames.PROXY_CODEC, this.wrapper); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java index 1eed9b83787b..4ebce483b1d4 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java @@ -43,7 +43,7 @@ public class Netty4ConnectionPoolTests { private static LocalTestServer server; private static EventLoopGroup eventLoopGroup; private static Bootstrap bootstrap; - private static SocketAddress serverAddress; + private static Netty4ConnectionPoolKey connectionPoolKey; @BeforeAll public static void startTestServerAndEventLoopGroup() { @@ -51,7 +51,8 @@ public static void startTestServerAndEventLoopGroup() { server.start(); eventLoopGroup = new NioEventLoopGroup(2); bootstrap = new Bootstrap().group(eventLoopGroup).channel(NioSocketChannel.class); - serverAddress = new InetSocketAddress("localhost", server.getPort()); + SocketAddress socketAddress = new InetSocketAddress("localhost", server.getPort()); + connectionPoolKey = new Netty4ConnectionPoolKey(socketAddress, socketAddress); } @AfterAll @@ -74,7 +75,7 @@ private Netty4ConnectionPool createPool(int maxConnections, Duration idleTimeout @Test public void testAcquireAndRelease() throws IOException { try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { - Future future = pool.acquire(serverAddress, false); + Future future = pool.acquire(connectionPoolKey, false); Channel channel = future.awaitUninterruptibly().getNow(); assertNotNull(channel); assertTrue(channel.isActive()); @@ -85,11 +86,11 @@ public void testAcquireAndRelease() throws IOException { @Test public void testConnectionIsReusedForSameRemoteAddress() throws IOException { try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { - Future future1 = pool.acquire(serverAddress, false); + Future future1 = pool.acquire(connectionPoolKey, false); Channel channel1 = future1.awaitUninterruptibly().getNow(); pool.release(channel1); - Future future2 = pool.acquire(serverAddress, false); + Future future2 = pool.acquire(connectionPoolKey, false); Channel channel2 = future2.awaitUninterruptibly().getNow(); assertSame(channel1, channel2); pool.release(channel2); @@ -103,11 +104,11 @@ public void testConnectionPoolSizeEnforced() throws IOException, InterruptedExce = createPool(maxConnections, Duration.ofSeconds(10), null, Duration.ofSeconds(10), maxConnections)) { List channels = new ArrayList<>(); for (int i = 0; i < maxConnections; i++) { - channels.add(pool.acquire(serverAddress, false).awaitUninterruptibly().getNow()); + channels.add(pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow()); } assertEquals(maxConnections, channels.size()); - Future pendingFuture = pool.acquire(serverAddress, false); + Future pendingFuture = pool.acquire(connectionPoolKey, false); Thread.sleep(100); assertFalse(pendingFuture.isDone()); @@ -124,9 +125,9 @@ public void testConnectionPoolSizeEnforced() throws IOException, InterruptedExce @Test public void testPendingAcquireTimeout() throws IOException, InterruptedException { try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofMillis(100), 1)) { - Channel channel = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + Channel channel = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); - Future timeoutFuture = pool.acquire(serverAddress, false); + Future timeoutFuture = pool.acquire(connectionPoolKey, false); assertTrue(timeoutFuture.await(500, TimeUnit.MILLISECONDS)); @@ -141,12 +142,12 @@ public void testPendingAcquireTimeout() throws IOException, InterruptedException @Test public void testIdleConnectionIsCleanedUp() throws IOException, InterruptedException { try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { - Channel channel1 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + Channel channel1 = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); pool.release(channel1); Thread.sleep(31000); // Wait for cleanup task to run (interval is 30s) assertFalse(channel1.isActive()); - Channel channel2 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + Channel channel2 = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); assertNotSame(channel1, channel2); pool.release(channel2); } @@ -156,13 +157,13 @@ public void testIdleConnectionIsCleanedUp() throws IOException, InterruptedExcep public void testMaxConnectionLifetimeEnforced() throws IOException, InterruptedException { try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), Duration.ofMillis(500), Duration.ofSeconds(10), 1)) { - Channel channel1 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + Channel channel1 = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); Thread.sleep(600); pool.release(channel1); Thread.sleep(100); // Give a moment for close to propagate assertFalse(channel1.isActive()); - Channel channel2 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + Channel channel2 = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); assertNotSame(channel1, channel2); pool.release(channel2); } @@ -171,11 +172,11 @@ public void testMaxConnectionLifetimeEnforced() throws IOException, InterruptedE @Test public void testUnhealthyConnectionIsDiscarded() throws IOException { try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { - Channel channel1 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + Channel channel1 = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); pool.release(channel1); channel1.close().awaitUninterruptibly(); - Channel channel2 = pool.acquire(serverAddress, false).awaitUninterruptibly().getNow(); + Channel channel2 = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); assertNotNull(channel2); assertTrue(channel2.isActive()); assertNotSame(channel1, channel2); @@ -187,7 +188,7 @@ public void testUnhealthyConnectionIsDiscarded() throws IOException { public void testAcquireOnClosedPoolFails() throws IOException { Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1); pool.close(); - assertThrows(IllegalStateException.class, () -> pool.acquire(serverAddress, false)); + assertThrows(IllegalStateException.class, () -> pool.acquire(connectionPoolKey, false)); } @Test @@ -201,12 +202,14 @@ public void testSeparatePoolsForSeparateRemoteAddresses() throws IOException { SocketAddress address1 = new InetSocketAddress("localhost", route1Server.getPort()); SocketAddress address2 = new InetSocketAddress("localhost", route2Server.getPort()); + Netty4ConnectionPoolKey key1 = new Netty4ConnectionPoolKey(address1, address1); + Netty4ConnectionPoolKey key2 = new Netty4ConnectionPoolKey(address2, address2); try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { - Channel channel1 = pool.acquire(address1, false).awaitUninterruptibly().getNow(); + Channel channel1 = pool.acquire(key1, false).awaitUninterruptibly().getNow(); assertNotNull(channel1); - Channel channel2 = pool.acquire(address2, false).awaitUninterruptibly().getNow(); + Channel channel2 = pool.acquire(key2, false).awaitUninterruptibly().getNow(); assertNotNull(channel2); assertNotSame(channel1, channel2); From 989b30211cbc667dddf458150de7a2e9fc0d774d Mon Sep 17 00:00:00 2001 From: George Banasios Date: Tue, 1 Jul 2025 13:44:39 +0300 Subject: [PATCH 10/57] cleanup the pipeline after the last content --- .../implementation/Netty4ConnectionPool.java | 10 ++++++ .../Netty4PipelineCleanupHandler.java | 20 +++++++----- .../http/netty4/NettyHttpClientTests.java | 31 ++++++++++++------- 3 files changed, 42 insertions(+), 19 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index cf9cd95680d8..a59be9183a55 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -30,6 +30,7 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; +import java.nio.channels.ClosedChannelException; import java.time.Duration; import java.time.OffsetDateTime; import java.time.ZoneOffset; @@ -371,6 +372,15 @@ public void initChannel(Channel channel) throws SSLException { // Wait for the proxy handshake to complete if proxy is being used. proxyHandler.connectFuture().addListener(proxyFuture -> { if (proxyFuture.isSuccess()) { + if (!newChannel.isActive()) { + promise.setFailure(new ClosedChannelException()); + + synchronized (this) { + activeConnections.decrementAndGet(); + satisfyWaiterWithNewConnection(); + } + return; + } newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); newChannel.attr(CONNECTION_POOL_KEY).set(key); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 29c71a7261af..37e4e36ba768 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -30,6 +30,7 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final Netty4ConnectionPool connectionPool; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); + private boolean lastContentRead; private static final List HANDLERS_TO_REMOVE; @@ -51,16 +52,21 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - ctx.fireChannelRead(msg); - - boolean lastRead = false; if (msg instanceof LastHttpContent) { - lastRead = true; + this.lastContentRead = true; } else if (msg instanceof Http2DataFrame) { - lastRead = ((Http2DataFrame) msg).isEndStream(); + this.lastContentRead = ((Http2DataFrame) msg).isEndStream(); } - if (lastRead) { + ctx.fireChannelRead(msg); + } + + @Override + public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { + // First, let other handlers process the channelReadComplete event. + ctx.fireChannelReadComplete(); + + if (lastContentRead) { cleanup(ctx); } } @@ -75,8 +81,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { + ctx.fireChannelInactive(); cleanup(ctx); - super.channelInactive(ctx); } public void cleanup(ChannelHandlerContext ctx) { diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java index 4ba9621b3c73..e113a6225d1c 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java @@ -42,6 +42,7 @@ import java.io.StringWriter; import java.net.URI; import java.nio.channels.Channels; +import java.nio.channels.ClosedChannelException; import java.nio.channels.WritableByteChannel; import java.nio.file.Files; import java.nio.file.Path; @@ -302,18 +303,24 @@ public void failedProxyAuthenticationReturnsCorrectError() { () -> httpClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(PROXY_TO_ADDRESS)))); Throwable exception = coreException.getCause(); - assertInstanceOf(ProxyConnectException.class, exception, () -> { - StringWriter stringWriter = new StringWriter(); - stringWriter.write(exception.toString()); - PrintWriter printWriter = new PrintWriter(stringWriter); - exception.printStackTrace(printWriter); - - return stringWriter.toString(); - }); - - assertTrue(coreException.getCause().getMessage().contains("Proxy Authentication Required"), - () -> "Expected exception message to contain \"Proxy Authentication Required\", it was: " - + coreException.getCause().getMessage()); + assertTrue(exception instanceof ProxyConnectException || exception instanceof ClosedChannelException, + "Exception was not of expected type ProxyConnectException or ClosedChannelException, but was " + + exception.getClass().getName()); + + if (exception instanceof ProxyConnectException) { + assertInstanceOf(ProxyConnectException.class, exception, () -> { + StringWriter stringWriter = new StringWriter(); + stringWriter.write(exception.toString()); + PrintWriter printWriter = new PrintWriter(stringWriter); + exception.printStackTrace(printWriter); + + return stringWriter.toString(); + }); + + assertTrue(coreException.getCause().getMessage().contains("Proxy Authentication Required"), + () -> "Expected exception message to contain \"Proxy Authentication Required\", it was: " + + coreException.getCause().getMessage()); + } } } From f0a78471c080bfcf9d0d30e2891249c0f7e5b301 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Tue, 1 Jul 2025 17:03:25 +0300 Subject: [PATCH 11/57] create channel wrapper --- .../implementation/Netty4ConnectionPool.java | 143 ++++++++++-------- 1 file changed, 80 insertions(+), 63 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index a59be9183a55..19e2c6899993 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -37,7 +37,6 @@ import java.util.Deque; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentMap; @@ -56,21 +55,13 @@ */ public final class Netty4ConnectionPool implements Closeable { - /** - * An attribute key to mark a channel as new, so that certain handlers (e.g. proxy, ssl) - * are only added once. - */ - public static final AttributeKey NEW_CHANNEL_KEY = AttributeKey.valueOf("new-channel"); - private static final AttributeKey CHANNEL_CREATION_TIME - = AttributeKey.valueOf("channel-creation-time"); - private static final AttributeKey CONNECTION_POOL_KEY - = AttributeKey.valueOf("connection-pool-key"); + private static final AttributeKey POOLED_CONNECTION_KEY + = AttributeKey.valueOf("pooled-connection-key"); private static final ClientLogger LOGGER = new ClientLogger(Netty4ConnectionPool.class); private static final String CLOSED_POOL_ERROR_MESSAGE = "Connection pool has been closed."; private final ConcurrentMap pool = new ConcurrentHashMap<>(); - private final ConcurrentMap channelIdleSince = new ConcurrentHashMap<>(); private final AtomicBoolean closed = new AtomicBoolean(false); private final Bootstrap bootstrap; @@ -143,22 +134,23 @@ public void release(Channel channel) { if (channel == null) { return; } - if (closed.get()) { + + PooledConnection pooledConnection = channel.attr(POOLED_CONNECTION_KEY).get(); + if (pooledConnection == null) { channel.close(); return; } - Netty4ConnectionPoolKey key = channel.attr(CONNECTION_POOL_KEY).get(); - if (key != null) { - PerRoutePool perRoutePool = pool.get(key); - if (perRoutePool != null) { - channel.attr(CONNECTION_POOL_KEY).set(null); - perRoutePool.release(channel); - } else { - channel.close(); - } + if (closed.get()) { + pooledConnection.close(); + return; + } + + PerRoutePool perRoutePool = pool.get(pooledConnection.key); + if (perRoutePool != null) { + perRoutePool.release(pooledConnection); } else { - channel.close(); + pooledConnection.close(); } } @@ -166,19 +158,12 @@ public void release(Channel channel) { * Periodically cleans up connections that have been idle for too long. */ private void cleanupIdleConnections() { - if (idleTimeoutNanos <= 0 || channelIdleSince.isEmpty()) { + if (idleTimeoutNanos <= 0 || pool.isEmpty()) { return; } - OffsetDateTime now = OffsetDateTime.now(ZoneOffset.UTC); - for (Iterator> it = channelIdleSince.entrySet().iterator(); it.hasNext();) { - Map.Entry entry = it.next(); - if (Duration.between(entry.getValue(), now).toNanos() >= idleTimeoutNanos) { - it.remove(); - if (entry.getKey().isActive()) { - entry.getKey().close(); - } - } + for (PerRoutePool perRoutePool : pool.values()) { + perRoutePool.cleanup(); } } @@ -190,7 +175,6 @@ public void close() throws IOException { } pool.values().forEach(PerRoutePool::close); pool.clear(); - channelIdleSince.clear(); } } @@ -202,11 +186,36 @@ private static long durationToNanos(Duration duration) { return (duration == null || duration.isNegative() || duration.isZero()) ? -1 : duration.toNanos(); } + /** + * A wrapper for a Netty Channel that holds pooling-related metadata. + */ + private static final class PooledConnection { + private final Channel channel; + private final Netty4ConnectionPoolKey key; + private final OffsetDateTime creationTime; + private volatile OffsetDateTime idleSince; + + PooledConnection(Channel channel, Netty4ConnectionPoolKey key) { + this.channel = channel; + this.key = key; + this.creationTime = OffsetDateTime.now(ZoneOffset.UTC); + channel.attr(POOLED_CONNECTION_KEY).set(this); + } + + private boolean isActiveAndWriteable() { + return channel.isActive() && channel.isWritable(); + } + + private void close() { + channel.close(); + } + } + /** * Manages connections and pending acquirers for a single route. */ private class PerRoutePool { - private final Deque idleConnections = new ConcurrentLinkedDeque<>(); + private final Deque idleConnections = new ConcurrentLinkedDeque<>(); private final Deque> pendingAcquirers = new ConcurrentLinkedDeque<>(); private final AtomicInteger activeConnections = new AtomicInteger(0); private final Netty4ConnectionPoolKey key; @@ -229,20 +238,18 @@ synchronized Future acquire() { .newFailedFuture(new IllegalStateException(CLOSED_POOL_ERROR_MESSAGE)); } - Channel channel; - while ((channel = idleConnections.poll()) != null) { - if (isHealthy(channel)) { + PooledConnection connection; + while ((connection = idleConnections.poll()) != null) { + if (isHealthy(connection)) { // Acquired an existing healthy connection. activeConnections count is not // yet incremented for idle channels, so we do it here. activeConnections.incrementAndGet(); - channel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(false)); - channelIdleSince.remove(channel); - channel.attr(CONNECTION_POOL_KEY).set(key); - return channel.eventLoop().newSucceededFuture(channel); + connection.idleSince = null; // Mark as active + return connection.channel.eventLoop().newSucceededFuture(connection.channel); } // Unhealthy idle connection was found and discarded. Don't decrement activeConnections // as it was already decremented when the channel was released to the idle queue. - channel.close(); + connection.close(); } // No idle connections available, create a new one. @@ -277,10 +284,10 @@ synchronized Future acquire() { return promise; } - synchronized void release(Channel channel) { - if (!isHealthy(channel)) { + synchronized void release(PooledConnection connection) { + if (!isHealthy(connection)) { activeConnections.decrementAndGet(); - channel.close(); + connection.close(); // Since a connection slot has been freed, we should try to create a new, // healthy connection for any request that might be waiting. @@ -293,17 +300,15 @@ synchronized void release(Channel channel) { while ((waiter = pendingAcquirers.poll()) != null) { // A waiter exists, hand over this channel directly. // The activeConnections count remains the same (one leaves, one joins). - if (waiter.trySuccess(channel)) { + if (waiter.trySuccess(connection.channel)) { return; } } // Channel is healthy and no waiters, return it to the idle queue. activeConnections.decrementAndGet(); - idleConnections.offer(channel); - if (idleTimeoutNanos > 0) { - channelIdleSince.put(channel, OffsetDateTime.now(ZoneOffset.UTC)); - } + connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); + idleConnections.offer(connection); } private void satisfyWaiterWithNewConnection() { @@ -332,6 +337,9 @@ private Future createNewConnection() { newConnectionBootstrap.handler(new ChannelInitializer() { @Override public void initChannel(Channel channel) throws SSLException { + // Create the connection wrapper and attach it to the channel. + new PooledConnection(channel, key); + ChannelPipeline pipeline = channel.pipeline(); // Test whether proxying should be applied to this Channel. If so, add it. // Proxy detection MUST use the final destination address from the key. @@ -381,9 +389,6 @@ public void initChannel(Channel channel) throws SSLException { } return; } - newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); - newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); - newChannel.attr(CONNECTION_POOL_KEY).set(key); promise.setSuccess(newChannel); } else { promise.setFailure(proxyFuture.cause()); @@ -396,9 +401,6 @@ public void initChannel(Channel channel) throws SSLException { } }); } else { - newChannel.attr(NEW_CHANNEL_KEY).set(new AtomicBoolean(true)); - newChannel.attr(CHANNEL_CREATION_TIME).set(OffsetDateTime.now(ZoneOffset.UTC)); - newChannel.attr(CONNECTION_POOL_KEY).set(key); promise.setSuccess(newChannel); } }); @@ -436,25 +438,40 @@ private SslContext buildSslContext() throws SSLException { return sslContextBuilder.build(); } - private boolean isHealthy(Channel channel) { - if (!channel.isActive() || !channel.isWritable()) { + private boolean isHealthy(PooledConnection connection) { + if (!connection.isActiveAndWriteable()) { return false; } if (maxLifetimeNanos > 0) { - OffsetDateTime creationTime = channel.attr(CHANNEL_CREATION_TIME).get(); OffsetDateTime now = OffsetDateTime.now(ZoneOffset.UTC); - if (creationTime != null && Duration.between(creationTime, now).toNanos() >= maxLifetimeNanos) { + if (Duration.between(connection.creationTime, now).toNanos() >= maxLifetimeNanos) { return false; } } return true; } + synchronized void cleanup() { + if (idleConnections.isEmpty()) { + return; + } + + OffsetDateTime now = OffsetDateTime.now(ZoneOffset.UTC); + for (Iterator it = idleConnections.iterator(); it.hasNext();) { + PooledConnection connection = it.next(); + if (connection.idleSince != null + && Duration.between(connection.idleSince, now).toNanos() >= idleTimeoutNanos) { + it.remove(); + connection.close(); + } + } + } + synchronized void close() { - Channel channel; - while ((channel = idleConnections.poll()) != null) { - channel.close(); + PooledConnection connection; + while ((connection = idleConnections.poll()) != null) { + connection.close(); } Promise waiter; while ((waiter = pendingAcquirers.poll()) != null) { From c8c648bc6c5d83835d3c7b6819dcbbbe3c284286 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Tue, 1 Jul 2025 18:17:41 +0300 Subject: [PATCH 12/57] remove channel closure from response handler --- .../netty4/implementation/Netty4AlpnHandler.java | 4 ++-- .../implementation/Netty4ResponseHandler.java | 13 +++++++++---- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index 5b1dfc4e7428..3258717c5246 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -97,7 +97,7 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { .addListener((ChannelFutureListener) sendListener -> { if (!sendListener.isSuccess()) { setOrSuppressError(errorReference, sendListener.cause()); - sendListener.channel().close(); + sendListener.channel().pipeline().fireExceptionCaught(sendListener.cause()); latch.countDown(); } else { sendListener.channel().read(); @@ -121,7 +121,7 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { .addListener((ChannelFutureListener) sendListener -> { if (!sendListener.isSuccess()) { setOrSuppressError(errorReference, sendListener.cause()); - sendListener.channel().close(); + sendListener.channel().pipeline().fireExceptionCaught(sendListener.cause()); latch.countDown(); } else { sendListener.channel().read(); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 5da09d7c8924..5d5bf61785ca 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -17,6 +17,7 @@ import io.netty.handler.codec.http.LastHttpContent; import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.util.Objects; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; @@ -168,12 +169,16 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { ctx.pipeline().remove(this); ctx.fireChannelReadComplete(); - if (complete) { - ctx.close(); - } - responseReference.set(new ResponseStateInfo(ctx.channel(), complete, statusCode, headers, eagerContent, ResponseBodyHandling.getBodyHandling(request, headers), false)); latch.countDown(); } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + setOrSuppressError(errorReference, + new IOException("The channel became inactive before a response was received.")); + ctx.fireChannelInactive(); + latch.countDown(); + } } From 8dea326edfe922aa41cad703d84fcb1fcba31b83 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Tue, 1 Jul 2025 19:59:02 +0300 Subject: [PATCH 13/57] try and fix the release deadlock --- .../implementation/Netty4ConnectionPool.java | 29 +++++++++++-------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 19e2c6899993..02a5ee740f26 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -296,19 +296,24 @@ synchronized void release(PooledConnection connection) { } // The channel is healthy. Now, check if anyone is waiting for a connection. - Promise waiter; - while ((waiter = pendingAcquirers.poll()) != null) { - // A waiter exists, hand over this channel directly. - // The activeConnections count remains the same (one leaves, one joins). - if (waiter.trySuccess(connection.channel)) { - return; - } + Promise waiterToNotify = pendingAcquirers.poll(); + + if (waiterToNotify != null) { + // A waiter exists. Fulfill the promise OUTSIDE the synchronized block. + // The activeConnections count remains the same. + connection.channel.eventLoop().execute(() -> { + if (!waiterToNotify.trySuccess(connection.channel)) { + // The waiter was cancelled or failed in the meantime. + // Release the connection again so it can be pooled or given to another waiter. + release(connection); + } + }); + } else { + // No waiters, return the connection to the idle queue. + activeConnections.decrementAndGet(); + connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); + idleConnections.offer(connection); } - - // Channel is healthy and no waiters, return it to the idle queue. - activeConnections.decrementAndGet(); - connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); - idleConnections.offer(connection); } private void satisfyWaiterWithNewConnection() { From e076602102189eb41ea4729883ec5ae2bdc89ad9 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 2 Jul 2025 10:33:03 +0300 Subject: [PATCH 14/57] handle case where read handlers are added to a closed channel --- .../Netty4EagerConsumeChannelHandler.java | 12 ++++++++++++ .../implementation/Netty4InitiateOneReadHandler.java | 12 ++++++++++++ 2 files changed, 24 insertions(+) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index cfeba45dc818..3c9712f462ab 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -13,6 +13,7 @@ import io.netty.util.ReferenceCountUtil; import java.io.IOException; +import java.nio.channels.ClosedChannelException; import java.util.concurrent.CountDownLatch; import java.util.function.Consumer; @@ -97,6 +98,17 @@ public void channelInactive(ChannelHandlerContext ctx) { ctx.fireChannelInactive(); } + @Override + public void handlerAdded(ChannelHandlerContext ctx) { + if (!ctx.channel().isActive()) { + // In case the read handler is added to a closed channel we fail loudly by firing + // an exception. Simply counting down the latch would cause the caller to receive + // an empty/incomplete data stream without any indication of the underlying network error. + ctx.fireExceptionCaught(new ClosedChannelException()); + ctx.pipeline().remove(this); + } + } + private void cleanup(ChannelHandlerContext ctx) { if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { ctx.pipeline().remove(this); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java index 6f287b04f05e..0606160b3cea 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java @@ -13,6 +13,7 @@ import io.netty.util.ReferenceCountUtil; import java.io.IOException; +import java.nio.channels.ClosedChannelException; import java.util.concurrent.CountDownLatch; /** @@ -136,4 +137,15 @@ public void channelInactive(ChannelHandlerContext ctx) { } ctx.fireChannelInactive(); } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) { + if (!ctx.channel().isActive()) { + // In case the read handler is added to a closed channel we fail loudly by firing + // an exception. Simply counting down the latch would cause the caller to receive + // an empty/incomplete data stream without any indication of the underlying network error. + ctx.fireExceptionCaught(new ClosedChannelException()); + ctx.pipeline().remove(this); + } + } } From 8de78bd6d66a29a5b9bd10d9f27a111c6ede7642 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 2 Jul 2025 12:42:27 +0300 Subject: [PATCH 15/57] remove synchronized --- .../implementation/Netty4ConnectionPool.java | 177 ++++++++++-------- .../Netty4ProgressAndTimeoutHandler.java | 6 +- 2 files changed, 97 insertions(+), 86 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 02a5ee740f26..52bd063f4cce 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -228,9 +228,7 @@ private class PerRoutePool { this.isHttps = isHttps; } - //TODO: Maybe this should be using CAS atomics and loops instead of synchronized - // in case the http-netty4 is also used in async flows instead of just sync - synchronized Future acquire() { + Future acquire() { if (closed.get()) { return bootstrap.config() .group() @@ -238,8 +236,13 @@ synchronized Future acquire() { .newFailedFuture(new IllegalStateException(CLOSED_POOL_ERROR_MESSAGE)); } - PooledConnection connection; - while ((connection = idleConnections.poll()) != null) { + // First, optimistically try to acquire an existing idle connection. + while (true) { + PooledConnection connection = idleConnections.poll(); + if (connection == null) { + break; // No idle connections available. + } + if (isHealthy(connection)) { // Acquired an existing healthy connection. activeConnections count is not // yet incremented for idle channels, so we do it here. @@ -247,96 +250,111 @@ synchronized Future acquire() { connection.idleSince = null; // Mark as active return connection.channel.eventLoop().newSucceededFuture(connection.channel); } + // Unhealthy idle connection was found and discarded. Don't decrement activeConnections // as it was already decremented when the channel was released to the idle queue. connection.close(); } - // No idle connections available, create a new one. - if (activeConnections.get() < maxConnectionsPerRoute) { - return createNewConnection(); - } - - // Pool is full, queue the request if there is space. - if (pendingAcquirers.size() >= maxPendingAcquires) { - return bootstrap.config() - .group() - .next() - .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); - } + // No idle connections. Try to create a new one or queue the request. + while (true) { + int currentActive = activeConnections.get(); + if (currentActive < maxConnectionsPerRoute) { + // Try to reserve a slot for a new connection. + if (activeConnections.compareAndSet(currentActive, currentActive + 1)) { + return createNewConnection(); + } + // CAS failed, another thread changed the count. Loop to retry. + } else { + // Pool is full, queue the request if there is space. + if (pendingAcquirers.size() >= maxPendingAcquires) { + return bootstrap.config() + .group() + .next() + .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); + } - Promise promise = bootstrap.config().group().next().newPromise(); - promise.addListener(future -> { - if (future.isCancelled()) { - pendingAcquirers.remove(promise); - } - }); - pendingAcquirers.offer(promise); + Promise promise = bootstrap.config().group().next().newPromise(); + promise.addListener(future -> { + if (future.isCancelled()) { + pendingAcquirers.remove(promise); + } + }); + pendingAcquirers.offer(promise); - if (pendingAcquireTimeout != null) { - bootstrap.config().group().schedule(() -> { - if (!promise.isDone()) { - promise.tryFailure( - CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout)); + if (pendingAcquireTimeout != null) { + bootstrap.config().group().schedule(() -> { + if (!promise.isDone()) { + promise.tryFailure(CoreException + .from("Connection acquisition timed out after " + pendingAcquireTimeout)); + } + }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); } - }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); + return promise; + } } - return promise; } - synchronized void release(PooledConnection connection) { + void release(PooledConnection connection) { if (!isHealthy(connection)) { activeConnections.decrementAndGet(); connection.close(); - - // Since a connection slot has been freed, we should try to create a new, - // healthy connection for any request that might be waiting. + // A slot has been freed. Try to satisfy a waiting acquirer with a new connection. satisfyWaiterWithNewConnection(); return; } // The channel is healthy. Now, check if anyone is waiting for a connection. - Promise waiterToNotify = pendingAcquirers.poll(); - - if (waiterToNotify != null) { - // A waiter exists. Fulfill the promise OUTSIDE the synchronized block. - // The activeConnections count remains the same. - connection.channel.eventLoop().execute(() -> { - if (!waiterToNotify.trySuccess(connection.channel)) { - // The waiter was cancelled or failed in the meantime. - // Release the connection again so it can be pooled or given to another waiter. - release(connection); - } - }); - } else { - // No waiters, return the connection to the idle queue. - activeConnections.decrementAndGet(); - connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); - idleConnections.offer(connection); + while (true) { + Promise waiterToNotify = pendingAcquirers.poll(); + if (waiterToNotify == null) { + // No waiters, return the connection to the idle queue. + activeConnections.decrementAndGet(); + connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); + idleConnections.offer(connection); + break; + } + + // A waiter exists. Fulfill the promise. Active connection count remains the same. + if (waiterToNotify.trySuccess(connection.channel)) { + // Waiter was notified successfully + return; + } + // If trySuccess fails, the waiter was cancelled. Loop again to find another waiter. } } private void satisfyWaiterWithNewConnection() { - // This method MUST be called from within a thread-safe block/method. - if (activeConnections.get() < maxConnectionsPerRoute) { - Promise waiter = pendingAcquirers.poll(); - if (waiter != null) { - // A waiter exists, and we have capacity, create a new connection for them. - Future newConnectionFuture = createNewConnection(); - newConnectionFuture.addListener(future -> { - if (future.isSuccess()) { - waiter.trySuccess((Channel) future.getNow()); - } else { - waiter.tryFailure(future.cause()); - } - }); + // This method is called when a connection slot is freed. + while (true) { + int currentActive = activeConnections.get(); + if (currentActive >= maxConnectionsPerRoute || pendingAcquirers.isEmpty()) { + return; } + + if (activeConnections.compareAndSet(currentActive, currentActive + 1)) { + Promise waiter = pendingAcquirers.poll(); + if (waiter != null) { + // A waiter exists, and we have capacity, create a new connection for them. + Future newConnectionFuture = createNewConnection(); + newConnectionFuture.addListener(future -> { + if (future.isSuccess()) { + waiter.trySuccess((Channel) future.getNow()); + } else { + waiter.tryFailure(future.cause()); + } + }); + } else { + // A waiter disappeared after we reserved a slot. Release the slot. + activeConnections.decrementAndGet(); + } + return; // Exit after attempting to satisfy one waiter. + } + // CAS failed, another thread is operating. Loop to re-evaluate. } } private Future createNewConnection() { - // This method MUST be called from within a thread-safe block/method. - activeConnections.incrementAndGet(); Bootstrap newConnectionBootstrap = bootstrap.clone(); newConnectionBootstrap.handler(new ChannelInitializer() { @@ -370,10 +388,9 @@ public void initChannel(Channel channel) throws SSLException { Promise promise = newConnectionBootstrap.config().group().next().newPromise(); newConnectionBootstrap.connect(route).addListener(future -> { if (!future.isSuccess()) { - synchronized (this) { - activeConnections.decrementAndGet(); - satisfyWaiterWithNewConnection(); - } + // Connect failed, release the slot and try to satisfy a waiter. + activeConnections.decrementAndGet(); + satisfyWaiterWithNewConnection(); promise.setFailure(future.cause()); return; } @@ -387,22 +404,16 @@ public void initChannel(Channel channel) throws SSLException { if (proxyFuture.isSuccess()) { if (!newChannel.isActive()) { promise.setFailure(new ClosedChannelException()); - - synchronized (this) { - activeConnections.decrementAndGet(); - satisfyWaiterWithNewConnection(); - } + activeConnections.decrementAndGet(); + satisfyWaiterWithNewConnection(); return; } promise.setSuccess(newChannel); } else { promise.setFailure(proxyFuture.cause()); newChannel.close(); - - synchronized (this) { - activeConnections.decrementAndGet(); - satisfyWaiterWithNewConnection(); - } + activeConnections.decrementAndGet(); + satisfyWaiterWithNewConnection(); } }); } else { @@ -457,7 +468,7 @@ private boolean isHealthy(PooledConnection connection) { return true; } - synchronized void cleanup() { + void cleanup() { if (idleConnections.isEmpty()) { return; } @@ -473,7 +484,7 @@ synchronized void cleanup() { } } - synchronized void close() { + void close() { PooledConnection connection; while ((connection = idleConnections.poll()) != null) { connection.close(); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java index bc637e04fc45..3df4ba10e7b0 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java @@ -168,7 +168,7 @@ void writeTimeoutRunnable(ChannelHandlerContext ctx, boolean trackingWriteTimeou private void disposeWriteTimeoutWatcher() { trackingWriteTimeout = false; - if (writeTimeoutWatcher != null && !writeTimeoutWatcher.isDone()) { + if (writeTimeoutWatcher != null) { writeTimeoutWatcher.cancel(false); writeTimeoutWatcher = null; } @@ -211,7 +211,7 @@ void responseTimedOut(ChannelHandlerContext ctx, boolean trackingResponseTimeout private void disposeResponseTimeoutWatcher() { trackingResponseTimeout = false; - if (responseTimeoutWatcher != null && !responseTimeoutWatcher.isDone()) { + if (responseTimeoutWatcher != null) { responseTimeoutWatcher.cancel(false); responseTimeoutWatcher = null; } @@ -283,7 +283,7 @@ void readTimeoutRunnable(ChannelHandlerContext ctx, boolean trackingReadTimeout) private void disposeReadTimeoutWatcher() { trackingReadTimeout = false; - if (readTimeoutWatcher != null && !readTimeoutWatcher.isDone()) { + if (readTimeoutWatcher != null) { readTimeoutWatcher.cancel(false); readTimeoutWatcher = null; } From ed27af790e5b6771cf01686277ec231ab965e6a9 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 2 Jul 2025 15:22:02 +0300 Subject: [PATCH 16/57] apply ALPN only on new connections --- .../http/netty4/NettyHttpClient.java | 38 ++++-- .../implementation/Netty4AlpnHandler.java | 108 +++++------------- .../netty4/implementation/Netty4Utility.java | 66 +++++++++++ 3 files changed, 124 insertions(+), 88 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index e17670683fe2..d251a04eba8e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -4,6 +4,7 @@ package io.clientcore.http.netty4; import io.clientcore.core.http.client.HttpClient; +import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.http.models.HttpHeaderName; import io.clientcore.core.http.models.HttpRequest; import io.clientcore.core.http.models.ProxyOptions; @@ -52,6 +53,7 @@ import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; import static io.clientcore.http.netty4.implementation.Netty4Utility.awaitLatch; +import static io.clientcore.http.netty4.implementation.Netty4Utility.configureHttpsPipeline; import static io.clientcore.http.netty4.implementation.Netty4Utility.createCodec; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; import static io.clientcore.http.netty4.implementation.Netty4Utility.setOrSuppressError; @@ -275,8 +277,17 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, // guaranteed to be complete by the time we get the channel because the Netty4AlpnHandler // reacts to the result of the ALPN negotiation that happened during the SSL handshake. if (isHttps) { - // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. - pipeline.addAfter(SSL, ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, latch)); + HttpProtocolVersion protocolVersion = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); + if (protocolVersion != null) { + // Connection is being reused, ALPN is already done. + // Manually configure the pipeline based on the stored protocol. + configureHttpsPipeline(pipeline, request, protocolVersion, responseReference, errorReference, latch); + send(request, channel, errorReference, latch); + } else { + // This is a new connection, let ALPN do the work. + // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. + pipeline.addAfter(SSL, ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, latch)); + } } else { // If there isn't an SslHandler, we can send the request immediately. // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. @@ -284,18 +295,23 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, new Netty4ResponseHandler(request, responseReference, errorReference, latch)); String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); - sendHttp11Request(request, channel, errorReference).addListener(f -> { - if (f.isSuccess()) { - channel.read(); - } else { - setOrSuppressError(errorReference, f.cause()); - pipeline.fireExceptionCaught(f.cause()); - latch.countDown(); - } - }); + send(request, channel, errorReference, latch); } } + private void send(HttpRequest request, Channel channel, AtomicReference errorReference, + CountDownLatch latch) { + sendHttp11Request(request, channel, errorReference).addListener(f -> { + if (f.isSuccess()) { + channel.read(); + } else { + setOrSuppressError(errorReference, f.cause()); + channel.pipeline().fireExceptionCaught(f.cause()); + latch.countDown(); + } + }); + } + public void close() { if (connectionPool != null) { try { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index 3258717c5246..ad3a90754780 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -2,25 +2,19 @@ // Licensed under the MIT License. package io.clientcore.http.netty4.implementation; +import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.http.models.HttpRequest; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; -import io.netty.handler.codec.http2.DefaultHttp2Connection; -import io.netty.handler.codec.http2.DelegatingDecompressorFrameListener; -import io.netty.handler.codec.http2.Http2Connection; -import io.netty.handler.codec.http2.Http2FrameListener; -import io.netty.handler.codec.http2.Http2Settings; -import io.netty.handler.codec.http2.HttpToHttp2ConnectionHandler; -import io.netty.handler.codec.http2.HttpToHttp2ConnectionHandlerBuilder; -import io.netty.handler.codec.http2.InboundHttp2ToHttpAdapterBuilder; import io.netty.handler.ssl.ApplicationProtocolNames; import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler; +import io.netty.util.AttributeKey; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; -import static io.clientcore.http.netty4.implementation.Netty4Utility.createCodec; +import static io.clientcore.http.netty4.implementation.Netty4Utility.configureHttpsPipeline; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; import static io.clientcore.http.netty4.implementation.Netty4Utility.setOrSuppressError; @@ -29,6 +23,14 @@ * either HTTP/1.1 or HTTP/2 based on the result of negotiation. */ public final class Netty4AlpnHandler extends ApplicationProtocolNegotiationHandler { + + /** + * An Attribute key for the channel storing the HTTP protocol version that was negotiated. + * This information will be used in case the same channel is reused in the future, so we can + * adjust the correct handlers because there's no need for ALPN to run again. + */ + public static final AttributeKey HTTP_PROTOCOL_VERSION_KEY + = AttributeKey.valueOf("http-protocol-version"); private static final int TWO_FIFTY_SIX_KB = 256 * 1024; private final HttpRequest request; private final AtomicReference responseReference; @@ -38,9 +40,9 @@ public final class Netty4AlpnHandler extends ApplicationProtocolNegotiationHandl /** * Creates a new instance of {@link Netty4AlpnHandler} with a fallback to using HTTP/1.1. * - * @param request The request to send once ALPN negotiation completes. + * @param request The request to send once ALPN negotiation completes. * @param errorReference An AtomicReference keeping track of errors during the request lifecycle. - * @param latch A CountDownLatch that will be released once the request completes. + * @param latch A CountDownLatch that will be released once the request completes. */ public Netty4AlpnHandler(HttpRequest request, AtomicReference responseReference, AtomicReference errorReference, CountDownLatch latch) { @@ -58,78 +60,30 @@ public boolean isSharable() { @Override protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { + HttpProtocolVersion protocolVersion; if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { - // TODO (alzimmer): InboundHttp2ToHttpAdapter buffers the entire response into a FullHttpResponse. Need to - // create a streaming version of this to support huge response payloads. - Http2Connection http2Connection = new DefaultHttp2Connection(false); - Http2Settings settings = new Http2Settings().headerTableSize(4096) - .maxHeaderListSize(TWO_FIFTY_SIX_KB) - .pushEnabled(false) - .initialWindowSize(TWO_FIFTY_SIX_KB); - Http2FrameListener frameListener = new DelegatingDecompressorFrameListener(http2Connection, - new InboundHttp2ToHttpAdapterBuilder(http2Connection).maxContentLength(Integer.MAX_VALUE) - .propagateSettings(true) - .validateHttpHeaders(true) - .build()); + protocolVersion = HttpProtocolVersion.HTTP_2; + } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { + protocolVersion = HttpProtocolVersion.HTTP_1_1; + } else { + ctx.fireExceptionCaught(new IllegalStateException("unknown protocol: " + protocol)); + return; + } - HttpToHttp2ConnectionHandler connectionHandler - = new HttpToHttp2ConnectionHandlerBuilder().initialSettings(settings) - .frameListener(frameListener) - .connection(http2Connection) - .validateHeaders(true) - .build(); + // Store the negotiated protocol for connection reuse. + ctx.channel().attr(HTTP_PROTOCOL_VERSION_KEY).set(protocolVersion); - if (ctx.pipeline().get(Netty4HandlerNames.PROGRESS_AND_TIMEOUT) != null) { - ctx.pipeline() - .addAfter(Netty4HandlerNames.PROGRESS_AND_TIMEOUT, Netty4HandlerNames.HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); - ctx.pipeline() - .addBefore(Netty4HandlerNames.PROGRESS_AND_TIMEOUT, Netty4HandlerNames.HTTP_CODEC, - connectionHandler); - } else { - ctx.pipeline().addAfter(Netty4HandlerNames.SSL, Netty4HandlerNames.HTTP_CODEC, connectionHandler); - ctx.pipeline() - .addAfter(Netty4HandlerNames.HTTP_CODEC, Netty4HandlerNames.HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); - } + configureHttpsPipeline(ctx.pipeline(), request, protocolVersion, responseReference, errorReference, latch); - sendHttp11Request(request, ctx.channel(), errorReference) - .addListener((ChannelFutureListener) sendListener -> { - if (!sendListener.isSuccess()) { - setOrSuppressError(errorReference, sendListener.cause()); - sendListener.channel().pipeline().fireExceptionCaught(sendListener.cause()); - latch.countDown(); - } else { - sendListener.channel().read(); - } - }); - } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { - if (ctx.pipeline().get(Netty4HandlerNames.PROGRESS_AND_TIMEOUT) != null) { - ctx.pipeline() - .addAfter(Netty4HandlerNames.PROGRESS_AND_TIMEOUT, Netty4HandlerNames.HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); - ctx.pipeline() - .addBefore(Netty4HandlerNames.PROGRESS_AND_TIMEOUT, Netty4HandlerNames.HTTP_CODEC, createCodec()); + sendHttp11Request(request, ctx.channel(), errorReference).addListener((ChannelFutureListener) sendListener -> { + if (!sendListener.isSuccess()) { + setOrSuppressError(errorReference, sendListener.cause()); + sendListener.channel().pipeline().fireExceptionCaught(sendListener.cause()); + latch.countDown(); } else { - ctx.pipeline().addAfter(Netty4HandlerNames.SSL, Netty4HandlerNames.HTTP_CODEC, createCodec()); - ctx.pipeline() - .addAfter(Netty4HandlerNames.HTTP_CODEC, Netty4HandlerNames.HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + sendListener.channel().read(); } - - sendHttp11Request(request, ctx.channel(), errorReference) - .addListener((ChannelFutureListener) sendListener -> { - if (!sendListener.isSuccess()) { - setOrSuppressError(errorReference, sendListener.cause()); - sendListener.channel().pipeline().fireExceptionCaught(sendListener.cause()); - latch.countDown(); - } else { - sendListener.channel().read(); - } - }); - } else { - throw new IllegalStateException("unknown protocol: " + protocol); - } + }); } @Override diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java index bc0daf8efe8c..006c859dcf11 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java @@ -3,6 +3,7 @@ package io.clientcore.http.netty4.implementation; +import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.http.models.HttpHeader; import io.clientcore.core.http.models.HttpHeaderName; import io.clientcore.core.http.models.HttpHeaders; @@ -19,6 +20,8 @@ import io.netty.buffer.Unpooled; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelPipeline; import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.DefaultHttpRequest; import io.netty.handler.codec.http.HttpChunkedInput; @@ -28,6 +31,13 @@ import io.netty.handler.codec.http.HttpHeadersFactory; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpVersion; +import io.netty.handler.codec.http2.DefaultHttp2Connection; +import io.netty.handler.codec.http2.DelegatingDecompressorFrameListener; +import io.netty.handler.codec.http2.Http2Connection; +import io.netty.handler.codec.http2.Http2FrameListener; +import io.netty.handler.codec.http2.Http2Settings; +import io.netty.handler.codec.http2.HttpToHttp2ConnectionHandlerBuilder; +import io.netty.handler.codec.http2.InboundHttp2ToHttpAdapterBuilder; import io.netty.handler.stream.ChunkedInput; import io.netty.handler.stream.ChunkedNioFile; import io.netty.handler.stream.ChunkedStream; @@ -69,6 +79,8 @@ public final class Netty4Utility { private static final List OPTIONAL_NETTY_VERSION_ARTIFACTS = Arrays .asList("netty-transport-native-unix-common", "netty-transport-native-epoll", "netty-transport-native-kqueue"); + private static final int TWO_FIFTY_SIX_KB = 256 * 1024; + /** * Converts Netty HttpHeaders to ClientCore HttpHeaders. *

@@ -438,6 +450,60 @@ public static HttpHeaderName fromPossibleAsciiString(CharSequence asciiString) { } } + /** + * Configures the pipeline for either HTTP/1.1 or HTTP/2 based on the negotiated protocol. + *

+ * This method adds the appropriate {@link Netty4HandlerNames#HTTP_CODEC} and + * {@link Netty4HandlerNames#HTTP_RESPONSE} handlers to the pipeline, positioned correctly + * relative to the {@link Netty4HandlerNames#PROGRESS_AND_TIMEOUT} or {@link Netty4HandlerNames#SSL} handlers. + * + * @param pipeline The channel pipeline to configure. + * @param request The HTTP request. + * @param protocol The negotiated HTTP protocol version. + * @param responseReference The atomic reference to hold the response state. + * @param errorReference The atomic reference to hold any errors. + * @param latch The countdown latch to signal completion. + */ + public static void configureHttpsPipeline(ChannelPipeline pipeline, HttpRequest request, + HttpProtocolVersion protocol, AtomicReference responseReference, + AtomicReference errorReference, CountDownLatch latch) { + final ChannelHandler httpCodec; + if (HttpProtocolVersion.HTTP_2 == protocol) { + // TODO (alzimmer): InboundHttp2ToHttpAdapter buffers the entire response into a FullHttpResponse. Need to + // create a streaming version of this to support huge response payloads. + Http2Connection http2Connection = new DefaultHttp2Connection(false); + Http2Settings settings = new Http2Settings().headerTableSize(4096) + .maxHeaderListSize(TWO_FIFTY_SIX_KB) + .pushEnabled(false) + .initialWindowSize(TWO_FIFTY_SIX_KB); + Http2FrameListener frameListener = new DelegatingDecompressorFrameListener(http2Connection, + new InboundHttp2ToHttpAdapterBuilder(http2Connection).maxContentLength(Integer.MAX_VALUE) + .propagateSettings(true) + .validateHttpHeaders(true) + .build()); + + httpCodec = new HttpToHttp2ConnectionHandlerBuilder().initialSettings(settings) + .frameListener(frameListener) + .connection(http2Connection) + .validateHeaders(true) + .build(); + } else { // HTTP/1.1 + httpCodec = createCodec(); + } + + Netty4ResponseHandler responseHandler + = new Netty4ResponseHandler(request, responseReference, errorReference, latch); + + if (pipeline.get(Netty4HandlerNames.PROGRESS_AND_TIMEOUT) != null) { + pipeline.addAfter(Netty4HandlerNames.PROGRESS_AND_TIMEOUT, Netty4HandlerNames.HTTP_RESPONSE, + responseHandler); + pipeline.addBefore(Netty4HandlerNames.PROGRESS_AND_TIMEOUT, Netty4HandlerNames.HTTP_CODEC, httpCodec); + } else { + pipeline.addAfter(Netty4HandlerNames.SSL, Netty4HandlerNames.HTTP_CODEC, httpCodec); + pipeline.addAfter(Netty4HandlerNames.HTTP_CODEC, Netty4HandlerNames.HTTP_RESPONSE, responseHandler); + } + } + private Netty4Utility() { } } From 51fa9389bfcc33e0be9657934be4c45b00d6a9a6 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 2 Jul 2025 17:07:07 +0300 Subject: [PATCH 17/57] try and fix deadlock --- .../implementation/Netty4AlpnHandler.java | 1 - .../Netty4ChannelBinaryData.java | 32 +++++++++++-------- .../Netty4EagerConsumeChannelHandler.java | 23 ++++++++----- .../Netty4PipelineCleanupHandler.java | 32 ++++++++++++------- .../implementation/Netty4ResponseHandler.java | 9 +++++- 5 files changed, 62 insertions(+), 35 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index ad3a90754780..514ab7dce34b 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -31,7 +31,6 @@ public final class Netty4AlpnHandler extends ApplicationProtocolNegotiationHandl */ public static final AttributeKey HTTP_PROTOCOL_VERSION_KEY = AttributeKey.valueOf("http-protocol-version"); - private static final int TWO_FIFTY_SIX_KB = 256 * 1024; private final HttpRequest request; private final AtomicReference responseReference; private final AtomicReference errorReference; diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 0f2351c63a69..8aaf42879eeb 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -66,7 +66,7 @@ public byte[] toBytes() { } if (bytes == null) { - drainStream(); + drainStream(true); // Must block to get bytes bytes = eagerContent.toByteArray(); eagerContent = null; } @@ -91,7 +91,8 @@ public T toObject(Type type, ObjectSerializer serializer) { @Override public InputStream toStream() { if (bytes == null) { - return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, this::drainStream); + // Pass a non-blocking drain runnable to the stream's onClose handler. + return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, () -> drainStream(false)); } else { return new ByteArrayInputStream(bytes); } @@ -179,15 +180,17 @@ public BinaryData toReplayableBinaryData() { /** * Ensures the underlying network stream is fully consumed but does not close the channel, - * allowing it to be reused by the connection pool. + * allowing it to be reused by the connection pool. This is a non-blocking operation + * that initiates the draining process. */ @Override public void close() { - drainStream(); + drainStream(false); } - private void drainStream() { + private void drainStream(boolean block) { if (streamDrained.compareAndSet(false, true)) { + // If the handler is already in the pipeline, another drain is in progress. if (channel.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { return; } @@ -203,14 +206,17 @@ private void drainStream() { channel.config().setAutoRead(true); channel.eventLoop().execute(channel::read); - awaitLatch(latch); - - Throwable exception = handler.channelException(); - if (exception != null) { - if (exception instanceof Error) { - throw (Error) exception; - } else { - throw CoreException.from(exception); + + if (block) { + awaitLatch(latch); + + Throwable exception = handler.channelException(); + if (exception != null) { + if (exception instanceof Error) { + throw (Error) exception; + } else { + throw CoreException.from(exception); + } } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 3c9712f462ab..263b1c883223 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -62,7 +62,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { } catch (IOException | RuntimeException ex) { ReferenceCountUtil.release(msg); ctx.fireExceptionCaught(ex); - cleanup(ctx); + triggerRequestComplete(ctx); } } @@ -70,7 +70,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { public void channelReadComplete(ChannelHandlerContext ctx) { ctx.fireChannelReadComplete(); if (lastRead) { - cleanup(ctx); + triggerRequestComplete(ctx); } } @@ -78,7 +78,7 @@ public void channelReadComplete(ChannelHandlerContext ctx) { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; ctx.fireExceptionCaught(cause); - cleanup(ctx); + triggerRequestComplete(ctx); } Throwable channelException() { @@ -88,13 +88,13 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - cleanup(ctx); + triggerRequestComplete(ctx); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - cleanup(ctx); + triggerRequestComplete(ctx); ctx.fireChannelInactive(); } @@ -109,11 +109,18 @@ public void handlerAdded(ChannelHandlerContext ctx) { } } - private void cleanup(ChannelHandlerContext ctx) { - if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { - ctx.pipeline().remove(this); + private void triggerRequestComplete(ChannelHandlerContext ctx) { + if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) == null) { + return; } latch.countDown(); + + Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.requestComplete(ctx); + } else { + ctx.pipeline().remove(this); + } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 37e4e36ba768..b71c69da9322 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -6,8 +6,6 @@ import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; -import io.netty.handler.codec.http.LastHttpContent; -import io.netty.handler.codec.http2.Http2DataFrame; import java.util.ArrayList; import java.util.Collections; @@ -30,7 +28,6 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final Netty4ConnectionPool connectionPool; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); - private boolean lastContentRead; private static final List HANDLERS_TO_REMOVE; @@ -52,11 +49,11 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof LastHttpContent) { - this.lastContentRead = true; - } else if (msg instanceof Http2DataFrame) { - this.lastContentRead = ((Http2DataFrame) msg).isEndStream(); - } + // if (msg instanceof LastHttpContent) { + // this.lastContentRead = true; + // } else if (msg instanceof Http2DataFrame) { + // this.lastContentRead = ((Http2DataFrame) msg).isEndStream(); + // } ctx.fireChannelRead(msg); } @@ -66,9 +63,11 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { // First, let other handlers process the channelReadComplete event. ctx.fireChannelReadComplete(); - if (lastContentRead) { - cleanup(ctx); - } + // if (lastContentRead) { + // // Schedule cleanup to run after the current event processing is complete + // // This prevents modifying the pipeline while it's still being traversed + // ctx.channel().eventLoop().execute(() -> cleanup(ctx)); + // } } @Override @@ -82,10 +81,19 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { ctx.fireChannelInactive(); + ctx.channel().eventLoop().execute(() -> cleanup(ctx)); + } + + /** + * Called externally when the request/response cycle is complete. + * This should be called by the code that manages the request lifecycle, + * not by detecting last content in the pipeline. + */ + public void requestComplete(ChannelHandlerContext ctx) { cleanup(ctx); } - public void cleanup(ChannelHandlerContext ctx) { + private void cleanup(ChannelHandlerContext ctx) { if (!cleanedUp.compareAndSet(false, true)) { return; } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 5d5bf61785ca..34add9cbdfc5 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -166,12 +166,19 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { return; } - ctx.pipeline().remove(this); ctx.fireChannelReadComplete(); responseReference.set(new ResponseStateInfo(ctx.channel(), complete, statusCode, headers, eagerContent, ResponseBodyHandling.getBodyHandling(request, headers), false)); + latch.countDown(); + + Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); + ctx.pipeline().remove(this); + + if (complete && cleanupHandler != null) { + cleanupHandler.requestComplete(ctx); + } } @Override From f3eee88b766e00c10ea1cc17d474486616a464ab Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 2 Jul 2025 19:12:48 +0300 Subject: [PATCH 18/57] Revert "try and fix deadlock" This reverts commit 51fa9389bfcc33e0be9657934be4c45b00d6a9a6. --- .../implementation/Netty4AlpnHandler.java | 1 + .../Netty4ChannelBinaryData.java | 32 ++++++++----------- .../Netty4EagerConsumeChannelHandler.java | 23 +++++-------- .../Netty4PipelineCleanupHandler.java | 32 +++++++------------ .../implementation/Netty4ResponseHandler.java | 9 +----- 5 files changed, 35 insertions(+), 62 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index 514ab7dce34b..ad3a90754780 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -31,6 +31,7 @@ public final class Netty4AlpnHandler extends ApplicationProtocolNegotiationHandl */ public static final AttributeKey HTTP_PROTOCOL_VERSION_KEY = AttributeKey.valueOf("http-protocol-version"); + private static final int TWO_FIFTY_SIX_KB = 256 * 1024; private final HttpRequest request; private final AtomicReference responseReference; private final AtomicReference errorReference; diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 8aaf42879eeb..0f2351c63a69 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -66,7 +66,7 @@ public byte[] toBytes() { } if (bytes == null) { - drainStream(true); // Must block to get bytes + drainStream(); bytes = eagerContent.toByteArray(); eagerContent = null; } @@ -91,8 +91,7 @@ public T toObject(Type type, ObjectSerializer serializer) { @Override public InputStream toStream() { if (bytes == null) { - // Pass a non-blocking drain runnable to the stream's onClose handler. - return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, () -> drainStream(false)); + return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, this::drainStream); } else { return new ByteArrayInputStream(bytes); } @@ -180,17 +179,15 @@ public BinaryData toReplayableBinaryData() { /** * Ensures the underlying network stream is fully consumed but does not close the channel, - * allowing it to be reused by the connection pool. This is a non-blocking operation - * that initiates the draining process. + * allowing it to be reused by the connection pool. */ @Override public void close() { - drainStream(false); + drainStream(); } - private void drainStream(boolean block) { + private void drainStream() { if (streamDrained.compareAndSet(false, true)) { - // If the handler is already in the pipeline, another drain is in progress. if (channel.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { return; } @@ -206,17 +203,14 @@ private void drainStream(boolean block) { channel.config().setAutoRead(true); channel.eventLoop().execute(channel::read); - - if (block) { - awaitLatch(latch); - - Throwable exception = handler.channelException(); - if (exception != null) { - if (exception instanceof Error) { - throw (Error) exception; - } else { - throw CoreException.from(exception); - } + awaitLatch(latch); + + Throwable exception = handler.channelException(); + if (exception != null) { + if (exception instanceof Error) { + throw (Error) exception; + } else { + throw CoreException.from(exception); } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 263b1c883223..3c9712f462ab 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -62,7 +62,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { } catch (IOException | RuntimeException ex) { ReferenceCountUtil.release(msg); ctx.fireExceptionCaught(ex); - triggerRequestComplete(ctx); + cleanup(ctx); } } @@ -70,7 +70,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { public void channelReadComplete(ChannelHandlerContext ctx) { ctx.fireChannelReadComplete(); if (lastRead) { - triggerRequestComplete(ctx); + cleanup(ctx); } } @@ -78,7 +78,7 @@ public void channelReadComplete(ChannelHandlerContext ctx) { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; ctx.fireExceptionCaught(cause); - triggerRequestComplete(ctx); + cleanup(ctx); } Throwable channelException() { @@ -88,13 +88,13 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - triggerRequestComplete(ctx); + cleanup(ctx); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - triggerRequestComplete(ctx); + cleanup(ctx); ctx.fireChannelInactive(); } @@ -109,18 +109,11 @@ public void handlerAdded(ChannelHandlerContext ctx) { } } - private void triggerRequestComplete(ChannelHandlerContext ctx) { - if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) == null) { - return; + private void cleanup(ChannelHandlerContext ctx) { + if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { + ctx.pipeline().remove(this); } latch.countDown(); - - Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.requestComplete(ctx); - } else { - ctx.pipeline().remove(this); - } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index b71c69da9322..37e4e36ba768 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -6,6 +6,8 @@ import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; +import io.netty.handler.codec.http.LastHttpContent; +import io.netty.handler.codec.http2.Http2DataFrame; import java.util.ArrayList; import java.util.Collections; @@ -28,6 +30,7 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final Netty4ConnectionPool connectionPool; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); + private boolean lastContentRead; private static final List HANDLERS_TO_REMOVE; @@ -49,11 +52,11 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - // if (msg instanceof LastHttpContent) { - // this.lastContentRead = true; - // } else if (msg instanceof Http2DataFrame) { - // this.lastContentRead = ((Http2DataFrame) msg).isEndStream(); - // } + if (msg instanceof LastHttpContent) { + this.lastContentRead = true; + } else if (msg instanceof Http2DataFrame) { + this.lastContentRead = ((Http2DataFrame) msg).isEndStream(); + } ctx.fireChannelRead(msg); } @@ -63,11 +66,9 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { // First, let other handlers process the channelReadComplete event. ctx.fireChannelReadComplete(); - // if (lastContentRead) { - // // Schedule cleanup to run after the current event processing is complete - // // This prevents modifying the pipeline while it's still being traversed - // ctx.channel().eventLoop().execute(() -> cleanup(ctx)); - // } + if (lastContentRead) { + cleanup(ctx); + } } @Override @@ -81,19 +82,10 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { ctx.fireChannelInactive(); - ctx.channel().eventLoop().execute(() -> cleanup(ctx)); - } - - /** - * Called externally when the request/response cycle is complete. - * This should be called by the code that manages the request lifecycle, - * not by detecting last content in the pipeline. - */ - public void requestComplete(ChannelHandlerContext ctx) { cleanup(ctx); } - private void cleanup(ChannelHandlerContext ctx) { + public void cleanup(ChannelHandlerContext ctx) { if (!cleanedUp.compareAndSet(false, true)) { return; } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 34add9cbdfc5..5d5bf61785ca 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -166,19 +166,12 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { return; } + ctx.pipeline().remove(this); ctx.fireChannelReadComplete(); responseReference.set(new ResponseStateInfo(ctx.channel(), complete, statusCode, headers, eagerContent, ResponseBodyHandling.getBodyHandling(request, headers), false)); - latch.countDown(); - - Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); - ctx.pipeline().remove(this); - - if (complete && cleanupHandler != null) { - cleanupHandler.requestComplete(ctx); - } } @Override From 61c273b9511d2f39a147720a2770e46f7565c768 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Thu, 3 Jul 2025 15:15:21 +0300 Subject: [PATCH 19/57] deadlock fix v2 --- .../core/shared/LocalTestServer.java | 46 +++++++++---------- .../http-netty4/spotbugs-exclude.xml | 4 ++ .../http/netty4/NettyHttpClient.java | 16 +++++++ .../Netty4ChannelBinaryData.java | 5 ++ .../Netty4ChannelInputStream.java | 35 ++++++++------ .../Netty4EagerConsumeChannelHandler.java | 30 ++++++------ .../Netty4InitiateOneReadHandler.java | 13 ------ .../Netty4PipelineCleanupHandler.java | 46 ++++++------------- .../implementation/Netty4ResponseHandler.java | 10 ++-- 9 files changed, 103 insertions(+), 102 deletions(-) diff --git a/sdk/clientcore/core/src/test/java/io/clientcore/core/shared/LocalTestServer.java b/sdk/clientcore/core/src/test/java/io/clientcore/core/shared/LocalTestServer.java index 97db35950466..66b34cd6ee55 100644 --- a/sdk/clientcore/core/src/test/java/io/clientcore/core/shared/LocalTestServer.java +++ b/sdk/clientcore/core/src/test/java/io/clientcore/core/shared/LocalTestServer.java @@ -60,14 +60,11 @@ public LocalTestServer(HttpProtocolVersion supportedProtocol, boolean includeTls httpConfig.addCustomizer(new SecureRequestCustomizer()); } - List connectionFactories = new ArrayList<>(); - - // SSL/TLS connection factory - if (includeTls) { - String nextProtocol = supportedProtocol == null - ? HttpVersion.HTTP_1_1.asString() - : (supportedProtocol == HttpProtocolVersion.HTTP_1_1) ? HttpVersion.HTTP_1_1.asString() : "alpn"; - + final ServerConnector httpConnector; + if (!includeTls) { + HttpConnectionFactory http11 = new HttpConnectionFactory(httpConfig); + httpConnector = new ServerConnector(server, http11); + } else { Security.addProvider(new OpenSSLProvider()); SslContextFactory.Server sslContextFactory = new SslContextFactory.Server(); sslContextFactory.setProvider("Conscrypt"); @@ -75,30 +72,29 @@ public LocalTestServer(HttpProtocolVersion supportedProtocol, boolean includeTls sslContextFactory.setKeyStorePath(mockKeyStore); sslContextFactory.setKeyStorePassword("password"); sslContextFactory.setKeyManagerPassword("password"); - sslContextFactory.setKeyStorePath(mockKeyStore); sslContextFactory.setTrustStorePassword("password"); sslContextFactory.setTrustAll(true); - connectionFactories.add(new SslConnectionFactory(sslContextFactory, nextProtocol)); - } + sslContextFactory.setCipherComparator(java.util.Comparator.comparingInt(String::hashCode)); - if (supportedProtocol == HttpProtocolVersion.HTTP_2) { - // ALPN connection factory - // HTTP/2 connection factory - ALPNServerConnectionFactory alpn = new ALPNServerConnectionFactory(); - alpn.setDefaultProtocol("h2"); - connectionFactories.add(alpn); - connectionFactories.add(new HTTP2ServerConnectionFactory(httpConfig)); - } + SslConnectionFactory ssl = new SslConnectionFactory(sslContextFactory, "alpn"); + + HttpConnectionFactory http11 = new HttpConnectionFactory(httpConfig); - if (supportedProtocol == null || supportedProtocol == HttpProtocolVersion.HTTP_1_1) { - // HTTP/1.1 connection factory - connectionFactories.add(new HttpConnectionFactory(httpConfig)); + if (supportedProtocol == HttpProtocolVersion.HTTP_2) { + HTTP2ServerConnectionFactory http2 = new HTTP2ServerConnectionFactory(httpConfig); + ALPNServerConnectionFactory alpn = new ALPNServerConnectionFactory(); + alpn.setDefaultProtocol(http11.getProtocol()); + + httpConnector = new ServerConnector(server, ssl, alpn, http2, http11); + } else { + httpConnector = new ServerConnector(server, ssl, http11); + } } - connector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); - connector.setHost("localhost"); - server.addConnector(connector); + this.connector = httpConnector; + this.connector.setHost("localhost"); + server.addConnector(this.connector); ServletContextHandler servletContextHandler = new ServletContextHandler(); servletContextHandler.setContextPath("/"); diff --git a/sdk/clientcore/http-netty4/spotbugs-exclude.xml b/sdk/clientcore/http-netty4/spotbugs-exclude.xml index e2139a1da9f8..77b83264b6e6 100644 --- a/sdk/clientcore/http-netty4/spotbugs-exclude.xml +++ b/sdk/clientcore/http-netty4/spotbugs-exclude.xml @@ -62,4 +62,8 @@ + + + + diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index d251a04eba8e..b285bba48418 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -156,12 +156,17 @@ public Response send(HttpRequest request) { } response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); + Channel channel = info.getResponseChannel(); + if (channel != null) { + cleanupPipeline(channel.pipeline()); + } } else { // Otherwise we aren't finished, handle the remaining content according to the documentation in // 'channelRead()'. BinaryData body = BinaryData.empty(); ResponseBodyHandling bodyHandling = info.getResponseBodyHandling(); Channel channel = info.getResponseChannel(); + ChannelPipeline pipeline = channel.pipeline(); if (bodyHandling == ResponseBodyHandling.IGNORE) { // We're ignoring the response content. CountDownLatch drainLatch = new CountDownLatch(1); @@ -169,6 +174,8 @@ public Response send(HttpRequest request) { }, info.isHttp2())); channel.config().setAutoRead(true); awaitLatch(drainLatch); + + cleanupPipeline(pipeline); } else if (bodyHandling == ResponseBodyHandling.STREAM) { // Body streaming uses a special BinaryData that tracks the firstContent read and the Channel it came // from so it can be consumed when the BinaryData is being used. @@ -198,6 +205,8 @@ public Response send(HttpRequest request) { channel.config().setAutoRead(true); awaitLatch(drainLatch); + cleanupPipeline(pipeline); + body = BinaryData.fromBytes(info.getEagerContent().toByteArray()); } @@ -353,4 +362,11 @@ private Netty4ConnectionPoolKey constructConnectionPoolKey(SocketAddress finalDe return key; } + private void cleanupPipeline(ChannelPipeline pipeline) { + Netty4PipelineCleanupHandler pipelineCleanupHandler = pipeline.get(Netty4PipelineCleanupHandler.class); + if (pipelineCleanupHandler != null) { + pipelineCleanupHandler.cleanup(pipeline.context(pipelineCleanupHandler), false); + } + } + } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 0f2351c63a69..2cd570117aa6 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -213,6 +213,11 @@ private void drainStream() { throw CoreException.from(exception); } } + + Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); + } } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java index 1d8882ffb9d8..5425b68424ca 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java @@ -7,8 +7,10 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; -import java.util.LinkedList; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; /** * Implementation of {@link InputStream} that reads contents from a Netty {@link Channel}. @@ -18,6 +20,9 @@ public final class Netty4ChannelInputStream extends InputStream { private final boolean isHttp2; private final Runnable onClose; + //Ensures the close operation is idempotent. + private final AtomicBoolean closed = new AtomicBoolean(false); + // Indicator for the Channel being fully read. // This will become true before 'streamDone' becomes true, but both may become true in the same operation. // Once this is true, the Channel will never be read again. @@ -28,9 +33,9 @@ public final class Netty4ChannelInputStream extends InputStream { // Once this is true, the stream will never return data again. private boolean streamDone = false; - // Linked list of byte[]s that maintains the last available contents from the Channel / eager content. - // A list is needed as each Channel.read() may result in many channelRead calls. - private final LinkedList additionalBuffers; + // Queue of byte[]s that maintains the last available contents from the Channel / eager content. + // A queue is needed as each Channel.read() may result in many channelRead calls. + private final Queue additionalBuffers; private byte[] currentBuffer; @@ -57,7 +62,7 @@ public final class Netty4ChannelInputStream extends InputStream { this.currentBuffer = new byte[0]; } this.readIndex = 0; - this.additionalBuffers = new LinkedList<>(); + this.additionalBuffers = new ConcurrentLinkedQueue<>(); this.channel = channel; if (channel.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { channel.pipeline().remove(Netty4InitiateOneReadHandler.class); @@ -178,19 +183,21 @@ public long skip(long n) throws IOException { */ @Override public void close() throws IOException { - try { - if (onClose != null && !streamDone) { - onClose.run(); + if (closed.compareAndSet(false, true)) { + try { + if (onClose != null) { + onClose.run(); + } + } finally { + super.close(); + streamDone = true; } - } finally { - super.close(); - streamDone = true; } } private boolean setupNextBuffer() throws IOException { if (!additionalBuffers.isEmpty()) { - currentBuffer = additionalBuffers.pop(); + currentBuffer = additionalBuffers.poll(); readIndex = 0; return true; } else if (readMore()) { @@ -225,7 +232,7 @@ private boolean readMore() throws IOException { byte[] buffer = new byte[byteBuf.readableBytes()]; byteBuf.readBytes(buffer); - additionalBuffers.add(buffer); + additionalBuffers.offer(buffer); }, isHttp2); channel.pipeline().addLast(Netty4HandlerNames.READ_ONE, handler); } @@ -253,7 +260,7 @@ private boolean readMore() throws IOException { } if (!additionalBuffers.isEmpty()) { - currentBuffer = additionalBuffers.pop(); + currentBuffer = additionalBuffers.poll(); readIndex = 0; } else if (channelDone) { // Don't listen to IntelliJ here, channelDone may be false. // This read contained no data and the channel completed, therefore the stream is also completed. diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 3c9712f462ab..0ac3327f9d63 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -57,12 +57,22 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { } else { lastRead = msg instanceof LastHttpContent; } + + if (lastRead) { + latch.countDown(); + } + ctx.fireChannelRead(msg); } catch (IOException | RuntimeException ex) { ReferenceCountUtil.release(msg); - ctx.fireExceptionCaught(ex); - cleanup(ctx); + latch.countDown(); + Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(ctx, true); + } else { + ctx.close(); + } } } @@ -70,15 +80,15 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { public void channelReadComplete(ChannelHandlerContext ctx) { ctx.fireChannelReadComplete(); if (lastRead) { - cleanup(ctx); + latch.countDown(); } } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; + latch.countDown(); ctx.fireExceptionCaught(cause); - cleanup(ctx); } Throwable channelException() { @@ -88,13 +98,13 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - cleanup(ctx); + latch.countDown(); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - cleanup(ctx); + latch.countDown(); ctx.fireChannelInactive(); } @@ -105,15 +115,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { // an exception. Simply counting down the latch would cause the caller to receive // an empty/incomplete data stream without any indication of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); - ctx.pipeline().remove(this); } } - private void cleanup(ChannelHandlerContext ctx) { - if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { - ctx.pipeline().remove(this); - } - - latch.countDown(); - } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java index 0606160b3cea..9ddef608a98c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java @@ -95,9 +95,6 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { @Override public void channelReadComplete(ChannelHandlerContext ctx) { latch.countDown(); - if (lastRead) { - ctx.pipeline().remove(this); - } ctx.fireChannelReadComplete(); } @@ -109,9 +106,6 @@ boolean isChannelConsumed() { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; latch.countDown(); - if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { - ctx.pipeline().remove(this); - } ctx.fireExceptionCaught(cause); } @@ -123,18 +117,12 @@ Throwable channelException() { @Override public void channelUnregistered(ChannelHandlerContext ctx) { latch.countDown(); - if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { - ctx.pipeline().remove(this); - } ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { latch.countDown(); - if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { - ctx.pipeline().remove(this); - } ctx.fireChannelInactive(); } @@ -145,7 +133,6 @@ public void handlerAdded(ChannelHandlerContext ctx) { // an exception. Simply counting down the latch would cause the caller to receive // an empty/incomplete data stream without any indication of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); - ctx.pipeline().remove(this); } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 37e4e36ba768..dc6d56d406a9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -6,8 +6,6 @@ import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; -import io.netty.handler.codec.http.LastHttpContent; -import io.netty.handler.codec.http2.Http2DataFrame; import java.util.ArrayList; import java.util.Collections; @@ -30,7 +28,6 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final Netty4ConnectionPool connectionPool; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); - private boolean lastContentRead; private static final List HANDLERS_TO_REMOVE; @@ -50,46 +47,27 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { this.connectionPool = connectionPool; } - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof LastHttpContent) { - this.lastContentRead = true; - } else if (msg instanceof Http2DataFrame) { - this.lastContentRead = ((Http2DataFrame) msg).isEndStream(); - } - - ctx.fireChannelRead(msg); - } - - @Override - public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { - // First, let other handlers process the channelReadComplete event. - ctx.fireChannelReadComplete(); - - if (lastContentRead) { - cleanup(ctx); - } - } - @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - // An exception has occurred, which means the channel is likely in a bad state. - // We handle this by closing the channel. This prevents it from being - // returned to the connection pool. - ctx.close(); + //ctx.fireExceptionCaught(cause); + cleanup(ctx, true); } @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { + public void channelInactive(ChannelHandlerContext ctx) { ctx.fireChannelInactive(); - cleanup(ctx); + cleanup(ctx, true); } - public void cleanup(ChannelHandlerContext ctx) { + public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { if (!cleanedUp.compareAndSet(false, true)) { return; } + // Always reset autoRead to false before returning a channel to the pool + // to ensure predictable behavior for the next request. + ctx.channel().config().setAutoRead(false); + ChannelPipeline pipeline = ctx.channel().pipeline(); for (String handlerName : HANDLERS_TO_REMOVE) { if (pipeline.get(handlerName) != null) { @@ -101,6 +79,10 @@ public void cleanup(ChannelHandlerContext ctx) { pipeline.remove(this); } - connectionPool.release(ctx.channel()); + if (closeChannel) { + ctx.channel().close(); + } else { + connectionPool.release(ctx.channel()); + } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 5d5bf61785ca..28e8e7cdc7ca 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -175,10 +175,12 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { } @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - setOrSuppressError(errorReference, - new IOException("The channel became inactive before a response was received.")); + public void channelInactive(ChannelHandlerContext ctx) { + if (!started) { + setOrSuppressError(errorReference, + new IOException("The channel became inactive before a response was received.")); + latch.countDown(); + } ctx.fireChannelInactive(); - latch.countDown(); } } From 40b929ab8ddc149a5fea18edeb4a2799d4c120d8 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Thu, 3 Jul 2025 19:38:30 +0300 Subject: [PATCH 20/57] Revert "deadlock fix v2" This reverts commit 61c273b9511d2f39a147720a2770e46f7565c768. --- .../core/shared/LocalTestServer.java | 46 ++++++++++--------- .../http-netty4/spotbugs-exclude.xml | 4 -- .../http/netty4/NettyHttpClient.java | 16 ------- .../Netty4ChannelBinaryData.java | 5 -- .../Netty4ChannelInputStream.java | 35 ++++++-------- .../Netty4EagerConsumeChannelHandler.java | 30 ++++++------ .../Netty4InitiateOneReadHandler.java | 13 ++++++ .../Netty4PipelineCleanupHandler.java | 46 +++++++++++++------ .../implementation/Netty4ResponseHandler.java | 10 ++-- 9 files changed, 102 insertions(+), 103 deletions(-) diff --git a/sdk/clientcore/core/src/test/java/io/clientcore/core/shared/LocalTestServer.java b/sdk/clientcore/core/src/test/java/io/clientcore/core/shared/LocalTestServer.java index 66b34cd6ee55..97db35950466 100644 --- a/sdk/clientcore/core/src/test/java/io/clientcore/core/shared/LocalTestServer.java +++ b/sdk/clientcore/core/src/test/java/io/clientcore/core/shared/LocalTestServer.java @@ -60,11 +60,14 @@ public LocalTestServer(HttpProtocolVersion supportedProtocol, boolean includeTls httpConfig.addCustomizer(new SecureRequestCustomizer()); } - final ServerConnector httpConnector; - if (!includeTls) { - HttpConnectionFactory http11 = new HttpConnectionFactory(httpConfig); - httpConnector = new ServerConnector(server, http11); - } else { + List connectionFactories = new ArrayList<>(); + + // SSL/TLS connection factory + if (includeTls) { + String nextProtocol = supportedProtocol == null + ? HttpVersion.HTTP_1_1.asString() + : (supportedProtocol == HttpProtocolVersion.HTTP_1_1) ? HttpVersion.HTTP_1_1.asString() : "alpn"; + Security.addProvider(new OpenSSLProvider()); SslContextFactory.Server sslContextFactory = new SslContextFactory.Server(); sslContextFactory.setProvider("Conscrypt"); @@ -72,29 +75,30 @@ public LocalTestServer(HttpProtocolVersion supportedProtocol, boolean includeTls sslContextFactory.setKeyStorePath(mockKeyStore); sslContextFactory.setKeyStorePassword("password"); sslContextFactory.setKeyManagerPassword("password"); + sslContextFactory.setKeyStorePath(mockKeyStore); sslContextFactory.setTrustStorePassword("password"); sslContextFactory.setTrustAll(true); - sslContextFactory.setCipherComparator(java.util.Comparator.comparingInt(String::hashCode)); - - SslConnectionFactory ssl = new SslConnectionFactory(sslContextFactory, "alpn"); - - HttpConnectionFactory http11 = new HttpConnectionFactory(httpConfig); + connectionFactories.add(new SslConnectionFactory(sslContextFactory, nextProtocol)); + } - if (supportedProtocol == HttpProtocolVersion.HTTP_2) { - HTTP2ServerConnectionFactory http2 = new HTTP2ServerConnectionFactory(httpConfig); - ALPNServerConnectionFactory alpn = new ALPNServerConnectionFactory(); - alpn.setDefaultProtocol(http11.getProtocol()); + if (supportedProtocol == HttpProtocolVersion.HTTP_2) { + // ALPN connection factory + // HTTP/2 connection factory + ALPNServerConnectionFactory alpn = new ALPNServerConnectionFactory(); + alpn.setDefaultProtocol("h2"); + connectionFactories.add(alpn); + connectionFactories.add(new HTTP2ServerConnectionFactory(httpConfig)); + } - httpConnector = new ServerConnector(server, ssl, alpn, http2, http11); - } else { - httpConnector = new ServerConnector(server, ssl, http11); - } + if (supportedProtocol == null || supportedProtocol == HttpProtocolVersion.HTTP_1_1) { + // HTTP/1.1 connection factory + connectionFactories.add(new HttpConnectionFactory(httpConfig)); } - this.connector = httpConnector; - this.connector.setHost("localhost"); - server.addConnector(this.connector); + connector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); + connector.setHost("localhost"); + server.addConnector(connector); ServletContextHandler servletContextHandler = new ServletContextHandler(); servletContextHandler.setContextPath("/"); diff --git a/sdk/clientcore/http-netty4/spotbugs-exclude.xml b/sdk/clientcore/http-netty4/spotbugs-exclude.xml index 77b83264b6e6..e2139a1da9f8 100644 --- a/sdk/clientcore/http-netty4/spotbugs-exclude.xml +++ b/sdk/clientcore/http-netty4/spotbugs-exclude.xml @@ -62,8 +62,4 @@ - - - - diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index b285bba48418..d251a04eba8e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -156,17 +156,12 @@ public Response send(HttpRequest request) { } response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); - Channel channel = info.getResponseChannel(); - if (channel != null) { - cleanupPipeline(channel.pipeline()); - } } else { // Otherwise we aren't finished, handle the remaining content according to the documentation in // 'channelRead()'. BinaryData body = BinaryData.empty(); ResponseBodyHandling bodyHandling = info.getResponseBodyHandling(); Channel channel = info.getResponseChannel(); - ChannelPipeline pipeline = channel.pipeline(); if (bodyHandling == ResponseBodyHandling.IGNORE) { // We're ignoring the response content. CountDownLatch drainLatch = new CountDownLatch(1); @@ -174,8 +169,6 @@ public Response send(HttpRequest request) { }, info.isHttp2())); channel.config().setAutoRead(true); awaitLatch(drainLatch); - - cleanupPipeline(pipeline); } else if (bodyHandling == ResponseBodyHandling.STREAM) { // Body streaming uses a special BinaryData that tracks the firstContent read and the Channel it came // from so it can be consumed when the BinaryData is being used. @@ -205,8 +198,6 @@ public Response send(HttpRequest request) { channel.config().setAutoRead(true); awaitLatch(drainLatch); - cleanupPipeline(pipeline); - body = BinaryData.fromBytes(info.getEagerContent().toByteArray()); } @@ -362,11 +353,4 @@ private Netty4ConnectionPoolKey constructConnectionPoolKey(SocketAddress finalDe return key; } - private void cleanupPipeline(ChannelPipeline pipeline) { - Netty4PipelineCleanupHandler pipelineCleanupHandler = pipeline.get(Netty4PipelineCleanupHandler.class); - if (pipelineCleanupHandler != null) { - pipelineCleanupHandler.cleanup(pipeline.context(pipelineCleanupHandler), false); - } - } - } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 2cd570117aa6..0f2351c63a69 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -213,11 +213,6 @@ private void drainStream() { throw CoreException.from(exception); } } - - Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); - } } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java index 5425b68424ca..1d8882ffb9d8 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java @@ -7,10 +7,8 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.LinkedList; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; /** * Implementation of {@link InputStream} that reads contents from a Netty {@link Channel}. @@ -20,9 +18,6 @@ public final class Netty4ChannelInputStream extends InputStream { private final boolean isHttp2; private final Runnable onClose; - //Ensures the close operation is idempotent. - private final AtomicBoolean closed = new AtomicBoolean(false); - // Indicator for the Channel being fully read. // This will become true before 'streamDone' becomes true, but both may become true in the same operation. // Once this is true, the Channel will never be read again. @@ -33,9 +28,9 @@ public final class Netty4ChannelInputStream extends InputStream { // Once this is true, the stream will never return data again. private boolean streamDone = false; - // Queue of byte[]s that maintains the last available contents from the Channel / eager content. - // A queue is needed as each Channel.read() may result in many channelRead calls. - private final Queue additionalBuffers; + // Linked list of byte[]s that maintains the last available contents from the Channel / eager content. + // A list is needed as each Channel.read() may result in many channelRead calls. + private final LinkedList additionalBuffers; private byte[] currentBuffer; @@ -62,7 +57,7 @@ public final class Netty4ChannelInputStream extends InputStream { this.currentBuffer = new byte[0]; } this.readIndex = 0; - this.additionalBuffers = new ConcurrentLinkedQueue<>(); + this.additionalBuffers = new LinkedList<>(); this.channel = channel; if (channel.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { channel.pipeline().remove(Netty4InitiateOneReadHandler.class); @@ -183,21 +178,19 @@ public long skip(long n) throws IOException { */ @Override public void close() throws IOException { - if (closed.compareAndSet(false, true)) { - try { - if (onClose != null) { - onClose.run(); - } - } finally { - super.close(); - streamDone = true; + try { + if (onClose != null && !streamDone) { + onClose.run(); } + } finally { + super.close(); + streamDone = true; } } private boolean setupNextBuffer() throws IOException { if (!additionalBuffers.isEmpty()) { - currentBuffer = additionalBuffers.poll(); + currentBuffer = additionalBuffers.pop(); readIndex = 0; return true; } else if (readMore()) { @@ -232,7 +225,7 @@ private boolean readMore() throws IOException { byte[] buffer = new byte[byteBuf.readableBytes()]; byteBuf.readBytes(buffer); - additionalBuffers.offer(buffer); + additionalBuffers.add(buffer); }, isHttp2); channel.pipeline().addLast(Netty4HandlerNames.READ_ONE, handler); } @@ -260,7 +253,7 @@ private boolean readMore() throws IOException { } if (!additionalBuffers.isEmpty()) { - currentBuffer = additionalBuffers.poll(); + currentBuffer = additionalBuffers.pop(); readIndex = 0; } else if (channelDone) { // Don't listen to IntelliJ here, channelDone may be false. // This read contained no data and the channel completed, therefore the stream is also completed. diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 0ac3327f9d63..3c9712f462ab 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -57,22 +57,12 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { } else { lastRead = msg instanceof LastHttpContent; } - - if (lastRead) { - latch.countDown(); - } - ctx.fireChannelRead(msg); } catch (IOException | RuntimeException ex) { ReferenceCountUtil.release(msg); - latch.countDown(); - Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(ctx, true); - } else { - ctx.close(); - } + ctx.fireExceptionCaught(ex); + cleanup(ctx); } } @@ -80,15 +70,15 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { public void channelReadComplete(ChannelHandlerContext ctx) { ctx.fireChannelReadComplete(); if (lastRead) { - latch.countDown(); + cleanup(ctx); } } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; - latch.countDown(); ctx.fireExceptionCaught(cause); + cleanup(ctx); } Throwable channelException() { @@ -98,13 +88,13 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - latch.countDown(); + cleanup(ctx); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - latch.countDown(); + cleanup(ctx); ctx.fireChannelInactive(); } @@ -115,7 +105,15 @@ public void handlerAdded(ChannelHandlerContext ctx) { // an exception. Simply counting down the latch would cause the caller to receive // an empty/incomplete data stream without any indication of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); + ctx.pipeline().remove(this); } } + private void cleanup(ChannelHandlerContext ctx) { + if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { + ctx.pipeline().remove(this); + } + + latch.countDown(); + } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java index 9ddef608a98c..0606160b3cea 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java @@ -95,6 +95,9 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { @Override public void channelReadComplete(ChannelHandlerContext ctx) { latch.countDown(); + if (lastRead) { + ctx.pipeline().remove(this); + } ctx.fireChannelReadComplete(); } @@ -106,6 +109,9 @@ boolean isChannelConsumed() { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; latch.countDown(); + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } ctx.fireExceptionCaught(cause); } @@ -117,12 +123,18 @@ Throwable channelException() { @Override public void channelUnregistered(ChannelHandlerContext ctx) { latch.countDown(); + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { latch.countDown(); + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } ctx.fireChannelInactive(); } @@ -133,6 +145,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { // an exception. Simply counting down the latch would cause the caller to receive // an empty/incomplete data stream without any indication of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); + ctx.pipeline().remove(this); } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index dc6d56d406a9..37e4e36ba768 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -6,6 +6,8 @@ import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; +import io.netty.handler.codec.http.LastHttpContent; +import io.netty.handler.codec.http2.Http2DataFrame; import java.util.ArrayList; import java.util.Collections; @@ -28,6 +30,7 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final Netty4ConnectionPool connectionPool; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); + private boolean lastContentRead; private static final List HANDLERS_TO_REMOVE; @@ -47,27 +50,46 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { this.connectionPool = connectionPool; } + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof LastHttpContent) { + this.lastContentRead = true; + } else if (msg instanceof Http2DataFrame) { + this.lastContentRead = ((Http2DataFrame) msg).isEndStream(); + } + + ctx.fireChannelRead(msg); + } + + @Override + public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { + // First, let other handlers process the channelReadComplete event. + ctx.fireChannelReadComplete(); + + if (lastContentRead) { + cleanup(ctx); + } + } + @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - //ctx.fireExceptionCaught(cause); - cleanup(ctx, true); + // An exception has occurred, which means the channel is likely in a bad state. + // We handle this by closing the channel. This prevents it from being + // returned to the connection pool. + ctx.close(); } @Override - public void channelInactive(ChannelHandlerContext ctx) { + public void channelInactive(ChannelHandlerContext ctx) throws Exception { ctx.fireChannelInactive(); - cleanup(ctx, true); + cleanup(ctx); } - public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { + public void cleanup(ChannelHandlerContext ctx) { if (!cleanedUp.compareAndSet(false, true)) { return; } - // Always reset autoRead to false before returning a channel to the pool - // to ensure predictable behavior for the next request. - ctx.channel().config().setAutoRead(false); - ChannelPipeline pipeline = ctx.channel().pipeline(); for (String handlerName : HANDLERS_TO_REMOVE) { if (pipeline.get(handlerName) != null) { @@ -79,10 +101,6 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { pipeline.remove(this); } - if (closeChannel) { - ctx.channel().close(); - } else { - connectionPool.release(ctx.channel()); - } + connectionPool.release(ctx.channel()); } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 28e8e7cdc7ca..5d5bf61785ca 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -175,12 +175,10 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { } @Override - public void channelInactive(ChannelHandlerContext ctx) { - if (!started) { - setOrSuppressError(errorReference, - new IOException("The channel became inactive before a response was received.")); - latch.countDown(); - } + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + setOrSuppressError(errorReference, + new IOException("The channel became inactive before a response was received.")); ctx.fireChannelInactive(); + latch.countDown(); } } From 792a06e23e4316f7f47e856bfae83188ea85d1b6 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Thu, 3 Jul 2025 19:59:23 +0300 Subject: [PATCH 21/57] deadlock fix v3 --- .../http-netty4/spotbugs-exclude.xml | 4 ++ .../http/netty4/NettyHttpClient.java | 16 +++++++ .../Netty4ChannelBinaryData.java | 5 +++ .../Netty4ChannelInputStream.java | 35 +++++++++------- .../Netty4EagerConsumeChannelHandler.java | 24 +++++------ .../Netty4InitiateOneReadHandler.java | 13 ------ .../Netty4PipelineCleanupHandler.java | 42 ++++++------------- .../implementation/Netty4ResponseHandler.java | 10 +++-- 8 files changed, 75 insertions(+), 74 deletions(-) diff --git a/sdk/clientcore/http-netty4/spotbugs-exclude.xml b/sdk/clientcore/http-netty4/spotbugs-exclude.xml index e2139a1da9f8..77b83264b6e6 100644 --- a/sdk/clientcore/http-netty4/spotbugs-exclude.xml +++ b/sdk/clientcore/http-netty4/spotbugs-exclude.xml @@ -62,4 +62,8 @@ + + + + diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index d251a04eba8e..b285bba48418 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -156,12 +156,17 @@ public Response send(HttpRequest request) { } response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); + Channel channel = info.getResponseChannel(); + if (channel != null) { + cleanupPipeline(channel.pipeline()); + } } else { // Otherwise we aren't finished, handle the remaining content according to the documentation in // 'channelRead()'. BinaryData body = BinaryData.empty(); ResponseBodyHandling bodyHandling = info.getResponseBodyHandling(); Channel channel = info.getResponseChannel(); + ChannelPipeline pipeline = channel.pipeline(); if (bodyHandling == ResponseBodyHandling.IGNORE) { // We're ignoring the response content. CountDownLatch drainLatch = new CountDownLatch(1); @@ -169,6 +174,8 @@ public Response send(HttpRequest request) { }, info.isHttp2())); channel.config().setAutoRead(true); awaitLatch(drainLatch); + + cleanupPipeline(pipeline); } else if (bodyHandling == ResponseBodyHandling.STREAM) { // Body streaming uses a special BinaryData that tracks the firstContent read and the Channel it came // from so it can be consumed when the BinaryData is being used. @@ -198,6 +205,8 @@ public Response send(HttpRequest request) { channel.config().setAutoRead(true); awaitLatch(drainLatch); + cleanupPipeline(pipeline); + body = BinaryData.fromBytes(info.getEagerContent().toByteArray()); } @@ -353,4 +362,11 @@ private Netty4ConnectionPoolKey constructConnectionPoolKey(SocketAddress finalDe return key; } + private void cleanupPipeline(ChannelPipeline pipeline) { + Netty4PipelineCleanupHandler pipelineCleanupHandler = pipeline.get(Netty4PipelineCleanupHandler.class); + if (pipelineCleanupHandler != null) { + pipelineCleanupHandler.cleanup(pipeline.context(pipelineCleanupHandler), false); + } + } + } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 0f2351c63a69..2cd570117aa6 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -213,6 +213,11 @@ private void drainStream() { throw CoreException.from(exception); } } + + Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); + } } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java index 1d8882ffb9d8..c2bc9af1a18a 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java @@ -7,8 +7,10 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; -import java.util.LinkedList; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; /** * Implementation of {@link InputStream} that reads contents from a Netty {@link Channel}. @@ -28,9 +30,12 @@ public final class Netty4ChannelInputStream extends InputStream { // Once this is true, the stream will never return data again. private boolean streamDone = false; - // Linked list of byte[]s that maintains the last available contents from the Channel / eager content. - // A list is needed as each Channel.read() may result in many channelRead calls. - private final LinkedList additionalBuffers; + //Ensures the close operation is idempotent. + private final AtomicBoolean closed = new AtomicBoolean(false); + + // Queue of byte[]s that maintains the last available contents from the Channel / eager content. + // A queue is needed as each Channel.read() may result in many channelRead calls. + private final Queue additionalBuffers; private byte[] currentBuffer; @@ -57,7 +62,7 @@ public final class Netty4ChannelInputStream extends InputStream { this.currentBuffer = new byte[0]; } this.readIndex = 0; - this.additionalBuffers = new LinkedList<>(); + this.additionalBuffers = new ConcurrentLinkedQueue<>(); this.channel = channel; if (channel.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { channel.pipeline().remove(Netty4InitiateOneReadHandler.class); @@ -178,19 +183,21 @@ public long skip(long n) throws IOException { */ @Override public void close() throws IOException { - try { - if (onClose != null && !streamDone) { - onClose.run(); + if (closed.compareAndSet(false, true)) { + try { + if (onClose != null) { + onClose.run(); + } + } finally { + super.close(); + streamDone = true; } - } finally { - super.close(); - streamDone = true; } } private boolean setupNextBuffer() throws IOException { if (!additionalBuffers.isEmpty()) { - currentBuffer = additionalBuffers.pop(); + currentBuffer = additionalBuffers.poll(); readIndex = 0; return true; } else if (readMore()) { @@ -225,7 +232,7 @@ private boolean readMore() throws IOException { byte[] buffer = new byte[byteBuf.readableBytes()]; byteBuf.readBytes(buffer); - additionalBuffers.add(buffer); + additionalBuffers.offer(buffer); }, isHttp2); channel.pipeline().addLast(Netty4HandlerNames.READ_ONE, handler); } @@ -253,7 +260,7 @@ private boolean readMore() throws IOException { } if (!additionalBuffers.isEmpty()) { - currentBuffer = additionalBuffers.pop(); + currentBuffer = additionalBuffers.poll(); readIndex = 0; } else if (channelDone) { // Don't listen to IntelliJ here, channelDone may be false. // This read contained no data and the channel completed, therefore the stream is also completed. diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 3c9712f462ab..fee6dfac3e31 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -57,12 +57,17 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { } else { lastRead = msg instanceof LastHttpContent; } + + if (lastRead) { + latch.countDown(); + } + ctx.fireChannelRead(msg); } catch (IOException | RuntimeException ex) { ReferenceCountUtil.release(msg); ctx.fireExceptionCaught(ex); - cleanup(ctx); + latch.countDown(); } } @@ -70,7 +75,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { public void channelReadComplete(ChannelHandlerContext ctx) { ctx.fireChannelReadComplete(); if (lastRead) { - cleanup(ctx); + latch.countDown(); } } @@ -78,7 +83,7 @@ public void channelReadComplete(ChannelHandlerContext ctx) { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; ctx.fireExceptionCaught(cause); - cleanup(ctx); + latch.countDown(); } Throwable channelException() { @@ -88,13 +93,13 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - cleanup(ctx); + latch.countDown(); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - cleanup(ctx); + latch.countDown(); ctx.fireChannelInactive(); } @@ -105,15 +110,6 @@ public void handlerAdded(ChannelHandlerContext ctx) { // an exception. Simply counting down the latch would cause the caller to receive // an empty/incomplete data stream without any indication of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); - ctx.pipeline().remove(this); } } - - private void cleanup(ChannelHandlerContext ctx) { - if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { - ctx.pipeline().remove(this); - } - - latch.countDown(); - } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java index 0606160b3cea..9ddef608a98c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java @@ -95,9 +95,6 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { @Override public void channelReadComplete(ChannelHandlerContext ctx) { latch.countDown(); - if (lastRead) { - ctx.pipeline().remove(this); - } ctx.fireChannelReadComplete(); } @@ -109,9 +106,6 @@ boolean isChannelConsumed() { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; latch.countDown(); - if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { - ctx.pipeline().remove(this); - } ctx.fireExceptionCaught(cause); } @@ -123,18 +117,12 @@ Throwable channelException() { @Override public void channelUnregistered(ChannelHandlerContext ctx) { latch.countDown(); - if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { - ctx.pipeline().remove(this); - } ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { latch.countDown(); - if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { - ctx.pipeline().remove(this); - } ctx.fireChannelInactive(); } @@ -145,7 +133,6 @@ public void handlerAdded(ChannelHandlerContext ctx) { // an exception. Simply counting down the latch would cause the caller to receive // an empty/incomplete data stream without any indication of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); - ctx.pipeline().remove(this); } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 37e4e36ba768..c4a9ac818f1c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -6,8 +6,6 @@ import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; -import io.netty.handler.codec.http.LastHttpContent; -import io.netty.handler.codec.http2.Http2DataFrame; import java.util.ArrayList; import java.util.Collections; @@ -30,7 +28,6 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final Netty4ConnectionPool connectionPool; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); - private boolean lastContentRead; private static final List HANDLERS_TO_REMOVE; @@ -50,46 +47,29 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { this.connectionPool = connectionPool; } - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof LastHttpContent) { - this.lastContentRead = true; - } else if (msg instanceof Http2DataFrame) { - this.lastContentRead = ((Http2DataFrame) msg).isEndStream(); - } - - ctx.fireChannelRead(msg); - } - - @Override - public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { - // First, let other handlers process the channelReadComplete event. - ctx.fireChannelReadComplete(); - - if (lastContentRead) { - cleanup(ctx); - } - } - @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { // An exception has occurred, which means the channel is likely in a bad state. // We handle this by closing the channel. This prevents it from being // returned to the connection pool. - ctx.close(); + cleanup(ctx, true); } @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { + public void channelInactive(ChannelHandlerContext ctx) { ctx.fireChannelInactive(); - cleanup(ctx); + cleanup(ctx, true); } - public void cleanup(ChannelHandlerContext ctx) { + public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { if (!cleanedUp.compareAndSet(false, true)) { return; } + // Always reset autoRead to false before returning a channel to the pool + // to ensure predictable behavior for the next request. + ctx.channel().config().setAutoRead(false); + ChannelPipeline pipeline = ctx.channel().pipeline(); for (String handlerName : HANDLERS_TO_REMOVE) { if (pipeline.get(handlerName) != null) { @@ -101,6 +81,10 @@ public void cleanup(ChannelHandlerContext ctx) { pipeline.remove(this); } - connectionPool.release(ctx.channel()); + if (closeChannel) { + ctx.channel().close(); + } else { + connectionPool.release(ctx.channel()); + } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 5d5bf61785ca..28e8e7cdc7ca 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -175,10 +175,12 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { } @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - setOrSuppressError(errorReference, - new IOException("The channel became inactive before a response was received.")); + public void channelInactive(ChannelHandlerContext ctx) { + if (!started) { + setOrSuppressError(errorReference, + new IOException("The channel became inactive before a response was received.")); + latch.countDown(); + } ctx.fireChannelInactive(); - latch.countDown(); } } From 56bbb978d9182ca3937190c39f9d4ab220bc6f15 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Thu, 3 Jul 2025 20:12:09 +0300 Subject: [PATCH 22/57] Revert "deadlock fix v3" This reverts commit 792a06e23e4316f7f47e856bfae83188ea85d1b6. --- .../http-netty4/spotbugs-exclude.xml | 4 -- .../http/netty4/NettyHttpClient.java | 16 ------- .../Netty4ChannelBinaryData.java | 5 --- .../Netty4ChannelInputStream.java | 35 +++++++--------- .../Netty4EagerConsumeChannelHandler.java | 24 ++++++----- .../Netty4InitiateOneReadHandler.java | 13 ++++++ .../Netty4PipelineCleanupHandler.java | 42 +++++++++++++------ .../implementation/Netty4ResponseHandler.java | 10 ++--- 8 files changed, 74 insertions(+), 75 deletions(-) diff --git a/sdk/clientcore/http-netty4/spotbugs-exclude.xml b/sdk/clientcore/http-netty4/spotbugs-exclude.xml index 77b83264b6e6..e2139a1da9f8 100644 --- a/sdk/clientcore/http-netty4/spotbugs-exclude.xml +++ b/sdk/clientcore/http-netty4/spotbugs-exclude.xml @@ -62,8 +62,4 @@ - - - - diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index b285bba48418..d251a04eba8e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -156,17 +156,12 @@ public Response send(HttpRequest request) { } response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); - Channel channel = info.getResponseChannel(); - if (channel != null) { - cleanupPipeline(channel.pipeline()); - } } else { // Otherwise we aren't finished, handle the remaining content according to the documentation in // 'channelRead()'. BinaryData body = BinaryData.empty(); ResponseBodyHandling bodyHandling = info.getResponseBodyHandling(); Channel channel = info.getResponseChannel(); - ChannelPipeline pipeline = channel.pipeline(); if (bodyHandling == ResponseBodyHandling.IGNORE) { // We're ignoring the response content. CountDownLatch drainLatch = new CountDownLatch(1); @@ -174,8 +169,6 @@ public Response send(HttpRequest request) { }, info.isHttp2())); channel.config().setAutoRead(true); awaitLatch(drainLatch); - - cleanupPipeline(pipeline); } else if (bodyHandling == ResponseBodyHandling.STREAM) { // Body streaming uses a special BinaryData that tracks the firstContent read and the Channel it came // from so it can be consumed when the BinaryData is being used. @@ -205,8 +198,6 @@ public Response send(HttpRequest request) { channel.config().setAutoRead(true); awaitLatch(drainLatch); - cleanupPipeline(pipeline); - body = BinaryData.fromBytes(info.getEagerContent().toByteArray()); } @@ -362,11 +353,4 @@ private Netty4ConnectionPoolKey constructConnectionPoolKey(SocketAddress finalDe return key; } - private void cleanupPipeline(ChannelPipeline pipeline) { - Netty4PipelineCleanupHandler pipelineCleanupHandler = pipeline.get(Netty4PipelineCleanupHandler.class); - if (pipelineCleanupHandler != null) { - pipelineCleanupHandler.cleanup(pipeline.context(pipelineCleanupHandler), false); - } - } - } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 2cd570117aa6..0f2351c63a69 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -213,11 +213,6 @@ private void drainStream() { throw CoreException.from(exception); } } - - Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); - } } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java index c2bc9af1a18a..1d8882ffb9d8 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java @@ -7,10 +7,8 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.LinkedList; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; /** * Implementation of {@link InputStream} that reads contents from a Netty {@link Channel}. @@ -30,12 +28,9 @@ public final class Netty4ChannelInputStream extends InputStream { // Once this is true, the stream will never return data again. private boolean streamDone = false; - //Ensures the close operation is idempotent. - private final AtomicBoolean closed = new AtomicBoolean(false); - - // Queue of byte[]s that maintains the last available contents from the Channel / eager content. - // A queue is needed as each Channel.read() may result in many channelRead calls. - private final Queue additionalBuffers; + // Linked list of byte[]s that maintains the last available contents from the Channel / eager content. + // A list is needed as each Channel.read() may result in many channelRead calls. + private final LinkedList additionalBuffers; private byte[] currentBuffer; @@ -62,7 +57,7 @@ public final class Netty4ChannelInputStream extends InputStream { this.currentBuffer = new byte[0]; } this.readIndex = 0; - this.additionalBuffers = new ConcurrentLinkedQueue<>(); + this.additionalBuffers = new LinkedList<>(); this.channel = channel; if (channel.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { channel.pipeline().remove(Netty4InitiateOneReadHandler.class); @@ -183,21 +178,19 @@ public long skip(long n) throws IOException { */ @Override public void close() throws IOException { - if (closed.compareAndSet(false, true)) { - try { - if (onClose != null) { - onClose.run(); - } - } finally { - super.close(); - streamDone = true; + try { + if (onClose != null && !streamDone) { + onClose.run(); } + } finally { + super.close(); + streamDone = true; } } private boolean setupNextBuffer() throws IOException { if (!additionalBuffers.isEmpty()) { - currentBuffer = additionalBuffers.poll(); + currentBuffer = additionalBuffers.pop(); readIndex = 0; return true; } else if (readMore()) { @@ -232,7 +225,7 @@ private boolean readMore() throws IOException { byte[] buffer = new byte[byteBuf.readableBytes()]; byteBuf.readBytes(buffer); - additionalBuffers.offer(buffer); + additionalBuffers.add(buffer); }, isHttp2); channel.pipeline().addLast(Netty4HandlerNames.READ_ONE, handler); } @@ -260,7 +253,7 @@ private boolean readMore() throws IOException { } if (!additionalBuffers.isEmpty()) { - currentBuffer = additionalBuffers.poll(); + currentBuffer = additionalBuffers.pop(); readIndex = 0; } else if (channelDone) { // Don't listen to IntelliJ here, channelDone may be false. // This read contained no data and the channel completed, therefore the stream is also completed. diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index fee6dfac3e31..3c9712f462ab 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -57,17 +57,12 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { } else { lastRead = msg instanceof LastHttpContent; } - - if (lastRead) { - latch.countDown(); - } - ctx.fireChannelRead(msg); } catch (IOException | RuntimeException ex) { ReferenceCountUtil.release(msg); ctx.fireExceptionCaught(ex); - latch.countDown(); + cleanup(ctx); } } @@ -75,7 +70,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { public void channelReadComplete(ChannelHandlerContext ctx) { ctx.fireChannelReadComplete(); if (lastRead) { - latch.countDown(); + cleanup(ctx); } } @@ -83,7 +78,7 @@ public void channelReadComplete(ChannelHandlerContext ctx) { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; ctx.fireExceptionCaught(cause); - latch.countDown(); + cleanup(ctx); } Throwable channelException() { @@ -93,13 +88,13 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - latch.countDown(); + cleanup(ctx); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - latch.countDown(); + cleanup(ctx); ctx.fireChannelInactive(); } @@ -110,6 +105,15 @@ public void handlerAdded(ChannelHandlerContext ctx) { // an exception. Simply counting down the latch would cause the caller to receive // an empty/incomplete data stream without any indication of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); + ctx.pipeline().remove(this); } } + + private void cleanup(ChannelHandlerContext ctx) { + if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { + ctx.pipeline().remove(this); + } + + latch.countDown(); + } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java index 9ddef608a98c..0606160b3cea 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java @@ -95,6 +95,9 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { @Override public void channelReadComplete(ChannelHandlerContext ctx) { latch.countDown(); + if (lastRead) { + ctx.pipeline().remove(this); + } ctx.fireChannelReadComplete(); } @@ -106,6 +109,9 @@ boolean isChannelConsumed() { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; latch.countDown(); + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } ctx.fireExceptionCaught(cause); } @@ -117,12 +123,18 @@ Throwable channelException() { @Override public void channelUnregistered(ChannelHandlerContext ctx) { latch.countDown(); + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { latch.countDown(); + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } ctx.fireChannelInactive(); } @@ -133,6 +145,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { // an exception. Simply counting down the latch would cause the caller to receive // an empty/incomplete data stream without any indication of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); + ctx.pipeline().remove(this); } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index c4a9ac818f1c..37e4e36ba768 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -6,6 +6,8 @@ import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; +import io.netty.handler.codec.http.LastHttpContent; +import io.netty.handler.codec.http2.Http2DataFrame; import java.util.ArrayList; import java.util.Collections; @@ -28,6 +30,7 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final Netty4ConnectionPool connectionPool; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); + private boolean lastContentRead; private static final List HANDLERS_TO_REMOVE; @@ -47,29 +50,46 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { this.connectionPool = connectionPool; } + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof LastHttpContent) { + this.lastContentRead = true; + } else if (msg instanceof Http2DataFrame) { + this.lastContentRead = ((Http2DataFrame) msg).isEndStream(); + } + + ctx.fireChannelRead(msg); + } + + @Override + public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { + // First, let other handlers process the channelReadComplete event. + ctx.fireChannelReadComplete(); + + if (lastContentRead) { + cleanup(ctx); + } + } + @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { // An exception has occurred, which means the channel is likely in a bad state. // We handle this by closing the channel. This prevents it from being // returned to the connection pool. - cleanup(ctx, true); + ctx.close(); } @Override - public void channelInactive(ChannelHandlerContext ctx) { + public void channelInactive(ChannelHandlerContext ctx) throws Exception { ctx.fireChannelInactive(); - cleanup(ctx, true); + cleanup(ctx); } - public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { + public void cleanup(ChannelHandlerContext ctx) { if (!cleanedUp.compareAndSet(false, true)) { return; } - // Always reset autoRead to false before returning a channel to the pool - // to ensure predictable behavior for the next request. - ctx.channel().config().setAutoRead(false); - ChannelPipeline pipeline = ctx.channel().pipeline(); for (String handlerName : HANDLERS_TO_REMOVE) { if (pipeline.get(handlerName) != null) { @@ -81,10 +101,6 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { pipeline.remove(this); } - if (closeChannel) { - ctx.channel().close(); - } else { - connectionPool.release(ctx.channel()); - } + connectionPool.release(ctx.channel()); } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 28e8e7cdc7ca..5d5bf61785ca 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -175,12 +175,10 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { } @Override - public void channelInactive(ChannelHandlerContext ctx) { - if (!started) { - setOrSuppressError(errorReference, - new IOException("The channel became inactive before a response was received.")); - latch.countDown(); - } + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + setOrSuppressError(errorReference, + new IOException("The channel became inactive before a response was received.")); ctx.fireChannelInactive(); + latch.countDown(); } } From f849ec8eab856b6d4721123f71bb8ce5d0f6d442 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Fri, 4 Jul 2025 08:09:43 +0300 Subject: [PATCH 23/57] deadlock fix attempt v4 --- .../http-netty4/spotbugs-exclude.xml | 4 ++ .../Netty4ChannelInputStream.java | 17 ++++---- .../Netty4EagerConsumeChannelHandler.java | 19 +++------ .../Netty4InitiateOneReadHandler.java | 13 ------ .../Netty4PipelineCleanupHandler.java | 42 ++++++------------- 5 files changed, 31 insertions(+), 64 deletions(-) diff --git a/sdk/clientcore/http-netty4/spotbugs-exclude.xml b/sdk/clientcore/http-netty4/spotbugs-exclude.xml index e2139a1da9f8..77b83264b6e6 100644 --- a/sdk/clientcore/http-netty4/spotbugs-exclude.xml +++ b/sdk/clientcore/http-netty4/spotbugs-exclude.xml @@ -62,4 +62,8 @@ + + + + diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java index 1d8882ffb9d8..952755171016 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java @@ -7,7 +7,8 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; -import java.util.LinkedList; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; /** @@ -28,9 +29,9 @@ public final class Netty4ChannelInputStream extends InputStream { // Once this is true, the stream will never return data again. private boolean streamDone = false; - // Linked list of byte[]s that maintains the last available contents from the Channel / eager content. - // A list is needed as each Channel.read() may result in many channelRead calls. - private final LinkedList additionalBuffers; + // Queue of byte[]s that maintains the last available contents from the Channel / eager content. + // A queue is needed as each Channel.read() may result in many channelRead calls. + private final Queue additionalBuffers; private byte[] currentBuffer; @@ -57,7 +58,7 @@ public final class Netty4ChannelInputStream extends InputStream { this.currentBuffer = new byte[0]; } this.readIndex = 0; - this.additionalBuffers = new LinkedList<>(); + this.additionalBuffers = new ConcurrentLinkedQueue<>(); this.channel = channel; if (channel.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { channel.pipeline().remove(Netty4InitiateOneReadHandler.class); @@ -190,7 +191,7 @@ public void close() throws IOException { private boolean setupNextBuffer() throws IOException { if (!additionalBuffers.isEmpty()) { - currentBuffer = additionalBuffers.pop(); + currentBuffer = additionalBuffers.poll(); readIndex = 0; return true; } else if (readMore()) { @@ -225,7 +226,7 @@ private boolean readMore() throws IOException { byte[] buffer = new byte[byteBuf.readableBytes()]; byteBuf.readBytes(buffer); - additionalBuffers.add(buffer); + additionalBuffers.offer(buffer); }, isHttp2); channel.pipeline().addLast(Netty4HandlerNames.READ_ONE, handler); } @@ -253,7 +254,7 @@ private boolean readMore() throws IOException { } if (!additionalBuffers.isEmpty()) { - currentBuffer = additionalBuffers.pop(); + currentBuffer = additionalBuffers.poll(); readIndex = 0; } else if (channelDone) { // Don't listen to IntelliJ here, channelDone may be false. // This read contained no data and the channel completed, therefore the stream is also completed. diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 3c9712f462ab..5f6ac8155696 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -62,7 +62,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { } catch (IOException | RuntimeException ex) { ReferenceCountUtil.release(msg); ctx.fireExceptionCaught(ex); - cleanup(ctx); + latch.countDown(); } } @@ -70,7 +70,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { public void channelReadComplete(ChannelHandlerContext ctx) { ctx.fireChannelReadComplete(); if (lastRead) { - cleanup(ctx); + latch.countDown(); } } @@ -78,7 +78,7 @@ public void channelReadComplete(ChannelHandlerContext ctx) { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; ctx.fireExceptionCaught(cause); - cleanup(ctx); + latch.countDown(); } Throwable channelException() { @@ -88,13 +88,13 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - cleanup(ctx); + latch.countDown(); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - cleanup(ctx); + latch.countDown(); ctx.fireChannelInactive(); } @@ -105,15 +105,6 @@ public void handlerAdded(ChannelHandlerContext ctx) { // an exception. Simply counting down the latch would cause the caller to receive // an empty/incomplete data stream without any indication of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); - ctx.pipeline().remove(this); - } - } - - private void cleanup(ChannelHandlerContext ctx) { - if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { - ctx.pipeline().remove(this); } - - latch.countDown(); } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java index 0606160b3cea..9ddef608a98c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java @@ -95,9 +95,6 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { @Override public void channelReadComplete(ChannelHandlerContext ctx) { latch.countDown(); - if (lastRead) { - ctx.pipeline().remove(this); - } ctx.fireChannelReadComplete(); } @@ -109,9 +106,6 @@ boolean isChannelConsumed() { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; latch.countDown(); - if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { - ctx.pipeline().remove(this); - } ctx.fireExceptionCaught(cause); } @@ -123,18 +117,12 @@ Throwable channelException() { @Override public void channelUnregistered(ChannelHandlerContext ctx) { latch.countDown(); - if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { - ctx.pipeline().remove(this); - } ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { latch.countDown(); - if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { - ctx.pipeline().remove(this); - } ctx.fireChannelInactive(); } @@ -145,7 +133,6 @@ public void handlerAdded(ChannelHandlerContext ctx) { // an exception. Simply counting down the latch would cause the caller to receive // an empty/incomplete data stream without any indication of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); - ctx.pipeline().remove(this); } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 37e4e36ba768..c4a9ac818f1c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -6,8 +6,6 @@ import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; -import io.netty.handler.codec.http.LastHttpContent; -import io.netty.handler.codec.http2.Http2DataFrame; import java.util.ArrayList; import java.util.Collections; @@ -30,7 +28,6 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final Netty4ConnectionPool connectionPool; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); - private boolean lastContentRead; private static final List HANDLERS_TO_REMOVE; @@ -50,46 +47,29 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { this.connectionPool = connectionPool; } - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof LastHttpContent) { - this.lastContentRead = true; - } else if (msg instanceof Http2DataFrame) { - this.lastContentRead = ((Http2DataFrame) msg).isEndStream(); - } - - ctx.fireChannelRead(msg); - } - - @Override - public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { - // First, let other handlers process the channelReadComplete event. - ctx.fireChannelReadComplete(); - - if (lastContentRead) { - cleanup(ctx); - } - } - @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { // An exception has occurred, which means the channel is likely in a bad state. // We handle this by closing the channel. This prevents it from being // returned to the connection pool. - ctx.close(); + cleanup(ctx, true); } @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { + public void channelInactive(ChannelHandlerContext ctx) { ctx.fireChannelInactive(); - cleanup(ctx); + cleanup(ctx, true); } - public void cleanup(ChannelHandlerContext ctx) { + public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { if (!cleanedUp.compareAndSet(false, true)) { return; } + // Always reset autoRead to false before returning a channel to the pool + // to ensure predictable behavior for the next request. + ctx.channel().config().setAutoRead(false); + ChannelPipeline pipeline = ctx.channel().pipeline(); for (String handlerName : HANDLERS_TO_REMOVE) { if (pipeline.get(handlerName) != null) { @@ -101,6 +81,10 @@ public void cleanup(ChannelHandlerContext ctx) { pipeline.remove(this); } - connectionPool.release(ctx.channel()); + if (closeChannel) { + ctx.channel().close(); + } else { + connectionPool.release(ctx.channel()); + } } } From 140a095e0d53c9dbe84f233d4a4d954fa3e413c9 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Fri, 4 Jul 2025 08:20:33 +0300 Subject: [PATCH 24/57] fix on input stream close method --- .../http/netty4/implementation/Netty4ChannelInputStream.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java index 952755171016..de5d4c96c0af 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java @@ -180,11 +180,13 @@ public long skip(long n) throws IOException { @Override public void close() throws IOException { try { - if (onClose != null && !streamDone) { + if (onClose != null) { onClose.run(); } } finally { super.close(); + currentBuffer = null; + additionalBuffers.clear(); streamDone = true; } } From c0f9beb364a99e2318136fa31da676e488890676 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Fri, 4 Jul 2025 10:14:08 +0300 Subject: [PATCH 25/57] deadlock fix attempt v5 --- .../clientcore/http/netty4/NettyHttpClient.java | 14 ++++++++++++-- .../implementation/Netty4ChannelBinaryData.java | 6 +++++- .../Netty4EagerConsumeChannelHandler.java | 10 ++++++++++ .../Netty4PipelineCleanupHandler.java | 15 ++++----------- 4 files changed, 31 insertions(+), 14 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index d251a04eba8e..4f9c2c42de92 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -144,6 +144,7 @@ public Response send(HttpRequest request) { } Response response; + Channel channelToRelease = null; if (info.isChannelConsumptionComplete()) { // The network response is already complete, handle creating our Response based on the request method and // response headers. @@ -154,7 +155,7 @@ public Response send(HttpRequest request) { // there was body content. body = BinaryData.fromBytes(eagerContent.toByteArray()); } - + channelToRelease = info.getResponseChannel(); response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); } else { // Otherwise we aren't finished, handle the remaining content according to the documentation in @@ -169,6 +170,7 @@ public Response send(HttpRequest request) { }, info.isHttp2())); channel.config().setAutoRead(true); awaitLatch(drainLatch); + channelToRelease = channel; } else if (bodyHandling == ResponseBodyHandling.STREAM) { // Body streaming uses a special BinaryData that tracks the firstContent read and the Channel it came // from so it can be consumed when the BinaryData is being used. @@ -197,13 +199,21 @@ public Response send(HttpRequest request) { }, info.isHttp2())); channel.config().setAutoRead(true); awaitLatch(drainLatch); - + channelToRelease = channel; body = BinaryData.fromBytes(info.getEagerContent().toByteArray()); } response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); } + if (channelToRelease != null) { + Netty4PipelineCleanupHandler cleanupHandler + = channelToRelease.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(channelToRelease.pipeline().context(cleanupHandler)); + } + } + if (response.getValue() != BinaryData.empty() && ServerSentEventUtils .isTextEventStreamContentType(response.getHeaders().getValue(HttpHeaderName.CONTENT_TYPE))) { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 0f2351c63a69..4f023feb399e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -91,7 +91,7 @@ public T toObject(Type type, ObjectSerializer serializer) { @Override public InputStream toStream() { if (bytes == null) { - return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, this::drainStream); + return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, this::close); } else { return new ByteArrayInputStream(bytes); } @@ -184,6 +184,10 @@ public BinaryData toReplayableBinaryData() { @Override public void close() { drainStream(); + Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler)); + } } private void drainStream() { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 5f6ac8155696..74627c00a6cf 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -107,4 +107,14 @@ public void handlerAdded(ChannelHandlerContext ctx) { ctx.fireExceptionCaught(new ClosedChannelException()); } } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) { + if (!lastRead) { + if (this.exception == null) { + this.exception = new IOException("Handler removed before stream was fully consumed."); + } + latch.countDown(); + } + } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index c4a9ac818f1c..58180006da26 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -49,19 +49,16 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - // An exception has occurred, which means the channel is likely in a bad state. - // We handle this by closing the channel. This prevents it from being - // returned to the connection pool. - cleanup(ctx, true); + cleanup(ctx); } @Override public void channelInactive(ChannelHandlerContext ctx) { + cleanup(ctx); ctx.fireChannelInactive(); - cleanup(ctx, true); } - public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { + public void cleanup(ChannelHandlerContext ctx) { if (!cleanedUp.compareAndSet(false, true)) { return; } @@ -81,10 +78,6 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { pipeline.remove(this); } - if (closeChannel) { - ctx.channel().close(); - } else { - connectionPool.release(ctx.channel()); - } + connectionPool.release(ctx.channel()); } } From a2d0bcd6b6aee46710b3ff96710544a7281de3ec Mon Sep 17 00:00:00 2001 From: George Banasios Date: Fri, 4 Jul 2025 11:37:39 +0300 Subject: [PATCH 26/57] Revert "deadlock fix attempt v5" This reverts commit c0f9beb364a99e2318136fa31da676e488890676. --- .../clientcore/http/netty4/NettyHttpClient.java | 14 ++------------ .../implementation/Netty4ChannelBinaryData.java | 6 +----- .../Netty4EagerConsumeChannelHandler.java | 10 ---------- .../Netty4PipelineCleanupHandler.java | 15 +++++++++++---- 4 files changed, 14 insertions(+), 31 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index 4f9c2c42de92..d251a04eba8e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -144,7 +144,6 @@ public Response send(HttpRequest request) { } Response response; - Channel channelToRelease = null; if (info.isChannelConsumptionComplete()) { // The network response is already complete, handle creating our Response based on the request method and // response headers. @@ -155,7 +154,7 @@ public Response send(HttpRequest request) { // there was body content. body = BinaryData.fromBytes(eagerContent.toByteArray()); } - channelToRelease = info.getResponseChannel(); + response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); } else { // Otherwise we aren't finished, handle the remaining content according to the documentation in @@ -170,7 +169,6 @@ public Response send(HttpRequest request) { }, info.isHttp2())); channel.config().setAutoRead(true); awaitLatch(drainLatch); - channelToRelease = channel; } else if (bodyHandling == ResponseBodyHandling.STREAM) { // Body streaming uses a special BinaryData that tracks the firstContent read and the Channel it came // from so it can be consumed when the BinaryData is being used. @@ -199,21 +197,13 @@ public Response send(HttpRequest request) { }, info.isHttp2())); channel.config().setAutoRead(true); awaitLatch(drainLatch); - channelToRelease = channel; + body = BinaryData.fromBytes(info.getEagerContent().toByteArray()); } response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); } - if (channelToRelease != null) { - Netty4PipelineCleanupHandler cleanupHandler - = channelToRelease.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(channelToRelease.pipeline().context(cleanupHandler)); - } - } - if (response.getValue() != BinaryData.empty() && ServerSentEventUtils .isTextEventStreamContentType(response.getHeaders().getValue(HttpHeaderName.CONTENT_TYPE))) { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 4f023feb399e..0f2351c63a69 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -91,7 +91,7 @@ public T toObject(Type type, ObjectSerializer serializer) { @Override public InputStream toStream() { if (bytes == null) { - return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, this::close); + return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, this::drainStream); } else { return new ByteArrayInputStream(bytes); } @@ -184,10 +184,6 @@ public BinaryData toReplayableBinaryData() { @Override public void close() { drainStream(); - Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler)); - } } private void drainStream() { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 74627c00a6cf..5f6ac8155696 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -107,14 +107,4 @@ public void handlerAdded(ChannelHandlerContext ctx) { ctx.fireExceptionCaught(new ClosedChannelException()); } } - - @Override - public void handlerRemoved(ChannelHandlerContext ctx) { - if (!lastRead) { - if (this.exception == null) { - this.exception = new IOException("Handler removed before stream was fully consumed."); - } - latch.countDown(); - } - } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 58180006da26..c4a9ac818f1c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -49,16 +49,19 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - cleanup(ctx); + // An exception has occurred, which means the channel is likely in a bad state. + // We handle this by closing the channel. This prevents it from being + // returned to the connection pool. + cleanup(ctx, true); } @Override public void channelInactive(ChannelHandlerContext ctx) { - cleanup(ctx); ctx.fireChannelInactive(); + cleanup(ctx, true); } - public void cleanup(ChannelHandlerContext ctx) { + public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { if (!cleanedUp.compareAndSet(false, true)) { return; } @@ -78,6 +81,10 @@ public void cleanup(ChannelHandlerContext ctx) { pipeline.remove(this); } - connectionPool.release(ctx.channel()); + if (closeChannel) { + ctx.channel().close(); + } else { + connectionPool.release(ctx.channel()); + } } } From d66fd1d0864b2ecfaea950bd0219f83155fb367d Mon Sep 17 00:00:00 2001 From: George Banasios Date: Fri, 4 Jul 2025 17:44:30 +0300 Subject: [PATCH 27/57] deadlock fix attempt v6 --- .../http/netty4/NettyHttpClient.java | 35 +- .../http/netty4/NettyHttpClientBuilder.java | 1 + .../implementation/Netty4AlpnHandler.java | 24 +- .../Netty4ChannelBinaryData.java | 33 +- .../Netty4ChannelInputStream.java | 7 +- .../Netty4EagerConsumeChannelHandler.java | 27 +- .../Netty4InitiateOneReadHandler.java | 18 +- .../Netty4PipelineCleanupHandler.java | 14 +- .../implementation/Netty4ResponseHandler.java | 9 +- .../netty4/implementation/Netty4Utility.java | 36 +- .../Netty4Http11ChannelInputStreamTests.java | 28 +- .../Netty4ResponseHandlerTests.java | 326 +++++++++--------- 12 files changed, 342 insertions(+), 216 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index d251a04eba8e..c2cb71dfcfcc 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -26,6 +26,7 @@ import io.clientcore.http.netty4.implementation.Netty4PipelineCleanupHandler; import io.clientcore.http.netty4.implementation.Netty4ProgressAndTimeoutHandler; import io.clientcore.http.netty4.implementation.Netty4ResponseHandler; +import io.clientcore.http.netty4.implementation.Netty4Utility; import io.clientcore.http.netty4.implementation.ResponseBodyHandling; import io.clientcore.http.netty4.implementation.ResponseStateInfo; import io.netty.bootstrap.Bootstrap; @@ -53,7 +54,6 @@ import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; import static io.clientcore.http.netty4.implementation.Netty4Utility.awaitLatch; -import static io.clientcore.http.netty4.implementation.Netty4Utility.configureHttpsPipeline; import static io.clientcore.http.netty4.implementation.Netty4Utility.createCodec; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; import static io.clientcore.http.netty4.implementation.Netty4Utility.setOrSuppressError; @@ -144,6 +144,8 @@ public Response send(HttpRequest request) { } Response response; + Channel channelToRelease; + if (info.isChannelConsumptionComplete()) { // The network response is already complete, handle creating our Response based on the request method and // response headers. @@ -154,7 +156,7 @@ public Response send(HttpRequest request) { // there was body content. body = BinaryData.fromBytes(eagerContent.toByteArray()); } - + channelToRelease = info.getResponseChannel(); response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); } else { // Otherwise we aren't finished, handle the remaining content according to the documentation in @@ -169,7 +171,9 @@ public Response send(HttpRequest request) { }, info.isHttp2())); channel.config().setAutoRead(true); awaitLatch(drainLatch); + channelToRelease = channel; } else if (bodyHandling == ResponseBodyHandling.STREAM) { + channelToRelease = null; // Body streaming uses a special BinaryData that tracks the firstContent read and the Channel it came // from so it can be consumed when the BinaryData is being used. // autoRead should have been disabled already but lets make sure that it is. @@ -197,6 +201,7 @@ public Response send(HttpRequest request) { }, info.isHttp2())); channel.config().setAutoRead(true); awaitLatch(drainLatch); + channelToRelease = channel; body = BinaryData.fromBytes(info.getEagerContent().toByteArray()); } @@ -204,6 +209,16 @@ public Response send(HttpRequest request) { response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); } + if (channelToRelease != null) { + channelToRelease.eventLoop().execute(() -> { + Netty4PipelineCleanupHandler cleanupHandler + = channelToRelease.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(channelToRelease.pipeline().context(cleanupHandler), false); + } + }); + } + if (response.getValue() != BinaryData.empty() && ServerSentEventUtils .isTextEventStreamContentType(response.getHeaders().getValue(HttpHeaderName.CONTENT_TYPE))) { @@ -281,8 +296,18 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, if (protocolVersion != null) { // Connection is being reused, ALPN is already done. // Manually configure the pipeline based on the stored protocol. - configureHttpsPipeline(pipeline, request, protocolVersion, responseReference, errorReference, latch); - send(request, channel, errorReference, latch); + boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; + pipeline.addLast(HTTP_RESPONSE, new Netty4ResponseHandler(request, responseReference, errorReference, latch, isHttp2)); + + if (!isHttp2 && pipeline.get(HTTP_CODEC) == null) { + pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); + } + + if (isHttp2) { + Netty4Utility.sendHttp2Request(request, channel, errorReference, latch); + } else { + send(request, channel, errorReference, latch); + } } else { // This is a new connection, let ALPN do the work. // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. @@ -292,7 +317,7 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, // If there isn't an SslHandler, we can send the request immediately. // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + new Netty4ResponseHandler(request, responseReference, errorReference, latch, false)); String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); send(request, channel, errorReference, latch); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 0e70174ad134..08c2a42b760f 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -395,6 +395,7 @@ public HttpClient build() { .channel(channelClass) .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, (int) getTimeoutMillis(connectTimeout, 10_000)); // Disable auto-read as we want to control when and how data is read from the channel. + bootstrap.option(ChannelOption.SO_KEEPALIVE, true); bootstrap.option(ChannelOption.AUTO_READ, false); Configuration buildConfiguration diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index ad3a90754780..dd12016d9212 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -31,7 +31,6 @@ public final class Netty4AlpnHandler extends ApplicationProtocolNegotiationHandl */ public static final AttributeKey HTTP_PROTOCOL_VERSION_KEY = AttributeKey.valueOf("http-protocol-version"); - private static final int TWO_FIFTY_SIX_KB = 256 * 1024; private final HttpRequest request; private final AtomicReference responseReference; private final AtomicReference errorReference; @@ -75,15 +74,20 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { configureHttpsPipeline(ctx.pipeline(), request, protocolVersion, responseReference, errorReference, latch); - sendHttp11Request(request, ctx.channel(), errorReference).addListener((ChannelFutureListener) sendListener -> { - if (!sendListener.isSuccess()) { - setOrSuppressError(errorReference, sendListener.cause()); - sendListener.channel().pipeline().fireExceptionCaught(sendListener.cause()); - latch.countDown(); - } else { - sendListener.channel().read(); - } - }); + if (protocolVersion == HttpProtocolVersion.HTTP_2) { + Netty4Utility.sendHttp2Request(request, ctx.channel(), errorReference, latch); + } else { + sendHttp11Request(request, ctx.channel(), errorReference) + .addListener((ChannelFutureListener) sendListener -> { + if (!sendListener.isSuccess()) { + setOrSuppressError(errorReference, sendListener.cause()); + sendListener.channel().pipeline().fireExceptionCaught(sendListener.cause()); + latch.countDown(); + } else { + sendListener.channel().read(); + } + }); + } } @Override diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 0f2351c63a69..ad85de1c9e5e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -70,7 +70,6 @@ public byte[] toBytes() { bytes = eagerContent.toByteArray(); eagerContent = null; } - return bytes; } @@ -91,7 +90,7 @@ public T toObject(Type type, ObjectSerializer serializer) { @Override public InputStream toStream() { if (bytes == null) { - return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, this::drainStream); + return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, streamDrained, this::close); } else { return new ByteArrayInputStream(bytes); } @@ -128,8 +127,6 @@ public void writeTo(OutputStream outputStream) { channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); channel.config().setAutoRead(true); - channel.eventLoop().execute(channel::read); - awaitLatch(latch); streamDrained.set(true); @@ -147,8 +144,6 @@ public void writeTo(OutputStream outputStream) { } } catch (IOException ex) { throw LOGGER.throwableAtError().log(ex, CoreException::from); - } finally { - close(); } } @@ -184,6 +179,21 @@ public BinaryData toReplayableBinaryData() { @Override public void close() { drainStream(); + + Runnable cleanupTask = () -> { + if (!this.isHttp2) { + Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); + } + } + }; + + if (channel.eventLoop().inEventLoop()) { + cleanupTask.run(); + } else { + channel.eventLoop().execute(cleanupTask); + } } private void drainStream() { @@ -191,6 +201,9 @@ private void drainStream() { if (channel.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { return; } + if (length != null && eagerContent.size() >= length) { + return; + } if (!channel.isActive()) { return; @@ -199,10 +212,12 @@ private void drainStream() { CountDownLatch latch = new CountDownLatch(1); Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, buf -> buf.readBytes(eagerContent, buf.readableBytes()), isHttp2); - channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); - channel.config().setAutoRead(true); + Runnable setupAndRead = () -> { + channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); + channel.config().setAutoRead(true); + }; - channel.eventLoop().execute(channel::read); + channel.eventLoop().execute(setupAndRead); awaitLatch(latch); Throwable exception = handler.channelException(); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java index de5d4c96c0af..a38c70987039 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java @@ -10,6 +10,7 @@ import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; /** * Implementation of {@link InputStream} that reads contents from a Netty {@link Channel}. @@ -18,6 +19,7 @@ public final class Netty4ChannelInputStream extends InputStream { private final Channel channel; private final boolean isHttp2; private final Runnable onClose; + private final AtomicBoolean parentStreamDrained; // Indicator for the Channel being fully read. // This will become true before 'streamDone' becomes true, but both may become true in the same operation. @@ -50,7 +52,8 @@ public final class Netty4ChannelInputStream extends InputStream { * @param isHttp2 Flag indicating whether the Channel is used for HTTP/2 or not. * @param onClose A runnable to execute when the stream is closed. */ - Netty4ChannelInputStream(ByteArrayOutputStream eagerContent, Channel channel, boolean isHttp2, Runnable onClose) { + Netty4ChannelInputStream(ByteArrayOutputStream eagerContent, Channel channel, boolean isHttp2, + AtomicBoolean parentStreamDrained, Runnable onClose) { if (eagerContent != null && eagerContent.size() > 0) { this.currentBuffer = eagerContent.toByteArray(); eagerContent.reset(); @@ -65,6 +68,7 @@ public final class Netty4ChannelInputStream extends InputStream { } this.isHttp2 = isHttp2; this.onClose = onClose; + this.parentStreamDrained = parentStreamDrained; } byte[] getCurrentBuffer() { @@ -200,6 +204,7 @@ private boolean setupNextBuffer() throws IOException { return true; } else { streamDone = true; + parentStreamDrained.set(true); return false; } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 5f6ac8155696..9154d3bdca65 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -45,6 +45,11 @@ public Netty4EagerConsumeChannelHandler(CountDownLatch latch, IOExceptionChecked @Override public void channelRead(ChannelHandlerContext ctx, Object msg) { + if (!(msg instanceof ByteBufHolder) && !(msg instanceof ByteBuf)) { + ctx.fireChannelRead(msg); + return; + } + try { ByteBuf buf = (msg instanceof ByteBufHolder) ? ((ByteBufHolder) msg).content() : (ByteBuf) msg; @@ -68,17 +73,18 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { @Override public void channelReadComplete(ChannelHandlerContext ctx) { - ctx.fireChannelReadComplete(); if (lastRead) { latch.countDown(); } + ctx.fireChannelReadComplete(); } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; - ctx.fireExceptionCaught(cause); + cleanup(ctx); latch.countDown(); + ctx.fireExceptionCaught(cause); } Throwable channelException() { @@ -88,12 +94,14 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { + cleanup(ctx); latch.countDown(); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { + cleanup(ctx); latch.countDown(); ctx.fireChannelInactive(); } @@ -101,10 +109,21 @@ public void channelInactive(ChannelHandlerContext ctx) { @Override public void handlerAdded(ChannelHandlerContext ctx) { if (!ctx.channel().isActive()) { - // In case the read handler is added to a closed channel we fail loudly by firing + // In case the read handler is added to a closed channel, we fail loudly by firing // an exception. Simply counting down the latch would cause the caller to receive - // an empty/incomplete data stream without any indication of the underlying network error. + // an empty/incomplete data stream without any sign of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); } } + + private void cleanup(ChannelHandlerContext ctx) { + if (latch.getCount() == 0) { + return; + } + + Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(ctx, true); + } + } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java index 9ddef608a98c..8042c42c72a7 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java @@ -105,6 +105,7 @@ boolean isChannelConsumed() { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; + performCleanup(ctx); latch.countDown(); ctx.fireExceptionCaught(cause); } @@ -116,12 +117,14 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { + performCleanup(ctx); latch.countDown(); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { + performCleanup(ctx); latch.countDown(); ctx.fireChannelInactive(); } @@ -129,10 +132,21 @@ public void channelInactive(ChannelHandlerContext ctx) { @Override public void handlerAdded(ChannelHandlerContext ctx) { if (!ctx.channel().isActive()) { - // In case the read handler is added to a closed channel we fail loudly by firing + // In case the read handler is added to a closed channel, we fail loudly by firing // an exception. Simply counting down the latch would cause the caller to receive - // an empty/incomplete data stream without any indication of the underlying network error. + // an empty/incomplete data stream without any sign of the underlying network error. ctx.fireExceptionCaught(new ClosedChannelException()); } } + + private void performCleanup(ChannelHandlerContext ctx) { + if (latch.getCount() == 0) { + return; + } + + Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(ctx, true); + } + } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index c4a9ac818f1c..174850394790 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -3,6 +3,7 @@ package io.clientcore.http.netty4.implementation; +import io.clientcore.core.http.client.HttpProtocolVersion; import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; @@ -49,9 +50,6 @@ public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - // An exception has occurred, which means the channel is likely in a bad state. - // We handle this by closing the channel. This prevents it from being - // returned to the connection pool. cleanup(ctx, true); } @@ -71,7 +69,15 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { ctx.channel().config().setAutoRead(false); ChannelPipeline pipeline = ctx.channel().pipeline(); + + HttpProtocolVersion protocolVersion = ctx.channel().attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); + boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; + for (String handlerName : HANDLERS_TO_REMOVE) { + if (isHttp2 && HTTP_CODEC.equals(handlerName)) { + continue; + } + if (pipeline.get(handlerName) != null) { pipeline.remove(handlerName); } @@ -81,7 +87,7 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { pipeline.remove(this); } - if (closeChannel) { + if (closeChannel || !ctx.channel().isActive()) { ctx.channel().close(); } else { connectionPool.release(ctx.channel()); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 5d5bf61785ca..585664f33ecf 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -48,6 +48,7 @@ public final class Netty4ResponseHandler extends ChannelInboundHandlerAdapter { // and initial response body content. private final ByteArrayOutputStream eagerContent = new ByteArrayOutputStream(); private boolean complete; + private boolean isHttp2; /** * Creates an instance of {@link Netty4ResponseHandler}. @@ -61,7 +62,7 @@ public final class Netty4ResponseHandler extends ChannelInboundHandlerAdapter { * @throws NullPointerException If {@code request}, {@code responseReference}, or {@code latch} is null. */ public Netty4ResponseHandler(HttpRequest request, AtomicReference responseReference, - AtomicReference errorReference, CountDownLatch latch) { + AtomicReference errorReference, CountDownLatch latch, boolean isHttp2) { this.request = Objects.requireNonNull(request, "Cannot create an instance of CoreResponseHandler with a null 'request'."); this.responseReference = Objects.requireNonNull(responseReference, @@ -166,11 +167,13 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { return; } - ctx.pipeline().remove(this); + if (!isHttp2) { + ctx.pipeline().remove(this); + } ctx.fireChannelReadComplete(); responseReference.set(new ResponseStateInfo(ctx.channel(), complete, statusCode, headers, eagerContent, - ResponseBodyHandling.getBodyHandling(request, headers), false)); + ResponseBodyHandling.getBodyHandling(request, headers), isHttp2)); latch.countDown(); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java index 006c859dcf11..5fa2b2184edd 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java @@ -492,7 +492,7 @@ public static void configureHttpsPipeline(ChannelPipeline pipeline, HttpRequest } Netty4ResponseHandler responseHandler - = new Netty4ResponseHandler(request, responseReference, errorReference, latch); + = new Netty4ResponseHandler(request, responseReference, errorReference, latch, protocol == HttpProtocolVersion.HTTP_2); if (pipeline.get(Netty4HandlerNames.PROGRESS_AND_TIMEOUT) != null) { pipeline.addAfter(Netty4HandlerNames.PROGRESS_AND_TIMEOUT, Netty4HandlerNames.HTTP_RESPONSE, @@ -504,6 +504,40 @@ public static void configureHttpsPipeline(ChannelPipeline pipeline, HttpRequest } } + public static void sendHttp2Request(HttpRequest request, Channel channel, + AtomicReference errorReference, CountDownLatch latch) { + io.netty.handler.codec.http.HttpRequest nettyRequest = toNettyHttpRequest(request); + + channel.writeAndFlush(nettyRequest).addListener(future -> { + if (future.isSuccess()) { + channel.read(); + } else { + setOrSuppressError(errorReference, future.cause()); + if (latch.getCount() > 0) { + latch.countDown(); + } + } + }); + + channel.read(); + } + + private static io.netty.handler.codec.http.HttpRequest toNettyHttpRequest(HttpRequest request) { + HttpMethod nettyMethod = HttpMethod.valueOf(request.getHttpMethod().toString()); + String uri = request.getUri().toString(); + WrappedHttp11Headers nettyHeaders = new WrappedHttp11Headers(request.getHeaders()); + nettyHeaders.getCoreHeaders().set(HttpHeaderName.HOST, request.getUri().getHost()); + + BinaryData body = request.getBody(); + if (body == null || body.getLength() == 0) { + return new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, nettyMethod, uri, Unpooled.EMPTY_BUFFER, + nettyHeaders, trailersFactory().newHeaders()); + } else { + return new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, nettyMethod, uri, + Unpooled.wrappedBuffer(body.toBytes()), nettyHeaders, trailersFactory().newHeaders()); + } + } + private Netty4Utility() { } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java index 3563d03c3813..2c88ef03e113 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java @@ -41,16 +41,16 @@ public class Netty4Http11ChannelInputStreamTests { @Test public void nullEagerContentResultsInEmptyInitialCurrentBuffer() throws IOException { - try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(null, createCloseableChannel(), false, null)) { + try (Netty4ChannelInputStream channelInputStream = new Netty4ChannelInputStream(null, + createCloseableChannel(), false, new AtomicBoolean(), null)) { assertEquals(0, channelInputStream.getCurrentBuffer().length); } } @Test public void emptyEagerContentResultsInEmptyInitialCurrentBuffer() throws IOException { - try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(new ByteArrayOutputStream(), createCloseableChannel(), false, null)) { + try (Netty4ChannelInputStream channelInputStream = new Netty4ChannelInputStream(new ByteArrayOutputStream(), + createCloseableChannel(), false, new AtomicBoolean(), null)) { assertEquals(0, channelInputStream.getCurrentBuffer().length); } } @@ -65,7 +65,7 @@ public void readConsumesCurrentBufferAndHasNoMoreData() throws IOException { // MockChannels aren't active by default, so once the eagerContent is consumed the stream will be done. Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(eagerContent, new MockChannel(), false, null); + = new Netty4ChannelInputStream(eagerContent, new MockChannel(), false, new AtomicBoolean(), null); // Make sure the Netty4ChannelInputStream copied the eager content correctly. assertArraysEqual(expected, channelInputStream.getCurrentBuffer()); @@ -97,7 +97,7 @@ public void readConsumesCurrentBufferAndRequestsMoreData() throws IOException { handler.channelRead(ctx, wrappedBuffer(expected, 16, 16)); handler.channelRead(ctx, LastHttpContent.EMPTY_LAST_CONTENT); handler.channelReadComplete(ctx); - }), false, null); + }), false, new AtomicBoolean(), null); int index = 0; byte[] actual = new byte[32]; @@ -119,7 +119,7 @@ public void multipleSmallerSkips() throws IOException { // MockChannels aren't active by default, so once the eagerContent is consumed the stream will be done. try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(eagerContent, createCloseableChannel(), false, null)) { + = new Netty4ChannelInputStream(eagerContent, createCloseableChannel(), false, new AtomicBoolean(), null)) { long skipped = channelInputStream.skip(16); assertEquals(16, skipped); @@ -141,8 +141,8 @@ public void largeReadTriggersMultipleChannelReads() throws IOException { byte[] expected = new byte[8192]; ThreadLocalRandom.current().nextBytes(expected); - try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(null, createChannelThatReads8Kb(expected), false, null)) { + try (Netty4ChannelInputStream channelInputStream = new Netty4ChannelInputStream(null, + createChannelThatReads8Kb(expected), false, new AtomicBoolean(), null)) { byte[] actual = new byte[8192]; int read = channelInputStream.read(actual); @@ -162,8 +162,8 @@ public void largeSkipTriggersMultipleChannelReads() throws IOException { byte[] expected = new byte[8192]; ThreadLocalRandom.current().nextBytes(expected); - try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(null, createChannelThatReads8Kb(expected), false, null)) { + try (Netty4ChannelInputStream channelInputStream = new Netty4ChannelInputStream(null, + createChannelThatReads8Kb(expected), false, new AtomicBoolean(), null)) { long skipped = channelInputStream.skip(8192); assertEquals(8192, skipped); @@ -176,8 +176,8 @@ public void largeSkipTriggersMultipleChannelReads() throws IOException { public void closingStreamTriggersOnCloseCallback() throws IOException { AtomicBoolean onCloseCalled = new AtomicBoolean(false); - try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(null, createCloseableChannel(), false, () -> onCloseCalled.set(true))) { + try (Netty4ChannelInputStream channelInputStream = new Netty4ChannelInputStream(null, createCloseableChannel(), + false, new AtomicBoolean(), () -> onCloseCalled.set(true))) { assertNotNull(channelInputStream); } @@ -188,7 +188,7 @@ public void closingStreamTriggersOnCloseCallback() throws IOException { @MethodSource("errorSupplier") public void streamPropagatesErrorFiredInChannel(Throwable expected) { InputStream inputStream - = new Netty4ChannelInputStream(null, createPartialReadThenErrorChannel(expected), false, null); + = new Netty4ChannelInputStream(null, createPartialReadThenErrorChannel(expected), false, new AtomicBoolean(), null); Throwable actual = assertThrows(Throwable.class, () -> inputStream.read(new byte[8192])); diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java index 8b01df09c2a8..f93383275e93 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java @@ -1,163 +1,163 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -package io.clientcore.http.netty4.implementation; - -import io.clientcore.core.http.models.HttpMethod; -import io.clientcore.core.http.models.HttpRequest; -import io.clientcore.http.netty4.mocking.MockChannelHandlerContext; -import io.netty.buffer.Unpooled; -import io.netty.channel.Channel; -import io.netty.handler.codec.http.DefaultFullHttpResponse; -import io.netty.handler.codec.http.DefaultHttpHeaders; -import io.netty.handler.codec.http.DefaultHttpResponse; -import io.netty.handler.codec.http.HttpResponseStatus; -import io.netty.handler.codec.http.HttpVersion; -import io.netty.handler.codec.http.LastHttpContent; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; - -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; - -import static io.clientcore.http.netty4.TestUtils.createChannelWithReadHandling; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertTrue; - -/** - * Tests {@link Netty4ResponseHandler}. - */ -@Timeout(value = 3, unit = TimeUnit.MINUTES) -public class Netty4ResponseHandlerTests { - @Test - public void firstReadIsFullHttpResponse() throws Exception { - HttpRequest request = new HttpRequest(); - AtomicReference responseReference = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - - Netty4ResponseHandler responseHandler - = new Netty4ResponseHandler(request, responseReference, new AtomicReference<>(), latch); - - Channel ch = createChannelWithReadHandling((ignored, channel) -> { - Netty4ResponseHandler handler = channel.pipeline().get(Netty4ResponseHandler.class); - MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); - - try { - handler.channelRead(ctx, new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, - Unpooled.EMPTY_BUFFER, new DefaultHttpHeaders(), new DefaultHttpHeaders())); - handler.channelReadComplete(ctx); - } catch (Exception ex) { - ctx.fireExceptionCaught(ex); - } - }); - - ch.pipeline().addLast(responseHandler); - ch.read(); - - assertEquals(0, latch.getCount()); - - ResponseStateInfo info = responseReference.get(); - assertNotNull(info); - - assertTrue(info.isChannelConsumptionComplete()); - assertEquals(0, info.getEagerContent().size()); - } - - @Test - public void incompleteIgnoredResponseBody() { - byte[] ignoredBodyData = new byte[32]; - ThreadLocalRandom.current().nextBytes(ignoredBodyData); - - HttpRequest request = new HttpRequest().setMethod(HttpMethod.HEAD); - AtomicReference responseReference = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - - Netty4ResponseHandler responseHandler - = new Netty4ResponseHandler(request, responseReference, new AtomicReference<>(), latch); - - Channel ch = createChannelWithReadHandling((readCount, channel) -> { - if (readCount == 0) { - Netty4ResponseHandler handler = channel.pipeline().get(Netty4ResponseHandler.class); - MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); - try { - handler.channelRead(ctx, - new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, new DefaultHttpHeaders())); - handler.channelReadComplete(ctx); - } catch (Exception ex) { - ctx.fireExceptionCaught(ex); - } - } else { - Netty4EagerConsumeChannelHandler handler - = channel.pipeline().get(Netty4EagerConsumeChannelHandler.class); - MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); - handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); - handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); - handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); - handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); - handler.channelRead(ctx, LastHttpContent.EMPTY_LAST_CONTENT); - handler.channelReadComplete(ctx); - } - }); - - ch.pipeline().addLast(responseHandler); - ch.read(); - - assertEquals(0, latch.getCount()); - - ResponseStateInfo info = responseReference.get(); - assertNotNull(info); - } - - @Test - public void bufferedResponseBodyLargerThanInitialRead() { - byte[] bodyPieces = new byte[32]; - ThreadLocalRandom.current().nextBytes(bodyPieces); - - byte[] expectedBody = new byte[bodyPieces.length * 4]; - System.arraycopy(bodyPieces, 0, expectedBody, 0, bodyPieces.length); - System.arraycopy(bodyPieces, 0, expectedBody, bodyPieces.length, bodyPieces.length); - System.arraycopy(bodyPieces, 0, expectedBody, bodyPieces.length * 2, bodyPieces.length); - System.arraycopy(bodyPieces, 0, expectedBody, bodyPieces.length * 3, bodyPieces.length); - - HttpRequest request = new HttpRequest(); - AtomicReference responseReference = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - - Netty4ResponseHandler responseHandler - = new Netty4ResponseHandler(request, responseReference, new AtomicReference<>(), latch); - - Channel ch = createChannelWithReadHandling((readCount, channel) -> { - if (readCount == 0) { - Netty4ResponseHandler handler = channel.pipeline().get(Netty4ResponseHandler.class); - MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); - try { - handler.channelRead(ctx, - new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, new DefaultHttpHeaders())); - handler.channelReadComplete(ctx); - } catch (Exception ex) { - ctx.fireExceptionCaught(ex); - } - } else { - Netty4EagerConsumeChannelHandler handler - = channel.pipeline().get(Netty4EagerConsumeChannelHandler.class); - MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); - handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); - handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); - handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); - handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); - handler.channelRead(ctx, LastHttpContent.EMPTY_LAST_CONTENT); - handler.channelReadComplete(ctx); - } - }); - - ch.pipeline().addLast(responseHandler); - ch.read(); - - assertEquals(0, latch.getCount()); - - ResponseStateInfo info = responseReference.get(); - assertNotNull(info); - } -} +//// Copyright (c) Microsoft Corporation. All rights reserved. +//// Licensed under the MIT License. +//package io.clientcore.http.netty4.implementation; +// +//import io.clientcore.core.http.models.HttpMethod; +//import io.clientcore.core.http.models.HttpRequest; +//import io.clientcore.http.netty4.mocking.MockChannelHandlerContext; +//import io.netty.buffer.Unpooled; +//import io.netty.channel.Channel; +//import io.netty.handler.codec.http.DefaultFullHttpResponse; +//import io.netty.handler.codec.http.DefaultHttpHeaders; +//import io.netty.handler.codec.http.DefaultHttpResponse; +//import io.netty.handler.codec.http.HttpResponseStatus; +//import io.netty.handler.codec.http.HttpVersion; +//import io.netty.handler.codec.http.LastHttpContent; +//import org.junit.jupiter.api.Test; +//import org.junit.jupiter.api.Timeout; +// +//import java.util.concurrent.CountDownLatch; +//import java.util.concurrent.ThreadLocalRandom; +//import java.util.concurrent.TimeUnit; +//import java.util.concurrent.atomic.AtomicReference; +// +//import static io.clientcore.http.netty4.TestUtils.createChannelWithReadHandling; +//import static org.junit.jupiter.api.Assertions.assertEquals; +//import static org.junit.jupiter.api.Assertions.assertNotNull; +//import static org.junit.jupiter.api.Assertions.assertTrue; +// +///** +// * Tests {@link Netty4ResponseHandler}. +// */ +//@Timeout(value = 3, unit = TimeUnit.MINUTES) +//public class Netty4ResponseHandlerTests { +// @Test +// public void firstReadIsFullHttpResponse() throws Exception { +// HttpRequest request = new HttpRequest(); +// AtomicReference responseReference = new AtomicReference<>(); +// CountDownLatch latch = new CountDownLatch(1); +// +// Netty4ResponseHandler responseHandler +// = new Netty4ResponseHandler(request, responseReference, new AtomicReference<>(), latch); +// +// Channel ch = createChannelWithReadHandling((ignored, channel) -> { +// Netty4ResponseHandler handler = channel.pipeline().get(Netty4ResponseHandler.class); +// MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); +// +// try { +// handler.channelRead(ctx, new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, +// Unpooled.EMPTY_BUFFER, new DefaultHttpHeaders(), new DefaultHttpHeaders())); +// handler.channelReadComplete(ctx); +// } catch (Exception ex) { +// ctx.fireExceptionCaught(ex); +// } +// }); +// +// ch.pipeline().addLast(responseHandler); +// ch.read(); +// +// assertEquals(0, latch.getCount()); +// +// ResponseStateInfo info = responseReference.get(); +// assertNotNull(info); +// +// assertTrue(info.isChannelConsumptionComplete()); +// assertEquals(0, info.getEagerContent().size()); +// } +// +// @Test +// public void incompleteIgnoredResponseBody() { +// byte[] ignoredBodyData = new byte[32]; +// ThreadLocalRandom.current().nextBytes(ignoredBodyData); +// +// HttpRequest request = new HttpRequest().setMethod(HttpMethod.HEAD); +// AtomicReference responseReference = new AtomicReference<>(); +// CountDownLatch latch = new CountDownLatch(1); +// +// Netty4ResponseHandler responseHandler +// = new Netty4ResponseHandler(request, responseReference, new AtomicReference<>(), latch); +// +// Channel ch = createChannelWithReadHandling((readCount, channel) -> { +// if (readCount == 0) { +// Netty4ResponseHandler handler = channel.pipeline().get(Netty4ResponseHandler.class); +// MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); +// try { +// handler.channelRead(ctx, +// new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, new DefaultHttpHeaders())); +// handler.channelReadComplete(ctx); +// } catch (Exception ex) { +// ctx.fireExceptionCaught(ex); +// } +// } else { +// Netty4EagerConsumeChannelHandler handler +// = channel.pipeline().get(Netty4EagerConsumeChannelHandler.class); +// MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); +// handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); +// handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); +// handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); +// handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); +// handler.channelRead(ctx, LastHttpContent.EMPTY_LAST_CONTENT); +// handler.channelReadComplete(ctx); +// } +// }); +// +// ch.pipeline().addLast(responseHandler); +// ch.read(); +// +// assertEquals(0, latch.getCount()); +// +// ResponseStateInfo info = responseReference.get(); +// assertNotNull(info); +// } +// +// @Test +// public void bufferedResponseBodyLargerThanInitialRead() { +// byte[] bodyPieces = new byte[32]; +// ThreadLocalRandom.current().nextBytes(bodyPieces); +// +// byte[] expectedBody = new byte[bodyPieces.length * 4]; +// System.arraycopy(bodyPieces, 0, expectedBody, 0, bodyPieces.length); +// System.arraycopy(bodyPieces, 0, expectedBody, bodyPieces.length, bodyPieces.length); +// System.arraycopy(bodyPieces, 0, expectedBody, bodyPieces.length * 2, bodyPieces.length); +// System.arraycopy(bodyPieces, 0, expectedBody, bodyPieces.length * 3, bodyPieces.length); +// +// HttpRequest request = new HttpRequest(); +// AtomicReference responseReference = new AtomicReference<>(); +// CountDownLatch latch = new CountDownLatch(1); +// +// Netty4ResponseHandler responseHandler +// = new Netty4ResponseHandler(request, responseReference, new AtomicReference<>(), latch); +// +// Channel ch = createChannelWithReadHandling((readCount, channel) -> { +// if (readCount == 0) { +// Netty4ResponseHandler handler = channel.pipeline().get(Netty4ResponseHandler.class); +// MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); +// try { +// handler.channelRead(ctx, +// new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, new DefaultHttpHeaders())); +// handler.channelReadComplete(ctx); +// } catch (Exception ex) { +// ctx.fireExceptionCaught(ex); +// } +// } else { +// Netty4EagerConsumeChannelHandler handler +// = channel.pipeline().get(Netty4EagerConsumeChannelHandler.class); +// MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); +// handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); +// handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); +// handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); +// handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); +// handler.channelRead(ctx, LastHttpContent.EMPTY_LAST_CONTENT); +// handler.channelReadComplete(ctx); +// } +// }); +// +// ch.pipeline().addLast(responseHandler); +// ch.read(); +// +// assertEquals(0, latch.getCount()); +// +// ResponseStateInfo info = responseReference.get(); +// assertNotNull(info); +// } +//} From 42f72c7d4c37571a62083218afa49cb453e392cc Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sat, 5 Jul 2025 12:11:39 +0300 Subject: [PATCH 28/57] make method more efficient --- .../http/netty4/NettyHttpClient.java | 11 +++-- .../implementation/Netty4AlpnHandler.java | 19 ++++---- .../Netty4ChannelBinaryData.java | 9 ++-- .../implementation/Netty4ConnectionPool.java | 1 + .../netty4/implementation/Netty4Utility.java | 48 ++++++++++++------- .../Netty4Http11ChannelInputStreamTests.java | 7 +-- 6 files changed, 57 insertions(+), 38 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index c2cb71dfcfcc..3af008dfa60c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -26,7 +26,6 @@ import io.clientcore.http.netty4.implementation.Netty4PipelineCleanupHandler; import io.clientcore.http.netty4.implementation.Netty4ProgressAndTimeoutHandler; import io.clientcore.http.netty4.implementation.Netty4ResponseHandler; -import io.clientcore.http.netty4.implementation.Netty4Utility; import io.clientcore.http.netty4.implementation.ResponseBodyHandling; import io.clientcore.http.netty4.implementation.ResponseStateInfo; import io.netty.bootstrap.Bootstrap; @@ -56,6 +55,7 @@ import static io.clientcore.http.netty4.implementation.Netty4Utility.awaitLatch; import static io.clientcore.http.netty4.implementation.Netty4Utility.createCodec; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; +import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp2Request; import static io.clientcore.http.netty4.implementation.Netty4Utility.setOrSuppressError; /** @@ -280,7 +280,7 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is // applying timeouts. - // This is done to keep the ChannelPipeline shorter, therefore more performant, if this would + // This is done to keep the ChannelPipeline shorter, therefore more performant if this would // effectively be a no-op. if (addProgressAndTimeoutHandler) { pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, @@ -294,17 +294,18 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, if (isHttps) { HttpProtocolVersion protocolVersion = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); if (protocolVersion != null) { - // Connection is being reused, ALPN is already done. + // The Connection is being reused, ALPN is already done. // Manually configure the pipeline based on the stored protocol. boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; - pipeline.addLast(HTTP_RESPONSE, new Netty4ResponseHandler(request, responseReference, errorReference, latch, isHttp2)); + pipeline.addLast(HTTP_RESPONSE, + new Netty4ResponseHandler(request, responseReference, errorReference, latch, isHttp2)); if (!isHttp2 && pipeline.get(HTTP_CODEC) == null) { pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); } if (isHttp2) { - Netty4Utility.sendHttp2Request(request, channel, errorReference, latch); + sendHttp2Request(request, channel, errorReference, latch); } else { send(request, channel, errorReference, latch); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index dd12016d9212..8dedf4c5fc5e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -16,6 +16,7 @@ import static io.clientcore.http.netty4.implementation.Netty4Utility.configureHttpsPipeline; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; +import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp2Request; import static io.clientcore.http.netty4.implementation.Netty4Utility.setOrSuppressError; /** @@ -75,18 +76,18 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { configureHttpsPipeline(ctx.pipeline(), request, protocolVersion, responseReference, errorReference, latch); if (protocolVersion == HttpProtocolVersion.HTTP_2) { - Netty4Utility.sendHttp2Request(request, ctx.channel(), errorReference, latch); + sendHttp2Request(request, ctx.channel(), errorReference, latch); } else { sendHttp11Request(request, ctx.channel(), errorReference) .addListener((ChannelFutureListener) sendListener -> { - if (!sendListener.isSuccess()) { - setOrSuppressError(errorReference, sendListener.cause()); - sendListener.channel().pipeline().fireExceptionCaught(sendListener.cause()); - latch.countDown(); - } else { - sendListener.channel().read(); - } - }); + if (!sendListener.isSuccess()) { + setOrSuppressError(errorReference, sendListener.cause()); + sendListener.channel().pipeline().fireExceptionCaught(sendListener.cause()); + latch.countDown(); + } else { + sendListener.channel().read(); + } + }); } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index ad85de1c9e5e..afdf8d38d0ac 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -181,11 +181,10 @@ public void close() { drainStream(); Runnable cleanupTask = () -> { - if (!this.isHttp2) { - Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); - } + Netty4PipelineCleanupHandler cleanupHandler + = channel.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); } }; diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 52bd063f4cce..1720db460bd4 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -388,6 +388,7 @@ public void initChannel(Channel channel) throws SSLException { Promise promise = newConnectionBootstrap.config().group().next().newPromise(); newConnectionBootstrap.connect(route).addListener(future -> { if (!future.isSuccess()) { + LOGGER.atError().setThrowable(future.cause()).log("Failed connection."); // Connect failed, release the slot and try to satisfy a waiter. activeConnections.decrementAndGet(); satisfyWaiterWithNewConnection(); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java index 5fa2b2184edd..9bb4a7d73233 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java @@ -24,8 +24,10 @@ import io.netty.channel.ChannelPipeline; import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.DefaultHttpRequest; +import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.HttpChunkedInput; import io.netty.handler.codec.http.HttpClientCodec; +import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.HttpDecoderConfig; import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpHeadersFactory; @@ -455,7 +457,7 @@ public static HttpHeaderName fromPossibleAsciiString(CharSequence asciiString) { *

* This method adds the appropriate {@link Netty4HandlerNames#HTTP_CODEC} and * {@link Netty4HandlerNames#HTTP_RESPONSE} handlers to the pipeline, positioned correctly - * relative to the {@link Netty4HandlerNames#PROGRESS_AND_TIMEOUT} or {@link Netty4HandlerNames#SSL} handlers. + * relatively to the {@link Netty4HandlerNames#PROGRESS_AND_TIMEOUT} or {@link Netty4HandlerNames#SSL} handlers. * * @param pipeline The channel pipeline to configure. * @param request The HTTP request. @@ -491,8 +493,8 @@ public static void configureHttpsPipeline(ChannelPipeline pipeline, HttpRequest httpCodec = createCodec(); } - Netty4ResponseHandler responseHandler - = new Netty4ResponseHandler(request, responseReference, errorReference, latch, protocol == HttpProtocolVersion.HTTP_2); + Netty4ResponseHandler responseHandler = new Netty4ResponseHandler(request, responseReference, errorReference, + latch, protocol == HttpProtocolVersion.HTTP_2); if (pipeline.get(Netty4HandlerNames.PROGRESS_AND_TIMEOUT) != null) { pipeline.addAfter(Netty4HandlerNames.PROGRESS_AND_TIMEOUT, Netty4HandlerNames.HTTP_RESPONSE, @@ -504,22 +506,31 @@ public static void configureHttpsPipeline(ChannelPipeline pipeline, HttpRequest } } - public static void sendHttp2Request(HttpRequest request, Channel channel, - AtomicReference errorReference, CountDownLatch latch) { + public static void sendHttp2Request(HttpRequest request, Channel channel, AtomicReference errorReference, + CountDownLatch latch) { io.netty.handler.codec.http.HttpRequest nettyRequest = toNettyHttpRequest(request); - channel.writeAndFlush(nettyRequest).addListener(future -> { + final ChannelFuture writeFuture; + + if (nettyRequest instanceof FullHttpRequest) { + writeFuture = channel.writeAndFlush(nettyRequest); + } else { + channel.write(nettyRequest); + + BinaryData requestBody = request.getBody(); + ChunkedInput chunkedInput = new HttpChunkedInput(new ChunkedStream(requestBody.toStream())); + + writeFuture = channel.writeAndFlush(chunkedInput); + } + + writeFuture.addListener(future -> { if (future.isSuccess()) { channel.read(); } else { setOrSuppressError(errorReference, future.cause()); - if (latch.getCount() > 0) { - latch.countDown(); - } + latch.countDown(); } }); - - channel.read(); } private static io.netty.handler.codec.http.HttpRequest toNettyHttpRequest(HttpRequest request) { @@ -529,12 +540,17 @@ private static io.netty.handler.codec.http.HttpRequest toNettyHttpRequest(HttpRe nettyHeaders.getCoreHeaders().set(HttpHeaderName.HOST, request.getUri().getHost()); BinaryData body = request.getBody(); - if (body == null || body.getLength() == 0) { - return new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, nettyMethod, uri, Unpooled.EMPTY_BUFFER, - nettyHeaders, trailersFactory().newHeaders()); + if (body == null || body.getLength() == 0 || body.isReplayable()) { + ByteBuf bodyBytes = (body == null || body.getLength() == 0) + ? Unpooled.EMPTY_BUFFER + : Unpooled.wrappedBuffer(body.toBytes()); + + nettyHeaders.getCoreHeaders().set(HttpHeaderName.CONTENT_LENGTH, String.valueOf(bodyBytes.readableBytes())); + return new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, nettyMethod, uri, bodyBytes, nettyHeaders, + trailersFactory().newHeaders()); } else { - return new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, nettyMethod, uri, - Unpooled.wrappedBuffer(body.toBytes()), nettyHeaders, trailersFactory().newHeaders()); + nettyHeaders.getCoreHeaders().set(HttpHeaderName.TRANSFER_ENCODING, "chunked"); + return new DefaultHttpRequest(HttpVersion.HTTP_1_1, nettyMethod, uri, nettyHeaders); } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java index 2c88ef03e113..aec45e0cb227 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java @@ -41,8 +41,8 @@ public class Netty4Http11ChannelInputStreamTests { @Test public void nullEagerContentResultsInEmptyInitialCurrentBuffer() throws IOException { - try (Netty4ChannelInputStream channelInputStream = new Netty4ChannelInputStream(null, - createCloseableChannel(), false, new AtomicBoolean(), null)) { + try (Netty4ChannelInputStream channelInputStream = + new Netty4ChannelInputStream(null, createCloseableChannel(), false, new AtomicBoolean(), null)) { assertEquals(0, channelInputStream.getCurrentBuffer().length); } } @@ -188,7 +188,8 @@ false, new AtomicBoolean(), () -> onCloseCalled.set(true))) { @MethodSource("errorSupplier") public void streamPropagatesErrorFiredInChannel(Throwable expected) { InputStream inputStream - = new Netty4ChannelInputStream(null, createPartialReadThenErrorChannel(expected), false, new AtomicBoolean(), null); + = new Netty4ChannelInputStream(null, createPartialReadThenErrorChannel(expected), false, + new AtomicBoolean(), null); Throwable actual = assertThrows(Throwable.class, () -> inputStream.read(new byte[8192])); From 4606fe4381d8318c06ac7fd081f19658702d4718 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sat, 5 Jul 2025 15:43:21 +0300 Subject: [PATCH 29/57] fix response handler tests --- .../http/netty4/NettyHttpClient.java | 4 +- .../Netty4ChannelBinaryData.java | 3 +- .../Netty4EagerConsumeChannelHandler.java | 11 +- .../implementation/Netty4ResponseHandler.java | 9 +- .../netty4/implementation/Netty4Utility.java | 4 +- .../io/clientcore/http/netty4/TestUtils.java | 19 ++ .../Netty4Http11ChannelInputStreamTests.java | 7 +- .../Netty4ResponseHandlerTests.java | 316 +++++++++--------- .../http/netty4/mocking/MockChannel.java | 4 +- 9 files changed, 197 insertions(+), 180 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index 3af008dfa60c..58fed716b202 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -298,7 +298,7 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, // Manually configure the pipeline based on the stored protocol. boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch, isHttp2)); + new Netty4ResponseHandler(request, responseReference, errorReference, latch)); if (!isHttp2 && pipeline.get(HTTP_CODEC) == null) { pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); @@ -318,7 +318,7 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, // If there isn't an SslHandler, we can send the request immediately. // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch, false)); + new Netty4ResponseHandler(request, responseReference, errorReference, latch)); String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); send(request, channel, errorReference, latch); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index afdf8d38d0ac..a2e40d55d97f 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -181,8 +181,7 @@ public void close() { drainStream(); Runnable cleanupTask = () -> { - Netty4PipelineCleanupHandler cleanupHandler - = channel.pipeline().get(Netty4PipelineCleanupHandler.class); + Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); if (cleanupHandler != null) { cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 9154d3bdca65..8feb2cbe5b12 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -45,13 +45,14 @@ public Netty4EagerConsumeChannelHandler(CountDownLatch latch, IOExceptionChecked @Override public void channelRead(ChannelHandlerContext ctx, Object msg) { - if (!(msg instanceof ByteBufHolder) && !(msg instanceof ByteBuf)) { - ctx.fireChannelRead(msg); - return; - } + ByteBuf buf = null; try { - ByteBuf buf = (msg instanceof ByteBufHolder) ? ((ByteBufHolder) msg).content() : (ByteBuf) msg; + if (msg instanceof ByteBufHolder) { + buf = ((ByteBufHolder) msg).content(); + } else if (msg instanceof ByteBuf) { + buf = (ByteBuf) msg; + } if (buf != null && buf.isReadable()) { byteBufConsumer.accept(buf); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 585664f33ecf..e768fb5867bd 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -2,6 +2,7 @@ // Licensed under the MIT License. package io.clientcore.http.netty4.implementation; +import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.http.models.HttpHeaders; import io.clientcore.core.http.models.HttpRequest; import io.clientcore.core.http.models.Response; @@ -62,7 +63,7 @@ public final class Netty4ResponseHandler extends ChannelInboundHandlerAdapter { * @throws NullPointerException If {@code request}, {@code responseReference}, or {@code latch} is null. */ public Netty4ResponseHandler(HttpRequest request, AtomicReference responseReference, - AtomicReference errorReference, CountDownLatch latch, boolean isHttp2) { + AtomicReference errorReference, CountDownLatch latch) { this.request = Objects.requireNonNull(request, "Cannot create an instance of CoreResponseHandler with a null 'request'."); this.responseReference = Objects.requireNonNull(responseReference, @@ -73,6 +74,12 @@ public Netty4ResponseHandler(HttpRequest request, AtomicReference readHandler) { + return createChannelWithReadHandling(readHandler, null); + } + + /** + * Creates a {@link Channel} that is able to mock {@link Channel#read()} operations. + * + * @param readHandler A {@link BiConsumer} that takes the current read count and the channel and mocks reading + * operations. + * @param protocolVersion The HTTP protocol version to set on the channel's attributes. Can be null. + * @return A {@link Channel}. + */ + public static Channel createChannelWithReadHandling(BiConsumer readHandler, + HttpProtocolVersion protocolVersion) { EventLoop eventLoop = new DefaultEventLoop() { @Override public boolean inEventLoop(Thread thread) { @@ -61,6 +76,10 @@ public boolean isActive() { } }; + if (protocolVersion != null) { + channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).set(protocolVersion); + } + try { eventLoop.register(channel).sync(); } catch (InterruptedException e) { diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java index aec45e0cb227..99aeb61873ae 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java @@ -41,8 +41,8 @@ public class Netty4Http11ChannelInputStreamTests { @Test public void nullEagerContentResultsInEmptyInitialCurrentBuffer() throws IOException { - try (Netty4ChannelInputStream channelInputStream = - new Netty4ChannelInputStream(null, createCloseableChannel(), false, new AtomicBoolean(), null)) { + try (Netty4ChannelInputStream channelInputStream + = new Netty4ChannelInputStream(null, createCloseableChannel(), false, new AtomicBoolean(), null)) { assertEquals(0, channelInputStream.getCurrentBuffer().length); } } @@ -187,8 +187,7 @@ false, new AtomicBoolean(), () -> onCloseCalled.set(true))) { @ParameterizedTest @MethodSource("errorSupplier") public void streamPropagatesErrorFiredInChannel(Throwable expected) { - InputStream inputStream - = new Netty4ChannelInputStream(null, createPartialReadThenErrorChannel(expected), false, + InputStream inputStream = new Netty4ChannelInputStream(null, createPartialReadThenErrorChannel(expected), false, new AtomicBoolean(), null); Throwable actual = assertThrows(Throwable.class, () -> inputStream.read(new byte[8192])); diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java index f93383275e93..091c96b033b7 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java @@ -1,163 +1,153 @@ -//// Copyright (c) Microsoft Corporation. All rights reserved. -//// Licensed under the MIT License. -//package io.clientcore.http.netty4.implementation; -// -//import io.clientcore.core.http.models.HttpMethod; -//import io.clientcore.core.http.models.HttpRequest; -//import io.clientcore.http.netty4.mocking.MockChannelHandlerContext; -//import io.netty.buffer.Unpooled; -//import io.netty.channel.Channel; -//import io.netty.handler.codec.http.DefaultFullHttpResponse; -//import io.netty.handler.codec.http.DefaultHttpHeaders; -//import io.netty.handler.codec.http.DefaultHttpResponse; -//import io.netty.handler.codec.http.HttpResponseStatus; -//import io.netty.handler.codec.http.HttpVersion; -//import io.netty.handler.codec.http.LastHttpContent; -//import org.junit.jupiter.api.Test; -//import org.junit.jupiter.api.Timeout; -// -//import java.util.concurrent.CountDownLatch; -//import java.util.concurrent.ThreadLocalRandom; -//import java.util.concurrent.TimeUnit; -//import java.util.concurrent.atomic.AtomicReference; -// -//import static io.clientcore.http.netty4.TestUtils.createChannelWithReadHandling; -//import static org.junit.jupiter.api.Assertions.assertEquals; -//import static org.junit.jupiter.api.Assertions.assertNotNull; -//import static org.junit.jupiter.api.Assertions.assertTrue; -// -///** -// * Tests {@link Netty4ResponseHandler}. -// */ -//@Timeout(value = 3, unit = TimeUnit.MINUTES) -//public class Netty4ResponseHandlerTests { -// @Test -// public void firstReadIsFullHttpResponse() throws Exception { -// HttpRequest request = new HttpRequest(); -// AtomicReference responseReference = new AtomicReference<>(); -// CountDownLatch latch = new CountDownLatch(1); -// -// Netty4ResponseHandler responseHandler -// = new Netty4ResponseHandler(request, responseReference, new AtomicReference<>(), latch); -// -// Channel ch = createChannelWithReadHandling((ignored, channel) -> { -// Netty4ResponseHandler handler = channel.pipeline().get(Netty4ResponseHandler.class); -// MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); -// -// try { -// handler.channelRead(ctx, new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, -// Unpooled.EMPTY_BUFFER, new DefaultHttpHeaders(), new DefaultHttpHeaders())); -// handler.channelReadComplete(ctx); -// } catch (Exception ex) { -// ctx.fireExceptionCaught(ex); -// } -// }); -// -// ch.pipeline().addLast(responseHandler); -// ch.read(); -// -// assertEquals(0, latch.getCount()); -// -// ResponseStateInfo info = responseReference.get(); -// assertNotNull(info); -// -// assertTrue(info.isChannelConsumptionComplete()); -// assertEquals(0, info.getEagerContent().size()); -// } -// -// @Test -// public void incompleteIgnoredResponseBody() { -// byte[] ignoredBodyData = new byte[32]; -// ThreadLocalRandom.current().nextBytes(ignoredBodyData); -// -// HttpRequest request = new HttpRequest().setMethod(HttpMethod.HEAD); -// AtomicReference responseReference = new AtomicReference<>(); -// CountDownLatch latch = new CountDownLatch(1); -// -// Netty4ResponseHandler responseHandler -// = new Netty4ResponseHandler(request, responseReference, new AtomicReference<>(), latch); -// -// Channel ch = createChannelWithReadHandling((readCount, channel) -> { -// if (readCount == 0) { -// Netty4ResponseHandler handler = channel.pipeline().get(Netty4ResponseHandler.class); -// MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); -// try { -// handler.channelRead(ctx, -// new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, new DefaultHttpHeaders())); -// handler.channelReadComplete(ctx); -// } catch (Exception ex) { -// ctx.fireExceptionCaught(ex); -// } -// } else { -// Netty4EagerConsumeChannelHandler handler -// = channel.pipeline().get(Netty4EagerConsumeChannelHandler.class); -// MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); -// handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); -// handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); -// handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); -// handler.channelRead(ctx, Unpooled.wrappedBuffer(ignoredBodyData)); -// handler.channelRead(ctx, LastHttpContent.EMPTY_LAST_CONTENT); -// handler.channelReadComplete(ctx); -// } -// }); -// -// ch.pipeline().addLast(responseHandler); -// ch.read(); -// -// assertEquals(0, latch.getCount()); -// -// ResponseStateInfo info = responseReference.get(); -// assertNotNull(info); -// } -// -// @Test -// public void bufferedResponseBodyLargerThanInitialRead() { -// byte[] bodyPieces = new byte[32]; -// ThreadLocalRandom.current().nextBytes(bodyPieces); -// -// byte[] expectedBody = new byte[bodyPieces.length * 4]; -// System.arraycopy(bodyPieces, 0, expectedBody, 0, bodyPieces.length); -// System.arraycopy(bodyPieces, 0, expectedBody, bodyPieces.length, bodyPieces.length); -// System.arraycopy(bodyPieces, 0, expectedBody, bodyPieces.length * 2, bodyPieces.length); -// System.arraycopy(bodyPieces, 0, expectedBody, bodyPieces.length * 3, bodyPieces.length); -// -// HttpRequest request = new HttpRequest(); -// AtomicReference responseReference = new AtomicReference<>(); -// CountDownLatch latch = new CountDownLatch(1); -// -// Netty4ResponseHandler responseHandler -// = new Netty4ResponseHandler(request, responseReference, new AtomicReference<>(), latch); -// -// Channel ch = createChannelWithReadHandling((readCount, channel) -> { -// if (readCount == 0) { -// Netty4ResponseHandler handler = channel.pipeline().get(Netty4ResponseHandler.class); -// MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); -// try { -// handler.channelRead(ctx, -// new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, new DefaultHttpHeaders())); -// handler.channelReadComplete(ctx); -// } catch (Exception ex) { -// ctx.fireExceptionCaught(ex); -// } -// } else { -// Netty4EagerConsumeChannelHandler handler -// = channel.pipeline().get(Netty4EagerConsumeChannelHandler.class); -// MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); -// handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); -// handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); -// handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); -// handler.channelRead(ctx, Unpooled.wrappedBuffer(bodyPieces)); -// handler.channelRead(ctx, LastHttpContent.EMPTY_LAST_CONTENT); -// handler.channelReadComplete(ctx); -// } -// }); -// -// ch.pipeline().addLast(responseHandler); -// ch.read(); -// -// assertEquals(0, latch.getCount()); -// -// ResponseStateInfo info = responseReference.get(); -// assertNotNull(info); -// } -//} +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package io.clientcore.http.netty4.implementation; + +import io.clientcore.core.http.models.HttpMethod; +import io.clientcore.core.http.models.HttpRequest; +import io.clientcore.http.netty4.mocking.MockChannelHandlerContext; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.DefaultHttpHeaders; +import io.netty.handler.codec.http.DefaultHttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpVersion; +import io.netty.handler.codec.http.LastHttpContent; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import static io.clientcore.http.netty4.TestUtils.createChannelWithReadHandling; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests {@link Netty4ResponseHandler}. + */ +@Timeout(value = 1, unit = TimeUnit.MINUTES) +public class Netty4ResponseHandlerTests { + @Test + public void firstReadIsFullHttpResponse() throws Exception { + HttpRequest request = new HttpRequest(); + AtomicReference responseReference = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + + Netty4ResponseHandler responseHandler + = new Netty4ResponseHandler(request, responseReference, new AtomicReference<>(), latch); + + Channel ch = createChannelWithReadHandling((ignored, channel) -> { + Netty4ResponseHandler handler = channel.pipeline().get(Netty4ResponseHandler.class); + MockChannelHandlerContext ctx = new MockChannelHandlerContext(channel); + + try { + handler.channelRead(ctx, new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, + Unpooled.EMPTY_BUFFER, new DefaultHttpHeaders(), new DefaultHttpHeaders())); + handler.channelReadComplete(ctx); + } catch (Exception ex) { + ctx.fireExceptionCaught(ex); + } + }); + + ch.pipeline().addLast(responseHandler); + ch.read(); + + assertEquals(0, latch.getCount()); + + ResponseStateInfo info = responseReference.get(); + assertNotNull(info); + + assertTrue(info.isChannelConsumptionComplete()); + assertEquals(0, info.getEagerContent().size()); + } + + @Test + public void incompleteIgnoredResponseBody() throws InterruptedException { + CountDownLatch headersLatch = new CountDownLatch(1); + Netty4ResponseHandler responseHandler = new Netty4ResponseHandler(new HttpRequest().setMethod(HttpMethod.HEAD), + new AtomicReference<>(), new AtomicReference<>(), headersLatch); + + CountDownLatch bodyLatch = new CountDownLatch(1); + + Channel ch = createChannelWithReadHandling((readCount, channel) -> { + try { + if (readCount == 0) { + responseHandler.channelRead(new MockChannelHandlerContext(channel), + new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK)); + responseHandler.channelReadComplete(new MockChannelHandlerContext(channel)); + } else { + Netty4EagerConsumeChannelHandler eagerConsumer + = channel.pipeline().get(Netty4EagerConsumeChannelHandler.class); + eagerConsumer.channelRead(new MockChannelHandlerContext(channel), + LastHttpContent.EMPTY_LAST_CONTENT); + eagerConsumer.channelReadComplete(new MockChannelHandlerContext(channel)); + } + } catch (Exception e) { + channel.pipeline().fireExceptionCaught(e); + } + }); + + ch.pipeline().addLast(responseHandler); + + ch.read(); + assertTrue(headersLatch.await(10, TimeUnit.SECONDS)); + + ch.pipeline().addLast(new Netty4EagerConsumeChannelHandler(bodyLatch, ignored -> { + }, false)); + + ch.read(); + assertTrue(bodyLatch.await(10, TimeUnit.SECONDS)); + } + + @Test + public void bufferedResponseBodyLargerThanInitialRead() throws InterruptedException { + AtomicReference responseReference = new AtomicReference<>(); + CountDownLatch headersLatch = new CountDownLatch(1); + + Netty4ResponseHandler responseHandler + = new Netty4ResponseHandler(new HttpRequest(), responseReference, new AtomicReference<>(), headersLatch); + + CountDownLatch bodyLatch = new CountDownLatch(1); + + Channel ch = createChannelWithReadHandling((readCount, channel) -> { + try { + if (readCount == 0) { + responseHandler.channelRead(new MockChannelHandlerContext(channel), + new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK)); + responseHandler.channelReadComplete(new MockChannelHandlerContext(channel)); + } else { + Netty4EagerConsumeChannelHandler eagerConsumer + = channel.pipeline().get(Netty4EagerConsumeChannelHandler.class); + eagerConsumer.channelRead(new MockChannelHandlerContext(channel), + LastHttpContent.EMPTY_LAST_CONTENT); + eagerConsumer.channelReadComplete(new MockChannelHandlerContext(channel)); + } + } catch (Exception e) { + channel.pipeline().fireExceptionCaught(e); + } + }); + + ch.pipeline().addLast(responseHandler); + + ch.read(); + assertTrue(headersLatch.await(10, TimeUnit.SECONDS)); + ResponseStateInfo info = responseReference.get(); + assertNotNull(info); + + ch.pipeline().addLast(new Netty4EagerConsumeChannelHandler(bodyLatch, buf -> { + try { + buf.readBytes(info.getEagerContent(), buf.readableBytes()); + } catch (IOException ex) { + throw new UncheckedIOException(ex); + } + }, false)); + + ch.read(); + assertTrue(bodyLatch.await(10, TimeUnit.SECONDS)); + } +} diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/mocking/MockChannel.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/mocking/MockChannel.java index 934fe564a5c9..f2b0e8057b81 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/mocking/MockChannel.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/mocking/MockChannel.java @@ -11,12 +11,14 @@ import io.netty.channel.EventLoop; import io.netty.util.Attribute; import io.netty.util.AttributeKey; +import io.netty.util.DefaultAttributeMap; import java.net.SocketAddress; public class MockChannel extends AbstractChannel { private final ChannelConfig config; private final ChannelMetadata metadata = new ChannelMetadata(false); + private final DefaultAttributeMap attributes = new DefaultAttributeMap(); public MockChannel() { super(null); @@ -25,7 +27,7 @@ public MockChannel() { @Override public Attribute attr(AttributeKey key) { - return null; + return attributes.attr(key); } @Override From 6fdf2e7b93f3a3895288f53ef7ea4c2c1400876e Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sat, 5 Jul 2025 19:28:22 +0300 Subject: [PATCH 30/57] make close method of netty binary data non blocking --- .../Netty4ChannelBinaryData.java | 86 +++++++++++++---- .../Netty4EagerConsumeChannelHandler.java | 95 +++++++++++++------ 2 files changed, 134 insertions(+), 47 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index a2e40d55d97f..070193efca0c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -66,8 +66,7 @@ public byte[] toBytes() { } if (bytes == null) { - drainStream(); - bytes = eagerContent.toByteArray(); + drainStreamSync(); eagerContent = null; } return bytes; @@ -121,6 +120,13 @@ public void writeTo(OutputStream outputStream) { return; } + if (!streamDrained.compareAndSet(false, true)) { + // Another drain operation (e.g., toBytes, toStream().close(), etc.) + // has already started. This call is either redundant or a race condition. + // We'll assume the first operation will handle cleanup. + return; + } + CountDownLatch latch = new CountDownLatch(1); Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, buf -> buf.readBytes(outputStream, buf.readableBytes()), isHttp2); @@ -131,6 +137,13 @@ public void writeTo(OutputStream outputStream) { streamDrained.set(true); Throwable exception = handler.channelException(); + + if (channel.eventLoop().inEventLoop()) { + cleanup(); + } else { + channel.eventLoop().execute(this::cleanup); + } + if (exception != null) { if (exception instanceof Error) { throw (Error) exception; @@ -178,53 +191,86 @@ public BinaryData toReplayableBinaryData() { */ @Override public void close() { - drainStream(); - - Runnable cleanupTask = () -> { - Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); + if (streamDrained.compareAndSet(false, true)) { + if (channel.eventLoop().inEventLoop()) { + drainAndCleanupAsync(); + } else { + channel.eventLoop().execute(this::drainAndCleanupAsync); } - }; + } + } - if (channel.eventLoop().inEventLoop()) { - cleanupTask.run(); - } else { - channel.eventLoop().execute(cleanupTask); + private void drainAndCleanupAsync() { + if (!channel.isActive()) { + cleanup(); + return; } + + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(this::cleanup, isHttp2); + channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); + channel.config().setAutoRead(true); } - private void drainStream() { + private void cleanup() { + Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); + } + } + +// private void drainAndCleanup() { +// drainStreamBlocking(); +// +// Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); +// if (cleanupHandler != null) { +// cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); +// } +// } + + private void drainStreamSync() { if (streamDrained.compareAndSet(false, true)) { if (channel.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { return; } - if (length != null && eagerContent.size() >= length) { + if (length != null && eagerContent != null && eagerContent.size() >= length) { + bytes = eagerContent.toByteArray(); + if (channel.eventLoop().inEventLoop()) { + cleanup(); + } else { + channel.eventLoop().execute(this::cleanup); + } return; } if (!channel.isActive()) { + this.bytes = (eagerContent == null) ? new byte[0] : eagerContent.toByteArray(); return; } CountDownLatch latch = new CountDownLatch(1); Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, buf -> buf.readBytes(eagerContent, buf.readableBytes()), isHttp2); - Runnable setupAndRead = () -> { - channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); - channel.config().setAutoRead(true); - }; + channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); + channel.config().setAutoRead(true); - channel.eventLoop().execute(setupAndRead); awaitLatch(latch); Throwable exception = handler.channelException(); + + if (channel.eventLoop().inEventLoop()) { + cleanup(); + } else { + channel.eventLoop().execute(this::cleanup); + } + if (exception != null) { if (exception instanceof Error) { throw (Error) exception; } else { throw CoreException.from(exception); } + } else { + bytes = eagerContent.toByteArray(); } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 8feb2cbe5b12..ccf17efd18c9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -24,6 +24,7 @@ public final class Netty4EagerConsumeChannelHandler extends ChannelInboundHandlerAdapter { private final CountDownLatch latch; private final IOExceptionCheckedConsumer byteBufConsumer; + private final Runnable onComplete; private final boolean isHttp2; private boolean lastRead; @@ -41,41 +42,67 @@ public Netty4EagerConsumeChannelHandler(CountDownLatch latch, IOExceptionChecked this.latch = latch; this.byteBufConsumer = byteBufConsumer; this.isHttp2 = isHttp2; + this.onComplete = null; + } + + /** + * Creates a new instance of {@link Netty4EagerConsumeChannelHandler} for non-blocking drain operations. + * + * @param onComplete The callback to run when the stream is fully drained or an error occurs. + * @param isHttp2 Flag indicating whether the handler is used for HTTP/2 or not. + */ + public Netty4EagerConsumeChannelHandler(Runnable onComplete, boolean isHttp2) { + this.latch = null; + this.byteBufConsumer = buf -> { + }; + this.onComplete = onComplete; + this.isHttp2 = isHttp2; } @Override public void channelRead(ChannelHandlerContext ctx, Object msg) { - ByteBuf buf = null; - try { - if (msg instanceof ByteBufHolder) { - buf = ((ByteBufHolder) msg).content(); - } else if (msg instanceof ByteBuf) { - buf = (ByteBuf) msg; - } - - if (buf != null && buf.isReadable()) { - byteBufConsumer.accept(buf); - } - - if (isHttp2) { - lastRead = msg instanceof Http2DataFrame && ((Http2DataFrame) msg).isEndStream(); - } else { - lastRead = msg instanceof LastHttpContent; + if (byteBufConsumer != null) { + ByteBuf buf = null; + + if (msg instanceof ByteBufHolder) { + buf = ((ByteBufHolder) msg).content(); + } else if (msg instanceof ByteBuf) { + buf = (ByteBuf) msg; + } + + if (buf != null && buf.isReadable()) { + byteBufConsumer.accept(buf); + } } - ctx.fireChannelRead(msg); - } catch (IOException | RuntimeException ex) { ReferenceCountUtil.release(msg); ctx.fireExceptionCaught(ex); - latch.countDown(); + if (latch != null) { + latch.countDown(); + } + return; + } finally { + if (latch == null) { + ReferenceCountUtil.release(msg); + } + } + + if (isHttp2) { + lastRead = msg instanceof Http2DataFrame && ((Http2DataFrame) msg).isEndStream(); + } else { + lastRead = msg instanceof LastHttpContent; + } + + if (latch != null) { + ctx.fireChannelRead(msg); } } @Override public void channelReadComplete(ChannelHandlerContext ctx) { if (lastRead) { - latch.countDown(); + signalComplete(ctx, false); } ctx.fireChannelReadComplete(); } @@ -83,8 +110,7 @@ public void channelReadComplete(ChannelHandlerContext ctx) { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; - cleanup(ctx); - latch.countDown(); + signalComplete(ctx, true); ctx.fireExceptionCaught(cause); } @@ -95,15 +121,13 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - cleanup(ctx); - latch.countDown(); + signalComplete(ctx, true); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - cleanup(ctx); - latch.countDown(); + signalComplete(ctx, true); ctx.fireChannelInactive(); } @@ -117,8 +141,25 @@ public void handlerAdded(ChannelHandlerContext ctx) { } } + private void signalComplete(ChannelHandlerContext ctx, boolean forceClose) { + if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { + ctx.pipeline().remove(this); + } + + if (latch != null) { + latch.countDown(); + } + if (onComplete != null) { + onComplete.run(); + } + + if (forceClose) { + cleanup(ctx); + } + } + private void cleanup(ChannelHandlerContext ctx) { - if (latch.getCount() == 0) { + if (latch != null && latch.getCount() == 0) { return; } From 623503c4aef4a389c5c42d5cdfb8135a798d2476 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sun, 6 Jul 2025 08:42:58 +0300 Subject: [PATCH 31/57] cleanup connection on netty binary data --- .../http/netty4/NettyHttpClient.java | 1 + .../Netty4ChannelBinaryData.java | 104 +++++------------- 2 files changed, 31 insertions(+), 74 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index 58fed716b202..cc99e89f64a2 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -236,6 +236,7 @@ public Response send(HttpRequest request) { // If an error occurred or we want to reconnect if (!Thread.currentThread().isInterrupted() && attemptRetry(serverSentResult, request)) { + response.close(); return this.send(request); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 070193efca0c..6f2f4131c662 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -38,6 +38,7 @@ public final class Netty4ChannelBinaryData extends BinaryData { private final Long length; private final boolean isHttp2; private final AtomicBoolean streamDrained = new AtomicBoolean(false); + private final CountDownLatch drainLatch = new CountDownLatch(1); // Non-final to allow nulling out after use. private ByteArrayOutputStream eagerContent; @@ -67,7 +68,6 @@ public byte[] toBytes() { if (bytes == null) { drainStreamSync(); - eagerContent = null; } return bytes; } @@ -108,55 +108,11 @@ public void writeTo(JsonWriter jsonWriter) { @Override public void writeTo(OutputStream outputStream) { + Objects.requireNonNull(outputStream, "'outputStream' cannot be null."); try { - if (bytes == null) { - // Channel hasn't been read yet, don't buffer it, just write it to the OutputStream as it's being read. - if (eagerContent.size() > 0) { - outputStream.write(eagerContent.toByteArray()); - eagerContent.reset(); - } - - if (!channel.isActive()) { - return; - } - - if (!streamDrained.compareAndSet(false, true)) { - // Another drain operation (e.g., toBytes, toStream().close(), etc.) - // has already started. This call is either redundant or a race condition. - // We'll assume the first operation will handle cleanup. - return; - } - - CountDownLatch latch = new CountDownLatch(1); - Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, - buf -> buf.readBytes(outputStream, buf.readableBytes()), isHttp2); - channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); - channel.config().setAutoRead(true); - - awaitLatch(latch); - streamDrained.set(true); - - Throwable exception = handler.channelException(); - - if (channel.eventLoop().inEventLoop()) { - cleanup(); - } else { - channel.eventLoop().execute(this::cleanup); - } - - if (exception != null) { - if (exception instanceof Error) { - throw (Error) exception; - } else { - throw CoreException.from(exception); - } - } - } else { - // Already converted the Channel to a byte[], use it. - outputStream.write(bytes); - } - } catch (IOException ex) { - throw LOGGER.throwableAtError().log(ex, CoreException::from); + outputStream.write(toBytes()); + } catch (IOException e) { + throw LOGGER.throwableAtError().log(e, CoreException::from); } } @@ -192,29 +148,30 @@ public BinaryData toReplayableBinaryData() { @Override public void close() { if (streamDrained.compareAndSet(false, true)) { - if (channel.eventLoop().inEventLoop()) { + try { drainAndCleanupAsync(); - } else { - channel.eventLoop().execute(this::drainAndCleanupAsync); + } finally { + drainLatch.countDown(); } } } private void drainAndCleanupAsync() { if (!channel.isActive()) { - cleanup(); + cleanup(true); return; } - Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(this::cleanup, isHttp2); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(() -> cleanup(false), isHttp2); channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); channel.config().setAutoRead(true); } - private void cleanup() { + private void cleanup(boolean closeChannel) { + //TODO: userTriggeredEvent Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); if (cleanupHandler != null) { - cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); + cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), closeChannel); } } @@ -228,40 +185,36 @@ private void cleanup() { // } private void drainStreamSync() { - if (streamDrained.compareAndSet(false, true)) { - if (channel.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { - return; - } + // First, check if another thread has already started the draining process. + if (!streamDrained.compareAndSet(false, true)) { + // If so, this thread becomes a "waiter". It blocks until the other thread is done. + awaitLatch(drainLatch); + return; // The other thread populated 'bytes', so we can just return. + } + + try { if (length != null && eagerContent != null && eagerContent.size() >= length) { bytes = eagerContent.toByteArray(); - if (channel.eventLoop().inEventLoop()) { - cleanup(); - } else { - channel.eventLoop().execute(this::cleanup); - } + cleanup(false); return; } if (!channel.isActive()) { this.bytes = (eagerContent == null) ? new byte[0] : eagerContent.toByteArray(); + cleanup(true); return; } - CountDownLatch latch = new CountDownLatch(1); - Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, + CountDownLatch ioLatch = new CountDownLatch(1); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(ioLatch, buf -> buf.readBytes(eagerContent, buf.readableBytes()), isHttp2); channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); channel.config().setAutoRead(true); - awaitLatch(latch); + awaitLatch(ioLatch); Throwable exception = handler.channelException(); - - if (channel.eventLoop().inEventLoop()) { - cleanup(); - } else { - channel.eventLoop().execute(this::cleanup); - } + cleanup(exception != null); if (exception != null) { if (exception instanceof Error) { @@ -272,6 +225,9 @@ private void drainStreamSync() { } else { bytes = eagerContent.toByteArray(); } + } finally { + drainLatch.countDown(); + eagerContent = null; } } } From f085ece7beb21c664e20c40ebdeb1349708aaec2 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sun, 6 Jul 2025 10:26:28 +0300 Subject: [PATCH 32/57] add exception handling on cleanup handler --- .../http/netty4/NettyHttpClient.java | 2 +- .../Netty4ChannelBinaryData.java | 29 +++++++------------ .../Netty4PipelineCleanupHandler.java | 14 ++++++++- .../implementation/Netty4ResponseHandler.java | 1 + 4 files changed, 26 insertions(+), 20 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index cc99e89f64a2..c18d141d418a 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -277,7 +277,7 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, // The first handler added is the cleanup handler. It will be the last to execute // in the outbound direction and the first in the inbound direction, but its main // purpose is to clean up all other request-specific handlers and release the channel. - pipeline.addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool)); + pipeline.addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is // applying timeouts. diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 6f2f4131c662..c41c7d5101cd 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -39,6 +39,8 @@ public final class Netty4ChannelBinaryData extends BinaryData { private final boolean isHttp2; private final AtomicBoolean streamDrained = new AtomicBoolean(false); private final CountDownLatch drainLatch = new CountDownLatch(1); + // Manages the "closed" state, ensuring cleanup happens only once. + private final AtomicBoolean closed = new AtomicBoolean(false); // Non-final to allow nulling out after use. private ByteArrayOutputStream eagerContent; @@ -67,7 +69,7 @@ public byte[] toBytes() { } if (bytes == null) { - drainStreamSync(); + drainStream(); } return bytes; } @@ -147,16 +149,19 @@ public BinaryData toReplayableBinaryData() { */ @Override public void close() { - if (streamDrained.compareAndSet(false, true)) { - try { + if (closed.compareAndSet(false, true)) { + if (!streamDrained.get()) { drainAndCleanupAsync(); - } finally { - drainLatch.countDown(); + } else { + cleanup(false); } } } private void drainAndCleanupAsync() { + streamDrained.set(true); + drainLatch.countDown(); + if (!channel.isActive()) { cleanup(true); return; @@ -175,16 +180,7 @@ private void cleanup(boolean closeChannel) { } } -// private void drainAndCleanup() { -// drainStreamBlocking(); -// -// Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); -// if (cleanupHandler != null) { -// cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), false); -// } -// } - - private void drainStreamSync() { + private void drainStream() { // First, check if another thread has already started the draining process. if (!streamDrained.compareAndSet(false, true)) { // If so, this thread becomes a "waiter". It blocks until the other thread is done. @@ -195,13 +191,11 @@ private void drainStreamSync() { try { if (length != null && eagerContent != null && eagerContent.size() >= length) { bytes = eagerContent.toByteArray(); - cleanup(false); return; } if (!channel.isActive()) { this.bytes = (eagerContent == null) ? new byte[0] : eagerContent.toByteArray(); - cleanup(true); return; } @@ -214,7 +208,6 @@ private void drainStreamSync() { awaitLatch(ioLatch); Throwable exception = handler.channelException(); - cleanup(exception != null); if (exception != null) { if (exception instanceof Error) { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 174850394790..db077d396932 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -11,7 +11,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.ALPN; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.CHUNKED_WRITER; @@ -20,6 +22,7 @@ import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_RESPONSE; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.READ_ONE; +import static io.clientcore.http.netty4.implementation.Netty4Utility.setOrSuppressError; /** * A handler that cleans up the pipeline after a request-response cycle and releases @@ -28,6 +31,8 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final Netty4ConnectionPool connectionPool; + private final AtomicReference errorReference; + private final CountDownLatch latch; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); private static final List HANDLERS_TO_REMOVE; @@ -44,12 +49,19 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { HANDLERS_TO_REMOVE = Collections.unmodifiableList(handlers); } - public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool) { + public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool, AtomicReference errorReference, + CountDownLatch latch) { this.connectionPool = connectionPool; + this.errorReference = errorReference; + this.latch = latch; } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + if (errorReference != null && latch != null) { + setOrSuppressError(errorReference, cause); + latch.countDown(); + } cleanup(ctx, true); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index e768fb5867bd..b5e9234e302f 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -89,6 +89,7 @@ public boolean isSharable() { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { setOrSuppressError(errorReference, cause); latch.countDown(); + ctx.fireExceptionCaught(cause); } @Override From d66d154aa049e5ade1f3a70668e3eb664e12ac65 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sun, 6 Jul 2025 16:06:29 +0300 Subject: [PATCH 33/57] Netty4ChannelBinaryData adjustments --- .../http/netty4/NettyHttpClient.java | 1 + .../Netty4ChannelBinaryData.java | 118 +++++++++++------- .../Netty4ChannelInputStream.java | 7 +- .../implementation/Netty4ConnectionPool.java | 2 +- .../Netty4EagerConsumeChannelHandler.java | 11 +- .../Netty4Http11ChannelInputStreamTests.java | 28 ++--- 6 files changed, 101 insertions(+), 66 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index c18d141d418a..e6789e6fa8da 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -165,6 +165,7 @@ public Response send(HttpRequest request) { ResponseBodyHandling bodyHandling = info.getResponseBodyHandling(); Channel channel = info.getResponseChannel(); if (bodyHandling == ResponseBodyHandling.IGNORE) { + // TODO: Don't block here? // We're ignoring the response content. CountDownLatch drainLatch = new CountDownLatch(1); channel.pipeline().addLast(new Netty4EagerConsumeChannelHandler(drainLatch, ignored -> { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index c41c7d5101cd..c26427914687 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -8,6 +8,7 @@ import io.clientcore.core.serialization.ObjectSerializer; import io.clientcore.core.serialization.json.JsonWriter; import io.netty.channel.Channel; +import io.netty.util.concurrent.Future; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -91,7 +92,7 @@ public T toObject(Type type, ObjectSerializer serializer) { @Override public InputStream toStream() { if (bytes == null) { - return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, streamDrained, this::close); + return new Netty4ChannelInputStream(eagerContent, channel, isHttp2, this::close); } else { return new ByteArrayInputStream(bytes); } @@ -150,30 +151,57 @@ public BinaryData toReplayableBinaryData() { @Override public void close() { if (closed.compareAndSet(false, true)) { + // If draining hasn't started, it means the stream was not consumed. + // We need to drain it to ensure the connection can be safely reused. if (!streamDrained.get()) { drainAndCleanupAsync(); - } else { - cleanup(false); } } } private void drainAndCleanupAsync() { - streamDrained.set(true); - drainLatch.countDown(); + if (streamDrained.compareAndSet(false, true)) { + if (!channel.isActive()) { + cleanup(true); + drainLatch.countDown(); + return; + } - if (!channel.isActive()) { - cleanup(true); - return; - } + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(() -> { + cleanup(false); + drainLatch.countDown(); + }, isHttp2); - Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(() -> cleanup(false), isHttp2); - channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); - channel.config().setAutoRead(true); + channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); + channel.config().setAutoRead(true); + } else { + awaitLatch(drainLatch); + } } private void cleanup(boolean closeChannel) { - //TODO: userTriggeredEvent + //TODO: use userTriggeredEvent + if (channel.eventLoop().inEventLoop()) { + // We are already on the correct thread, so we can execute directly. + doCleanup(closeChannel); + } else { + Future cleanupFuture = channel.eventLoop().submit(() -> doCleanup(closeChannel)); + + try { + cleanupFuture.get(); + } catch (InterruptedException | java.util.concurrent.ExecutionException e) { + LOGGER.atWarning() + .setThrowable(e) + .log("Failed to wait for channel cleanup. Closing channel forcefully."); + channel.close(); + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + } + } + } + + private void doCleanup(boolean closeChannel) { Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); if (cleanupHandler != null) { cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), closeChannel); @@ -181,46 +209,48 @@ private void cleanup(boolean closeChannel) { } private void drainStream() { - // First, check if another thread has already started the draining process. - if (!streamDrained.compareAndSet(false, true)) { - // If so, this thread becomes a "waiter". It blocks until the other thread is done. - awaitLatch(drainLatch); - return; // The other thread populated 'bytes', so we can just return. - } + if (streamDrained.compareAndSet(false, true)) { + try { + if (length != null && eagerContent != null && eagerContent.size() >= length) { + bytes = eagerContent.toByteArray(); + cleanup(false); + return; + } - try { - if (length != null && eagerContent != null && eagerContent.size() >= length) { - bytes = eagerContent.toByteArray(); - return; - } + if (!channel.isActive()) { + bytes = (eagerContent == null) ? new byte[0] : eagerContent.toByteArray(); + cleanup(true); + return; + } - if (!channel.isActive()) { - this.bytes = (eagerContent == null) ? new byte[0] : eagerContent.toByteArray(); - return; - } + CountDownLatch ioLatch = new CountDownLatch(1); + // This handler is responsible for triggering the cleanup when it completes. + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(ioLatch, + buf -> buf.readBytes(eagerContent, buf.readableBytes()), isHttp2); - CountDownLatch ioLatch = new CountDownLatch(1); - Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(ioLatch, - buf -> buf.readBytes(eagerContent, buf.readableBytes()), isHttp2); - channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); - channel.config().setAutoRead(true); + channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); + channel.config().setAutoRead(true); - awaitLatch(ioLatch); + awaitLatch(ioLatch); - Throwable exception = handler.channelException(); + Throwable exception = handler.channelException(); - if (exception != null) { - if (exception instanceof Error) { - throw (Error) exception; + if (exception != null) { + if (exception instanceof Error) { + throw (Error) exception; + } else { + throw CoreException.from(exception); + } } else { - throw CoreException.from(exception); + bytes = eagerContent.toByteArray(); } - } else { - bytes = eagerContent.toByteArray(); + } finally { + closed.set(true); + drainLatch.countDown(); + eagerContent = null; } - } finally { - drainLatch.countDown(); - eagerContent = null; + } else { + awaitLatch(drainLatch); } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java index a38c70987039..de5d4c96c0af 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelInputStream.java @@ -10,7 +10,6 @@ import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; /** * Implementation of {@link InputStream} that reads contents from a Netty {@link Channel}. @@ -19,7 +18,6 @@ public final class Netty4ChannelInputStream extends InputStream { private final Channel channel; private final boolean isHttp2; private final Runnable onClose; - private final AtomicBoolean parentStreamDrained; // Indicator for the Channel being fully read. // This will become true before 'streamDone' becomes true, but both may become true in the same operation. @@ -52,8 +50,7 @@ public final class Netty4ChannelInputStream extends InputStream { * @param isHttp2 Flag indicating whether the Channel is used for HTTP/2 or not. * @param onClose A runnable to execute when the stream is closed. */ - Netty4ChannelInputStream(ByteArrayOutputStream eagerContent, Channel channel, boolean isHttp2, - AtomicBoolean parentStreamDrained, Runnable onClose) { + Netty4ChannelInputStream(ByteArrayOutputStream eagerContent, Channel channel, boolean isHttp2, Runnable onClose) { if (eagerContent != null && eagerContent.size() > 0) { this.currentBuffer = eagerContent.toByteArray(); eagerContent.reset(); @@ -68,7 +65,6 @@ public final class Netty4ChannelInputStream extends InputStream { } this.isHttp2 = isHttp2; this.onClose = onClose; - this.parentStreamDrained = parentStreamDrained; } byte[] getCurrentBuffer() { @@ -204,7 +200,6 @@ private boolean setupNextBuffer() throws IOException { return true; } else { streamDone = true; - parentStreamDrained.set(true); return false; } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 1720db460bd4..f63fb486af5a 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -240,7 +240,7 @@ Future acquire() { while (true) { PooledConnection connection = idleConnections.poll(); if (connection == null) { - break; // No idle connections available. + break; } if (isHealthy(connection)) { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index ccf17efd18c9..7fc0cde63840 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -142,10 +142,19 @@ public void handlerAdded(ChannelHandlerContext ctx) { } private void signalComplete(ChannelHandlerContext ctx, boolean forceClose) { + boolean isSyncDrain = (latch != null); + if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { ctx.pipeline().remove(this); } + if (isSyncDrain) { + Netty4PipelineCleanupHandler mainCleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); + if (mainCleanupHandler != null) { + mainCleanupHandler.cleanup(ctx, forceClose); + } + } + if (latch != null) { latch.countDown(); } @@ -153,7 +162,7 @@ private void signalComplete(ChannelHandlerContext ctx, boolean forceClose) { onComplete.run(); } - if (forceClose) { + if (!isSyncDrain && forceClose) { cleanup(ctx); } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java index 99aeb61873ae..3563d03c3813 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelInputStreamTests.java @@ -42,15 +42,15 @@ public class Netty4Http11ChannelInputStreamTests { @Test public void nullEagerContentResultsInEmptyInitialCurrentBuffer() throws IOException { try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(null, createCloseableChannel(), false, new AtomicBoolean(), null)) { + = new Netty4ChannelInputStream(null, createCloseableChannel(), false, null)) { assertEquals(0, channelInputStream.getCurrentBuffer().length); } } @Test public void emptyEagerContentResultsInEmptyInitialCurrentBuffer() throws IOException { - try (Netty4ChannelInputStream channelInputStream = new Netty4ChannelInputStream(new ByteArrayOutputStream(), - createCloseableChannel(), false, new AtomicBoolean(), null)) { + try (Netty4ChannelInputStream channelInputStream + = new Netty4ChannelInputStream(new ByteArrayOutputStream(), createCloseableChannel(), false, null)) { assertEquals(0, channelInputStream.getCurrentBuffer().length); } } @@ -65,7 +65,7 @@ public void readConsumesCurrentBufferAndHasNoMoreData() throws IOException { // MockChannels aren't active by default, so once the eagerContent is consumed the stream will be done. Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(eagerContent, new MockChannel(), false, new AtomicBoolean(), null); + = new Netty4ChannelInputStream(eagerContent, new MockChannel(), false, null); // Make sure the Netty4ChannelInputStream copied the eager content correctly. assertArraysEqual(expected, channelInputStream.getCurrentBuffer()); @@ -97,7 +97,7 @@ public void readConsumesCurrentBufferAndRequestsMoreData() throws IOException { handler.channelRead(ctx, wrappedBuffer(expected, 16, 16)); handler.channelRead(ctx, LastHttpContent.EMPTY_LAST_CONTENT); handler.channelReadComplete(ctx); - }), false, new AtomicBoolean(), null); + }), false, null); int index = 0; byte[] actual = new byte[32]; @@ -119,7 +119,7 @@ public void multipleSmallerSkips() throws IOException { // MockChannels aren't active by default, so once the eagerContent is consumed the stream will be done. try (Netty4ChannelInputStream channelInputStream - = new Netty4ChannelInputStream(eagerContent, createCloseableChannel(), false, new AtomicBoolean(), null)) { + = new Netty4ChannelInputStream(eagerContent, createCloseableChannel(), false, null)) { long skipped = channelInputStream.skip(16); assertEquals(16, skipped); @@ -141,8 +141,8 @@ public void largeReadTriggersMultipleChannelReads() throws IOException { byte[] expected = new byte[8192]; ThreadLocalRandom.current().nextBytes(expected); - try (Netty4ChannelInputStream channelInputStream = new Netty4ChannelInputStream(null, - createChannelThatReads8Kb(expected), false, new AtomicBoolean(), null)) { + try (Netty4ChannelInputStream channelInputStream + = new Netty4ChannelInputStream(null, createChannelThatReads8Kb(expected), false, null)) { byte[] actual = new byte[8192]; int read = channelInputStream.read(actual); @@ -162,8 +162,8 @@ public void largeSkipTriggersMultipleChannelReads() throws IOException { byte[] expected = new byte[8192]; ThreadLocalRandom.current().nextBytes(expected); - try (Netty4ChannelInputStream channelInputStream = new Netty4ChannelInputStream(null, - createChannelThatReads8Kb(expected), false, new AtomicBoolean(), null)) { + try (Netty4ChannelInputStream channelInputStream + = new Netty4ChannelInputStream(null, createChannelThatReads8Kb(expected), false, null)) { long skipped = channelInputStream.skip(8192); assertEquals(8192, skipped); @@ -176,8 +176,8 @@ public void largeSkipTriggersMultipleChannelReads() throws IOException { public void closingStreamTriggersOnCloseCallback() throws IOException { AtomicBoolean onCloseCalled = new AtomicBoolean(false); - try (Netty4ChannelInputStream channelInputStream = new Netty4ChannelInputStream(null, createCloseableChannel(), - false, new AtomicBoolean(), () -> onCloseCalled.set(true))) { + try (Netty4ChannelInputStream channelInputStream + = new Netty4ChannelInputStream(null, createCloseableChannel(), false, () -> onCloseCalled.set(true))) { assertNotNull(channelInputStream); } @@ -187,8 +187,8 @@ false, new AtomicBoolean(), () -> onCloseCalled.set(true))) { @ParameterizedTest @MethodSource("errorSupplier") public void streamPropagatesErrorFiredInChannel(Throwable expected) { - InputStream inputStream = new Netty4ChannelInputStream(null, createPartialReadThenErrorChannel(expected), false, - new AtomicBoolean(), null); + InputStream inputStream + = new Netty4ChannelInputStream(null, createPartialReadThenErrorChannel(expected), false, null); Throwable actual = assertThrows(Throwable.class, () -> inputStream.read(new byte[8192])); From aa1fba91f174116b1d8a9b4b9fac1e1b71d6f7f1 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Mon, 7 Jul 2025 10:22:29 +0300 Subject: [PATCH 34/57] try to not block on send method --- .../http/netty4/NettyHttpClient.java | 92 +++++-------------- .../Netty4ChannelBinaryData.java | 10 +- .../implementation/Netty4ConnectionPool.java | 17 +++- .../Netty4EagerConsumeChannelHandler.java | 35 ++----- .../Netty4PipelineCleanupHandler.java | 2 +- .../implementation/Netty4ResponseHandler.java | 1 + 6 files changed, 55 insertions(+), 102 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index e6789e6fa8da..90c800ed4bed 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -22,11 +22,9 @@ import io.clientcore.http.netty4.implementation.Netty4ChannelBinaryData; import io.clientcore.http.netty4.implementation.Netty4ConnectionPool; import io.clientcore.http.netty4.implementation.Netty4ConnectionPoolKey; -import io.clientcore.http.netty4.implementation.Netty4EagerConsumeChannelHandler; import io.clientcore.http.netty4.implementation.Netty4PipelineCleanupHandler; import io.clientcore.http.netty4.implementation.Netty4ProgressAndTimeoutHandler; import io.clientcore.http.netty4.implementation.Netty4ResponseHandler; -import io.clientcore.http.netty4.implementation.ResponseBodyHandling; import io.clientcore.http.netty4.implementation.ResponseStateInfo; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; @@ -143,74 +141,18 @@ public Response send(HttpRequest request) { throw LOGGER.throwableAtError().log(errorReference.get(), CoreException::from); } - Response response; - Channel channelToRelease; - + //Response response; + //Channel channelToRelease; + BinaryData body; if (info.isChannelConsumptionComplete()) { - // The network response is already complete, handle creating our Response based on the request method and - // response headers. - BinaryData body = BinaryData.empty(); ByteArrayOutputStream eagerContent = info.getEagerContent(); - if (info.getResponseBodyHandling() != ResponseBodyHandling.IGNORE && eagerContent.size() > 0) { - // Set the response body as the first HttpContent received if the request wasn't a HEAD request and - // there was body content. - body = BinaryData.fromBytes(eagerContent.toByteArray()); - } - channelToRelease = info.getResponseChannel(); - response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); - } else { - // Otherwise we aren't finished, handle the remaining content according to the documentation in - // 'channelRead()'. - BinaryData body = BinaryData.empty(); - ResponseBodyHandling bodyHandling = info.getResponseBodyHandling(); - Channel channel = info.getResponseChannel(); - if (bodyHandling == ResponseBodyHandling.IGNORE) { - // TODO: Don't block here? - // We're ignoring the response content. - CountDownLatch drainLatch = new CountDownLatch(1); - channel.pipeline().addLast(new Netty4EagerConsumeChannelHandler(drainLatch, ignored -> { - }, info.isHttp2())); - channel.config().setAutoRead(true); - awaitLatch(drainLatch); - channelToRelease = channel; - } else if (bodyHandling == ResponseBodyHandling.STREAM) { - channelToRelease = null; - // Body streaming uses a special BinaryData that tracks the firstContent read and the Channel it came - // from so it can be consumed when the BinaryData is being used. - // autoRead should have been disabled already but lets make sure that it is. - channel.config().setAutoRead(false); - String contentLength = info.getHeaders().getValue(HttpHeaderName.CONTENT_LENGTH); - Long length = null; - if (!CoreUtils.isNullOrEmpty(contentLength)) { - try { - length = Long.parseLong(contentLength); - } catch (NumberFormatException ignored) { - // Ignore, we'll just read until the channel is closed. - } - } - body = new Netty4ChannelBinaryData(info.getEagerContent(), channel, length, info.isHttp2()); - } else { - // All cases otherwise assume BUFFER. - CountDownLatch drainLatch = new CountDownLatch(1); - channel.pipeline().addLast(new Netty4EagerConsumeChannelHandler(drainLatch, buf -> { - try { - buf.readBytes(info.getEagerContent(), buf.readableBytes()); - } catch (IOException ex) { - throw LOGGER.throwableAtError().log(ex, CoreException::from); - } - }, info.isHttp2())); - channel.config().setAutoRead(true); - awaitLatch(drainLatch); - channelToRelease = channel; - - body = BinaryData.fromBytes(info.getEagerContent().toByteArray()); - } + body = (eagerContent == null || eagerContent.size() == 0) + ? BinaryData.empty() + : BinaryData.fromBytes(eagerContent.toByteArray()); - response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); - } - - if (channelToRelease != null) { + // Since the response is complete, we are responsible for cleaning up the channel. + Channel channelToRelease = info.getResponseChannel(); channelToRelease.eventLoop().execute(() -> { Netty4PipelineCleanupHandler cleanupHandler = channelToRelease.pipeline().get(Netty4PipelineCleanupHandler.class); @@ -218,8 +160,24 @@ public Response send(HttpRequest request) { cleanupHandler.cleanup(channelToRelease.pipeline().context(cleanupHandler), false); } }); + } else { + // For all other cases, create a streaming response body. + // This delegates all body consumption and cleanup logic to Netty4ChannelBinaryData. + String contentLength = info.getHeaders().getValue(HttpHeaderName.CONTENT_LENGTH); + Long length = null; + if (!CoreUtils.isNullOrEmpty(contentLength)) { + try { + length = Long.parseLong(contentLength); + } catch (NumberFormatException ignored) { + // Ignore, we'll just read until the channel is closed. + } + } + body = new Netty4ChannelBinaryData(info.getEagerContent(), info.getResponseChannel(), length, + info.isHttp2()); } + Response response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); + if (response.getValue() != BinaryData.empty() && ServerSentEventUtils .isTextEventStreamContentType(response.getHeaders().getValue(HttpHeaderName.CONTENT_TYPE))) { @@ -247,10 +205,10 @@ public Response send(HttpRequest request) { throw LOGGER.throwableAtError().log(ex, CoreException::from); } } else { + response.close(); throw LOGGER.throwableAtError().log(NO_LISTENER_ERROR_MESSAGE, IllegalStateException::new); } } - return response; } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index c26427914687..ada45891a3de 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -8,7 +8,6 @@ import io.clientcore.core.serialization.ObjectSerializer; import io.clientcore.core.serialization.json.JsonWriter; import io.netty.channel.Channel; -import io.netty.util.concurrent.Future; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -180,19 +179,15 @@ private void drainAndCleanupAsync() { } private void cleanup(boolean closeChannel) { - //TODO: use userTriggeredEvent if (channel.eventLoop().inEventLoop()) { - // We are already on the correct thread, so we can execute directly. doCleanup(closeChannel); } else { - Future cleanupFuture = channel.eventLoop().submit(() -> doCleanup(closeChannel)); - try { - cleanupFuture.get(); + channel.eventLoop().submit(() -> doCleanup(closeChannel)).get(); } catch (InterruptedException | java.util.concurrent.ExecutionException e) { LOGGER.atWarning() .setThrowable(e) - .log("Failed to wait for channel cleanup. Closing channel forcefully."); + .log("Exception during synchronous channel cleanup. Forcing channel close."); channel.close(); if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); @@ -202,6 +197,7 @@ private void cleanup(boolean closeChannel) { } private void doCleanup(boolean closeChannel) { + // TODO: userTriggeredEvent Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); if (cleanupHandler != null) { cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), closeChannel); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index f63fb486af5a..155ce485e563 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -10,6 +10,8 @@ import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; @@ -356,7 +358,7 @@ private void satisfyWaiterWithNewConnection() { private Future createNewConnection() { Bootstrap newConnectionBootstrap = bootstrap.clone(); - + Promise promise = newConnectionBootstrap.config().group().next().newPromise(); newConnectionBootstrap.handler(new ChannelInitializer() { @Override public void initChannel(Channel channel) throws SSLException { @@ -370,6 +372,14 @@ public void initChannel(Channel channel) throws SSLException { if (hasProxy) { ProxyHandler proxyHandler = channelInitializationProxyHandler.createProxy(proxyChallenges); pipeline.addFirst(PROXY, proxyHandler); + + pipeline.addLast("proxy-exception-handler", new ChannelInboundHandlerAdapter() { + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + promise.tryFailure(cause); + ctx.close(); + } + }); } // Add SSL handling if the request is HTTPS. @@ -385,7 +395,6 @@ public void initChannel(Channel channel) throws SSLException { } }); - Promise promise = newConnectionBootstrap.config().group().next().newPromise(); newConnectionBootstrap.connect(route).addListener(future -> { if (!future.isSuccess()) { LOGGER.atError().setThrowable(future.cause()).log("Failed connection."); @@ -409,6 +418,10 @@ public void initChannel(Channel channel) throws SSLException { satisfyWaiterWithNewConnection(); return; } + ChannelPipeline pipeline = newChannel.pipeline(); + if (pipeline.get("proxy-exception-handler") != null) { + pipeline.remove("proxy-exception-handler"); + } promise.setSuccess(newChannel); } else { promise.setFailure(proxyFuture.cause()); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 7fc0cde63840..3cda51e893c9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -75,43 +75,32 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { byteBufConsumer.accept(buf); } } - } catch (IOException | RuntimeException ex) { - ReferenceCountUtil.release(msg); - ctx.fireExceptionCaught(ex); - if (latch != null) { - latch.countDown(); + + if (isHttp2) { + lastRead = msg instanceof Http2DataFrame && ((Http2DataFrame) msg).isEndStream(); + } else { + lastRead = msg instanceof LastHttpContent; } - return; + } catch (IOException | RuntimeException ex) { + exceptionCaught(ctx, ex); } finally { - if (latch == null) { - ReferenceCountUtil.release(msg); - } - } - - if (isHttp2) { - lastRead = msg instanceof Http2DataFrame && ((Http2DataFrame) msg).isEndStream(); - } else { - lastRead = msg instanceof LastHttpContent; - } - - if (latch != null) { - ctx.fireChannelRead(msg); + ReferenceCountUtil.release(msg); } } @Override public void channelReadComplete(ChannelHandlerContext ctx) { + ctx.fireChannelReadComplete(); if (lastRead) { signalComplete(ctx, false); } - ctx.fireChannelReadComplete(); } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; - signalComplete(ctx, true); ctx.fireExceptionCaught(cause); + signalComplete(ctx, true); } Throwable channelException() { @@ -168,10 +157,6 @@ private void signalComplete(ChannelHandlerContext ctx, boolean forceClose) { } private void cleanup(ChannelHandlerContext ctx) { - if (latch != null && latch.getCount() == 0) { - return; - } - Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); if (cleanupHandler != null) { cleanupHandler.cleanup(ctx, true); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index db077d396932..0fff754fc037 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -68,7 +68,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { @Override public void channelInactive(ChannelHandlerContext ctx) { ctx.fireChannelInactive(); - cleanup(ctx, true); + ctx.channel().eventLoop().execute(() -> cleanup(ctx, true)); } public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index b5e9234e302f..afa549e085e7 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -171,6 +171,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { // Reading hasn't started yet. if (!started) { + ctx.read(); ctx.fireChannelReadComplete(); return; } From 720b3953e95a9938acec54aea21902b1cd7f4699 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Tue, 8 Jul 2025 13:50:50 +0300 Subject: [PATCH 35/57] disable connection pooling functionality --- sdk/clientcore/http-netty4/pom.xml | 6 + .../http-netty4/spotbugs-exclude.xml | 7 + .../http/netty4/NettyHttpClient.java | 295 +++++++++++++---- .../http/netty4/NettyHttpClientBuilder.java | 26 +- .../Netty4ChannelBinaryData.java | 5 +- .../implementation/Netty4ConnectionPool.java | 54 +--- .../implementation/Netty4HandlerNames.java | 2 + .../Netty4PipelineCleanupHandler.java | 2 +- .../netty4/implementation/Netty4Utility.java | 40 +++ .../netty4/NettyHttpClientBuilderTests.java | 32 ++ .../http/netty4/NettyHttpClientTests.java | 114 +++++++ .../Netty4ConnectionPoolTests.java | 95 ++++++ .../Netty4Http11ChannelBinaryDataTests.java | 207 +++++++++++- .../Netty4PipelineCleanupHandlerTests.java | 301 ++++++++++++++++++ 14 files changed, 1065 insertions(+), 121 deletions(-) create mode 100644 sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java diff --git a/sdk/clientcore/http-netty4/pom.xml b/sdk/clientcore/http-netty4/pom.xml index c1af06050748..5e3d722bcbf6 100644 --- a/sdk/clientcore/http-netty4/pom.xml +++ b/sdk/clientcore/http-netty4/pom.xml @@ -205,6 +205,12 @@ 2.5.2 test + + org.mockito + mockito-core + 4.11.0 + test + diff --git a/sdk/clientcore/http-netty4/spotbugs-exclude.xml b/sdk/clientcore/http-netty4/spotbugs-exclude.xml index 77b83264b6e6..b29b42c590e0 100644 --- a/sdk/clientcore/http-netty4/spotbugs-exclude.xml +++ b/sdk/clientcore/http-netty4/spotbugs-exclude.xml @@ -66,4 +66,11 @@ + + + + + + + diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index 90c800ed4bed..83d0de24fbc4 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -14,6 +14,7 @@ import io.clientcore.core.models.CoreException; import io.clientcore.core.models.ServerSentResult; import io.clientcore.core.models.binarydata.BinaryData; +import io.clientcore.core.utils.AuthenticateChallenge; import io.clientcore.core.utils.CoreUtils; import io.clientcore.core.utils.ProgressReporter; import io.clientcore.core.utils.ServerSentEventUtils; @@ -25,22 +26,34 @@ import io.clientcore.http.netty4.implementation.Netty4PipelineCleanupHandler; import io.clientcore.http.netty4.implementation.Netty4ProgressAndTimeoutHandler; import io.clientcore.http.netty4.implementation.Netty4ResponseHandler; +import io.clientcore.http.netty4.implementation.Netty4SslInitializationHandler; +import io.clientcore.http.netty4.implementation.ResponseBodyHandling; import io.clientcore.http.netty4.implementation.ResponseStateInfo; import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.Unpooled; import io.netty.channel.Channel; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; +import io.netty.handler.proxy.ProxyHandler; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslContextBuilder; +import io.netty.handler.ssl.SslHandler; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; +import javax.net.ssl.SSLException; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; import java.net.URI; import java.nio.channels.ClosedChannelException; +import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import static io.clientcore.core.utils.ServerSentEventUtils.attemptRetry; import static io.clientcore.core.utils.ServerSentEventUtils.processTextEventStream; @@ -49,8 +62,11 @@ import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_RESPONSE; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PIPELINE_CLEANUP; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL_INITIALIZER; import static io.clientcore.http.netty4.implementation.Netty4Utility.awaitLatch; +import static io.clientcore.http.netty4.implementation.Netty4Utility.buildSslContext; import static io.clientcore.http.netty4.implementation.Netty4Utility.createCodec; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp2Request; @@ -76,24 +92,36 @@ class NettyHttpClient implements HttpClient { private final long responseTimeoutMillis; private final long writeTimeoutMillis; - NettyHttpClient(EventLoopGroup eventLoopGroup, Netty4ConnectionPool connectionPool, ProxyOptions proxyOptions, - ChannelInitializationProxyHandler channelInitializationProxyHandler, long readTimeoutMillis, + private final Bootstrap bootstrap; + private final Consumer sslContextModifier; + private final HttpProtocolVersion maximumHttpVersion; + + NettyHttpClient(Bootstrap bootstrap, EventLoopGroup eventLoopGroup, Netty4ConnectionPool connectionPool, + ProxyOptions proxyOptions, ChannelInitializationProxyHandler channelInitializationProxyHandler, + Consumer sslContextModifier, HttpProtocolVersion maximumHttpVersion, long readTimeoutMillis, long responseTimeoutMillis, long writeTimeoutMillis) { + this.bootstrap = bootstrap; this.eventLoopGroup = eventLoopGroup; this.connectionPool = connectionPool; this.proxyOptions = proxyOptions; this.channelInitializationProxyHandler = channelInitializationProxyHandler; + this.sslContextModifier = sslContextModifier; + this.maximumHttpVersion = maximumHttpVersion; this.readTimeoutMillis = readTimeoutMillis; this.responseTimeoutMillis = responseTimeoutMillis; this.writeTimeoutMillis = writeTimeoutMillis; } Bootstrap getBootstrap() { - return connectionPool.getBootstrap(); + return connectionPool != null ? connectionPool.getBootstrap() : bootstrap; } @Override public Response send(HttpRequest request) { + return connectionPool != null ? sendWithConnectionPool(request) : sendWithoutConnectionPool(request); + } + + private Response sendWithConnectionPool(HttpRequest request) { final URI uri = request.getUri(); final boolean isHttps = "https".equalsIgnoreCase(uri.getScheme()); final int port = uri.getPort() == -1 ? (isHttps ? 443 : 80) : uri.getPort(); @@ -117,7 +145,7 @@ public Response send(HttpRequest request) { Channel channel = future.getNow(); try { - configureRequestPipeline(channel, request, responseReference, errorReference, latch, isHttps); + configurePooledRequestPipeline(channel, request, responseReference, errorReference, latch, isHttps); } catch (Exception e) { // An exception occurred during the pipeline setup. // We fire the exception through the pipeline to trigger the cleanup handler, @@ -141,78 +169,145 @@ public Response send(HttpRequest request) { throw LOGGER.throwableAtError().log(errorReference.get(), CoreException::from); } - //Response response; - //Channel channelToRelease; - BinaryData body; - if (info.isChannelConsumptionComplete()) { - ByteArrayOutputStream eagerContent = info.getEagerContent(); + return createResponse(request, info); + } - body = (eagerContent == null || eagerContent.size() == 0) - ? BinaryData.empty() - : BinaryData.fromBytes(eagerContent.toByteArray()); + private Response sendWithoutConnectionPool(HttpRequest request) { + URI uri = request.getUri(); + String host = uri.getHost(); + int port = uri.getPort() == -1 ? ("https".equalsIgnoreCase(uri.getScheme()) ? 443 : 80) : uri.getPort(); + boolean isHttps = "https".equalsIgnoreCase(uri.getScheme()); + ProgressReporter progressReporter = (request.getContext() == null) + ? null + : (ProgressReporter) request.getContext().getMetadata("progressReporter"); + boolean addProgressAndTimeoutHandler + = progressReporter != null || writeTimeoutMillis > 0 || responseTimeoutMillis > 0 || readTimeoutMillis > 0; - // Since the response is complete, we are responsible for cleaning up the channel. - Channel channelToRelease = info.getResponseChannel(); - channelToRelease.eventLoop().execute(() -> { - Netty4PipelineCleanupHandler cleanupHandler - = channelToRelease.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(channelToRelease.pipeline().context(cleanupHandler), false); + AtomicReference responseReference = new AtomicReference<>(); + AtomicReference errorReference = new AtomicReference<>(); + AtomicReference> proxyChallenges = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + + // Configure an immutable ChannelInitializer in the builder with everything that can be added on a non-per + // request basis. + bootstrap.handler(new ChannelInitializer() { + @Override + protected void initChannel(Channel channel) throws SSLException { + // Test whether proxying should be applied to this Channel. If so, add it. + boolean hasProxy = channelInitializationProxyHandler.test(channel.remoteAddress()); + if (hasProxy) { + ProxyHandler proxyHandler = channelInitializationProxyHandler.createProxy(proxyChallenges); + proxyHandler.connectFuture().addListener(future -> { + if (!future.isSuccess()) { + setOrSuppressError(errorReference, future.cause()); + } + }); + + channel.pipeline().addFirst(PROXY, proxyHandler); } - }); - } else { - // For all other cases, create a streaming response body. - // This delegates all body consumption and cleanup logic to Netty4ChannelBinaryData. - String contentLength = info.getHeaders().getValue(HttpHeaderName.CONTENT_LENGTH); - Long length = null; - if (!CoreUtils.isNullOrEmpty(contentLength)) { - try { - length = Long.parseLong(contentLength); - } catch (NumberFormatException ignored) { - // Ignore, we'll just read until the channel is closed. + + // Add SSL handling if the request is HTTPS. + if (isHttps) { + SslContext ssl = buildSslContext(maximumHttpVersion, sslContextModifier); + // SSL handling is added last here. This is done as proxying could require SSL handling too. + channel.pipeline().addLast(SSL, ssl.newHandler(channel.alloc(), host, port)); + channel.pipeline().addLast(SSL_INITIALIZER, new Netty4SslInitializationHandler()); + + channel.pipeline() + .addLast(ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, latch)); } } - body = new Netty4ChannelBinaryData(info.getEagerContent(), info.getResponseChannel(), length, - info.isHttp2()); - } + }); - Response response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); + bootstrap.connect(host, port).addListener((ChannelFutureListener) connectListener -> { + if (!connectListener.isSuccess()) { + LOGGER.atError().setThrowable(connectListener.cause()).log("Failed connection."); + errorReference.set(connectListener.cause()); + connectListener.channel().close(); + latch.countDown(); + return; + } - if (response.getValue() != BinaryData.empty() - && ServerSentEventUtils - .isTextEventStreamContentType(response.getHeaders().getValue(HttpHeaderName.CONTENT_TYPE))) { - ServerSentEventListener listener = request.getServerSentEventListener(); + Channel channel = connectListener.channel(); + channel.closeFuture().addListener(closeListener -> { + if (!closeListener.isSuccess()) { + LOGGER.atError().setThrowable(closeListener.cause()).log("Channel closed with error"); + setOrSuppressError(errorReference, closeListener.cause()); + } + }); - if (listener != null) { - try { - ServerSentResult serverSentResult - = processTextEventStream(response.getValue().toStream(), listener); + // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is + // applying timeouts. + // This is done to keep the ChannelPipeline shorter, therefore more performant, if this would + // effectively be a no-op. + if (addProgressAndTimeoutHandler) { + channel.pipeline() + .addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, + writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); + } - if (serverSentResult.getException() != null) { - // If an exception occurred while processing the text event stream, emit listener onError. - listener.onError(serverSentResult.getException()); - } + Throwable earlyError = errorReference.get(); + if (earlyError != null) { + // If an error occurred between the connect and the request being sent, don't proceed with sending + // the request. + latch.countDown(); + return; + } - // If an error occurred or we want to reconnect - if (!Thread.currentThread().isInterrupted() && attemptRetry(serverSentResult, request)) { - response.close(); - return this.send(request); + // What I basically want here is the following logic in Netty: + // 1. If a proxy exists, it should be added first. When the connection is activated, we should connect + // to the proxy (with or without SSL). If there is no proxy, skip this step. + // 2. Once step 1 is complete, we should wait until the SSL handshake is complete (if applicable). + // If SSL isn't being used, skip this step. + // 3. Once step 2 is complete, we should send the request. + // + // None of the steps should block the event loop, so we need to use listeners to ensure that the next + // step is only executed once the previous step is complete. + SslHandler sslHandler = channel.pipeline().get(SslHandler.class); + if (sslHandler != null) { + // If the SslHandler is present, trigger the SSL handshake to complete before sending the request. + sslHandler.handshakeFuture().addListener(handshakeListener -> { + if (!handshakeListener.isSuccess()) { + LOGGER.atError().setThrowable(handshakeListener.cause()).log("Failed SSL handshake."); + errorReference.set(handshakeListener.cause()); + latch.countDown(); } - - response = new Response<>(response.getRequest(), response.getStatusCode(), response.getHeaders(), - createBodyFromServerSentResult(serverSentResult)); - } catch (IOException ex) { - throw LOGGER.throwableAtError().log(ex, CoreException::from); - } + }); + channel.write(Unpooled.EMPTY_BUFFER); } else { - response.close(); - throw LOGGER.throwableAtError().log(NO_LISTENER_ERROR_MESSAGE, IllegalStateException::new); + // If there isn't an SslHandler, we can send the request immediately. + // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. + Netty4ResponseHandler responseHandler + = new Netty4ResponseHandler(request, responseReference, errorReference, latch); + channel.pipeline().addLast(HTTP_RESPONSE, responseHandler); + + String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; + channel.pipeline().addBefore(addBefore, HTTP_CODEC, createCodec()); + + sendHttp11Request(request, channel, errorReference) + .addListener((ChannelFutureListener) sendListener -> { + if (!sendListener.isSuccess()) { + setOrSuppressError(errorReference, sendListener.cause()); + sendListener.channel().close(); + latch.countDown(); + } else { + sendListener.channel().read(); + } + }); } + }); + + awaitLatch(latch); + + ResponseStateInfo info = responseReference.get(); + if (info == null) { + throw LOGGER.throwableAtError().log(errorReference.get(), CoreException::from); } - return response; + + return createResponse(request, info); } - private void configureRequestPipeline(Channel channel, HttpRequest request, + private void configurePooledRequestPipeline(Channel channel, HttpRequest request, AtomicReference responseReference, AtomicReference errorReference, CountDownLatch latch, boolean isHttps) { @@ -233,11 +328,6 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, ChannelPipeline pipeline = channel.pipeline(); - // The first handler added is the cleanup handler. It will be the last to execute - // in the outbound direction and the first in the inbound direction, but its main - // purpose is to clean up all other request-specific handlers and release the channel. - pipeline.addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); - // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is // applying timeouts. // This is done to keep the ChannelPipeline shorter, therefore more performant if this would @@ -283,6 +373,8 @@ private void configureRequestPipeline(Channel channel, HttpRequest request, pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); send(request, channel, errorReference, latch); } + + pipeline.addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); } private void send(HttpRequest request, Channel channel, AtomicReference errorReference, @@ -298,6 +390,79 @@ private void send(HttpRequest request, Channel channel, AtomicReference createResponse(HttpRequest request, ResponseStateInfo info) { + BinaryData body; + Response response; + + if (info.isChannelConsumptionComplete()) { + ByteArrayOutputStream eagerContent = info.getEagerContent(); + + body = (info.getResponseBodyHandling() != ResponseBodyHandling.IGNORE + && eagerContent != null + && eagerContent.size() > 0) ? BinaryData.fromBytes(eagerContent.toByteArray()) : BinaryData.empty(); + + Channel channelToCleanup = info.getResponseChannel(); + channelToCleanup.eventLoop().execute(() -> { + Netty4PipelineCleanupHandler cleanupHandler + = channelToCleanup.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(channelToCleanup.pipeline().context(cleanupHandler), false); + } else { + channelToCleanup.close(); + } + }); + } else { + // For all other cases, create a streaming response body. + // This delegates all body consumption and cleanup logic to Netty4ChannelBinaryData. + String contentLength = info.getHeaders().getValue(HttpHeaderName.CONTENT_LENGTH); + Long length = null; + if (!CoreUtils.isNullOrEmpty(contentLength)) { + try { + length = Long.parseLong(contentLength); + } catch (NumberFormatException ignored) { + // Ignore, we'll just read until the channel is closed. + } + } + body = new Netty4ChannelBinaryData(info.getEagerContent(), info.getResponseChannel(), length, + info.isHttp2()); + } + + response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); + + if (response.getValue() != BinaryData.empty() + && ServerSentEventUtils + .isTextEventStreamContentType(response.getHeaders().getValue(HttpHeaderName.CONTENT_TYPE))) { + ServerSentEventListener listener = request.getServerSentEventListener(); + + if (listener != null) { + try { + ServerSentResult serverSentResult + = processTextEventStream(response.getValue().toStream(), listener); + + if (serverSentResult.getException() != null) { + // If an exception occurred while processing the text event stream, emit listener onError. + listener.onError(serverSentResult.getException()); + } + + // If an error occurred or we want to reconnect + if (!Thread.currentThread().isInterrupted() && attemptRetry(serverSentResult, request)) { + response.close(); + return this.send(request); + } + + response = new Response<>(response.getRequest(), response.getStatusCode(), response.getHeaders(), + createBodyFromServerSentResult(serverSentResult)); + } catch (IOException ex) { + throw LOGGER.throwableAtError().log(ex, CoreException::from); + } + } else { + response.close(); + throw LOGGER.throwableAtError().log(NO_LISTENER_ERROR_MESSAGE, IllegalStateException::new); + } + } + return response; + } + public void close() { if (connectionPool != null) { try { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 08c2a42b760f..0bd6725ce259 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -293,15 +293,14 @@ public NettyHttpClientBuilder maximumHttpVersion(HttpProtocolVersion httpVersion * Sets the maximum number of connections allowed per remote address in the connection pool. *

* If not set, a default value of 500 is used. + *

+ * A value of {@code 0} or less will disable connection pooling, and hence each request will + * get a newly created connection. * - * @param connectionPoolSize The maximum number of connections. Must be greater than or equal to 0. + * @param connectionPoolSize The maximum number of connections. * @return The updated builder. */ public NettyHttpClientBuilder connectionPoolSize(int connectionPoolSize) { - if (connectionPoolSize < 0) { - throw LOGGER.throwableAtError() - .log("connectionPoolSize must be greater than or equal to 0", IllegalArgumentException::new); - } this.connectionPoolSize = connectionPoolSize; return this; } @@ -404,14 +403,17 @@ public HttpClient build() { ProxyOptions buildProxyOptions = (proxyOptions == null) ? ProxyOptions.fromConfiguration(buildConfiguration, true) : proxyOptions; - Netty4ConnectionPool connectionPool - = new Netty4ConnectionPool(bootstrap, new ChannelInitializationProxyHandler(buildProxyOptions), - sslContextModifier, connectionPoolSize, connectionIdleTimeout, maxConnectionLifetime, - pendingAcquireTimeout, maxPendingAcquires, maximumHttpVersion); + Netty4ConnectionPool connectionPool = null; + if (connectionPoolSize > 0) { + connectionPool + = new Netty4ConnectionPool(bootstrap, new ChannelInitializationProxyHandler(buildProxyOptions), + sslContextModifier, connectionPoolSize, connectionIdleTimeout, maxConnectionLifetime, + pendingAcquireTimeout, maxPendingAcquires, maximumHttpVersion); + } - return new NettyHttpClient(group, connectionPool, buildProxyOptions, - new ChannelInitializationProxyHandler(buildProxyOptions), getTimeoutMillis(readTimeout), - getTimeoutMillis(responseTimeout), getTimeoutMillis(writeTimeout)); + return new NettyHttpClient(bootstrap, group, connectionPool, buildProxyOptions, + new ChannelInitializationProxyHandler(buildProxyOptions), sslContextModifier, maximumHttpVersion, + getTimeoutMillis(readTimeout), getTimeoutMillis(responseTimeout), getTimeoutMillis(writeTimeout)); } static long getTimeoutMillis(Duration duration) { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index ada45891a3de..fcdc6c90917e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -154,6 +154,10 @@ public void close() { // We need to drain it to ensure the connection can be safely reused. if (!streamDrained.get()) { drainAndCleanupAsync(); + } else { + eagerContent = null; + channel.disconnect(); + channel.close(); } } } @@ -241,7 +245,6 @@ private void drainStream() { bytes = eagerContent.toByteArray(); } } finally { - closed.set(true); drainLatch.countDown(); eagerContent = null; } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 155ce485e563..27aac6d7686d 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -15,14 +15,9 @@ import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; -import io.netty.handler.codec.http2.Http2SecurityUtil; import io.netty.handler.proxy.ProxyHandler; -import io.netty.handler.ssl.ApplicationProtocolConfig; -import io.netty.handler.ssl.ApplicationProtocolNames; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslContextBuilder; -import io.netty.handler.ssl.SslProvider; -import io.netty.handler.ssl.SupportedCipherSuiteFilter; import io.netty.util.AttributeKey; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; @@ -48,14 +43,16 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.CONNECTION_SETUP_ERROR; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL_INITIALIZER; +import static io.clientcore.http.netty4.implementation.Netty4Utility.buildSslContext; /** * A pool of Netty channels that can be reused for requests to the same remote address. */ -public final class Netty4ConnectionPool implements Closeable { +public class Netty4ConnectionPool implements Closeable { private static final AttributeKey POOLED_CONNECTION_KEY = AttributeKey.valueOf("pooled-connection-key"); @@ -373,7 +370,9 @@ public void initChannel(Channel channel) throws SSLException { ProxyHandler proxyHandler = channelInitializationProxyHandler.createProxy(proxyChallenges); pipeline.addFirst(PROXY, proxyHandler); - pipeline.addLast("proxy-exception-handler", new ChannelInboundHandlerAdapter() { + // This handler's only job is to catch any exception during the + // connection setup (proxying, SSL handshake) and fail the promise. + pipeline.addLast(CONNECTION_SETUP_ERROR, new ChannelInboundHandlerAdapter() { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { promise.tryFailure(cause); @@ -385,7 +384,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { // Add SSL handling if the request is HTTPS. if (isHttps) { InetSocketAddress inetSocketAddress = (InetSocketAddress) key.getFinalDestination(); - SslContext ssl = buildSslContext(); + SslContext ssl = buildSslContext(maximumHttpVersion, sslContextModifier); // SSL handling is added last here. This is done as proxying could require SSL handling too. channel.pipeline() .addLast(SSL, ssl.newHandler(channel.alloc(), inetSocketAddress.getHostString(), @@ -419,11 +418,15 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { return; } ChannelPipeline pipeline = newChannel.pipeline(); - if (pipeline.get("proxy-exception-handler") != null) { - pipeline.remove("proxy-exception-handler"); + if (pipeline.get(CONNECTION_SETUP_ERROR) != null) { + pipeline.remove(CONNECTION_SETUP_ERROR); } promise.setSuccess(newChannel); } else { + ChannelPipeline pipeline = newChannel.pipeline(); + if (pipeline.get(CONNECTION_SETUP_ERROR) != null) { + pipeline.remove(CONNECTION_SETUP_ERROR); + } promise.setFailure(proxyFuture.cause()); newChannel.close(); activeConnections.decrementAndGet(); @@ -437,37 +440,6 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { return promise; } - private SslContext buildSslContext() throws SSLException { - SslContextBuilder sslContextBuilder - = SslContextBuilder.forClient().endpointIdentificationAlgorithm("HTTPS"); - if (maximumHttpVersion == HttpProtocolVersion.HTTP_2) { - // If HTTP/2 is the maximum version, we need to ensure that ALPN is enabled. - SslProvider sslProvider = SslContext.defaultClientProvider(); - ApplicationProtocolConfig.SelectorFailureBehavior selectorBehavior; - ApplicationProtocolConfig.SelectedListenerFailureBehavior selectedBehavior; - if (sslProvider == SslProvider.JDK) { - selectorBehavior = ApplicationProtocolConfig.SelectorFailureBehavior.FATAL_ALERT; - selectedBehavior = ApplicationProtocolConfig.SelectedListenerFailureBehavior.FATAL_ALERT; - } else { - // Netty OpenSslContext doesn't support FATAL_ALERT, use NO_ADVERTISE and ACCEPT - // instead. - selectorBehavior = ApplicationProtocolConfig.SelectorFailureBehavior.NO_ADVERTISE; - selectedBehavior = ApplicationProtocolConfig.SelectedListenerFailureBehavior.ACCEPT; - } - - sslContextBuilder.ciphers(Http2SecurityUtil.CIPHERS, SupportedCipherSuiteFilter.INSTANCE) - .applicationProtocolConfig( - new ApplicationProtocolConfig(ApplicationProtocolConfig.Protocol.ALPN, selectorBehavior, - selectedBehavior, ApplicationProtocolNames.HTTP_2, ApplicationProtocolNames.HTTP_1_1)); - } - if (sslContextModifier != null) { - // Allow the caller to modify the SslContextBuilder before it is built. - sslContextModifier.accept(sslContextBuilder); - } - - return sslContextBuilder.build(); - } - private boolean isHealthy(PooledConnection connection) { if (!connection.isActiveAndWriteable()) { return false; diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java index 8cae50f4a624..2e7975fb37d8 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java @@ -74,6 +74,8 @@ public final class Netty4HandlerNames { */ public static final String PIPELINE_CLEANUP = "clientcore.pipelinecleanup"; + public static final String CONNECTION_SETUP_ERROR = "clientcore.connectionsetuperror"; + private Netty4HandlerNames() { } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 0fff754fc037..ce409d9226de 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -99,7 +99,7 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { pipeline.remove(this); } - if (closeChannel || !ctx.channel().isActive()) { + if (closeChannel || !ctx.channel().isActive() || connectionPool == null) { ctx.channel().close(); } else { connectionPool.release(ctx.channel()); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java index 9fdf17b9d831..771c7092e4f7 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java @@ -37,15 +37,23 @@ import io.netty.handler.codec.http2.DelegatingDecompressorFrameListener; import io.netty.handler.codec.http2.Http2Connection; import io.netty.handler.codec.http2.Http2FrameListener; +import io.netty.handler.codec.http2.Http2SecurityUtil; import io.netty.handler.codec.http2.Http2Settings; import io.netty.handler.codec.http2.HttpToHttp2ConnectionHandlerBuilder; import io.netty.handler.codec.http2.InboundHttp2ToHttpAdapterBuilder; +import io.netty.handler.ssl.ApplicationProtocolConfig; +import io.netty.handler.ssl.ApplicationProtocolNames; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslContextBuilder; +import io.netty.handler.ssl.SslProvider; +import io.netty.handler.ssl.SupportedCipherSuiteFilter; import io.netty.handler.stream.ChunkedInput; import io.netty.handler.stream.ChunkedNioFile; import io.netty.handler.stream.ChunkedStream; import io.netty.handler.stream.ChunkedWriteHandler; import io.netty.util.Version; +import javax.net.ssl.SSLException; import java.io.IOException; import java.io.OutputStream; import java.nio.channels.FileChannel; @@ -58,6 +66,7 @@ import java.util.Objects; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import static io.netty.handler.codec.http.DefaultHttpHeadersFactory.trailersFactory; @@ -554,6 +563,37 @@ private static io.netty.handler.codec.http.HttpRequest toNettyHttpRequest(HttpRe } } + public static SslContext buildSslContext(HttpProtocolVersion maximumHttpVersion, + Consumer sslContextModifier) throws SSLException { + SslContextBuilder sslContextBuilder = SslContextBuilder.forClient().endpointIdentificationAlgorithm("HTTPS"); + if (maximumHttpVersion == HttpProtocolVersion.HTTP_2) { + // If HTTP/2 is the maximum version, we need to ensure that ALPN is enabled. + SslProvider sslProvider = SslContext.defaultClientProvider(); + ApplicationProtocolConfig.SelectorFailureBehavior selectorBehavior; + ApplicationProtocolConfig.SelectedListenerFailureBehavior selectedBehavior; + if (sslProvider == SslProvider.JDK) { + selectorBehavior = ApplicationProtocolConfig.SelectorFailureBehavior.FATAL_ALERT; + selectedBehavior = ApplicationProtocolConfig.SelectedListenerFailureBehavior.FATAL_ALERT; + } else { + // Netty OpenSslContext doesn't support FATAL_ALERT, use NO_ADVERTISE and ACCEPT + // instead. + selectorBehavior = ApplicationProtocolConfig.SelectorFailureBehavior.NO_ADVERTISE; + selectedBehavior = ApplicationProtocolConfig.SelectedListenerFailureBehavior.ACCEPT; + } + + sslContextBuilder.ciphers(Http2SecurityUtil.CIPHERS, SupportedCipherSuiteFilter.INSTANCE) + .applicationProtocolConfig( + new ApplicationProtocolConfig(ApplicationProtocolConfig.Protocol.ALPN, selectorBehavior, + selectedBehavior, ApplicationProtocolNames.HTTP_2, ApplicationProtocolNames.HTTP_1_1)); + } + if (sslContextModifier != null) { + // Allow the caller to modify the SslContextBuilder before it is built. + sslContextModifier.accept(sslContextBuilder); + } + + return sslContextBuilder.build(); + } + private Netty4Utility() { } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientBuilderTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientBuilderTests.java index 1c18c9618f5a..a417ef32f253 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientBuilderTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientBuilderTests.java @@ -3,6 +3,7 @@ package io.clientcore.http.netty4; +import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.http.models.HttpMethod; import io.clientcore.core.http.models.HttpRequest; import io.clientcore.core.http.models.ProxyOptions; @@ -32,6 +33,7 @@ import java.io.IOException; import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodHandles; +import java.lang.reflect.Field; import java.net.InetSocketAddress; import java.time.Duration; import java.util.ArrayList; @@ -47,6 +49,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; /** * Tests {@link NettyHttpClientBuilder}. @@ -372,6 +375,35 @@ public void getEventLoopGroupToUse(Class expected, EventLoopGroup configuredG assertInstanceOf(expected, eventLoopGroup); } + @Test + public void buildNettyClientWithoutConnectionPool() throws NoSuchFieldException, IllegalAccessException { + NettyHttpClient client = (NettyHttpClient) new NettyHttpClientBuilder().connectionPoolSize(0).build(); + + Field connectionPoolField = NettyHttpClient.class.getDeclaredField("connectionPool"); + connectionPoolField.setAccessible(true); + assertNull(connectionPoolField.get(client), "Connection pool should be null when pool size is 0."); + } + + @Test + public void testInvalidMaxPendingAcquires() { + NettyHttpClientBuilder builder = new NettyHttpClientBuilder(); + assertThrows(IllegalArgumentException.class, () -> builder.maxPendingAcquires(0)); + assertThrows(IllegalArgumentException.class, () -> builder.maxPendingAcquires(-1)); + } + + @Test + public void testMaximumHttpVersion() throws NoSuchFieldException, IllegalAccessException { + NettyHttpClientBuilder builder = new NettyHttpClientBuilder(); + + NettyHttpClient clientv1 = (NettyHttpClient) builder.maximumHttpVersion(HttpProtocolVersion.HTTP_1_1).build(); + Field httpVersionField = NettyHttpClient.class.getDeclaredField("maximumHttpVersion"); + httpVersionField.setAccessible(true); + assertEquals(HttpProtocolVersion.HTTP_1_1, httpVersionField.get(clientv1)); + + NettyHttpClient clientv2 = (NettyHttpClient) builder.maximumHttpVersion(null).build(); + assertEquals(HttpProtocolVersion.HTTP_2, httpVersionField.get(clientv2)); + } + private static Stream getEventLoopGroupToUseSupplier() throws ReflectiveOperationException { // Doesn't matter what this is calling, just needs to throw an exception. // This will as it doesn't accept the arguments that it will be called with. diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java index e113a6225d1c..9aff69c0f1f0 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java @@ -4,6 +4,7 @@ package io.clientcore.http.netty4; import io.clientcore.core.http.client.HttpClient; +import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.http.models.HttpHeader; import io.clientcore.core.http.models.HttpHeaderName; import io.clientcore.core.http.models.HttpHeaders; @@ -12,6 +13,7 @@ import io.clientcore.core.http.models.ProxyOptions; import io.clientcore.core.http.models.RequestContext; import io.clientcore.core.http.models.Response; +import io.clientcore.core.http.models.ServerSentEvent; import io.clientcore.core.http.pipeline.HttpPipeline; import io.clientcore.core.http.pipeline.HttpPipelineBuilder; import io.clientcore.core.http.pipeline.HttpPipelinePolicy; @@ -19,6 +21,7 @@ import io.clientcore.core.http.pipeline.HttpRetryPolicy; import io.clientcore.core.models.CoreException; import io.clientcore.core.models.binarydata.BinaryData; +import io.clientcore.core.shared.LocalTestServer; import io.clientcore.core.utils.ProgressReporter; import io.clientcore.http.netty4.implementation.MockProxyServer; import io.clientcore.http.netty4.implementation.Netty4ProgressAndTimeoutHandler; @@ -34,16 +37,21 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import javax.servlet.http.HttpServletResponse; import java.io.ByteArrayOutputStream; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.io.PrintWriter; import java.io.StringWriter; +import java.net.ServerSocket; +import java.net.Socket; import java.net.URI; import java.nio.channels.Channels; import java.nio.channels.ClosedChannelException; import java.nio.channels.WritableByteChannel; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.time.Duration; @@ -51,11 +59,13 @@ import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -363,6 +373,110 @@ public void progressAndTimeoutHandlerNotAdded() throws IOException { } } + @Test + public void sendWithServerSentEvents() throws InterruptedException { + LocalTestServer sseServer = new LocalTestServer(HttpProtocolVersion.HTTP_1_1, false, (req, res, body) -> { + res.setContentType("text/event-stream"); + res.setCharacterEncoding(StandardCharsets.UTF_8.name()); + res.setStatus(HttpServletResponse.SC_OK); + try (PrintWriter writer = res.getWriter()) { + writer.println("id: 1"); + writer.println("event: message"); + writer.println("data: event-1"); + writer.println(); + writer.flush(); + + writer.println("id: 2"); + writer.println("event: message"); + writer.println("data: event-2"); + writer.println(); + writer.flush(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + try { + sseServer.start(); + final CountDownLatch latch = new CountDownLatch(2); + final AtomicReference lastEvent = new AtomicReference<>(); + + HttpClient client = new NettyHttpClientBuilder().build(); + HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(URI.create(sseServer.getUri())); + request.setServerSentEventListener(event -> { + lastEvent.set(event); + latch.countDown(); + }); + + try (Response response = client.send(request)) { + assertEquals(200, response.getStatusCode()); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertNotNull(lastEvent.get()); + assertEquals("2", lastEvent.get().getId()); + assertEquals("message", lastEvent.get().getEvent()); + } + } finally { + sseServer.stop(); + } + } + + @Test + public void sendWithServerSentEventsAndNoListenerThrows() { + LocalTestServer sseServer = new LocalTestServer(HttpProtocolVersion.HTTP_1_1, false, (req, res, body) -> { + res.setContentType("text/event-stream"); + res.setStatus(HttpServletResponse.SC_OK); + try { + res.getWriter().println("data: event-1\n\n"); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + try { + sseServer.start(); + HttpClient client = new NettyHttpClientBuilder().build(); + HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(URI.create(sseServer.getUri())); + + IllegalStateException ex = assertThrows(IllegalStateException.class, () -> client.send(request)); + assertTrue(ex.getMessage().contains("No ServerSentEventListener attached")); + } finally { + sseServer.stop(); + } + } + + @Test + public void malformedContentLengthIsIgnored() throws IOException { + String rawResponse = "HTTP/1.1 200 OK\r\n" + "Content-Type: application/octet-stream\r\n" + + "Content-Length: not-a-number\r\n" + "\r\n"; + + try (ServerSocket serverSocket = new ServerSocket(0)) { + int port = serverSocket.getLocalPort(); + URI url = URI.create("http://localhost:" + port); + + Thread clientThread = new Thread(() -> { + HttpClient client = new NettyHttpClientBuilder().build(); + HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(url); + try (Response response = client.send(request)) { + assertEquals(200, response.getStatusCode()); + TestUtils.assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); + } + }); + clientThread.start(); + + try (Socket socket = serverSocket.accept()) { + OutputStream out = socket.getOutputStream(); + out.write(rawResponse.getBytes(StandardCharsets.UTF_8)); + out.write(SHORT_BODY); + out.flush(); + } + + clientThread.join(10000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + private static Stream requestHeaderSupplier() { return Stream.of(Arguments.of(null, NULL_REPLACEMENT), Arguments.of("", ""), Arguments.of("aValue", "aValue")); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java index 4ebce483b1d4..b65d2b321f56 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java @@ -18,6 +18,7 @@ import org.junit.jupiter.api.Timeout; import java.io.IOException; +import java.lang.reflect.Field; import java.net.InetSocketAddress; import java.net.SocketAddress; import java.time.Duration; @@ -25,11 +26,13 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -72,6 +75,42 @@ private Netty4ConnectionPool createPool(int maxConnections, Duration idleTimeout HttpProtocolVersion.HTTP_1_1); } + @Test + public void releaseNullChannelDoesNotThrow() throws IOException { + try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { + assertDoesNotThrow(() -> pool.release(null)); + } + } + + @Test + public void releaseUnknownChannelClosesChannel() throws IOException { + Channel unknownChannel = new NioSocketChannel(); + eventLoopGroup.register(unknownChannel); + + try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { + pool.release(unknownChannel); + // The pool doesn't know this channel, so it should close it. + unknownChannel.closeFuture().awaitUninterruptibly(); + assertFalse(unknownChannel.isOpen()); + } + } + + @Test + public void releaseToClosedPoolClosesChannel() throws IOException { + Bootstrap realBootstrap = bootstrap.clone().remoteAddress(connectionPoolKey.getConnectionTarget()); + Netty4ConnectionPool pool = new Netty4ConnectionPool(realBootstrap, new ChannelInitializationProxyHandler(null), + null, 1, null, null, null, 1, null); + + Channel channel = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); + assertTrue(channel.isActive()); + + pool.close(); + pool.release(channel); + + channel.closeFuture().awaitUninterruptibly(); + assertFalse(channel.isOpen()); + } + @Test public void testAcquireAndRelease() throws IOException { try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { @@ -83,6 +122,62 @@ public void testAcquireAndRelease() throws IOException { } } + @Test + public void closeIsIdempotent() throws IOException { + Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1); + pool.close(); + assertDoesNotThrow(pool::close); + } + + @Test + public void poolWithNoIdleTimeoutHasNoCleanupTask() + throws IOException, NoSuchFieldException, IllegalAccessException { + try (Netty4ConnectionPool pool = createPool(1, null, null, Duration.ofSeconds(10), 1)) { + Field cleanupTaskField = Netty4ConnectionPool.class.getDeclaredField("cleanupTask"); + cleanupTaskField.setAccessible(true); + assertNull(cleanupTaskField.get(pool)); + } + } + + @Test + public void pendingAcquireQueueIsFull() throws IOException { + try (Netty4ConnectionPool pool = createPool(1, null, null, Duration.ofSeconds(10), 1)) { + Channel channel = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); + assertNotNull(channel); + + Future pendingFuture = pool.acquire(connectionPoolKey, false); + assertFalse(pendingFuture.isDone()); + + Future failedFuture = pool.acquire(connectionPoolKey, false); + assertTrue(failedFuture.isDone()); + assertFalse(failedFuture.isSuccess()); + assertInstanceOf(CoreException.class, failedFuture.cause()); + + pool.release(channel); + pendingFuture.awaitUninterruptibly(); + pool.release(pendingFuture.getNow()); + } + } + + @Test + public void cancelledPendingAcquireIsRemovedFromQueue() throws IOException, InterruptedException { + try (Netty4ConnectionPool pool = createPool(1, null, Duration.ofSeconds(5), Duration.ofSeconds(10), 1)) { + Channel channel1 = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); + + Future pendingFuture = pool.acquire(connectionPoolKey, false); + pendingFuture.cancel(true); + + Thread.sleep(100); + + pool.release(channel1); + + Channel channel2 = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); + assertSame(channel1, channel2); + + pool.release(channel2); + } + } + @Test public void testConnectionIsReusedForSameRemoteAddress() throws IOException { try (Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1)) { diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java index a51af8e6be19..3ef115097911 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java @@ -2,20 +2,36 @@ // Licensed under the MIT License. package io.clientcore.http.netty4.implementation; +import io.clientcore.core.models.CoreException; import io.clientcore.core.models.binarydata.BinaryData; import io.clientcore.core.models.binarydata.ByteArrayBinaryData; +import io.clientcore.core.serialization.ObjectSerializer; import io.clientcore.http.netty4.mocking.MockChannelHandlerContext; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.AbstractChannel; import io.netty.channel.Channel; +import io.netty.channel.ChannelConfig; +import io.netty.channel.ChannelMetadata; +import io.netty.channel.ChannelOutboundBuffer; +import io.netty.channel.ChannelPromise; +import io.netty.channel.DefaultChannelConfig; +import io.netty.channel.DefaultEventLoop; +import io.netty.channel.EventLoop; import io.netty.handler.codec.http.LastHttpContent; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.mockito.Mockito; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; +import java.lang.reflect.Type; +import java.net.SocketAddress; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import static io.clientcore.http.netty4.TestUtils.assertArraysEqual; import static io.clientcore.http.netty4.TestUtils.createChannelWithReadHandling; @@ -25,12 +41,22 @@ import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * Tests {@link Netty4ChannelBinaryData}. */ @Timeout(value = 3, unit = TimeUnit.MINUTES) public class Netty4Http11ChannelBinaryDataTests { + private static final byte[] HELLO_BYTES = "Hello".getBytes(StandardCharsets.UTF_8); + private static final byte[] WORLD_BYTES = " World!".getBytes(StandardCharsets.UTF_8); + private static final byte[] HELLO_WORLD_BYTES = "Hello World!".getBytes(StandardCharsets.UTF_8); + @Test public void toBytesWillThrowIsLengthIsTooLarge() { assertThrows(IllegalStateException.class, @@ -139,7 +165,7 @@ public void channelBinaryDataIsNeverReplayable() { @Test public void channelBinaryDataToReplayableReturnsAByteArrayBinaryData() throws IOException { - byte[] expected = "Hello world!".getBytes(StandardCharsets.UTF_8); + byte[] expected = HELLO_WORLD_BYTES; ByteArrayOutputStream eagerContent = new ByteArrayOutputStream(); eagerContent.write(expected); @@ -154,6 +180,185 @@ public void channelBinaryDataToReplayableReturnsAByteArrayBinaryData() throws IO assertArraysEqual(expected, replayable.toBytes()); } + @Test + public void toStreamReturnsNettyStreamWhenNotDrained() throws IOException { + ByteArrayOutputStream eagerContent = new ByteArrayOutputStream(); + eagerContent.write(HELLO_BYTES); + Channel channel = createChannelWithReadHandling((ignored, ch) -> { + ByteBuf content = Unpooled.wrappedBuffer(WORLD_BYTES); + ch.pipeline().fireChannelRead(content); + ch.pipeline().fireChannelRead(LastHttpContent.EMPTY_LAST_CONTENT); + ch.pipeline().fireChannelReadComplete(); + }); + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(eagerContent, channel, (long) HELLO_WORLD_BYTES.length, false); + + InputStream stream = binaryData.toStream(); + + assertInstanceOf(Netty4ChannelInputStream.class, stream); + + ByteArrayOutputStream result = new ByteArrayOutputStream(); + byte[] buffer = new byte[1024]; + int length; + while ((length = stream.read(buffer)) != -1) { + result.write(buffer, 0, length); + } + assertArraysEqual(HELLO_WORLD_BYTES, result.toByteArray()); + } + + @Test + public void toBytesDrainsFromLiveChannel() throws IOException { + ByteArrayOutputStream eagerContent = new ByteArrayOutputStream(); + eagerContent.write(HELLO_BYTES); + + Channel channel = createChannelWithReadHandling((ignored, ch) -> { + ByteBuf content = Unpooled.wrappedBuffer(WORLD_BYTES); + ch.pipeline().fireChannelRead(content); + ch.pipeline().fireChannelRead(LastHttpContent.EMPTY_LAST_CONTENT); + ch.pipeline().fireChannelReadComplete(); + }); + + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(eagerContent, channel, (long) HELLO_WORLD_BYTES.length, false); + + byte[] result = binaryData.toBytes(); + + assertArraysEqual(HELLO_WORLD_BYTES, result); + assertTrue(channel.config().isAutoRead()); + } + + @Test + public void toBytesThrowsIfChannelErrors() { + IOException testException = new IOException("test error"); + Channel channel + = createChannelWithReadHandling((ignored, ch) -> ch.pipeline().fireExceptionCaught(testException)); + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), channel, 10L, false); + + CoreException exception = assertThrows(CoreException.class, binaryData::toBytes); + assertEquals(testException, exception.getCause()); + } + + @Test + public void closeAfterDrainingDisconnectsChannel() { + TestMockChannel realChannel = new TestMockChannel(); + new DefaultEventLoop().register(realChannel); + Channel spiedChannel = spy(realChannel); + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), spiedChannel, 0L, false); + + binaryData.toBytes(); + binaryData.close(); + + verify(spiedChannel).disconnect(); + verify(spiedChannel).close(); + } + + @Test + public void toObjectThrowsCoreExceptionOnSerializationError() throws IOException { + ObjectSerializer mockSerializer = Mockito.mock(ObjectSerializer.class); + when(mockSerializer.deserializeFromBytes(any(), any(Type.class))) + .thenThrow(new IOException("deserialization failed")); + + TestMockChannel channel = new TestMockChannel(); + new DefaultEventLoop().register(channel); + + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), channel, 0L, false); + + CoreException ex = assertThrows(CoreException.class, () -> binaryData.toObject(String.class, mockSerializer)); + assertInstanceOf(IOException.class, ex.getCause()); + } + + @Test + public void cleanupDoesNothingIfHandlerIsMissing() { + TestMockChannel realChannel = new TestMockChannel(); + new DefaultEventLoop().register(realChannel); + Channel spiedChannel = spy(realChannel); + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), spiedChannel, 0L, false); + + binaryData.toBytes(); + + verify(spiedChannel, never()).close(); + } + + private static class TestMockChannel extends AbstractChannel { + private final AtomicBoolean disconnectCalled = new AtomicBoolean(false); + private final AtomicBoolean closeCalled = new AtomicBoolean(false); + + protected TestMockChannel() { + super(null); + } + + @Override + protected AbstractUnsafe newUnsafe() { + return new AbstractUnsafe() { + @Override + public void connect(SocketAddress remoteAddress, SocketAddress localAddress, ChannelPromise promise) { + promise.setSuccess(); + } + }; + } + + @Override + protected boolean isCompatible(EventLoop loop) { + return true; + } + + @Override + protected SocketAddress localAddress0() { + return null; + } + + @Override + protected SocketAddress remoteAddress0() { + return null; + } + + @Override + protected void doBind(SocketAddress localAddress) { + } + + @Override + protected void doDisconnect() { + disconnectCalled.set(true); + } + + @Override + protected void doClose() { + closeCalled.set(true); + } + + @Override + protected void doBeginRead() { + } + + @Override + protected void doWrite(ChannelOutboundBuffer in) { + } + + @Override + public ChannelConfig config() { + return new DefaultChannelConfig(this); + } + + @Override + public boolean isOpen() { + return true; + } + + @Override + public boolean isActive() { + return true; + } + + @Override + public ChannelMetadata metadata() { + return new ChannelMetadata(false); + } + } + private static Channel channelWithNoData() { return createChannelWithReadHandling((ignored, channel) -> { Netty4EagerConsumeChannelHandler handler = channel.pipeline().get(Netty4EagerConsumeChannelHandler.class); diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java new file mode 100644 index 000000000000..d2953cbb19b8 --- /dev/null +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java @@ -0,0 +1,301 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package io.clientcore.http.netty4.implementation; + +import io.clientcore.core.http.client.HttpProtocolVersion; +import io.netty.channel.AbstractChannel; +import io.netty.channel.ChannelConfig; +import io.netty.channel.ChannelHandlerAdapter; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelMetadata; +import io.netty.channel.ChannelOutboundBuffer; +import io.netty.channel.ChannelPromise; +import io.netty.channel.DefaultChannelConfig; +import io.netty.channel.DefaultEventLoop; +import io.netty.channel.EventLoop; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.net.SocketAddress; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_CODEC; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_RESPONSE; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +/** + * Tests for {@link Netty4PipelineCleanupHandler}. + */ +public class Netty4PipelineCleanupHandlerTests { + + @Mock + private Netty4ConnectionPool connectionPool; + + private TestMockChannel testChannel; + private AtomicReference errorReference; + private CountDownLatch latch; + + @BeforeEach + public void setup() { + MockitoAnnotations.openMocks(this); + testChannel = new TestMockChannel(new MockEventLoop()); + errorReference = new AtomicReference<>(); + latch = new CountDownLatch(1); + } + + @Test + public void cleanupWhenPooledAndActiveReleasesChannel() { + testChannel.setActive(true); + testChannel.pipeline().addLast(HTTP_CODEC, new MockChannelHandler()); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + testChannel.pipeline().addLast(handler); + ChannelHandlerContext ctx = testChannel.pipeline().context(handler); + + handler.cleanup(ctx, false); + + verify(connectionPool).release(testChannel); + assertEquals(0, testChannel.getCloseCallCount()); + assertNull(testChannel.pipeline().get(HTTP_CODEC)); + assertFalse(testChannel.config().isAutoRead()); + } + + @Test + public void cleanupWhenForceCloseClosesChannel() { + testChannel.setActive(true); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + testChannel.pipeline().addLast(handler); + ChannelHandlerContext ctx = testChannel.pipeline().context(handler); + + handler.cleanup(ctx, true); + + assertEquals(1, testChannel.getCloseCallCount()); + verify(connectionPool, never()).release(testChannel); + } + + @Test + public void cleanupWhenNonPooledClosesChannel() { + testChannel.setActive(true); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(null, errorReference, latch); + testChannel.pipeline().addLast(handler); + ChannelHandlerContext ctx = testChannel.pipeline().context(handler); + + handler.cleanup(ctx, false); + + assertEquals(1, testChannel.getCloseCallCount()); + } + + @Test + public void cleanupWhenChannelInactiveClosesChannel() { + testChannel.setActive(false); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + testChannel.pipeline().addLast(handler); + ChannelHandlerContext ctx = testChannel.pipeline().context(handler); + + handler.cleanup(ctx, false); + + assertEquals(1, testChannel.getCloseCallCount()); + verify(connectionPool, never()).release(testChannel); + } + + @Test + public void cleanupWhenHttp2PreservesHttpCodec() { + testChannel.setActive(true); + testChannel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).set(HttpProtocolVersion.HTTP_2); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + populatePipelineWithStandardHandlers(handler); + ChannelHandlerContext ctx = testChannel.pipeline().context(handler); + + handler.cleanup(ctx, false); + + assertNotNull(testChannel.pipeline().get(HTTP_CODEC)); + assertNull(testChannel.pipeline().get(HTTP_RESPONSE)); + verify(connectionPool).release(testChannel); + } + + @Test + public void cleanupIsIdempotent() { + testChannel.setActive(true); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + testChannel.pipeline().addLast(handler); + ChannelHandlerContext ctx = testChannel.pipeline().context(handler); + + handler.cleanup(ctx, true); + handler.cleanup(ctx, true); + + assertEquals(1, testChannel.getCloseCallCount()); + } + + @Test + public void exceptionCaughtSetsErrorAndClosesChannel() { + testChannel.setActive(true); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + testChannel.pipeline().addLast(handler); + ChannelHandlerContext ctx = testChannel.pipeline().context(handler); + Throwable testException = new IOException("Test Exception"); + + handler.exceptionCaught(ctx, testException); + + assertEquals(testException, errorReference.get()); + assertEquals(0, latch.getCount()); + assertEquals(1, testChannel.getCloseCallCount()); + verify(connectionPool, never()).release(testChannel); + } + + @Test + public void exceptionCaughtWithNullsStillClosesChannel() { + testChannel.setActive(true); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, null, null); + testChannel.pipeline().addLast(handler); + ChannelHandlerContext ctx = testChannel.pipeline().context(handler); + Throwable testException = new IOException("Test Exception"); + + handler.exceptionCaught(ctx, testException); + + assertEquals(1, testChannel.getCloseCallCount()); + verify(connectionPool, never()).release(testChannel); + } + + @Test + public void channelInactiveSchedulesAndExecutesCleanup() { + testChannel.setActive(true); + assertTrue(testChannel.isActive()); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + testChannel.pipeline().addLast(handler); + + testChannel.close(); + + assertEquals(1, testChannel.getCloseCallCount(), "close() should be called once."); + verify(connectionPool, never()).release(testChannel); + } + + private void populatePipelineWithStandardHandlers(Netty4PipelineCleanupHandler handler) { + testChannel.pipeline().addLast(PROGRESS_AND_TIMEOUT, new MockChannelHandler()); + testChannel.pipeline().addLast(HTTP_RESPONSE, new MockChannelHandler()); + testChannel.pipeline().addLast(HTTP_CODEC, new MockChannelHandler()); + testChannel.pipeline().addLast(handler); + } + + private static class MockChannelHandler extends ChannelHandlerAdapter { + } + + private static class TestMockChannel extends AbstractChannel { + private static final ChannelMetadata METADATA = new ChannelMetadata(false); + private final ChannelConfig config = new DefaultChannelConfig(this); + private final AtomicInteger closeCallCount = new AtomicInteger(0); + private final EventLoop eventLoop; + + private volatile boolean active; + private volatile boolean open = true; + + protected TestMockChannel(EventLoop eventLoop) { + super(null); + this.eventLoop = eventLoop; + } + + @Override + public EventLoop eventLoop() { + return this.eventLoop; + } + + @Override + public ChannelConfig config() { + return this.config; + } + + @Override + public boolean isOpen() { + return open; + } + + @Override + public boolean isActive() { + return active; + } + + @Override + public ChannelMetadata metadata() { + return METADATA; + } + + @Override + protected AbstractUnsafe newUnsafe() { + return new AbstractUnsafe() { + @Override + public void connect(SocketAddress remoteAddress, SocketAddress localAddress, ChannelPromise promise) { + active = true; + promise.setSuccess(); + } + }; + } + + @Override + protected boolean isCompatible(EventLoop loop) { + return loop == this.eventLoop; + } + + @Override + protected SocketAddress localAddress0() { + return null; + } + + @Override + protected SocketAddress remoteAddress0() { + return null; + } + + @Override + protected void doBind(SocketAddress localAddress) { + } + + @Override + protected void doDisconnect() { + active = false; + } + + @Override + protected void doClose() { + active = false; + open = false; + closeCallCount.incrementAndGet(); + } + + @Override + protected void doBeginRead() { + } + + @Override + protected void doWrite(ChannelOutboundBuffer in) { + } + + public void setActive(boolean isActive) { + this.active = isActive; + } + + public int getCloseCallCount() { + return closeCallCount.get(); + } + } + + private static class MockEventLoop extends DefaultEventLoop { + @Override + public void execute(Runnable task) { + if (task == null) { + throw new NullPointerException("task"); + } + task.run(); + } + } +} From 670687f460803a17813a74af77497599989516fb Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 9 Jul 2025 14:02:13 +0300 Subject: [PATCH 36/57] adjust cleanups --- .../http/netty4/NettyHttpClient.java | 34 ++-- .../http/netty4/NettyHttpClientBuilder.java | 4 +- .../implementation/Netty4AlpnHandler.java | 24 +++ .../Netty4ChannelBinaryData.java | 58 +++--- .../implementation/Netty4ConnectionPool.java | 21 --- .../Netty4EagerConsumeChannelHandler.java | 23 +-- .../implementation/Netty4HandlerNames.java | 2 - .../Netty4PipelineCleanupHandler.java | 4 + .../http/netty4/NettyHttpClientTests.java | 11 ++ ...Netty4EagerConsumeChannelHandlerTests.java | 166 ++++++++++++++++++ .../Netty4Http11ChannelBinaryDataTests.java | 91 +++++++++- 11 files changed, 335 insertions(+), 103 deletions(-) create mode 100644 sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandlerTests.java diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index 83d0de24fbc4..35b7e81474e9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -354,6 +354,8 @@ private void configurePooledRequestPipeline(Channel channel, HttpRequest request pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); } + pipeline.addLast(PIPELINE_CLEANUP, + new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); if (isHttp2) { sendHttp2Request(request, channel, errorReference, latch); } else { @@ -362,7 +364,8 @@ private void configurePooledRequestPipeline(Channel channel, HttpRequest request } else { // This is a new connection, let ALPN do the work. // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. - pipeline.addAfter(SSL, ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, latch)); + pipeline.addAfter(SSL, ALPN, + new Netty4AlpnHandler(request, responseReference, errorReference, latch, connectionPool)); } } else { // If there isn't an SslHandler, we can send the request immediately. @@ -371,10 +374,9 @@ private void configurePooledRequestPipeline(Channel channel, HttpRequest request new Netty4ResponseHandler(request, responseReference, errorReference, latch)); String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); + pipeline.addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); send(request, channel, errorReference, latch); } - - pipeline.addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); } private void send(HttpRequest request, Channel channel, AtomicReference errorReference, @@ -393,6 +395,19 @@ private void send(HttpRequest request, Channel channel, AtomicReference createResponse(HttpRequest request, ResponseStateInfo info) { BinaryData body; Response response; + Channel channelToCleanup = info.getResponseChannel(); + + final Runnable cleanupTask = () -> { + if (connectionPool != null) { + Netty4PipelineCleanupHandler cleanupHandler + = channelToCleanup.pipeline().get(Netty4PipelineCleanupHandler.class); + if (cleanupHandler != null) { + cleanupHandler.cleanup(channelToCleanup.pipeline().context(cleanupHandler), false); + } + } else { + channelToCleanup.close(); + } + }; if (info.isChannelConsumptionComplete()) { ByteArrayOutputStream eagerContent = info.getEagerContent(); @@ -401,16 +416,7 @@ private Response createResponse(HttpRequest request, ResponseStateIn && eagerContent != null && eagerContent.size() > 0) ? BinaryData.fromBytes(eagerContent.toByteArray()) : BinaryData.empty(); - Channel channelToCleanup = info.getResponseChannel(); - channelToCleanup.eventLoop().execute(() -> { - Netty4PipelineCleanupHandler cleanupHandler - = channelToCleanup.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(channelToCleanup.pipeline().context(cleanupHandler), false); - } else { - channelToCleanup.close(); - } - }); + channelToCleanup.eventLoop().execute(cleanupTask); } else { // For all other cases, create a streaming response body. // This delegates all body consumption and cleanup logic to Netty4ChannelBinaryData. @@ -424,7 +430,7 @@ private Response createResponse(HttpRequest request, ResponseStateIn } } body = new Netty4ChannelBinaryData(info.getEagerContent(), info.getResponseChannel(), length, - info.isHttp2()); + info.isHttp2(), cleanupTask); } response = new Response<>(request, info.getStatusCode(), info.getHeaders(), body); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 0bd6725ce259..aa12a285ab26 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -137,7 +137,7 @@ private static Class getChannelClass(String className) private HttpProtocolVersion maximumHttpVersion = HttpProtocolVersion.HTTP_2; // --- Connection Pool Configuration --- - private int connectionPoolSize = 500; + private int connectionPoolSize = 1000; private Duration connectionIdleTimeout = Duration.ofSeconds(60); private Duration maxConnectionLifetime; private Duration pendingAcquireTimeout = Duration.ofSeconds(60); // Default wait time for a connection @@ -292,7 +292,7 @@ public NettyHttpClientBuilder maximumHttpVersion(HttpProtocolVersion httpVersion /** * Sets the maximum number of connections allowed per remote address in the connection pool. *

- * If not set, a default value of 500 is used. + * If not set, a default value of 1000 is used. *

* A value of {@code 0} or less will disable connection pooling, and hence each request will * get a newly created connection. diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index 8dedf4c5fc5e..ab5c65aa2fbf 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -14,6 +14,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PIPELINE_CLEANUP; import static io.clientcore.http.netty4.implementation.Netty4Utility.configureHttpsPipeline; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp2Request; @@ -36,6 +37,25 @@ public final class Netty4AlpnHandler extends ApplicationProtocolNegotiationHandl private final AtomicReference responseReference; private final AtomicReference errorReference; private final CountDownLatch latch; + private final Netty4ConnectionPool connectionPool; + + /** + * Creates a new instance of {@link Netty4AlpnHandler} with a fallback to using HTTP/1.1. + * + * @param request The request to send once ALPN negotiation completes. + * @param errorReference An AtomicReference keeping track of errors during the request lifecycle. + * @param latch A CountDownLatch that will be released once the request completes. + * @param connectionPool The connection pool. + */ + public Netty4AlpnHandler(HttpRequest request, AtomicReference responseReference, + AtomicReference errorReference, CountDownLatch latch, Netty4ConnectionPool connectionPool) { + super(ApplicationProtocolNames.HTTP_1_1); + this.request = request; + this.responseReference = responseReference; + this.errorReference = errorReference; + this.latch = latch; + this.connectionPool = connectionPool; + } /** * Creates a new instance of {@link Netty4AlpnHandler} with a fallback to using HTTP/1.1. @@ -51,6 +71,7 @@ public Netty4AlpnHandler(HttpRequest request, AtomicReference this.responseReference = responseReference; this.errorReference = errorReference; this.latch = latch; + this.connectionPool = null; } @Override @@ -75,6 +96,9 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { configureHttpsPipeline(ctx.pipeline(), request, protocolVersion, responseReference, errorReference, latch); + ctx.pipeline() + .addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); + if (protocolVersion == HttpProtocolVersion.HTTP_2) { sendHttp2Request(request, ctx.channel(), errorReference, latch); } else { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index fcdc6c90917e..8932fe3e327a 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -41,6 +41,7 @@ public final class Netty4ChannelBinaryData extends BinaryData { private final CountDownLatch drainLatch = new CountDownLatch(1); // Manages the "closed" state, ensuring cleanup happens only once. private final AtomicBoolean closed = new AtomicBoolean(false); + private final Runnable onClose; // Non-final to allow nulling out after use. private ByteArrayOutputStream eagerContent; @@ -54,12 +55,23 @@ public final class Netty4ChannelBinaryData extends BinaryData { * @param channel The Netty {@link Channel}. * @param length Size of the response body (if known). * @param isHttp2 Flag indicating whether the handler is used for HTTP/2 or not. + * @param onClose The Runnable to run when the {@code close()} method is triggered. */ + public Netty4ChannelBinaryData(ByteArrayOutputStream eagerContent, Channel channel, Long length, boolean isHttp2, + Runnable onClose) { + this.eagerContent = eagerContent; + this.channel = channel; + this.length = length; + this.isHttp2 = isHttp2; + this.onClose = onClose; + } + public Netty4ChannelBinaryData(ByteArrayOutputStream eagerContent, Channel channel, Long length, boolean isHttp2) { this.eagerContent = eagerContent; this.channel = channel; this.length = length; this.isHttp2 = isHttp2; + this.onClose = null; } @Override @@ -155,9 +167,9 @@ public void close() { if (!streamDrained.get()) { drainAndCleanupAsync(); } else { - eagerContent = null; - channel.disconnect(); - channel.close(); + if (onClose != null) { + onClose.run(); + } } } } @@ -165,13 +177,17 @@ public void close() { private void drainAndCleanupAsync() { if (streamDrained.compareAndSet(false, true)) { if (!channel.isActive()) { - cleanup(true); + if (onClose != null) { + onClose.run(); + } drainLatch.countDown(); return; } Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(() -> { - cleanup(false); + if (onClose != null) { + onClose.run(); + } drainLatch.countDown(); }, isHttp2); @@ -182,44 +198,16 @@ private void drainAndCleanupAsync() { } } - private void cleanup(boolean closeChannel) { - if (channel.eventLoop().inEventLoop()) { - doCleanup(closeChannel); - } else { - try { - channel.eventLoop().submit(() -> doCleanup(closeChannel)).get(); - } catch (InterruptedException | java.util.concurrent.ExecutionException e) { - LOGGER.atWarning() - .setThrowable(e) - .log("Exception during synchronous channel cleanup. Forcing channel close."); - channel.close(); - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - } - } - } - - private void doCleanup(boolean closeChannel) { - // TODO: userTriggeredEvent - Netty4PipelineCleanupHandler cleanupHandler = channel.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(channel.pipeline().context(cleanupHandler), closeChannel); - } - } - private void drainStream() { if (streamDrained.compareAndSet(false, true)) { try { if (length != null && eagerContent != null && eagerContent.size() >= length) { bytes = eagerContent.toByteArray(); - cleanup(false); return; } if (!channel.isActive()) { bytes = (eagerContent == null) ? new byte[0] : eagerContent.toByteArray(); - cleanup(true); return; } @@ -232,7 +220,7 @@ private void drainStream() { channel.config().setAutoRead(true); awaitLatch(ioLatch); - + bytes = eagerContent.toByteArray(); Throwable exception = handler.channelException(); if (exception != null) { @@ -245,8 +233,8 @@ private void drainStream() { bytes = eagerContent.toByteArray(); } } finally { - drainLatch.countDown(); eagerContent = null; + drainLatch.countDown(); } } else { awaitLatch(drainLatch); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 27aac6d7686d..575d1d3e3029 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -10,8 +10,6 @@ import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; @@ -43,7 +41,6 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; -import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.CONNECTION_SETUP_ERROR; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL_INITIALIZER; @@ -369,16 +366,6 @@ public void initChannel(Channel channel) throws SSLException { if (hasProxy) { ProxyHandler proxyHandler = channelInitializationProxyHandler.createProxy(proxyChallenges); pipeline.addFirst(PROXY, proxyHandler); - - // This handler's only job is to catch any exception during the - // connection setup (proxying, SSL handshake) and fail the promise. - pipeline.addLast(CONNECTION_SETUP_ERROR, new ChannelInboundHandlerAdapter() { - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - promise.tryFailure(cause); - ctx.close(); - } - }); } // Add SSL handling if the request is HTTPS. @@ -417,16 +404,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { satisfyWaiterWithNewConnection(); return; } - ChannelPipeline pipeline = newChannel.pipeline(); - if (pipeline.get(CONNECTION_SETUP_ERROR) != null) { - pipeline.remove(CONNECTION_SETUP_ERROR); - } promise.setSuccess(newChannel); } else { - ChannelPipeline pipeline = newChannel.pipeline(); - if (pipeline.get(CONNECTION_SETUP_ERROR) != null) { - pipeline.remove(CONNECTION_SETUP_ERROR); - } promise.setFailure(proxyFuture.cause()); newChannel.close(); activeConnections.decrementAndGet(); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 3cda51e893c9..6d1fb7770a83 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -131,35 +131,18 @@ public void handlerAdded(ChannelHandlerContext ctx) { } private void signalComplete(ChannelHandlerContext ctx, boolean forceClose) { - boolean isSyncDrain = (latch != null); - if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { ctx.pipeline().remove(this); } - if (isSyncDrain) { - Netty4PipelineCleanupHandler mainCleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); - if (mainCleanupHandler != null) { - mainCleanupHandler.cleanup(ctx, forceClose); - } - } - + // If in sync mode (for toBytes()), just signal completion. if (latch != null) { latch.countDown(); } + + // If in async mode (for close()), run the cleanup callback. if (onComplete != null) { onComplete.run(); } - - if (!isSyncDrain && forceClose) { - cleanup(ctx); - } - } - - private void cleanup(ChannelHandlerContext ctx) { - Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(ctx, true); - } } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java index 2e7975fb37d8..8cae50f4a624 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java @@ -74,8 +74,6 @@ public final class Netty4HandlerNames { */ public static final String PIPELINE_CLEANUP = "clientcore.pipelinecleanup"; - public static final String CONNECTION_SETUP_ERROR = "clientcore.connectionsetuperror"; - private Netty4HandlerNames() { } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index ce409d9226de..db29dbde1e89 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -68,6 +68,9 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { @Override public void channelInactive(ChannelHandlerContext ctx) { ctx.fireChannelInactive(); + if (latch != null) { + latch.countDown(); + } ctx.channel().eventLoop().execute(() -> cleanup(ctx, true)); } @@ -76,6 +79,7 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { return; } + // TODO: use userTriggeredEvent // Always reset autoRead to false before returning a channel to the pool // to ensure predictable behavior for the next request. ctx.channel().config().setAutoRead(false); diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java index 9aff69c0f1f0..f304cd091333 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java @@ -477,6 +477,17 @@ public void malformedContentLengthIsIgnored() throws IOException { } } + @Test + public void nonPooledClientSendsRequestSuccessfully() { + HttpClient client = new NettyHttpClientBuilder().connectionPoolSize(0).build(); + + try (Response response + = client.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(SHORT_BODY_PATH)))) { + assertEquals(200, response.getStatusCode()); + assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); + } + } + private static Stream requestHeaderSupplier() { return Stream.of(Arguments.of(null, NULL_REPLACEMENT), Arguments.of("", ""), Arguments.of("aValue", "aValue")); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandlerTests.java new file mode 100644 index 000000000000..3442eef4d529 --- /dev/null +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandlerTests.java @@ -0,0 +1,166 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package io.clientcore.http.netty4.implementation; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.http.DefaultHttpContent; +import io.netty.handler.codec.http.LastHttpContent; +import io.netty.handler.codec.http2.DefaultHttp2DataFrame; +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.channels.ClosedChannelException; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests for {@link Netty4EagerConsumeChannelHandler}. + */ +public class Netty4EagerConsumeChannelHandlerTests { + private static final byte[] HELLO_BYTES = "Hello".getBytes(StandardCharsets.UTF_8); + + @Test + public void syncDrainConsumesHttp1Content() throws InterruptedException { + ByteArrayOutputStream receivedBytes = new ByteArrayOutputStream(); + CountDownLatch latch = new CountDownLatch(1); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, + buf -> buf.readBytes(receivedBytes, buf.readableBytes()), false); + + EmbeddedChannel channel = new EmbeddedChannel(handler); + + channel.writeInbound(new DefaultHttpContent(Unpooled.wrappedBuffer(HELLO_BYTES))); + assertFalse(latch.await(50, TimeUnit.MILLISECONDS), "Latch should not count down on partial content."); + + channel.writeInbound(LastHttpContent.EMPTY_LAST_CONTENT); + assertTrue(latch.await(1, TimeUnit.SECONDS), "Latch should count down on last content."); + + assertArrayEquals(HELLO_BYTES, receivedBytes.toByteArray()); + assertNull(channel.pipeline().get(Netty4EagerConsumeChannelHandler.class)); + } + + @Test + public void syncDrainConsumesHttp2Content() throws InterruptedException { + ByteArrayOutputStream receivedBytes = new ByteArrayOutputStream(); + CountDownLatch latch = new CountDownLatch(1); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, + buf -> buf.readBytes(receivedBytes, buf.readableBytes()), true); + + EmbeddedChannel channel = new EmbeddedChannel(handler); + + channel.writeInbound(new DefaultHttp2DataFrame(Unpooled.wrappedBuffer(HELLO_BYTES), false)); + assertFalse(latch.await(50, TimeUnit.MILLISECONDS), "Latch should not count down on partial content."); + + channel.writeInbound(new DefaultHttp2DataFrame(true)); + assertTrue(latch.await(1, TimeUnit.SECONDS), "Latch should count down on last content."); + + assertArrayEquals(HELLO_BYTES, receivedBytes.toByteArray()); + assertNull(channel.pipeline().get(Netty4EagerConsumeChannelHandler.class)); + } + + @Test + public void asyncDrainCallsOnComplete() { + AtomicBoolean onCompleteCalled = new AtomicBoolean(false); + Runnable onComplete = () -> onCompleteCalled.set(true); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(onComplete, false); + + EmbeddedChannel channel = new EmbeddedChannel(handler); + channel.writeInbound(LastHttpContent.EMPTY_LAST_CONTENT); + + channel.runPendingTasks(); + + assertTrue(onCompleteCalled.get(), "onComplete should have been called."); + assertNull(channel.pipeline().get(Netty4EagerConsumeChannelHandler.class)); + } + + @Test + public void consumerExceptionIsPropagated() { + IOException testException = new IOException("test"); + CountDownLatch latch = new CountDownLatch(1); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, buf -> { + throw testException; + }, false); + + EmbeddedChannel channel = new EmbeddedChannel(handler); + ByteBuf content = Unpooled.wrappedBuffer(HELLO_BYTES); + + IOException thrown = assertThrows(IOException.class, () -> { + channel.writeInbound(content); + channel.checkException(); + }); + + assertEquals(testException, thrown); + + assertNull(channel.pipeline().get(Netty4EagerConsumeChannelHandler.class)); + } + + @Test + public void exceptionCaughtSignalsCompletion() throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, buf -> { + }, false); + + EmbeddedChannel channel = new EmbeddedChannel(handler); + + channel.pipeline().fireExceptionCaught(new RuntimeException("test")); + + assertTrue(latch.await(1, TimeUnit.SECONDS), "Latch should count down on exceptionCaught."); + assertNotNull(handler.channelException()); + } + + @Test + public void channelInactiveSignalsCompletion() throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, buf -> { + }, false); + + EmbeddedChannel channel = new EmbeddedChannel(handler); + assertTrue(channel.isActive()); + + channel.close().awaitUninterruptibly(); + + assertTrue(latch.await(1, TimeUnit.SECONDS), "Latch should count down on channelInactive."); + } + + @Test + public void addingToInactiveChannelFiresException() { + EmbeddedChannel channel = new EmbeddedChannel(); + channel.close().awaitUninterruptibly(); + + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(() -> { + }, false); + + channel.pipeline().addLast(handler); + + assertThrows(ClosedChannelException.class, channel::checkException); + } + + @Test + public void handlesByteBufMessage() { + ByteArrayOutputStream receivedBytes = new ByteArrayOutputStream(); + CountDownLatch latch = new CountDownLatch(1); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, + buf -> buf.readBytes(receivedBytes, buf.readableBytes()), false); + + EmbeddedChannel channel = new EmbeddedChannel(handler); + + channel.writeInbound(Unpooled.wrappedBuffer(HELLO_BYTES)); + channel.finishAndReleaseAll(); + + assertEquals(0, latch.getCount()); + assertArrayEquals(HELLO_BYTES, receivedBytes.toByteArray()); + } +} diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java index 3ef115097911..202172f95eef 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java @@ -18,6 +18,8 @@ import io.netty.channel.DefaultChannelConfig; import io.netty.channel.DefaultEventLoop; import io.netty.channel.EventLoop; +import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.http.DefaultHttpContent; import io.netty.handler.codec.http.LastHttpContent; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -30,11 +32,13 @@ import java.net.SocketAddress; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import static io.clientcore.http.netty4.TestUtils.assertArraysEqual; import static io.clientcore.http.netty4.TestUtils.createChannelWithReadHandling; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -211,19 +215,26 @@ public void toBytesDrainsFromLiveChannel() throws IOException { ByteArrayOutputStream eagerContent = new ByteArrayOutputStream(); eagerContent.write(HELLO_BYTES); - Channel channel = createChannelWithReadHandling((ignored, ch) -> { - ByteBuf content = Unpooled.wrappedBuffer(WORLD_BYTES); - ch.pipeline().fireChannelRead(content); - ch.pipeline().fireChannelRead(LastHttpContent.EMPTY_LAST_CONTENT); - ch.pipeline().fireChannelReadComplete(); - }); + EmbeddedChannel channel = new EmbeddedChannel(); Netty4ChannelBinaryData binaryData - = new Netty4ChannelBinaryData(eagerContent, channel, (long) HELLO_WORLD_BYTES.length, false); + = new Netty4ChannelBinaryData(eagerContent, channel, (long) HELLO_WORLD_BYTES.length, false, null); + + Thread serverThread = new Thread(() -> { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + channel.writeInbound(new DefaultHttpContent(Unpooled.wrappedBuffer(WORLD_BYTES))); + channel.writeInbound(LastHttpContent.EMPTY_LAST_CONTENT); + }); + + serverThread.start(); byte[] result = binaryData.toBytes(); - assertArraysEqual(HELLO_WORLD_BYTES, result); + assertArrayEquals(HELLO_WORLD_BYTES, result); assertTrue(channel.config().isAutoRead()); } @@ -244,8 +255,13 @@ public void closeAfterDrainingDisconnectsChannel() { TestMockChannel realChannel = new TestMockChannel(); new DefaultEventLoop().register(realChannel); Channel spiedChannel = spy(realChannel); + Runnable cleanupTask = () -> { + spiedChannel.disconnect(); + spiedChannel.close(); + }; + Netty4ChannelBinaryData binaryData - = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), spiedChannel, 0L, false); + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), spiedChannel, 0L, false, cleanupTask); binaryData.toBytes(); binaryData.close(); @@ -283,6 +299,63 @@ public void cleanupDoesNothingIfHandlerIsMissing() { verify(spiedChannel, never()).close(); } + @Test + public void toBytesOnInactiveChannelReturnsEagerContent() throws IOException { + byte[] eagerBytes = "eager".getBytes(StandardCharsets.UTF_8); + ByteArrayOutputStream eagerContent = new ByteArrayOutputStream(); + eagerContent.write(eagerBytes); + + TestMockChannel channel = new TestMockChannel(); + new DefaultEventLoop().register(channel); + + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(eagerContent, channel, (long) eagerBytes.length, false); + + channel.close().awaitUninterruptibly(); + byte[] result = binaryData.toBytes(); + + assertArraysEqual(eagerBytes, result); + } + + @Test + public void toBytesUsesEagerContentWhenSufficient() throws IOException { + byte[] fullBody = "Full body".getBytes(StandardCharsets.UTF_8); + ByteArrayOutputStream eagerContent = new ByteArrayOutputStream(); + eagerContent.write(fullBody); + + TestMockChannel realChannel = new TestMockChannel(); + new DefaultEventLoop().register(realChannel); + Channel spiedChannel = spy(realChannel); + + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(eagerContent, spiedChannel, (long) fullBody.length, false); + + byte[] result = binaryData.toBytes(); + + assertArraysEqual(fullBody, result); + verify(spiedChannel, never()).read(); + verify(spiedChannel, never()).config(); + } + + @Test + public void closeBeforeDrainingEventuallyCleansUp() throws InterruptedException { + EmbeddedChannel channel = new EmbeddedChannel(); + assertTrue(channel.isActive()); + + CountDownLatch cleanupLatch = new CountDownLatch(1); + Runnable cleanupTask = cleanupLatch::countDown; + + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), channel, 1L, false, cleanupTask); + + binaryData.close(); + + channel.close().awaitUninterruptibly(); + + assertTrue(cleanupLatch.await(10, TimeUnit.SECONDS), + "Cleanup task was not called after the channel became inactive."); + } + private static class TestMockChannel extends AbstractChannel { private final AtomicBoolean disconnectCalled = new AtomicBoolean(false); private final AtomicBoolean closeCalled = new AtomicBoolean(false); From c0022f4c29cc12b9f023f3dea7e94ba76081ed99 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 9 Jul 2025 19:14:22 +0300 Subject: [PATCH 37/57] use event for pipeline cleanup --- .../io/clientcore/http/netty4/NettyHttpClient.java | 7 ++----- .../Netty4InitiateOneReadHandler.java | 13 ------------- .../implementation/Netty4PipelineCleanupEvent.java | 12 ++++++++++++ .../Netty4PipelineCleanupHandler.java | 10 +++++++++- 4 files changed, 23 insertions(+), 19 deletions(-) create mode 100644 sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupEvent.java diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index 35b7e81474e9..e159e0078c9d 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -23,6 +23,7 @@ import io.clientcore.http.netty4.implementation.Netty4ChannelBinaryData; import io.clientcore.http.netty4.implementation.Netty4ConnectionPool; import io.clientcore.http.netty4.implementation.Netty4ConnectionPoolKey; +import io.clientcore.http.netty4.implementation.Netty4PipelineCleanupEvent; import io.clientcore.http.netty4.implementation.Netty4PipelineCleanupHandler; import io.clientcore.http.netty4.implementation.Netty4ProgressAndTimeoutHandler; import io.clientcore.http.netty4.implementation.Netty4ResponseHandler; @@ -399,11 +400,7 @@ private Response createResponse(HttpRequest request, ResponseStateIn final Runnable cleanupTask = () -> { if (connectionPool != null) { - Netty4PipelineCleanupHandler cleanupHandler - = channelToCleanup.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(channelToCleanup.pipeline().context(cleanupHandler), false); - } + channelToCleanup.pipeline().fireUserEventTriggered(Netty4PipelineCleanupEvent.CLEANUP_PIPELINE); } else { channelToCleanup.close(); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java index 8042c42c72a7..ccfc65214104 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java @@ -105,7 +105,6 @@ boolean isChannelConsumed() { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; - performCleanup(ctx); latch.countDown(); ctx.fireExceptionCaught(cause); } @@ -117,14 +116,12 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - performCleanup(ctx); latch.countDown(); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - performCleanup(ctx); latch.countDown(); ctx.fireChannelInactive(); } @@ -139,14 +136,4 @@ public void handlerAdded(ChannelHandlerContext ctx) { } } - private void performCleanup(ChannelHandlerContext ctx) { - if (latch.getCount() == 0) { - return; - } - - Netty4PipelineCleanupHandler cleanupHandler = ctx.pipeline().get(Netty4PipelineCleanupHandler.class); - if (cleanupHandler != null) { - cleanupHandler.cleanup(ctx, true); - } - } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupEvent.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupEvent.java new file mode 100644 index 000000000000..b72f0b60fe5b --- /dev/null +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupEvent.java @@ -0,0 +1,12 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package io.clientcore.http.netty4.implementation; + +public enum Netty4PipelineCleanupEvent { + + /** + * Event indicating that the channel will be released back to the connection pool. + */ + CLEANUP_PIPELINE +} diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index db29dbde1e89..1921333944c6 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -74,12 +74,20 @@ public void channelInactive(ChannelHandlerContext ctx) { ctx.channel().eventLoop().execute(() -> cleanup(ctx, true)); } + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) { + if (evt instanceof Netty4PipelineCleanupEvent) { + cleanup(ctx, false); + return; + } + ctx.fireUserEventTriggered(evt); + } + public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { if (!cleanedUp.compareAndSet(false, true)) { return; } - // TODO: use userTriggeredEvent // Always reset autoRead to false before returning a channel to the pool // to ensure predictable behavior for the next request. ctx.channel().config().setAutoRead(false); From 7b3c8feff9e0f0e1758244d34ec3360a2fb739d3 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Thu, 10 Jul 2025 11:25:58 +0300 Subject: [PATCH 38/57] attempt to synchronize the pipeline modification and cleaup for the same channel to prevent race conditions --- .../http/netty4/NettyHttpClient.java | 133 ++++++++++-------- .../implementation/Netty4ConnectionPool.java | 13 ++ .../Netty4PipelineCleanupEvent.java | 2 +- .../Netty4PipelineCleanupHandler.java | 38 +++-- .../implementation/Netty4ResponseHandler.java | 11 +- .../Netty4PipelineCleanupHandlerTests.java | 3 + 6 files changed, 118 insertions(+), 82 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index e159e0078c9d..a94646a82c6f 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -54,6 +54,7 @@ import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import static io.clientcore.core.utils.ServerSentEventUtils.attemptRetry; @@ -144,7 +145,7 @@ private Response sendWithConnectionPool(HttpRequest request) { return; } - Channel channel = future.getNow(); + final Channel channel = future.getNow(); try { configurePooledRequestPipeline(channel, request, responseReference, errorReference, latch, isHttps); } catch (Exception e) { @@ -239,8 +240,8 @@ protected void initChannel(Channel channel) throws SSLException { // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is // applying timeouts. - // This is done to keep the ChannelPipeline shorter, therefore more performant, if this would - // effectively be a no-op. + // This is done to keep the ChannelPipeline shorter, therefore more performant if this + // effectively is a no-op. if (addProgressAndTimeoutHandler) { channel.pipeline() .addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, @@ -249,7 +250,7 @@ protected void initChannel(Channel channel) throws SSLException { Throwable earlyError = errorReference.get(); if (earlyError != null) { - // If an error occurred between the connect and the request being sent, don't proceed with sending + // If an error occurred between the connecting and the request being sent, don't proceed with sending // the request. latch.countDown(); return; @@ -312,71 +313,81 @@ private void configurePooledRequestPipeline(Channel channel, HttpRequest request AtomicReference responseReference, AtomicReference errorReference, CountDownLatch latch, boolean isHttps) { - // It's possible that the channel was closed between the time it was acquired and now. - // This check ensures that we don't try to add handlers to a closed channel. - if (!channel.isActive()) { - LOGGER.atWarning().log("Channel acquired from the pool is inactive, failing the request."); - setOrSuppressError(errorReference, new ClosedChannelException()); - latch.countDown(); - return; - } - - ProgressReporter progressReporter = (request.getContext() == null) - ? null - : (ProgressReporter) request.getContext().getMetadata("progressReporter"); - boolean addProgressAndTimeoutHandler - = progressReporter != null || writeTimeoutMillis > 0 || responseTimeoutMillis > 0 || readTimeoutMillis > 0; + ReentrantLock lock = channel.attr(Netty4ConnectionPool.CHANNEL_LOCK).get(); + lock.lock(); + try { + // It's possible that the channel was closed between the time it was acquired and now. + // This check ensures that we don't try to add handlers to a closed channel. + // Read handlers are responsible after this check for not being added in a closed channel. + if (!channel.isActive()) { + LOGGER.atWarning().log("Channel acquired from the pool is inactive, failing the request."); + setOrSuppressError(errorReference, new ClosedChannelException()); + latch.countDown(); + return; + } - ChannelPipeline pipeline = channel.pipeline(); + ProgressReporter progressReporter = (request.getContext() == null) + ? null + : (ProgressReporter) request.getContext().getMetadata("progressReporter"); + boolean addProgressAndTimeoutHandler = progressReporter != null + || writeTimeoutMillis > 0 + || responseTimeoutMillis > 0 + || readTimeoutMillis > 0; - // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is - // applying timeouts. - // This is done to keep the ChannelPipeline shorter, therefore more performant if this would - // effectively be a no-op. - if (addProgressAndTimeoutHandler) { - pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, - writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); - } + ChannelPipeline pipeline = channel.pipeline(); - // The SslHandler is already in the pipeline if this is an HTTPS request, as it's added - // by the connection pool during the initial connection setup. The SSL handshake is also - // guaranteed to be complete by the time we get the channel because the Netty4AlpnHandler - // reacts to the result of the ALPN negotiation that happened during the SSL handshake. - if (isHttps) { - HttpProtocolVersion protocolVersion = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); - if (protocolVersion != null) { - // The Connection is being reused, ALPN is already done. - // Manually configure the pipeline based on the stored protocol. - boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; - pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is + // applying timeouts. + // This is done to keep the ChannelPipeline shorter, therefore more performant if this + // effectively is a no-op. + if (addProgressAndTimeoutHandler) { + pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, + writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); + } - if (!isHttp2 && pipeline.get(HTTP_CODEC) == null) { - pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); - } + // The SslHandler is already in the pipeline if this is an HTTPS request, as it's added + // by the connection pool during the initial connection setup. The SSL handshake is also + // guaranteed to be complete by the time we get the channel because the Netty4AlpnHandler + // reacts to the result of the ALPN negotiation that happened during the SSL handshake. + if (isHttps) { + HttpProtocolVersion protocolVersion = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); + if (protocolVersion != null) { + // The Connection is being reused, ALPN is already done. + // Manually configure the pipeline based on the stored protocol. + boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; + pipeline.addLast(HTTP_RESPONSE, + new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + + if (!isHttp2 && pipeline.get(HTTP_CODEC) == null) { + pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); + } - pipeline.addLast(PIPELINE_CLEANUP, - new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); - if (isHttp2) { - sendHttp2Request(request, channel, errorReference, latch); + pipeline.addLast(PIPELINE_CLEANUP, + new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); + if (isHttp2) { + sendHttp2Request(request, channel, errorReference, latch); + } else { + send(request, channel, errorReference, latch); + } } else { - send(request, channel, errorReference, latch); + // This is a new connection, let ALPN do the work. + // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. + pipeline.addAfter(SSL, ALPN, + new Netty4AlpnHandler(request, responseReference, errorReference, latch, connectionPool)); } } else { - // This is a new connection, let ALPN do the work. - // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. - pipeline.addAfter(SSL, ALPN, - new Netty4AlpnHandler(request, responseReference, errorReference, latch, connectionPool)); + // If there isn't an SslHandler, we can send the request immediately. + // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. + pipeline.addLast(HTTP_RESPONSE, + new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; + pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); + pipeline.addLast(PIPELINE_CLEANUP, + new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); + send(request, channel, errorReference, latch); } - } else { - // If there isn't an SslHandler, we can send the request immediately. - // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. - pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); - String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; - pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); - pipeline.addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); - send(request, channel, errorReference, latch); + } finally { + lock.unlock(); } } @@ -471,7 +482,7 @@ public void close() { try { connectionPool.close(); } catch (IOException e) { - LOGGER.atWarning().setThrowable(e).log("Failed to close Netty4ConnectionPool."); + LOGGER.atWarning().setThrowable(e).log("Failed to close the Netty Connection pool."); } } if (eventLoopGroup != null && !eventLoopGroup.isShuttingDown()) { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 575d1d3e3029..35b8ffe00bf2 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -39,6 +39,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY; @@ -51,6 +52,16 @@ */ public class Netty4ConnectionPool implements Closeable { + /** + * An AttributeKey referring to a channel-specific {@link ReentrantLock}. + *

+ * This lock is used to ensure that the setup and cleanup of a channel's pipeline are atomic operations. + * It protects against race conditions where a channel might be acquired from the pool and configured for a new + * request before the cleanup from the previous request has fully completed. Each channel gets its own unique + * lock instance, making the lock contention extremely low. + */ + public static final AttributeKey CHANNEL_LOCK = AttributeKey.valueOf("channel-lock"); + private static final AttributeKey POOLED_CONNECTION_KEY = AttributeKey.valueOf("pooled-connection-key"); @@ -356,6 +367,8 @@ private Future createNewConnection() { newConnectionBootstrap.handler(new ChannelInitializer() { @Override public void initChannel(Channel channel) throws SSLException { + channel.attr(CHANNEL_LOCK).set(new ReentrantLock()); + // Create the connection wrapper and attach it to the channel. new PooledConnection(channel, key); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupEvent.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupEvent.java index b72f0b60fe5b..3fc5e1806d9e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupEvent.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupEvent.java @@ -6,7 +6,7 @@ public enum Netty4PipelineCleanupEvent { /** - * Event indicating that the channel will be released back to the connection pool. + * Event used to indicate that the Netty channel will be released back to the connection pool. */ CLEANUP_PIPELINE } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 1921333944c6..773cbbfce767 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -14,6 +14,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.ALPN; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.CHUNKED_WRITER; @@ -88,27 +89,34 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { return; } - // Always reset autoRead to false before returning a channel to the pool - // to ensure predictable behavior for the next request. - ctx.channel().config().setAutoRead(false); + ReentrantLock lock = ctx.channel().attr(Netty4ConnectionPool.CHANNEL_LOCK).get(); + lock.lock(); - ChannelPipeline pipeline = ctx.channel().pipeline(); + try { + // Always reset autoRead to false before returning a channel to the pool + // to ensure predictable behavior for the next request. + ctx.channel().config().setAutoRead(false); - HttpProtocolVersion protocolVersion = ctx.channel().attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); - boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; + ChannelPipeline pipeline = ctx.channel().pipeline(); - for (String handlerName : HANDLERS_TO_REMOVE) { - if (isHttp2 && HTTP_CODEC.equals(handlerName)) { - continue; - } + HttpProtocolVersion protocolVersion = ctx.channel().attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); + boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; + + for (String handlerName : HANDLERS_TO_REMOVE) { + if (isHttp2 && HTTP_CODEC.equals(handlerName)) { + continue; + } - if (pipeline.get(handlerName) != null) { - pipeline.remove(handlerName); + if (pipeline.get(handlerName) != null) { + pipeline.remove(handlerName); + } } - } - if (pipeline.get(Netty4PipelineCleanupHandler.class) != null) { - pipeline.remove(this); + if (pipeline.get(Netty4PipelineCleanupHandler.class) != null) { + pipeline.remove(this); + } + } finally { + lock.unlock(); } if (closeChannel || !ctx.channel().isActive() || connectionPool == null) { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index afa549e085e7..a9c7a16e511a 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -176,14 +176,15 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { return; } - if (!isHttp2) { - ctx.pipeline().remove(this); - } - ctx.fireChannelReadComplete(); - responseReference.set(new ResponseStateInfo(ctx.channel(), complete, statusCode, headers, eagerContent, ResponseBodyHandling.getBodyHandling(request, headers), isHttp2)); latch.countDown(); + + if (!isHttp2 && ctx.pipeline().get(this.getClass()) != null) { + ctx.pipeline().remove(this); + } + + ctx.fireChannelReadComplete(); } @Override diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java index d2953cbb19b8..fdb137a57301 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java @@ -14,6 +14,7 @@ import io.netty.channel.DefaultChannelConfig; import io.netty.channel.DefaultEventLoop; import io.netty.channel.EventLoop; +import io.netty.util.AttributeKey; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mock; @@ -24,6 +25,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_CODEC; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_RESPONSE; @@ -52,6 +54,7 @@ public class Netty4PipelineCleanupHandlerTests { public void setup() { MockitoAnnotations.openMocks(this); testChannel = new TestMockChannel(new MockEventLoop()); + testChannel.attr(AttributeKey.valueOf("channel-lock")).set(new ReentrantLock()); errorReference = new AtomicReference<>(); latch = new CountDownLatch(1); } From 1a2840900baf98c0ef19a84fba10e390041acd59 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Thu, 10 Jul 2025 18:03:08 +0300 Subject: [PATCH 39/57] self removal of handlers --- .../netty4/implementation/Netty4AlpnHandler.java | 5 +++++ .../Netty4EagerConsumeChannelHandler.java | 10 +++++----- .../Netty4InitiateOneReadHandler.java | 16 ++++++++++++++-- .../Netty4ProgressAndTimeoutHandler.java | 9 +++++++++ 4 files changed, 33 insertions(+), 7 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index ab5c65aa2fbf..41bc4c475122 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -14,6 +14,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.ALPN; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PIPELINE_CLEANUP; import static io.clientcore.http.netty4.implementation.Netty4Utility.configureHttpsPipeline; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; @@ -113,6 +114,10 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { } }); } + + if (ctx.pipeline().get(ALPN) != null) { + ctx.pipeline().remove(this); + } } @Override diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 6d1fb7770a83..60b969aac13e 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -92,7 +92,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { public void channelReadComplete(ChannelHandlerContext ctx) { ctx.fireChannelReadComplete(); if (lastRead) { - signalComplete(ctx, false); + signalComplete(ctx); } } @@ -100,7 +100,7 @@ public void channelReadComplete(ChannelHandlerContext ctx) { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; ctx.fireExceptionCaught(cause); - signalComplete(ctx, true); + signalComplete(ctx); } Throwable channelException() { @@ -110,13 +110,13 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - signalComplete(ctx, true); + signalComplete(ctx); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - signalComplete(ctx, true); + signalComplete(ctx); ctx.fireChannelInactive(); } @@ -130,7 +130,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { } } - private void signalComplete(ChannelHandlerContext ctx, boolean forceClose) { + private void signalComplete(ChannelHandlerContext ctx) { if (ctx.pipeline().get(Netty4EagerConsumeChannelHandler.class) != null) { ctx.pipeline().remove(this); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java index ccfc65214104..031b8ea459ab 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4InitiateOneReadHandler.java @@ -95,6 +95,11 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { @Override public void channelReadComplete(ChannelHandlerContext ctx) { latch.countDown(); + if (lastRead) { + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } + } ctx.fireChannelReadComplete(); } @@ -116,13 +121,13 @@ Throwable channelException() { // TODO (alzimmer): Are the latch countdowns needed for unregistering and inactivity? @Override public void channelUnregistered(ChannelHandlerContext ctx) { - latch.countDown(); + signalComplete(ctx); ctx.fireChannelUnregistered(); } @Override public void channelInactive(ChannelHandlerContext ctx) { - latch.countDown(); + signalComplete(ctx); ctx.fireChannelInactive(); } @@ -136,4 +141,11 @@ public void handlerAdded(ChannelHandlerContext ctx) { } } + private void signalComplete(ChannelHandlerContext ctx) { + latch.countDown(); + if (ctx.pipeline().get(Netty4InitiateOneReadHandler.class) != null) { + ctx.pipeline().remove(this); + } + } + } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java index 3df4ba10e7b0..ae6ee03021bb 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java @@ -3,6 +3,7 @@ package io.clientcore.http.netty4.implementation; +import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.utils.ProgressReporter; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufHolder; @@ -18,6 +19,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; + /** * Class containing all ChannelHandler concepts that ClientCore SDKs use. *

@@ -233,6 +236,12 @@ public void channelReadComplete(ChannelHandlerContext ctx) { this.lastReadMillis = System.currentTimeMillis(); if (lastRead && trackingReadTimeout) { endReadTracking(); + + HttpProtocolVersion protocolVersion = ctx.channel().attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); + + if (protocolVersion != HttpProtocolVersion.HTTP_2 && ctx.pipeline().get(PROGRESS_AND_TIMEOUT) != null) { + ctx.pipeline().remove(this); + } } ctx.fireChannelReadComplete(); } From ed9b857f854a3a64a12eecd473fb916df9bddfcd Mon Sep 17 00:00:00 2001 From: George Banasios Date: Thu, 10 Jul 2025 19:03:35 +0300 Subject: [PATCH 40/57] attempt to fix race condition --- .../http/netty4/NettyHttpClient.java | 150 ++++---- .../http/netty4/NettyHttpClientBuilder.java | 2 +- .../implementation/Netty4AlpnHandler.java | 20 +- .../implementation/Netty4ConnectionPool.java | 341 ++++++++++++------ .../Netty4PipelineCleanupHandler.java | 11 +- .../Netty4ProgressAndTimeoutHandler.java | 9 - .../implementation/Netty4ResponseHandler.java | 4 - .../http/netty4/NettyHttpClientTests.java | 17 +- .../Netty4PipelineCleanupHandlerTests.java | 27 +- 9 files changed, 376 insertions(+), 205 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index a94646a82c6f..0373905ef357 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -313,82 +313,90 @@ private void configurePooledRequestPipeline(Channel channel, HttpRequest request AtomicReference responseReference, AtomicReference errorReference, CountDownLatch latch, boolean isHttps) { - ReentrantLock lock = channel.attr(Netty4ConnectionPool.CHANNEL_LOCK).get(); - lock.lock(); - try { - // It's possible that the channel was closed between the time it was acquired and now. - // This check ensures that we don't try to add handlers to a closed channel. - // Read handlers are responsible after this check for not being added in a closed channel. - if (!channel.isActive()) { - LOGGER.atWarning().log("Channel acquired from the pool is inactive, failing the request."); - setOrSuppressError(errorReference, new ClosedChannelException()); - latch.countDown(); - return; - } - - ProgressReporter progressReporter = (request.getContext() == null) - ? null - : (ProgressReporter) request.getContext().getMetadata("progressReporter"); - boolean addProgressAndTimeoutHandler = progressReporter != null - || writeTimeoutMillis > 0 - || responseTimeoutMillis > 0 - || readTimeoutMillis > 0; - - ChannelPipeline pipeline = channel.pipeline(); - - // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is - // applying timeouts. - // This is done to keep the ChannelPipeline shorter, therefore more performant if this - // effectively is a no-op. - if (addProgressAndTimeoutHandler) { - pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, - writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); - } + channel.eventLoop().execute(() -> { + ReentrantLock lock = channel.attr(Netty4ConnectionPool.CHANNEL_LOCK).get(); + lock.lock(); + try { + channel.config().setAutoRead(false); + + // It's possible that the channel was closed between the time it was acquired and now. + // This check ensures that we don't try to add handlers to a closed channel. + // Read handlers are responsible after this check for not being added in a closed channel. + if (!channel.isActive()) { + LOGGER.atWarning().log("Channel acquired from the pool is inactive, failing the request."); + setOrSuppressError(errorReference, new ClosedChannelException()); + latch.countDown(); + return; + } - // The SslHandler is already in the pipeline if this is an HTTPS request, as it's added - // by the connection pool during the initial connection setup. The SSL handshake is also - // guaranteed to be complete by the time we get the channel because the Netty4AlpnHandler - // reacts to the result of the ALPN negotiation that happened during the SSL handshake. - if (isHttps) { - HttpProtocolVersion protocolVersion = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); - if (protocolVersion != null) { - // The Connection is being reused, ALPN is already done. - // Manually configure the pipeline based on the stored protocol. - boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; - pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + final Object pipelineOwnerToken = new Object(); + channel.attr(Netty4ConnectionPool.PIPELINE_OWNER_TOKEN).set(pipelineOwnerToken); + + ProgressReporter progressReporter = (request.getContext() == null) + ? null + : (ProgressReporter) request.getContext().getMetadata("progressReporter"); + boolean addProgressAndTimeoutHandler = progressReporter != null + || writeTimeoutMillis > 0 + || responseTimeoutMillis > 0 + || readTimeoutMillis > 0; + + ChannelPipeline pipeline = channel.pipeline(); + + // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is + // applying timeouts. + // This is done to keep the ChannelPipeline shorter, therefore more performant if this + // effectively is a no-op. + if (addProgressAndTimeoutHandler) { + pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, + writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); + } - if (!isHttp2 && pipeline.get(HTTP_CODEC) == null) { - pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); - } + // The SslHandler is already in the pipeline if this is an HTTPS request, as it's added + // by the connection pool during the initial connection setup. The SSL handshake is also + // guaranteed to be complete by the time we get the channel because the Netty4AlpnHandler + // reacts to the result of the ALPN negotiation that happened during the SSL handshake. + if (isHttps) { + HttpProtocolVersion protocolVersion + = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); + if (protocolVersion != null) { + // The Connection is being reused, ALPN is already done. + // Manually configure the pipeline based on the stored protocol. + boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; + pipeline.addLast(HTTP_RESPONSE, + new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + + if (!isHttp2 && pipeline.get(HTTP_CODEC) == null) { + pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); + } - pipeline.addLast(PIPELINE_CLEANUP, - new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); - if (isHttp2) { - sendHttp2Request(request, channel, errorReference, latch); + pipeline.addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool, + errorReference, latch, pipelineOwnerToken)); + if (isHttp2) { + sendHttp2Request(request, channel, errorReference, latch); + } else { + send(request, channel, errorReference, latch); + } } else { - send(request, channel, errorReference, latch); + // This is a new connection, let ALPN do the work. + // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. + pipeline.addAfter(SSL, ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, + latch, connectionPool, pipelineOwnerToken)); } } else { - // This is a new connection, let ALPN do the work. - // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. - pipeline.addAfter(SSL, ALPN, - new Netty4AlpnHandler(request, responseReference, errorReference, latch, connectionPool)); + // If there isn't an SslHandler, we can send the request immediately. + // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. + pipeline.addLast(HTTP_RESPONSE, + new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; + pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); + pipeline.addLast(PIPELINE_CLEANUP, + new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, pipelineOwnerToken)); + send(request, channel, errorReference, latch); } - } else { - // If there isn't an SslHandler, we can send the request immediately. - // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. - pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); - String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; - pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); - pipeline.addLast(PIPELINE_CLEANUP, - new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); - send(request, channel, errorReference, latch); + } finally { + lock.unlock(); } - } finally { - lock.unlock(); - } + }); } private void send(HttpRequest request, Channel channel, AtomicReference errorReference, @@ -409,6 +417,12 @@ private Response createResponse(HttpRequest request, ResponseStateIn Response response; Channel channelToCleanup = info.getResponseChannel(); + channelToCleanup.eventLoop().execute(() -> { + if (channelToCleanup.pipeline().get(Netty4ResponseHandler.class) != null) { + channelToCleanup.pipeline().remove(Netty4ResponseHandler.class); + } + }); + final Runnable cleanupTask = () -> { if (connectionPool != null) { channelToCleanup.pipeline().fireUserEventTriggered(Netty4PipelineCleanupEvent.CLEANUP_PIPELINE); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index aa12a285ab26..708d7dca945b 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -138,7 +138,7 @@ private static Class getChannelClass(String className) // --- Connection Pool Configuration --- private int connectionPoolSize = 1000; - private Duration connectionIdleTimeout = Duration.ofSeconds(60); + private Duration connectionIdleTimeout = Duration.ofSeconds(50); private Duration maxConnectionLifetime; private Duration pendingAcquireTimeout = Duration.ofSeconds(60); // Default wait time for a connection private int maxPendingAcquires = 10_000; // Default pending queue size diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index 41bc4c475122..df01024d8980 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -34,11 +34,15 @@ public final class Netty4AlpnHandler extends ApplicationProtocolNegotiationHandl */ public static final AttributeKey HTTP_PROTOCOL_VERSION_KEY = AttributeKey.valueOf("http-protocol-version"); + private static final Netty4ConnectionPool.Http2GoAwayHandler GO_AWAY_HANDLER + = new Netty4ConnectionPool.Http2GoAwayHandler(); + private final HttpRequest request; private final AtomicReference responseReference; private final AtomicReference errorReference; private final CountDownLatch latch; private final Netty4ConnectionPool connectionPool; + private final Object pipelineOwnerToken; /** * Creates a new instance of {@link Netty4AlpnHandler} with a fallback to using HTTP/1.1. @@ -49,13 +53,15 @@ public final class Netty4AlpnHandler extends ApplicationProtocolNegotiationHandl * @param connectionPool The connection pool. */ public Netty4AlpnHandler(HttpRequest request, AtomicReference responseReference, - AtomicReference errorReference, CountDownLatch latch, Netty4ConnectionPool connectionPool) { + AtomicReference errorReference, CountDownLatch latch, Netty4ConnectionPool connectionPool, + Object pipelineOwnerToken) { super(ApplicationProtocolNames.HTTP_1_1); this.request = request; this.responseReference = responseReference; this.errorReference = errorReference; this.latch = latch; this.connectionPool = connectionPool; + this.pipelineOwnerToken = pipelineOwnerToken; } /** @@ -73,6 +79,7 @@ public Netty4AlpnHandler(HttpRequest request, AtomicReference this.errorReference = errorReference; this.latch = latch; this.connectionPool = null; + this.pipelineOwnerToken = null; } @Override @@ -97,8 +104,15 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { configureHttpsPipeline(ctx.pipeline(), request, protocolVersion, responseReference, errorReference, latch); - ctx.pipeline() - .addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch)); + if (protocolVersion == HttpProtocolVersion.HTTP_2) { + ctx.pipeline().addLast(GO_AWAY_HANDLER); + } + + if (connectionPool != null) { + ctx.pipeline() + .addLast(PIPELINE_CLEANUP, + new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, pipelineOwnerToken)); + } if (protocolVersion == HttpProtocolVersion.HTTP_2) { sendHttp2Request(request, ctx.channel(), errorReference, latch); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 35b8ffe00bf2..ff29e9ef40fc 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -10,12 +10,21 @@ import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; +import io.netty.handler.codec.http2.Http2Connection; +import io.netty.handler.codec.http2.Http2GoAwayFrame; +import io.netty.handler.codec.http2.HttpToHttp2ConnectionHandler; +import io.netty.handler.proxy.HttpProxyHandler; import io.netty.handler.proxy.ProxyHandler; +import io.netty.handler.ssl.SslCloseCompletionEvent; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslContextBuilder; +import io.netty.handler.ssl.SslHandler; import io.netty.util.AttributeKey; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; @@ -61,6 +70,10 @@ public class Netty4ConnectionPool implements Closeable { * lock instance, making the lock contention extremely low. */ public static final AttributeKey CHANNEL_LOCK = AttributeKey.valueOf("channel-lock"); + public static final AttributeKey HTTP2_GOAWAY_RECEIVED = AttributeKey.valueOf("http2-goaway-received"); + + // A unique token to identify the current owner of a channel pipeline + public static final AttributeKey PIPELINE_OWNER_TOKEN = AttributeKey.valueOf("pipeline-owner-token"); private static final AttributeKey POOLED_CONNECTION_KEY = AttributeKey.valueOf("pooled-connection-key"); @@ -84,6 +97,43 @@ public class Netty4ConnectionPool implements Closeable { private final AtomicReference> proxyChallenges; private final HttpProtocolVersion maximumHttpVersion; + @ChannelHandler.Sharable + public static class Http2GoAwayHandler extends ChannelInboundHandlerAdapter { + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof Http2GoAwayFrame) { + // A GOAWAY frame was received. Mark the channel so the pool knows + // not to reuse it for new requests. + ctx.channel().attr(HTTP2_GOAWAY_RECEIVED).set(true); + } + super.channelRead(ctx, msg); + } + } + + @ChannelHandler.Sharable + public static final class SuppressProxyConnectExceptionWarningHandler extends ChannelInboundHandlerAdapter { + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + if (cause instanceof HttpProxyHandler.HttpProxyConnectException) { + return; + } + ctx.fireExceptionCaught(cause); + } + } + + @ChannelHandler.Sharable + private static class SslGracefulShutdownHandler extends ChannelInboundHandlerAdapter { + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof SslCloseCompletionEvent) { + ctx.channel().close(); + } + super.userEventTriggered(ctx, evt); + } + } + + private static final SslGracefulShutdownHandler SSL_GRACEFUL_SHUTDOWN_HANDLER = new SslGracefulShutdownHandler(); + public Netty4ConnectionPool(Bootstrap bootstrap, ChannelInitializationProxyHandler channelInitializationProxyHandler, Consumer sslContextModifier, int maxConnectionsPerRoute, Duration connectionIdleTimeout, @@ -115,7 +165,7 @@ public Netty4ConnectionPool(Bootstrap bootstrap, } /** - * Acquires a channel for the given remote address from the pool. + * Acquires a channel for the given composite key from the pool. * * @param key The composite key representing the connection route. * @param isHttps Flag indicating whether connections for this route should be secured using TLS/SSL. @@ -132,10 +182,11 @@ public Future acquire(Netty4ConnectionPoolKey key, boolean isHttps) { } /** - * Releases a channel back to the pool. + * Releases a channel back to the connection pool. * The channel pipeline must be cleaned of request-specific handlers before releasing. + * This method is not responsible for that. * - * @param channel The channel to release. + * @param channel The channel to release back to the connection pool. */ public void release(Channel channel) { if (channel == null) { @@ -224,7 +275,8 @@ private void close() { private class PerRoutePool { private final Deque idleConnections = new ConcurrentLinkedDeque<>(); private final Deque> pendingAcquirers = new ConcurrentLinkedDeque<>(); - private final AtomicInteger activeConnections = new AtomicInteger(0); + // Counter for all connections for a specific route (active and idle). + private final AtomicInteger totalConnections = new AtomicInteger(0); private final Netty4ConnectionPoolKey key; private final SocketAddress route; private final boolean isHttps; @@ -235,6 +287,17 @@ private class PerRoutePool { this.isHttps = isHttps; } + /** + * Acquires a connection. + * + *

+ * This method is the entry point for getting a connection. It will first try to poll from the idle queue. + * If it can't, it will attempt to create a new one if pool capacity is not reached. If capacity is reached, + * it will queue the request. + *

+ * + * @return A {@link Future} that completes with a {@link Channel}. + */ Future acquire() { if (closed.get()) { return bootstrap.config() @@ -243,121 +306,145 @@ Future acquire() { .newFailedFuture(new IllegalStateException(CLOSED_POOL_ERROR_MESSAGE)); } - // First, optimistically try to acquire an existing idle connection. + // First, try the optimistic fast-path. + PooledConnection connection = pollIdleConnection(); + if (connection != null) { + return connection.channel.eventLoop().newSucceededFuture(connection.channel); + } + + // No idle connections, we need to either create a new one or queue. + int currentTotal = totalConnections.getAndIncrement(); + if (currentTotal < maxConnectionsPerRoute) { + return createNewConnection(); + } + + // Pool is full, decrement the counter back and queue the request. + totalConnections.getAndDecrement(); + return queueAcquireRequest(); + } + + void release(PooledConnection connection) { + if (!isHealthy(connection)) { + connection.close(); // The close listener will handle decrementing the counter. + return; + } + + connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); + + // Offer to the idle queue and then try to satisfy pending waiters. + idleConnections.offer(connection); + drainPendingAcquirers(); + } + + private PooledConnection pollIdleConnection() { while (true) { PooledConnection connection = idleConnections.poll(); if (connection == null) { - break; + return null; } if (isHealthy(connection)) { - // Acquired an existing healthy connection. activeConnections count is not - // yet incremented for idle channels, so we do it here. - activeConnections.incrementAndGet(); connection.idleSince = null; // Mark as active - return connection.channel.eventLoop().newSucceededFuture(connection.channel); + return connection; } - // Unhealthy idle connection was found and discarded. Don't decrement activeConnections - // as it was already decremented when the channel was released to the idle queue. - connection.close(); + // Unhealthy idle connection was found and discarded. + connection.close(); // The close listener will handle decrementing the counter. } + } - // No idle connections. Try to create a new one or queue the request. - while (true) { - int currentActive = activeConnections.get(); - if (currentActive < maxConnectionsPerRoute) { - // Try to reserve a slot for a new connection. - if (activeConnections.compareAndSet(currentActive, currentActive + 1)) { - return createNewConnection(); - } - // CAS failed, another thread changed the count. Loop to retry. - } else { - // Pool is full, queue the request if there is space. - if (pendingAcquirers.size() >= maxPendingAcquires) { - return bootstrap.config() - .group() - .next() - .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); - } + /** + * Queues a new promise for a connection. + * This is called when the pool is at max capacity. + * + * @return A Future that will be completed later. + */ + private Future queueAcquireRequest() { + if (pendingAcquirers.size() >= maxPendingAcquires) { + return bootstrap.config() + .group() + .next() + .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); + } - Promise promise = bootstrap.config().group().next().newPromise(); - promise.addListener(future -> { - if (future.isCancelled()) { - pendingAcquirers.remove(promise); - } - }); - pendingAcquirers.offer(promise); + Promise promise = bootstrap.config().group().next().newPromise(); + promise.addListener(future -> { + if (future.isCancelled()) { + pendingAcquirers.remove(promise); + } + }); + pendingAcquirers.offer(promise); - if (pendingAcquireTimeout != null) { - bootstrap.config().group().schedule(() -> { - if (!promise.isDone()) { - promise.tryFailure(CoreException - .from("Connection acquisition timed out after " + pendingAcquireTimeout)); - } - }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); + if (pendingAcquireTimeout != null) { + bootstrap.config().group().schedule(() -> { + if (!promise.isDone()) { + promise.tryFailure( + CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout)); } - return promise; - } + }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); } + + // After queueing, try to drain in case a connection was released in the meantime. + drainPendingAcquirers(); + + return promise; } - void release(PooledConnection connection) { - if (!isHealthy(connection)) { - activeConnections.decrementAndGet(); - connection.close(); - // A slot has been freed. Try to satisfy a waiting acquirer with a new connection. - satisfyWaiterWithNewConnection(); + /** + * This is the core logic that matches pending waiters with available resources. + * It can be triggered when a connection is released, or when a slot opens up. + */ + private void drainPendingAcquirers() { + if (pendingAcquirers.isEmpty()) { return; } - // The channel is healthy. Now, check if anyone is waiting for a connection. - while (true) { - Promise waiterToNotify = pendingAcquirers.poll(); - if (waiterToNotify == null) { - // No waiters, return the connection to the idle queue. - activeConnections.decrementAndGet(); - connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); - idleConnections.offer(connection); - break; + // Try to satisfy a waiter with an idle connection. + PooledConnection idleConnection = pollIdleConnection(); + if (idleConnection != null) { + Promise waiter = pollNextWaiter(); + if (waiter != null) { + waiter.trySuccess(idleConnection.channel); + } else { + // No waiter, put the connection back. + idleConnections.addFirst(idleConnection); } + return; + } - // A waiter exists. Fulfill the promise. Active connection count remains the same. - if (waiterToNotify.trySuccess(connection.channel)) { - // Waiter was notified successfully - return; + // No idle connections, try to satisfy a waiter with a new connection if we have capacity. + int currentTotal = totalConnections.getAndIncrement(); + if (currentTotal < maxConnectionsPerRoute) { + Promise waiter = pollNextWaiter(); + if (waiter != null) { + Future newConnectionFuture = createNewConnection(); + newConnectionFuture.addListener(future -> { + if (future.isSuccess()) { + waiter.trySuccess((Channel) future.getNow()); + } else { + waiter.tryFailure(future.cause()); + } + }); + } else { + // No waiter was found, decrement our capacity reservation. + totalConnections.getAndDecrement(); } - // If trySuccess fails, the waiter was cancelled. Loop again to find another waiter. + } else { + // No capacity, decrement our speculative increment. + totalConnections.getAndDecrement(); } } - private void satisfyWaiterWithNewConnection() { - // This method is called when a connection slot is freed. + private Promise pollNextWaiter() { while (true) { - int currentActive = activeConnections.get(); - if (currentActive >= maxConnectionsPerRoute || pendingAcquirers.isEmpty()) { - return; + Promise waiter = pendingAcquirers.poll(); + if (waiter == null) { + return null; // Queue is empty } - - if (activeConnections.compareAndSet(currentActive, currentActive + 1)) { - Promise waiter = pendingAcquirers.poll(); - if (waiter != null) { - // A waiter exists, and we have capacity, create a new connection for them. - Future newConnectionFuture = createNewConnection(); - newConnectionFuture.addListener(future -> { - if (future.isSuccess()) { - waiter.trySuccess((Channel) future.getNow()); - } else { - waiter.tryFailure(future.cause()); - } - }); - } else { - // A waiter disappeared after we reserved a slot. Release the slot. - activeConnections.decrementAndGet(); - } - return; // Exit after attempting to satisfy one waiter. + if (!waiter.isCancelled()) { + return waiter; // Found a valid waiter } - // CAS failed, another thread is operating. Loop to re-evaluate. + // Discard the canceled waiter and try again. } } @@ -379,17 +466,18 @@ public void initChannel(Channel channel) throws SSLException { if (hasProxy) { ProxyHandler proxyHandler = channelInitializationProxyHandler.createProxy(proxyChallenges); pipeline.addFirst(PROXY, proxyHandler); + pipeline.addAfter(PROXY, "clientcore.suppressproxyexception", + new SuppressProxyConnectExceptionWarningHandler()); } // Add SSL handling if the request is HTTPS. if (isHttps) { InetSocketAddress inetSocketAddress = (InetSocketAddress) key.getFinalDestination(); SslContext ssl = buildSslContext(maximumHttpVersion, sslContextModifier); - // SSL handling is added last here. This is done as proxying could require SSL handling too. - channel.pipeline() - .addLast(SSL, ssl.newHandler(channel.alloc(), inetSocketAddress.getHostString(), - inetSocketAddress.getPort())); - channel.pipeline().addLast(SSL_INITIALIZER, new Netty4SslInitializationHandler()); + pipeline.addLast(SSL, ssl.newHandler(channel.alloc(), inetSocketAddress.getHostString(), + inetSocketAddress.getPort())); + pipeline.addAfter(SSL, "clientcore.sslshutdown", SSL_GRACEFUL_SHUTDOWN_HANDLER); + pipeline.addLast(SSL_INITIALIZER, new Netty4SslInitializationHandler()); } } }); @@ -397,14 +485,34 @@ public void initChannel(Channel channel) throws SSLException { newConnectionBootstrap.connect(route).addListener(future -> { if (!future.isSuccess()) { LOGGER.atError().setThrowable(future.cause()).log("Failed connection."); - // Connect failed, release the slot and try to satisfy a waiter. - activeConnections.decrementAndGet(); - satisfyWaiterWithNewConnection(); + totalConnections.getAndDecrement(); + drainPendingAcquirers(); promise.setFailure(future.cause()); return; } Channel newChannel = ((ChannelFuture) future).channel(); + newChannel.closeFuture().addListener(closeFuture -> { + totalConnections.getAndDecrement(); + drainPendingAcquirers(); + }); + + Runnable connectionReadyRunner = () -> { + SslHandler sslHandler = newChannel.pipeline().get(SslHandler.class); + if (sslHandler != null) { + sslHandler.handshakeFuture().addListener(sslFuture -> { + if (sslFuture.isSuccess()) { + promise.setSuccess(newChannel); + } else { + promise.setFailure(sslFuture.cause()); + newChannel.close(); + } + }); + } else { + promise.setSuccess(newChannel); + } + }; + ProxyHandler proxyHandler = (ProxyHandler) newChannel.pipeline().get(PROXY); if (proxyHandler != null) { @@ -413,16 +521,13 @@ public void initChannel(Channel channel) throws SSLException { if (proxyFuture.isSuccess()) { if (!newChannel.isActive()) { promise.setFailure(new ClosedChannelException()); - activeConnections.decrementAndGet(); - satisfyWaiterWithNewConnection(); + newChannel.close(); return; } - promise.setSuccess(newChannel); + connectionReadyRunner.run(); } else { promise.setFailure(proxyFuture.cause()); newChannel.close(); - activeConnections.decrementAndGet(); - satisfyWaiterWithNewConnection(); } }); } else { @@ -433,16 +538,43 @@ public void initChannel(Channel channel) throws SSLException { } private boolean isHealthy(PooledConnection connection) { + Channel channel = connection.channel; + if (!connection.isActiveAndWriteable()) { return false; } + OffsetDateTime now = null; // To be initialized only if needed. + if (maxLifetimeNanos > 0) { - OffsetDateTime now = OffsetDateTime.now(ZoneOffset.UTC); + now = OffsetDateTime.now(ZoneOffset.UTC); if (Duration.between(connection.creationTime, now).toNanos() >= maxLifetimeNanos) { return false; } } + + if (connection.idleSince != null && idleTimeoutNanos > 0) { + if (now == null) { + now = OffsetDateTime.now(ZoneOffset.UTC); + } + if (Duration.between(connection.idleSince, now).toNanos() >= idleTimeoutNanos) { + return false; + } + } + + HttpProtocolVersion protocol = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); + if (protocol == HttpProtocolVersion.HTTP_2) { + if (Boolean.TRUE.equals(channel.attr(HTTP2_GOAWAY_RECEIVED).get())) { + return false; + } + + HttpToHttp2ConnectionHandler http2Handler = channel.pipeline().get(HttpToHttp2ConnectionHandler.class); + if (http2Handler != null) { + Http2Connection.Endpoint clientEndpoint = http2Handler.connection().local(); + return clientEndpoint.numActiveStreams() <= 0; + } + } + return true; } @@ -473,5 +605,4 @@ void close() { } } } - } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 773cbbfce767..97d6273bbd64 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -35,6 +35,7 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final AtomicReference errorReference; private final CountDownLatch latch; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); + private final Object pipelineOwnerToken; private static final List HANDLERS_TO_REMOVE; @@ -51,10 +52,11 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { } public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool, AtomicReference errorReference, - CountDownLatch latch) { + CountDownLatch latch, Object pipelineOwnerToken) { this.connectionPool = connectionPool; this.errorReference = errorReference; this.latch = latch; + this.pipelineOwnerToken = pipelineOwnerToken; } @Override @@ -85,6 +87,13 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) { } public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { + // Check if this handler is still the rightful owner of the pipeline. + // If the tokens don't match, a new request has taken over this channel, + // so this stale cleanup handler must not do anything. + if (ctx.channel().attr(Netty4ConnectionPool.PIPELINE_OWNER_TOKEN).get() != pipelineOwnerToken) { + return; + } + if (!cleanedUp.compareAndSet(false, true)) { return; } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java index ae6ee03021bb..3df4ba10e7b0 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ProgressAndTimeoutHandler.java @@ -3,7 +3,6 @@ package io.clientcore.http.netty4.implementation; -import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.utils.ProgressReporter; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufHolder; @@ -19,8 +18,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; - /** * Class containing all ChannelHandler concepts that ClientCore SDKs use. *

@@ -236,12 +233,6 @@ public void channelReadComplete(ChannelHandlerContext ctx) { this.lastReadMillis = System.currentTimeMillis(); if (lastRead && trackingReadTimeout) { endReadTracking(); - - HttpProtocolVersion protocolVersion = ctx.channel().attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); - - if (protocolVersion != HttpProtocolVersion.HTTP_2 && ctx.pipeline().get(PROGRESS_AND_TIMEOUT) != null) { - ctx.pipeline().remove(this); - } } ctx.fireChannelReadComplete(); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index a9c7a16e511a..4b915a53052d 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -180,10 +180,6 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { ResponseBodyHandling.getBodyHandling(request, headers), isHttp2)); latch.countDown(); - if (!isHttp2 && ctx.pipeline().get(this.getClass()) != null) { - ctx.pipeline().remove(this); - } - ctx.fireChannelReadComplete(); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java index f304cd091333..d97e61b46192 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java @@ -454,11 +454,18 @@ public void malformedContentLengthIsIgnored() throws IOException { URI url = URI.create("http://localhost:" + port); Thread clientThread = new Thread(() -> { - HttpClient client = new NettyHttpClientBuilder().build(); - HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(url); - try (Response response = client.send(request)) { - assertEquals(200, response.getStatusCode()); - TestUtils.assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); + NettyHttpClient client = null; + try { + client = (NettyHttpClient) new NettyHttpClientBuilder().build(); + HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(url); + try (Response response = client.send(request)) { + assertEquals(200, response.getStatusCode()); + TestUtils.assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); + } + } finally { + if (client != null) { + client.close(); + } } }); clientThread.start(); diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java index fdb137a57301..2b2e74d3efc3 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java @@ -46,6 +46,7 @@ public class Netty4PipelineCleanupHandlerTests { @Mock private Netty4ConnectionPool connectionPool; + private static final Object OBJECT = new Object(); private TestMockChannel testChannel; private AtomicReference errorReference; private CountDownLatch latch; @@ -55,6 +56,7 @@ public void setup() { MockitoAnnotations.openMocks(this); testChannel = new TestMockChannel(new MockEventLoop()); testChannel.attr(AttributeKey.valueOf("channel-lock")).set(new ReentrantLock()); + testChannel.attr(AttributeKey.valueOf("pipeline-owner-token")).set(OBJECT); errorReference = new AtomicReference<>(); latch = new CountDownLatch(1); } @@ -63,7 +65,8 @@ public void setup() { public void cleanupWhenPooledAndActiveReleasesChannel() { testChannel.setActive(true); testChannel.pipeline().addLast(HTTP_CODEC, new MockChannelHandler()); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + Netty4PipelineCleanupHandler handler + = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -78,7 +81,8 @@ public void cleanupWhenPooledAndActiveReleasesChannel() { @Test public void cleanupWhenForceCloseClosesChannel() { testChannel.setActive(true); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + Netty4PipelineCleanupHandler handler + = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -91,7 +95,7 @@ public void cleanupWhenForceCloseClosesChannel() { @Test public void cleanupWhenNonPooledClosesChannel() { testChannel.setActive(true); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(null, errorReference, latch); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(null, errorReference, latch, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -103,7 +107,8 @@ public void cleanupWhenNonPooledClosesChannel() { @Test public void cleanupWhenChannelInactiveClosesChannel() { testChannel.setActive(false); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + Netty4PipelineCleanupHandler handler + = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -117,7 +122,8 @@ public void cleanupWhenChannelInactiveClosesChannel() { public void cleanupWhenHttp2PreservesHttpCodec() { testChannel.setActive(true); testChannel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).set(HttpProtocolVersion.HTTP_2); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + Netty4PipelineCleanupHandler handler + = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); populatePipelineWithStandardHandlers(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -131,7 +137,8 @@ public void cleanupWhenHttp2PreservesHttpCodec() { @Test public void cleanupIsIdempotent() { testChannel.setActive(true); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + Netty4PipelineCleanupHandler handler + = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -144,7 +151,8 @@ public void cleanupIsIdempotent() { @Test public void exceptionCaughtSetsErrorAndClosesChannel() { testChannel.setActive(true); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + Netty4PipelineCleanupHandler handler + = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); Throwable testException = new IOException("Test Exception"); @@ -160,7 +168,7 @@ public void exceptionCaughtSetsErrorAndClosesChannel() { @Test public void exceptionCaughtWithNullsStillClosesChannel() { testChannel.setActive(true); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, null, null); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, null, null, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); Throwable testException = new IOException("Test Exception"); @@ -175,7 +183,8 @@ public void exceptionCaughtWithNullsStillClosesChannel() { public void channelInactiveSchedulesAndExecutesCleanup() { testChannel.setActive(true); assertTrue(testChannel.isActive()); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch); + Netty4PipelineCleanupHandler handler + = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); testChannel.pipeline().addLast(handler); testChannel.close(); From fec1ffc104c8da1e448bca02b9b24cc87205a349 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sun, 13 Jul 2025 10:31:39 +0300 Subject: [PATCH 41/57] close clients after tests --- sdk/clientcore/http-netty4/pom.xml | 8 + .../netty4/NettyHttp2HttpClientTests.java | 4 + .../NettyHttpClientHttpClientTests.java | 8 +- ...ttyHttpClientHttpClientWithHttpsTests.java | 4 + .../http/netty4/NettyHttpClientTests.java | 206 +++++++++++------- 5 files changed, 154 insertions(+), 76 deletions(-) diff --git a/sdk/clientcore/http-netty4/pom.xml b/sdk/clientcore/http-netty4/pom.xml index 5e3d722bcbf6..5fbe8c526c69 100644 --- a/sdk/clientcore/http-netty4/pom.xml +++ b/sdk/clientcore/http-netty4/pom.xml @@ -238,6 +238,14 @@ + + org.apache.maven.plugins + maven-surefire-plugin + 3.5.2 + + false + + diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java index 9fb189ccd741..d17fcbc9cb26 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java @@ -47,6 +47,10 @@ public static void startTestServer() { @AfterAll public static void stopTestServer() { + if (HTTP_CLIENT_INSTANCE instanceof NettyHttpClient) { + ((NettyHttpClient) HTTP_CLIENT_INSTANCE).close(); + } + if (server != null) { server.stop(); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java index e31381e1edaa..422594662d8e 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java @@ -20,15 +20,21 @@ @Timeout(value = 3, unit = TimeUnit.MINUTES) public class NettyHttpClientHttpClientTests extends HttpClientTests { private static LocalTestServer server; + private static HttpClient sharedClient; @BeforeAll public static void startTestServer() { server = HttpClientTestsServer.getHttpClientTestsServer(HttpProtocolVersion.HTTP_1_1, false); server.start(); + sharedClient = new NettyHttpClientBuilder().build(); } @AfterAll public static void stopTestServer() { + if (sharedClient instanceof NettyHttpClient) { + ((NettyHttpClient) sharedClient).close(); + } + if (server != null) { server.stop(); } @@ -47,6 +53,6 @@ protected String getServerUri(boolean secure) { @Override protected HttpClient getHttpClient() { - return new NettyHttpClientBuilder().build(); + return sharedClient; } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java index 38a00845ce1a..dc1bc914d19e 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java @@ -40,6 +40,10 @@ public static void startTestServer() { @AfterAll public static void stopTestServer() { + if (HTTP_CLIENT_INSTANCE instanceof NettyHttpClient) { + ((NettyHttpClient) HTTP_CLIENT_INSTANCE).close(); + } + if (server != null) { server.stop(); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java index d97e61b46192..085a3296d905 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java @@ -28,7 +28,9 @@ import io.clientcore.http.netty4.implementation.NettyHttpClientLocalTestServer; import io.netty.channel.ChannelPipeline; import io.netty.handler.proxy.ProxyConnectException; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; @@ -95,10 +97,24 @@ public class NettyHttpClientTests { private static final String SERVER_HTTP_URI = NettyHttpClientLocalTestServer.getServer().getUri(); + private static HttpClient sharedClient; + + @BeforeAll + public static void setup() { + sharedClient = new NettyHttpClientProvider().getSharedInstance(); + } + + @AfterAll + public static void cleanup() { + if (sharedClient instanceof NettyHttpClient) { + ((NettyHttpClient) sharedClient).close(); + } + } + @Test public void testConcurrentRequestsSync() throws InterruptedException, ExecutionException { int numRequests = 100; // 100 = 1GB of data read - HttpClient client = new NettyHttpClientProvider().getSharedInstance(); + HttpClient client = sharedClient; ForkJoinPool pool = new ForkJoinPool((int) Math.ceil(Runtime.getRuntime().availableProcessors() / 2.0)); try { @@ -129,7 +145,7 @@ public void testResponseLongBodyAsByteArray() { @Test public void testProgressReporterSync() { - HttpClient client = new NettyHttpClientProvider().getSharedInstance(); + HttpClient client = sharedClient; ConcurrentLinkedDeque progress = new ConcurrentLinkedDeque<>(); HttpRequest request = new HttpRequest().setMethod(HttpMethod.POST) @@ -154,7 +170,7 @@ public void testFileUpload() throws IOException { tempFile.toFile().deleteOnExit(); BinaryData body = BinaryData.fromFile(tempFile, 1L, 42L); - HttpClient client = new NettyHttpClientProvider().getSharedInstance(); + HttpClient client = sharedClient; HttpRequest request = new HttpRequest().setMethod(HttpMethod.POST) .setUri(uri(SHORT_POST_BODY_WITH_VALIDATION_PATH)) .setBody(body); @@ -166,7 +182,7 @@ public void testFileUpload() throws IOException { @Test public void testRequestBodyIsErrorShouldPropagateToResponse() { - HttpClient client = new NettyHttpClientProvider().getSharedInstance(); + HttpClient client = sharedClient; HttpRequest request = new HttpRequest().setMethod(HttpMethod.POST) .setUri(uri(SHORT_POST_BODY_PATH)) .setHeaders(new HttpHeaders().set(HttpHeaderName.CONTENT_LENGTH, "132")) @@ -184,7 +200,7 @@ public int read() { @Test public void testRequestBodyIsErrorShouldPropagateToResponseSyncInGetMethod() { - HttpClient client = new NettyHttpClientProvider().getSharedInstance(); + HttpClient client = sharedClient; try (Response response = client.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(LONG_BODY_PATH)))) { @@ -197,7 +213,7 @@ public void testRequestBodyIsErrorShouldPropagateToResponseSyncInGetMethod() { @Test @Timeout(20) public void testFlowableWhenServerReturnsBodyAndNoErrorsWhenHttp500Returned() { - HttpClient client = new NettyHttpClientProvider().getSharedInstance(); + HttpClient client = sharedClient; try (Response response = sendRequest(client, "/error")) { assertEquals(500, response.getStatusCode()); assertEquals("error", response.getValue().toString()); @@ -207,7 +223,7 @@ public void testFlowableWhenServerReturnsBodyAndNoErrorsWhenHttp500Returned() { @ParameterizedTest @MethodSource("requestHeaderSupplier") public void requestHeader(String headerValue, String expectedValue) { - HttpClient client = new NettyHttpClientProvider().getSharedInstance(); + HttpClient client = sharedClient; HttpRequest request = new HttpRequest().setMethod(HttpMethod.POST) .setUri(uri(HTTP_HEADERS_PATH)) @@ -221,7 +237,7 @@ public void requestHeader(String headerValue, String expectedValue) { @Test public void validateRequestHasOneUserAgentHeader() { - HttpClient httpClient = new NettyHttpClientProvider().getSharedInstance(); + HttpClient httpClient = sharedClient; try (Response response = httpClient.send(new HttpRequest().setMethod(HttpMethod.GET) .setUri(uri(NO_DOUBLE_UA_PATH)) @@ -233,7 +249,7 @@ public void validateRequestHasOneUserAgentHeader() { @Test public void validateHeadersReturnAsIs() { - HttpClient client = new NettyHttpClientProvider().getSharedInstance(); + HttpClient client = sharedClient; HttpHeaderName singleValueHeaderName = HttpHeaderName.fromString("singleValue"); final String singleValueHeaderValue = "value"; @@ -283,17 +299,25 @@ public void proxyAuthenticationErrorEagerlyRetries() throws IOException { = new ProxyOptions(ProxyOptions.Type.HTTP, mockProxyServer.socketAddress()).setCredentials("1", "1"); // Create an HttpPipeline where any exception has a retry delay of 10 seconds. - HttpPipeline httpPipeline = new HttpPipelineBuilder().addPolicy(retryPolicy) - .addPolicy(callCountingPolicy) - .httpClient(new NettyHttpClientBuilder().proxy(proxyOptions).build()) - .build(); + NettyHttpClient httpClient = null; + try { + httpClient = (NettyHttpClient) new NettyHttpClientBuilder().proxy(proxyOptions).build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().addPolicy(retryPolicy) + .addPolicy(callCountingPolicy) + .httpClient(httpClient) + .build(); + + try (Response response + = httpPipeline.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(PROXY_TO_ADDRESS)))) { + assertEquals(418, response.getStatusCode()); + } - try (Response response - = httpPipeline.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(PROXY_TO_ADDRESS)))) { - assertEquals(418, response.getStatusCode()); + assertEquals(1, callCount.get()); + } finally { + if (httpClient != null) { + httpClient.close(); + } } - - assertEquals(1, callCount.get()); } } @@ -304,32 +328,38 @@ public void proxyAuthenticationErrorEagerlyRetries() throws IOException { @RepeatedTest(100) public void failedProxyAuthenticationReturnsCorrectError() { try (MockProxyServer mockProxyServer = new MockProxyServer("1", "1")) { - HttpClient httpClient = new NettyHttpClientBuilder() - .proxy( + NettyHttpClient httpClient = null; + try { + httpClient = (NettyHttpClient) new NettyHttpClientBuilder().proxy( new ProxyOptions(ProxyOptions.Type.HTTP, mockProxyServer.socketAddress()).setCredentials("2", "2")) - .build(); - - CoreException coreException = assertThrows(CoreException.class, - () -> httpClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(PROXY_TO_ADDRESS)))); - - Throwable exception = coreException.getCause(); - assertTrue(exception instanceof ProxyConnectException || exception instanceof ClosedChannelException, - "Exception was not of expected type ProxyConnectException or ClosedChannelException, but was " - + exception.getClass().getName()); - - if (exception instanceof ProxyConnectException) { - assertInstanceOf(ProxyConnectException.class, exception, () -> { - StringWriter stringWriter = new StringWriter(); - stringWriter.write(exception.toString()); - PrintWriter printWriter = new PrintWriter(stringWriter); - exception.printStackTrace(printWriter); - - return stringWriter.toString(); - }); - - assertTrue(coreException.getCause().getMessage().contains("Proxy Authentication Required"), - () -> "Expected exception message to contain \"Proxy Authentication Required\", it was: " - + coreException.getCause().getMessage()); + .build(); + final HttpClient finalClient = httpClient; + CoreException coreException = assertThrows(CoreException.class, + () -> finalClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(PROXY_TO_ADDRESS)))); + + Throwable exception = coreException.getCause(); + assertTrue(exception instanceof ProxyConnectException || exception instanceof ClosedChannelException, + "Exception was not of expected type ProxyConnectException or ClosedChannelException, but was " + + exception.getClass().getName()); + + if (exception instanceof ProxyConnectException) { + assertInstanceOf(ProxyConnectException.class, exception, () -> { + StringWriter stringWriter = new StringWriter(); + stringWriter.write(exception.toString()); + PrintWriter printWriter = new PrintWriter(stringWriter); + exception.printStackTrace(printWriter); + + return stringWriter.toString(); + }); + + assertTrue(coreException.getCause().getMessage().contains("Proxy Authentication Required"), + () -> "Expected exception message to contain \"Proxy Authentication Required\", it was: " + + coreException.getCause().getMessage()); + } + } finally { + if (httpClient != null) { + httpClient.close(); + } } } } @@ -359,17 +389,24 @@ public void failedProxyAuthenticationReturnsCorrectError() { */ @Test public void progressAndTimeoutHandlerNotAdded() throws IOException { - HttpClient client = new NettyHttpClientBuilder().connectTimeout(Duration.ZERO) - .writeTimeout(Duration.ZERO) - .responseTimeout(Duration.ZERO) - .readTimeout(Duration.ZERO) - .build(); + NettyHttpClient httpClient = null; + try { + httpClient = (NettyHttpClient) new NettyHttpClientBuilder().connectTimeout(Duration.ZERO) + .writeTimeout(Duration.ZERO) + .responseTimeout(Duration.ZERO) + .readTimeout(Duration.ZERO) + .build(); - try (Response response - = client.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(LONG_BODY_PATH)))) { - assertNotNull(response); - assertEquals(200, response.getStatusCode()); - assertArraysEqual(LONG_BODY, response.getValue().toBytes()); + try (Response response + = httpClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(LONG_BODY_PATH)))) { + assertNotNull(response); + assertEquals(200, response.getStatusCode()); + assertArraysEqual(LONG_BODY, response.getValue().toBytes()); + } + } finally { + if (httpClient != null) { + httpClient.close(); + } } } @@ -400,20 +437,27 @@ public void sendWithServerSentEvents() throws InterruptedException { sseServer.start(); final CountDownLatch latch = new CountDownLatch(2); final AtomicReference lastEvent = new AtomicReference<>(); + NettyHttpClient httpClient = null; + try { + httpClient = (NettyHttpClient) new NettyHttpClientBuilder().build(); + HttpRequest request + = new HttpRequest().setMethod(HttpMethod.GET).setUri(URI.create(sseServer.getUri())); + request.setServerSentEventListener(event -> { + lastEvent.set(event); + latch.countDown(); + }); - HttpClient client = new NettyHttpClientBuilder().build(); - HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(URI.create(sseServer.getUri())); - request.setServerSentEventListener(event -> { - lastEvent.set(event); - latch.countDown(); - }); - - try (Response response = client.send(request)) { - assertEquals(200, response.getStatusCode()); - assertTrue(latch.await(10, TimeUnit.SECONDS)); - assertNotNull(lastEvent.get()); - assertEquals("2", lastEvent.get().getId()); - assertEquals("message", lastEvent.get().getEvent()); + try (Response response = httpClient.send(request)) { + assertEquals(200, response.getStatusCode()); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertNotNull(lastEvent.get()); + assertEquals("2", lastEvent.get().getId()); + assertEquals("message", lastEvent.get().getEvent()); + } + } finally { + if (httpClient != null) { + httpClient.close(); + } } } finally { sseServer.stop(); @@ -432,14 +476,19 @@ public void sendWithServerSentEventsAndNoListenerThrows() { } }); + NettyHttpClient client = null; try { sseServer.start(); - HttpClient client = new NettyHttpClientBuilder().build(); + client = (NettyHttpClient) new NettyHttpClientBuilder().build(); HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(URI.create(sseServer.getUri())); - IllegalStateException ex = assertThrows(IllegalStateException.class, () -> client.send(request)); + final NettyHttpClient finalClient = client; + IllegalStateException ex = assertThrows(IllegalStateException.class, () -> finalClient.send(request)); assertTrue(ex.getMessage().contains("No ServerSentEventListener attached")); } finally { + if (client != null) { + client.close(); + } sseServer.stop(); } } @@ -486,12 +535,19 @@ public void malformedContentLengthIsIgnored() throws IOException { @Test public void nonPooledClientSendsRequestSuccessfully() { - HttpClient client = new NettyHttpClientBuilder().connectionPoolSize(0).build(); + NettyHttpClient client = null; + try { + client = (NettyHttpClient) new NettyHttpClientBuilder().connectionPoolSize(0).build(); - try (Response response - = client.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(SHORT_BODY_PATH)))) { - assertEquals(200, response.getStatusCode()); - assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); + try (Response response + = client.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(SHORT_BODY_PATH)))) { + assertEquals(200, response.getStatusCode()); + assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); + } + } finally { + if (client != null) { + client.close(); + } } } @@ -500,7 +556,7 @@ private static Stream requestHeaderSupplier() { } private static Response getResponse(String path) { - HttpClient client = new NettyHttpClientProvider().getSharedInstance(); + HttpClient client = sharedClient; return getResponse(client, path); } @@ -514,7 +570,7 @@ public static URI uri(String path) { } private static void checkBodyReceived(byte[] expectedBody, String path) { - HttpClient client = new NettyHttpClientProvider().getSharedInstance(); + HttpClient client = sharedClient; try (Response response = sendRequest(client, path)) { ByteArrayOutputStream outStream = new ByteArrayOutputStream(); WritableByteChannel body = Channels.newChannel(outStream); From ca3624508d571d24625e2f2c3b1591a6ce66c5dd Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sun, 13 Jul 2025 12:12:09 +0300 Subject: [PATCH 42/57] Revert "close clients after tests" This reverts commit fec1ffc104c8da1e448bca02b9b24cc87205a349. --- sdk/clientcore/http-netty4/pom.xml | 8 - .../netty4/NettyHttp2HttpClientTests.java | 4 - .../NettyHttpClientHttpClientTests.java | 8 +- ...ttyHttpClientHttpClientWithHttpsTests.java | 4 - .../http/netty4/NettyHttpClientTests.java | 206 +++++++----------- 5 files changed, 76 insertions(+), 154 deletions(-) diff --git a/sdk/clientcore/http-netty4/pom.xml b/sdk/clientcore/http-netty4/pom.xml index 5fbe8c526c69..5e3d722bcbf6 100644 --- a/sdk/clientcore/http-netty4/pom.xml +++ b/sdk/clientcore/http-netty4/pom.xml @@ -238,14 +238,6 @@ - - org.apache.maven.plugins - maven-surefire-plugin - 3.5.2 - - false - - diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java index d17fcbc9cb26..9fb189ccd741 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java @@ -47,10 +47,6 @@ public static void startTestServer() { @AfterAll public static void stopTestServer() { - if (HTTP_CLIENT_INSTANCE instanceof NettyHttpClient) { - ((NettyHttpClient) HTTP_CLIENT_INSTANCE).close(); - } - if (server != null) { server.stop(); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java index 422594662d8e..e31381e1edaa 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java @@ -20,21 +20,15 @@ @Timeout(value = 3, unit = TimeUnit.MINUTES) public class NettyHttpClientHttpClientTests extends HttpClientTests { private static LocalTestServer server; - private static HttpClient sharedClient; @BeforeAll public static void startTestServer() { server = HttpClientTestsServer.getHttpClientTestsServer(HttpProtocolVersion.HTTP_1_1, false); server.start(); - sharedClient = new NettyHttpClientBuilder().build(); } @AfterAll public static void stopTestServer() { - if (sharedClient instanceof NettyHttpClient) { - ((NettyHttpClient) sharedClient).close(); - } - if (server != null) { server.stop(); } @@ -53,6 +47,6 @@ protected String getServerUri(boolean secure) { @Override protected HttpClient getHttpClient() { - return sharedClient; + return new NettyHttpClientBuilder().build(); } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java index dc1bc914d19e..38a00845ce1a 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java @@ -40,10 +40,6 @@ public static void startTestServer() { @AfterAll public static void stopTestServer() { - if (HTTP_CLIENT_INSTANCE instanceof NettyHttpClient) { - ((NettyHttpClient) HTTP_CLIENT_INSTANCE).close(); - } - if (server != null) { server.stop(); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java index 085a3296d905..d97e61b46192 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java @@ -28,9 +28,7 @@ import io.clientcore.http.netty4.implementation.NettyHttpClientLocalTestServer; import io.netty.channel.ChannelPipeline; import io.netty.handler.proxy.ProxyConnectException; -import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; @@ -97,24 +95,10 @@ public class NettyHttpClientTests { private static final String SERVER_HTTP_URI = NettyHttpClientLocalTestServer.getServer().getUri(); - private static HttpClient sharedClient; - - @BeforeAll - public static void setup() { - sharedClient = new NettyHttpClientProvider().getSharedInstance(); - } - - @AfterAll - public static void cleanup() { - if (sharedClient instanceof NettyHttpClient) { - ((NettyHttpClient) sharedClient).close(); - } - } - @Test public void testConcurrentRequestsSync() throws InterruptedException, ExecutionException { int numRequests = 100; // 100 = 1GB of data read - HttpClient client = sharedClient; + HttpClient client = new NettyHttpClientProvider().getSharedInstance(); ForkJoinPool pool = new ForkJoinPool((int) Math.ceil(Runtime.getRuntime().availableProcessors() / 2.0)); try { @@ -145,7 +129,7 @@ public void testResponseLongBodyAsByteArray() { @Test public void testProgressReporterSync() { - HttpClient client = sharedClient; + HttpClient client = new NettyHttpClientProvider().getSharedInstance(); ConcurrentLinkedDeque progress = new ConcurrentLinkedDeque<>(); HttpRequest request = new HttpRequest().setMethod(HttpMethod.POST) @@ -170,7 +154,7 @@ public void testFileUpload() throws IOException { tempFile.toFile().deleteOnExit(); BinaryData body = BinaryData.fromFile(tempFile, 1L, 42L); - HttpClient client = sharedClient; + HttpClient client = new NettyHttpClientProvider().getSharedInstance(); HttpRequest request = new HttpRequest().setMethod(HttpMethod.POST) .setUri(uri(SHORT_POST_BODY_WITH_VALIDATION_PATH)) .setBody(body); @@ -182,7 +166,7 @@ public void testFileUpload() throws IOException { @Test public void testRequestBodyIsErrorShouldPropagateToResponse() { - HttpClient client = sharedClient; + HttpClient client = new NettyHttpClientProvider().getSharedInstance(); HttpRequest request = new HttpRequest().setMethod(HttpMethod.POST) .setUri(uri(SHORT_POST_BODY_PATH)) .setHeaders(new HttpHeaders().set(HttpHeaderName.CONTENT_LENGTH, "132")) @@ -200,7 +184,7 @@ public int read() { @Test public void testRequestBodyIsErrorShouldPropagateToResponseSyncInGetMethod() { - HttpClient client = sharedClient; + HttpClient client = new NettyHttpClientProvider().getSharedInstance(); try (Response response = client.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(LONG_BODY_PATH)))) { @@ -213,7 +197,7 @@ public void testRequestBodyIsErrorShouldPropagateToResponseSyncInGetMethod() { @Test @Timeout(20) public void testFlowableWhenServerReturnsBodyAndNoErrorsWhenHttp500Returned() { - HttpClient client = sharedClient; + HttpClient client = new NettyHttpClientProvider().getSharedInstance(); try (Response response = sendRequest(client, "/error")) { assertEquals(500, response.getStatusCode()); assertEquals("error", response.getValue().toString()); @@ -223,7 +207,7 @@ public void testFlowableWhenServerReturnsBodyAndNoErrorsWhenHttp500Returned() { @ParameterizedTest @MethodSource("requestHeaderSupplier") public void requestHeader(String headerValue, String expectedValue) { - HttpClient client = sharedClient; + HttpClient client = new NettyHttpClientProvider().getSharedInstance(); HttpRequest request = new HttpRequest().setMethod(HttpMethod.POST) .setUri(uri(HTTP_HEADERS_PATH)) @@ -237,7 +221,7 @@ public void requestHeader(String headerValue, String expectedValue) { @Test public void validateRequestHasOneUserAgentHeader() { - HttpClient httpClient = sharedClient; + HttpClient httpClient = new NettyHttpClientProvider().getSharedInstance(); try (Response response = httpClient.send(new HttpRequest().setMethod(HttpMethod.GET) .setUri(uri(NO_DOUBLE_UA_PATH)) @@ -249,7 +233,7 @@ public void validateRequestHasOneUserAgentHeader() { @Test public void validateHeadersReturnAsIs() { - HttpClient client = sharedClient; + HttpClient client = new NettyHttpClientProvider().getSharedInstance(); HttpHeaderName singleValueHeaderName = HttpHeaderName.fromString("singleValue"); final String singleValueHeaderValue = "value"; @@ -299,25 +283,17 @@ public void proxyAuthenticationErrorEagerlyRetries() throws IOException { = new ProxyOptions(ProxyOptions.Type.HTTP, mockProxyServer.socketAddress()).setCredentials("1", "1"); // Create an HttpPipeline where any exception has a retry delay of 10 seconds. - NettyHttpClient httpClient = null; - try { - httpClient = (NettyHttpClient) new NettyHttpClientBuilder().proxy(proxyOptions).build(); - HttpPipeline httpPipeline = new HttpPipelineBuilder().addPolicy(retryPolicy) - .addPolicy(callCountingPolicy) - .httpClient(httpClient) - .build(); - - try (Response response - = httpPipeline.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(PROXY_TO_ADDRESS)))) { - assertEquals(418, response.getStatusCode()); - } + HttpPipeline httpPipeline = new HttpPipelineBuilder().addPolicy(retryPolicy) + .addPolicy(callCountingPolicy) + .httpClient(new NettyHttpClientBuilder().proxy(proxyOptions).build()) + .build(); - assertEquals(1, callCount.get()); - } finally { - if (httpClient != null) { - httpClient.close(); - } + try (Response response + = httpPipeline.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(PROXY_TO_ADDRESS)))) { + assertEquals(418, response.getStatusCode()); } + + assertEquals(1, callCount.get()); } } @@ -328,38 +304,32 @@ public void proxyAuthenticationErrorEagerlyRetries() throws IOException { @RepeatedTest(100) public void failedProxyAuthenticationReturnsCorrectError() { try (MockProxyServer mockProxyServer = new MockProxyServer("1", "1")) { - NettyHttpClient httpClient = null; - try { - httpClient = (NettyHttpClient) new NettyHttpClientBuilder().proxy( + HttpClient httpClient = new NettyHttpClientBuilder() + .proxy( new ProxyOptions(ProxyOptions.Type.HTTP, mockProxyServer.socketAddress()).setCredentials("2", "2")) - .build(); - final HttpClient finalClient = httpClient; - CoreException coreException = assertThrows(CoreException.class, - () -> finalClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(PROXY_TO_ADDRESS)))); - - Throwable exception = coreException.getCause(); - assertTrue(exception instanceof ProxyConnectException || exception instanceof ClosedChannelException, - "Exception was not of expected type ProxyConnectException or ClosedChannelException, but was " - + exception.getClass().getName()); - - if (exception instanceof ProxyConnectException) { - assertInstanceOf(ProxyConnectException.class, exception, () -> { - StringWriter stringWriter = new StringWriter(); - stringWriter.write(exception.toString()); - PrintWriter printWriter = new PrintWriter(stringWriter); - exception.printStackTrace(printWriter); - - return stringWriter.toString(); - }); - - assertTrue(coreException.getCause().getMessage().contains("Proxy Authentication Required"), - () -> "Expected exception message to contain \"Proxy Authentication Required\", it was: " - + coreException.getCause().getMessage()); - } - } finally { - if (httpClient != null) { - httpClient.close(); - } + .build(); + + CoreException coreException = assertThrows(CoreException.class, + () -> httpClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(PROXY_TO_ADDRESS)))); + + Throwable exception = coreException.getCause(); + assertTrue(exception instanceof ProxyConnectException || exception instanceof ClosedChannelException, + "Exception was not of expected type ProxyConnectException or ClosedChannelException, but was " + + exception.getClass().getName()); + + if (exception instanceof ProxyConnectException) { + assertInstanceOf(ProxyConnectException.class, exception, () -> { + StringWriter stringWriter = new StringWriter(); + stringWriter.write(exception.toString()); + PrintWriter printWriter = new PrintWriter(stringWriter); + exception.printStackTrace(printWriter); + + return stringWriter.toString(); + }); + + assertTrue(coreException.getCause().getMessage().contains("Proxy Authentication Required"), + () -> "Expected exception message to contain \"Proxy Authentication Required\", it was: " + + coreException.getCause().getMessage()); } } } @@ -389,24 +359,17 @@ public void failedProxyAuthenticationReturnsCorrectError() { */ @Test public void progressAndTimeoutHandlerNotAdded() throws IOException { - NettyHttpClient httpClient = null; - try { - httpClient = (NettyHttpClient) new NettyHttpClientBuilder().connectTimeout(Duration.ZERO) - .writeTimeout(Duration.ZERO) - .responseTimeout(Duration.ZERO) - .readTimeout(Duration.ZERO) - .build(); + HttpClient client = new NettyHttpClientBuilder().connectTimeout(Duration.ZERO) + .writeTimeout(Duration.ZERO) + .responseTimeout(Duration.ZERO) + .readTimeout(Duration.ZERO) + .build(); - try (Response response - = httpClient.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(LONG_BODY_PATH)))) { - assertNotNull(response); - assertEquals(200, response.getStatusCode()); - assertArraysEqual(LONG_BODY, response.getValue().toBytes()); - } - } finally { - if (httpClient != null) { - httpClient.close(); - } + try (Response response + = client.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(LONG_BODY_PATH)))) { + assertNotNull(response); + assertEquals(200, response.getStatusCode()); + assertArraysEqual(LONG_BODY, response.getValue().toBytes()); } } @@ -437,27 +400,20 @@ public void sendWithServerSentEvents() throws InterruptedException { sseServer.start(); final CountDownLatch latch = new CountDownLatch(2); final AtomicReference lastEvent = new AtomicReference<>(); - NettyHttpClient httpClient = null; - try { - httpClient = (NettyHttpClient) new NettyHttpClientBuilder().build(); - HttpRequest request - = new HttpRequest().setMethod(HttpMethod.GET).setUri(URI.create(sseServer.getUri())); - request.setServerSentEventListener(event -> { - lastEvent.set(event); - latch.countDown(); - }); - try (Response response = httpClient.send(request)) { - assertEquals(200, response.getStatusCode()); - assertTrue(latch.await(10, TimeUnit.SECONDS)); - assertNotNull(lastEvent.get()); - assertEquals("2", lastEvent.get().getId()); - assertEquals("message", lastEvent.get().getEvent()); - } - } finally { - if (httpClient != null) { - httpClient.close(); - } + HttpClient client = new NettyHttpClientBuilder().build(); + HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(URI.create(sseServer.getUri())); + request.setServerSentEventListener(event -> { + lastEvent.set(event); + latch.countDown(); + }); + + try (Response response = client.send(request)) { + assertEquals(200, response.getStatusCode()); + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertNotNull(lastEvent.get()); + assertEquals("2", lastEvent.get().getId()); + assertEquals("message", lastEvent.get().getEvent()); } } finally { sseServer.stop(); @@ -476,19 +432,14 @@ public void sendWithServerSentEventsAndNoListenerThrows() { } }); - NettyHttpClient client = null; try { sseServer.start(); - client = (NettyHttpClient) new NettyHttpClientBuilder().build(); + HttpClient client = new NettyHttpClientBuilder().build(); HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(URI.create(sseServer.getUri())); - final NettyHttpClient finalClient = client; - IllegalStateException ex = assertThrows(IllegalStateException.class, () -> finalClient.send(request)); + IllegalStateException ex = assertThrows(IllegalStateException.class, () -> client.send(request)); assertTrue(ex.getMessage().contains("No ServerSentEventListener attached")); } finally { - if (client != null) { - client.close(); - } sseServer.stop(); } } @@ -535,19 +486,12 @@ public void malformedContentLengthIsIgnored() throws IOException { @Test public void nonPooledClientSendsRequestSuccessfully() { - NettyHttpClient client = null; - try { - client = (NettyHttpClient) new NettyHttpClientBuilder().connectionPoolSize(0).build(); + HttpClient client = new NettyHttpClientBuilder().connectionPoolSize(0).build(); - try (Response response - = client.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(SHORT_BODY_PATH)))) { - assertEquals(200, response.getStatusCode()); - assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); - } - } finally { - if (client != null) { - client.close(); - } + try (Response response + = client.send(new HttpRequest().setMethod(HttpMethod.GET).setUri(uri(SHORT_BODY_PATH)))) { + assertEquals(200, response.getStatusCode()); + assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); } } @@ -556,7 +500,7 @@ private static Stream requestHeaderSupplier() { } private static Response getResponse(String path) { - HttpClient client = sharedClient; + HttpClient client = new NettyHttpClientProvider().getSharedInstance(); return getResponse(client, path); } @@ -570,7 +514,7 @@ public static URI uri(String path) { } private static void checkBodyReceived(byte[] expectedBody, String path) { - HttpClient client = sharedClient; + HttpClient client = new NettyHttpClientProvider().getSharedInstance(); try (Response response = sendRequest(client, path)) { ByteArrayOutputStream outStream = new ByteArrayOutputStream(); WritableByteChannel body = Channels.newChannel(outStream); From 437616b515027994c17a769d2f9a54b2119e7a94 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sun, 13 Jul 2025 12:42:04 +0300 Subject: [PATCH 43/57] http2 connecetions --- .../implementation/Netty4ConnectionPool.java | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index ff29e9ef40fc..366c5bda70a3 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -16,9 +16,7 @@ import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; -import io.netty.handler.codec.http2.Http2Connection; import io.netty.handler.codec.http2.Http2GoAwayFrame; -import io.netty.handler.codec.http2.HttpToHttp2ConnectionHandler; import io.netty.handler.proxy.HttpProxyHandler; import io.netty.handler.proxy.ProxyHandler; import io.netty.handler.ssl.SslCloseCompletionEvent; @@ -348,7 +346,6 @@ private PooledConnection pollIdleConnection() { return connection; } - // Unhealthy idle connection was found and discarded. connection.close(); // The close listener will handle decrementing the counter. } } @@ -426,7 +423,7 @@ private void drainPendingAcquirers() { } }); } else { - // No waiter was found, decrement our capacity reservation. + // No waiter was found, decrement the capacity reservation. totalConnections.getAndDecrement(); } } else { @@ -442,9 +439,8 @@ private Promise pollNextWaiter() { return null; // Queue is empty } if (!waiter.isCancelled()) { - return waiter; // Found a valid waiter + return waiter; } - // Discard the canceled waiter and try again. } } @@ -564,15 +560,7 @@ private boolean isHealthy(PooledConnection connection) { HttpProtocolVersion protocol = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); if (protocol == HttpProtocolVersion.HTTP_2) { - if (Boolean.TRUE.equals(channel.attr(HTTP2_GOAWAY_RECEIVED).get())) { - return false; - } - - HttpToHttp2ConnectionHandler http2Handler = channel.pipeline().get(HttpToHttp2ConnectionHandler.class); - if (http2Handler != null) { - Http2Connection.Endpoint clientEndpoint = http2Handler.connection().local(); - return clientEndpoint.numActiveStreams() <= 0; - } + return !Boolean.TRUE.equals(channel.attr(HTTP2_GOAWAY_RECEIVED).get()); } return true; From 40ef6ee20ca503b38ab3861abb48d0cc69581009 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sun, 13 Jul 2025 16:22:17 +0300 Subject: [PATCH 44/57] synchronize connection pool --- .../http/netty4/NettyHttpClientBuilder.java | 2 +- .../implementation/Netty4ConnectionPool.java | 202 +++++++++--------- .../Netty4PipelineCleanupHandler.java | 47 +++- .../netty4/implementation/Netty4Utility.java | 27 ++- .../src/main/java/module-info.java | 1 + .../Netty4ConnectionPoolTests.java | 2 + .../Netty4PipelineCleanupHandlerTests.java | 6 +- 7 files changed, 161 insertions(+), 126 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 708d7dca945b..2935bc8ed0b6 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -394,8 +394,8 @@ public HttpClient build() { .channel(channelClass) .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, (int) getTimeoutMillis(connectTimeout, 10_000)); // Disable auto-read as we want to control when and how data is read from the channel. - bootstrap.option(ChannelOption.SO_KEEPALIVE, true); bootstrap.option(ChannelOption.AUTO_READ, false); + bootstrap.option(ChannelOption.SO_KEEPALIVE, true); Configuration buildConfiguration = (configuration == null) ? Configuration.getGlobalConfiguration() : configuration; diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 366c5bda70a3..f0f83c7cf70b 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -271,6 +271,8 @@ private void close() { * Manages connections and pending acquirers for a single route. */ private class PerRoutePool { + private final Object lock = new Object(); + private final Deque idleConnections = new ConcurrentLinkedDeque<>(); private final Deque> pendingAcquirers = new ConcurrentLinkedDeque<>(); // Counter for all connections for a specific route (active and idle). @@ -304,131 +306,89 @@ Future acquire() { .newFailedFuture(new IllegalStateException(CLOSED_POOL_ERROR_MESSAGE)); } - // First, try the optimistic fast-path. - PooledConnection connection = pollIdleConnection(); - if (connection != null) { - return connection.channel.eventLoop().newSucceededFuture(connection.channel); - } - - // No idle connections, we need to either create a new one or queue. - int currentTotal = totalConnections.getAndIncrement(); - if (currentTotal < maxConnectionsPerRoute) { - return createNewConnection(); - } - - // Pool is full, decrement the counter back and queue the request. - totalConnections.getAndDecrement(); - return queueAcquireRequest(); - } - - void release(PooledConnection connection) { - if (!isHealthy(connection)) { - connection.close(); // The close listener will handle decrementing the counter. - return; - } - - connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); - - // Offer to the idle queue and then try to satisfy pending waiters. - idleConnections.offer(connection); - drainPendingAcquirers(); - } - - private PooledConnection pollIdleConnection() { + // First, try the optimistic fast-path without locking. while (true) { PooledConnection connection = idleConnections.poll(); if (connection == null) { - return null; + break; } - if (isHealthy(connection)) { connection.idleSince = null; // Mark as active - return connection; + return connection.channel.eventLoop().newSucceededFuture(connection.channel); } - - connection.close(); // The close listener will handle decrementing the counter. - } - } - - /** - * Queues a new promise for a connection. - * This is called when the pool is at max capacity. - * - * @return A Future that will be completed later. - */ - private Future queueAcquireRequest() { - if (pendingAcquirers.size() >= maxPendingAcquires) { - return bootstrap.config() - .group() - .next() - .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); + // Unhealthy connection, close it. The close listener will handle bookkeeping. + connection.close(); } - Promise promise = bootstrap.config().group().next().newPromise(); - promise.addListener(future -> { - if (future.isCancelled()) { - pendingAcquirers.remove(promise); + synchronized (lock) { + // Re-check idle queue while holding the lock, in case a connection was released + // between the optimistic check and acquiring the lock. + PooledConnection connection = idleConnections.poll(); + if (connection != null) { + if (isHealthy(connection)) { + connection.idleSince = null; + return connection.channel.eventLoop().newSucceededFuture(connection.channel); + } else { + connection.close(); + } } - }); - pendingAcquirers.offer(promise); - if (pendingAcquireTimeout != null) { - bootstrap.config().group().schedule(() -> { - if (!promise.isDone()) { - promise.tryFailure( - CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout)); + if (totalConnections.get() < maxConnectionsPerRoute) { + // Capacity is available, reserve a slot. + totalConnections.getAndIncrement(); + } else { + // Pool is full, queue the request if there is space. + if (pendingAcquirers.size() >= maxPendingAcquires) { + return bootstrap.config() + .group() + .next() + .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); } - }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); - } - // After queueing, try to drain in case a connection was released in the meantime. - drainPendingAcquirers(); + Promise promise = bootstrap.config().group().next().newPromise(); + promise.addListener(future -> { + if (future.isCancelled()) { + synchronized (lock) { + pendingAcquirers.remove(promise); + } + } + }); + pendingAcquirers.offer(promise); - return promise; - } + if (pendingAcquireTimeout != null) { + bootstrap.config().group().schedule(() -> { + if (!promise.isDone()) { + promise.tryFailure(CoreException + .from("Connection acquisition timed out after " + pendingAcquireTimeout)); + } + }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); + } - /** - * This is the core logic that matches pending waiters with available resources. - * It can be triggered when a connection is released, or when a slot opens up. - */ - private void drainPendingAcquirers() { - if (pendingAcquirers.isEmpty()) { - return; + return promise; + } } - // Try to satisfy a waiter with an idle connection. - PooledConnection idleConnection = pollIdleConnection(); - if (idleConnection != null) { - Promise waiter = pollNextWaiter(); - if (waiter != null) { - waiter.trySuccess(idleConnection.channel); - } else { - // No waiter, put the connection back. - idleConnections.addFirst(idleConnection); - } + return createNewConnection(); + } + + void release(PooledConnection connection) { + if (!isHealthy(connection)) { + connection.close(); // The close listener will handle everything. return; } - // No idle connections, try to satisfy a waiter with a new connection if we have capacity. - int currentTotal = totalConnections.getAndIncrement(); - if (currentTotal < maxConnectionsPerRoute) { + synchronized (lock) { Promise waiter = pollNextWaiter(); if (waiter != null) { - Future newConnectionFuture = createNewConnection(); - newConnectionFuture.addListener(future -> { - if (future.isSuccess()) { - waiter.trySuccess((Channel) future.getNow()); - } else { - waiter.tryFailure(future.cause()); - } - }); + // A waiter exists, pass the connection directly to them. + // The totalConnections count is unchanged because one user is swapping for another. + connection.idleSince = null; // Mark as active for the new owner. + waiter.trySuccess(connection.channel); } else { - // No waiter was found, decrement the capacity reservation. - totalConnections.getAndDecrement(); + // No waiters, return the connection to the idle queue. + connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); + idleConnections.offer(connection); } - } else { - // No capacity, decrement our speculative increment. - totalConnections.getAndDecrement(); } } @@ -441,6 +401,40 @@ private Promise pollNextWaiter() { if (!waiter.isCancelled()) { return waiter; } + // If waiter was canceled, loop to find the next valid one. + } + } + + private void tryToSatisfyWaiterWithNewConnection() { + boolean createNew = false; + synchronized (lock) { + if (!pendingAcquirers.isEmpty() && totalConnections.get() < maxConnectionsPerRoute) { + // A slot is free and someone is waiting. Reserve the slot for them. + totalConnections.getAndIncrement(); + createNew = true; + } + } + + if (createNew) { + Future newConnectionFuture = createNewConnection(); + newConnectionFuture.addListener(future -> { + synchronized (lock) { + Promise waiter = pollNextWaiter(); + if (waiter != null) { + if (future.isSuccess()) { + waiter.trySuccess((Channel) future.getNow()); + } else { + waiter.tryFailure(future.cause()); + } + } else { + // A connection was created, but no one was waiting for it. + // This can happen if the waiter timed out. Close the new connection. + if (future.isSuccess()) { + ((Channel) future.getNow()).close(); + } + } + } + }); } } @@ -482,7 +476,7 @@ public void initChannel(Channel channel) throws SSLException { if (!future.isSuccess()) { LOGGER.atError().setThrowable(future.cause()).log("Failed connection."); totalConnections.getAndDecrement(); - drainPendingAcquirers(); + tryToSatisfyWaiterWithNewConnection(); promise.setFailure(future.cause()); return; } @@ -490,7 +484,7 @@ public void initChannel(Channel channel) throws SSLException { Channel newChannel = ((ChannelFuture) future).channel(); newChannel.closeFuture().addListener(closeFuture -> { totalConnections.getAndDecrement(); - drainPendingAcquirers(); + tryToSatisfyWaiterWithNewConnection(); }); Runnable connectionReadyRunner = () -> { @@ -536,7 +530,7 @@ public void initChannel(Channel channel) throws SSLException { private boolean isHealthy(PooledConnection connection) { Channel channel = connection.channel; - if (!connection.isActiveAndWriteable()) { + if (!connection.isActiveAndWriteable() || channel.config().isAutoRead()) { return false; } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 97d6273bbd64..eada110f0e99 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -4,6 +4,8 @@ package io.clientcore.http.netty4.implementation; import io.clientcore.core.http.client.HttpProtocolVersion; +import io.clientcore.core.instrumentation.logging.ClientLogger; +import io.netty.channel.Channel; import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; @@ -22,7 +24,10 @@ import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_CODEC; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_RESPONSE; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.READ_ONE; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL_INITIALIZER; import static io.clientcore.http.netty4.implementation.Netty4Utility.setOrSuppressError; /** @@ -31,6 +36,7 @@ */ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { + private static final ClientLogger LOGGER = new ClientLogger(Netty4PipelineCleanupHandler.class); private final Netty4ConnectionPool connectionPool; private final AtomicReference errorReference; private final CountDownLatch latch; @@ -98,6 +104,7 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { return; } + final Channel channel = ctx.channel(); ReentrantLock lock = ctx.channel().attr(Netty4ConnectionPool.CHANNEL_LOCK).get(); lock.lock(); @@ -124,14 +131,46 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { if (pipeline.get(Netty4PipelineCleanupHandler.class) != null) { pipeline.remove(this); } + + if (!isPipelineClean(pipeline, isHttp2)) { + closeChannel = true; + } + } finally { lock.unlock(); + if (closeChannel || !channel.isActive() || connectionPool == null) { + channel.close(); + } else { + channel.eventLoop().execute(() -> connectionPool.release(channel)); + } } + } + + private boolean isPipelineClean(ChannelPipeline pipeline, boolean isHttp2) { + + for (String handlerName : pipeline.names()) { + if (handlerName.contains("HeadContext") || handlerName.contains("TailContext")) { + continue; + } + + if (isHttp2 && handlerName.equals(Netty4HandlerNames.HTTP_CODEC)) { + continue; + } + + if (handlerName.equals(PROXY) + || handlerName.equals("clientcore.suppressproxyexception") // TODO: move those to handler names class + || handlerName.equals(SSL) + || handlerName.equals("clientcore.sslshutdown") + || handlerName.equals(SSL_INITIALIZER) + || handlerName.equals(Netty4ConnectionPool.Http2GoAwayHandler.class.getName())) { + continue; + } - if (closeChannel || !ctx.channel().isActive() || connectionPool == null) { - ctx.channel().close(); - } else { - connectionPool.release(ctx.channel()); + LOGGER.atWarning() + .addKeyValue("handlerName", handlerName) + .log("Found unexpected handler in pipeline during cleanup."); + return false; } + return true; } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java index 771c7092e4f7..98706893e1e9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java @@ -519,20 +519,7 @@ public static void sendHttp2Request(HttpRequest request, Channel channel, Atomic CountDownLatch latch) { io.netty.handler.codec.http.HttpRequest nettyRequest = toNettyHttpRequest(request); - final ChannelFuture writeFuture; - - if (nettyRequest instanceof FullHttpRequest) { - writeFuture = channel.writeAndFlush(nettyRequest); - } else { - channel.write(nettyRequest); - - BinaryData requestBody = request.getBody(); - ChunkedInput chunkedInput = new HttpChunkedInput(new ChunkedStream(requestBody.toStream())); - - writeFuture = channel.writeAndFlush(chunkedInput); - } - - writeFuture.addListener(future -> { + channel.writeAndFlush(nettyRequest).addListener(future -> { if (future.isSuccess()) { channel.read(); } else { @@ -540,6 +527,18 @@ public static void sendHttp2Request(HttpRequest request, Channel channel, Atomic latch.countDown(); } }); + + if (!(nettyRequest instanceof FullHttpRequest)) { + BinaryData requestBody = request.getBody(); + ChunkedInput chunkedInput = new HttpChunkedInput(new ChunkedStream(requestBody.toStream())); + + channel.writeAndFlush(chunkedInput).addListener(future -> { + if (!future.isSuccess()) { + setOrSuppressError(errorReference, future.cause()); + latch.countDown(); + } + }); + } } private static io.netty.handler.codec.http.HttpRequest toNettyHttpRequest(HttpRequest request) { diff --git a/sdk/clientcore/http-netty4/src/main/java/module-info.java b/sdk/clientcore/http-netty4/src/main/java/module-info.java index 2ab0fc5e7223..85431bbc2ae2 100644 --- a/sdk/clientcore/http-netty4/src/main/java/module-info.java +++ b/sdk/clientcore/http-netty4/src/main/java/module-info.java @@ -17,6 +17,7 @@ requires io.netty.resolver; requires io.netty.codec.http2; requires java.sql; + requires java.management; exports io.clientcore.http.netty4; diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java index b65d2b321f56..0826e73d67f9 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java @@ -8,6 +8,7 @@ import io.clientcore.core.shared.LocalTestServer; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; +import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.nio.NioSocketChannel; @@ -54,6 +55,7 @@ public static void startTestServerAndEventLoopGroup() { server.start(); eventLoopGroup = new NioEventLoopGroup(2); bootstrap = new Bootstrap().group(eventLoopGroup).channel(NioSocketChannel.class); + bootstrap.option(ChannelOption.AUTO_READ, false); SocketAddress socketAddress = new InetSocketAddress("localhost", server.getPort()); connectionPoolKey = new Netty4ConnectionPoolKey(socketAddress, socketAddress); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java index 2b2e74d3efc3..a4ee27afe26c 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java @@ -72,8 +72,8 @@ public void cleanupWhenPooledAndActiveReleasesChannel() { handler.cleanup(ctx, false); - verify(connectionPool).release(testChannel); - assertEquals(0, testChannel.getCloseCallCount()); + //verify(connectionPool).release(testChannel); + //assertEquals(0, testChannel.getCloseCallCount()); assertNull(testChannel.pipeline().get(HTTP_CODEC)); assertFalse(testChannel.config().isAutoRead()); } @@ -131,7 +131,7 @@ public void cleanupWhenHttp2PreservesHttpCodec() { assertNotNull(testChannel.pipeline().get(HTTP_CODEC)); assertNull(testChannel.pipeline().get(HTTP_RESPONSE)); - verify(connectionPool).release(testChannel); + //verify(connectionPool).release(testChannel); } @Test From 59f92409a87d526d42c6b470bb09161b9ec1c15e Mon Sep 17 00:00:00 2001 From: George Banasios Date: Mon, 14 Jul 2025 09:47:38 +0300 Subject: [PATCH 45/57] Revert "synchronize connection pool" This reverts commit 40ef6ee20ca503b38ab3861abb48d0cc69581009. --- .../http/netty4/NettyHttpClientBuilder.java | 3 + .../implementation/Netty4ConnectionPool.java | 218 ++++++++++-------- .../Netty4PipelineCleanupHandler.java | 47 +--- .../netty4/implementation/Netty4Utility.java | 27 +-- .../src/main/java/module-info.java | 1 - .../http/netty4/NettyHttpClientTests.java | 78 +++---- .../Netty4PipelineCleanupHandlerTests.java | 7 +- 7 files changed, 182 insertions(+), 199 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 2935bc8ed0b6..30699759b3d8 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -14,6 +14,7 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; +import io.netty.channel.WriteBufferWaterMark; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioSocketChannel; @@ -396,6 +397,8 @@ public HttpClient build() { // Disable auto-read as we want to control when and how data is read from the channel. bootstrap.option(ChannelOption.AUTO_READ, false); bootstrap.option(ChannelOption.SO_KEEPALIVE, true); + bootstrap.option(ChannelOption.WRITE_BUFFER_WATER_MARK, + new WriteBufferWaterMark(8 * 1024 * 1024, 16 * 1024 * 1024)); Configuration buildConfiguration = (configuration == null) ? Configuration.getGlobalConfiguration() : configuration; diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index f0f83c7cf70b..d39e2d8b56f9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -271,8 +271,6 @@ private void close() { * Manages connections and pending acquirers for a single route. */ private class PerRoutePool { - private final Object lock = new Object(); - private final Deque idleConnections = new ConcurrentLinkedDeque<>(); private final Deque> pendingAcquirers = new ConcurrentLinkedDeque<>(); // Counter for all connections for a specific route (active and idle). @@ -306,89 +304,143 @@ Future acquire() { .newFailedFuture(new IllegalStateException(CLOSED_POOL_ERROR_MESSAGE)); } - // First, try the optimistic fast-path without locking. + // First, try the optimistic fast-path. + PooledConnection connection = pollIdleConnection(); + if (connection != null) { + return connection.channel.eventLoop().newSucceededFuture(connection.channel); + } + + // No idle connections, we need to either create a new one or queue. + int currentTotal = totalConnections.getAndIncrement(); + if (currentTotal < maxConnectionsPerRoute) { + return createNewConnection(); + } + + // Pool is full, decrement the counter back and queue the request. + totalConnections.getAndDecrement(); + return queueAcquireRequest(); + } + + void release(PooledConnection connection) { + if (!isHealthy(connection)) { + connection.close(); // The close listener will handle decrementing the counter. + return; + } + + connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); + + // Offer to the idle queue and then try to satisfy pending waiters. + idleConnections.offer(connection); + satisfyWaiter(); + } + + private PooledConnection pollIdleConnection() { while (true) { PooledConnection connection = idleConnections.poll(); if (connection == null) { - break; + return null; } + if (isHealthy(connection)) { connection.idleSince = null; // Mark as active - return connection.channel.eventLoop().newSucceededFuture(connection.channel); + return connection; } - // Unhealthy connection, close it. The close listener will handle bookkeeping. - connection.close(); - } - synchronized (lock) { - // Re-check idle queue while holding the lock, in case a connection was released - // between the optimistic check and acquiring the lock. - PooledConnection connection = idleConnections.poll(); - if (connection != null) { - if (isHealthy(connection)) { - connection.idleSince = null; - return connection.channel.eventLoop().newSucceededFuture(connection.channel); - } else { - connection.close(); - } - } - - if (totalConnections.get() < maxConnectionsPerRoute) { - // Capacity is available, reserve a slot. - totalConnections.getAndIncrement(); - } else { - // Pool is full, queue the request if there is space. - if (pendingAcquirers.size() >= maxPendingAcquires) { - return bootstrap.config() - .group() - .next() - .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); - } - - Promise promise = bootstrap.config().group().next().newPromise(); - promise.addListener(future -> { - if (future.isCancelled()) { - synchronized (lock) { - pendingAcquirers.remove(promise); - } - } - }); - pendingAcquirers.offer(promise); + connection.close(); // The close listener will handle decrementing the counter. + } + } - if (pendingAcquireTimeout != null) { - bootstrap.config().group().schedule(() -> { - if (!promise.isDone()) { - promise.tryFailure(CoreException - .from("Connection acquisition timed out after " + pendingAcquireTimeout)); - } - }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); - } + /** + * Queues a new promise for a connection. + * This is called when the pool is at max capacity. + * + * @return A Future that will be completed later. + */ + private Future queueAcquireRequest() { + if (pendingAcquirers.size() >= maxPendingAcquires) { + return bootstrap.config() + .group() + .next() + .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); + } - return promise; + Promise promise = bootstrap.config().group().next().newPromise(); + promise.addListener(future -> { + if (future.isCancelled()) { + pendingAcquirers.remove(promise); } + }); + pendingAcquirers.offer(promise); + if (pendingAcquireTimeout != null) { + bootstrap.config().group().schedule(() -> { + if (!promise.isDone()) { + promise.tryFailure( + CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout)); + } + }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); } - - return createNewConnection(); + satisfyWaiter(); + return promise; } - void release(PooledConnection connection) { - if (!isHealthy(connection)) { - connection.close(); // The close listener will handle everything. + /** + * This is the core logic that matches pending waiters with available resources. + * It can be triggered when a connection is released, or when a slot opens up. + */ + private void satisfyWaiter() { + if (pendingAcquirers.isEmpty()) { return; } - synchronized (lock) { + // First, try to get a ready-to-use idle connection. + PooledConnection idleConnection = pollIdleConnection(); + if (idleConnection != null) { Promise waiter = pollNextWaiter(); if (waiter != null) { - // A waiter exists, pass the connection directly to them. - // The totalConnections count is unchanged because one user is swapping for another. - connection.idleSince = null; // Mark as active for the new owner. - waiter.trySuccess(connection.channel); + if (!waiter.trySuccess(idleConnection.channel)) { + // Waiter was canceled, release the connection back. + release(idleConnection); + } } else { - // No waiters, return the connection to the idle queue. - connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); - idleConnections.offer(connection); + // No waiter, put the connection back in the idle queue. + idleConnections.addFirst(idleConnection); + } + return; + } + + // No idle connections, try to create a new one if there is capacity. + while (true) { + int currentTotal = totalConnections.get(); + if (currentTotal >= maxConnectionsPerRoute) { + // No capacity, can't create a new connection. + return; + } + + if (totalConnections.compareAndSet(currentTotal, currentTotal + 1)) { + // We successfully reserved a slot for a new connection. + Promise waiter = pollNextWaiter(); + if (waiter != null) { + // Create a new connection for this specific waiter. + createNewConnection().addListener(future -> { + if (future.isSuccess()) { + if (!waiter.trySuccess((Channel) future.getNow())) { + // The Waiter was canceled while we were connecting. + // Release the new connection. + release(((Channel) future.getNow()).attr(POOLED_CONNECTION_KEY).get()); + } + } else { + // Connection failed. The close listener on the (failed) channel + // will decrement totalConnections and trigger another satisfyWaiter call. + waiter.tryFailure(future.cause()); + } + }); + } else { + // We reserved a slot, but there's no waiter. Release the slot. + totalConnections.decrementAndGet(); + } + return; } + // CAS failed, another thread acted. Loop to retry. } } @@ -401,40 +453,6 @@ private Promise pollNextWaiter() { if (!waiter.isCancelled()) { return waiter; } - // If waiter was canceled, loop to find the next valid one. - } - } - - private void tryToSatisfyWaiterWithNewConnection() { - boolean createNew = false; - synchronized (lock) { - if (!pendingAcquirers.isEmpty() && totalConnections.get() < maxConnectionsPerRoute) { - // A slot is free and someone is waiting. Reserve the slot for them. - totalConnections.getAndIncrement(); - createNew = true; - } - } - - if (createNew) { - Future newConnectionFuture = createNewConnection(); - newConnectionFuture.addListener(future -> { - synchronized (lock) { - Promise waiter = pollNextWaiter(); - if (waiter != null) { - if (future.isSuccess()) { - waiter.trySuccess((Channel) future.getNow()); - } else { - waiter.tryFailure(future.cause()); - } - } else { - // A connection was created, but no one was waiting for it. - // This can happen if the waiter timed out. Close the new connection. - if (future.isSuccess()) { - ((Channel) future.getNow()).close(); - } - } - } - }); } } @@ -476,7 +494,7 @@ public void initChannel(Channel channel) throws SSLException { if (!future.isSuccess()) { LOGGER.atError().setThrowable(future.cause()).log("Failed connection."); totalConnections.getAndDecrement(); - tryToSatisfyWaiterWithNewConnection(); + satisfyWaiter(); promise.setFailure(future.cause()); return; } @@ -484,7 +502,7 @@ public void initChannel(Channel channel) throws SSLException { Channel newChannel = ((ChannelFuture) future).channel(); newChannel.closeFuture().addListener(closeFuture -> { totalConnections.getAndDecrement(); - tryToSatisfyWaiterWithNewConnection(); + satisfyWaiter(); }); Runnable connectionReadyRunner = () -> { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index eada110f0e99..97d6273bbd64 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -4,8 +4,6 @@ package io.clientcore.http.netty4.implementation; import io.clientcore.core.http.client.HttpProtocolVersion; -import io.clientcore.core.instrumentation.logging.ClientLogger; -import io.netty.channel.Channel; import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; @@ -24,10 +22,7 @@ import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_CODEC; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_RESPONSE; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; -import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.READ_ONE; -import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; -import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL_INITIALIZER; import static io.clientcore.http.netty4.implementation.Netty4Utility.setOrSuppressError; /** @@ -36,7 +31,6 @@ */ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { - private static final ClientLogger LOGGER = new ClientLogger(Netty4PipelineCleanupHandler.class); private final Netty4ConnectionPool connectionPool; private final AtomicReference errorReference; private final CountDownLatch latch; @@ -104,7 +98,6 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { return; } - final Channel channel = ctx.channel(); ReentrantLock lock = ctx.channel().attr(Netty4ConnectionPool.CHANNEL_LOCK).get(); lock.lock(); @@ -131,46 +124,14 @@ public void cleanup(ChannelHandlerContext ctx, boolean closeChannel) { if (pipeline.get(Netty4PipelineCleanupHandler.class) != null) { pipeline.remove(this); } - - if (!isPipelineClean(pipeline, isHttp2)) { - closeChannel = true; - } - } finally { lock.unlock(); - if (closeChannel || !channel.isActive() || connectionPool == null) { - channel.close(); - } else { - channel.eventLoop().execute(() -> connectionPool.release(channel)); - } } - } - - private boolean isPipelineClean(ChannelPipeline pipeline, boolean isHttp2) { - - for (String handlerName : pipeline.names()) { - if (handlerName.contains("HeadContext") || handlerName.contains("TailContext")) { - continue; - } - - if (isHttp2 && handlerName.equals(Netty4HandlerNames.HTTP_CODEC)) { - continue; - } - - if (handlerName.equals(PROXY) - || handlerName.equals("clientcore.suppressproxyexception") // TODO: move those to handler names class - || handlerName.equals(SSL) - || handlerName.equals("clientcore.sslshutdown") - || handlerName.equals(SSL_INITIALIZER) - || handlerName.equals(Netty4ConnectionPool.Http2GoAwayHandler.class.getName())) { - continue; - } - LOGGER.atWarning() - .addKeyValue("handlerName", handlerName) - .log("Found unexpected handler in pipeline during cleanup."); - return false; + if (closeChannel || !ctx.channel().isActive() || connectionPool == null) { + ctx.channel().close(); + } else { + connectionPool.release(ctx.channel()); } - return true; } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java index 98706893e1e9..771c7092e4f7 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java @@ -519,7 +519,20 @@ public static void sendHttp2Request(HttpRequest request, Channel channel, Atomic CountDownLatch latch) { io.netty.handler.codec.http.HttpRequest nettyRequest = toNettyHttpRequest(request); - channel.writeAndFlush(nettyRequest).addListener(future -> { + final ChannelFuture writeFuture; + + if (nettyRequest instanceof FullHttpRequest) { + writeFuture = channel.writeAndFlush(nettyRequest); + } else { + channel.write(nettyRequest); + + BinaryData requestBody = request.getBody(); + ChunkedInput chunkedInput = new HttpChunkedInput(new ChunkedStream(requestBody.toStream())); + + writeFuture = channel.writeAndFlush(chunkedInput); + } + + writeFuture.addListener(future -> { if (future.isSuccess()) { channel.read(); } else { @@ -527,18 +540,6 @@ public static void sendHttp2Request(HttpRequest request, Channel channel, Atomic latch.countDown(); } }); - - if (!(nettyRequest instanceof FullHttpRequest)) { - BinaryData requestBody = request.getBody(); - ChunkedInput chunkedInput = new HttpChunkedInput(new ChunkedStream(requestBody.toStream())); - - channel.writeAndFlush(chunkedInput).addListener(future -> { - if (!future.isSuccess()) { - setOrSuppressError(errorReference, future.cause()); - latch.countDown(); - } - }); - } } private static io.netty.handler.codec.http.HttpRequest toNettyHttpRequest(HttpRequest request) { diff --git a/sdk/clientcore/http-netty4/src/main/java/module-info.java b/sdk/clientcore/http-netty4/src/main/java/module-info.java index 85431bbc2ae2..2ab0fc5e7223 100644 --- a/sdk/clientcore/http-netty4/src/main/java/module-info.java +++ b/sdk/clientcore/http-netty4/src/main/java/module-info.java @@ -17,7 +17,6 @@ requires io.netty.resolver; requires io.netty.codec.http2; requires java.sql; - requires java.management; exports io.clientcore.http.netty4; diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java index d97e61b46192..4704a5553dba 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java @@ -444,45 +444,45 @@ public void sendWithServerSentEventsAndNoListenerThrows() { } } - @Test - public void malformedContentLengthIsIgnored() throws IOException { - String rawResponse = "HTTP/1.1 200 OK\r\n" + "Content-Type: application/octet-stream\r\n" - + "Content-Length: not-a-number\r\n" + "\r\n"; - - try (ServerSocket serverSocket = new ServerSocket(0)) { - int port = serverSocket.getLocalPort(); - URI url = URI.create("http://localhost:" + port); - - Thread clientThread = new Thread(() -> { - NettyHttpClient client = null; - try { - client = (NettyHttpClient) new NettyHttpClientBuilder().build(); - HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(url); - try (Response response = client.send(request)) { - assertEquals(200, response.getStatusCode()); - TestUtils.assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); - } - } finally { - if (client != null) { - client.close(); - } - } - }); - clientThread.start(); - - try (Socket socket = serverSocket.accept()) { - OutputStream out = socket.getOutputStream(); - out.write(rawResponse.getBytes(StandardCharsets.UTF_8)); - out.write(SHORT_BODY); - out.flush(); - } - - clientThread.join(10000); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } + // @Test + // public void malformedContentLengthIsIgnored() throws IOException { + // String rawResponse = "HTTP/1.1 200 OK\r\n" + "Content-Type: application/octet-stream\r\n" + // + "Content-Length: not-a-number\r\n" + "\r\n"; + // + // try (ServerSocket serverSocket = new ServerSocket(0)) { + // int port = serverSocket.getLocalPort(); + // URI url = URI.create("http://localhost:" + port); + // + // Thread clientThread = new Thread(() -> { + // NettyHttpClient client = null; + // try { + // client = (NettyHttpClient) new NettyHttpClientBuilder().build(); + // HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(url); + // try (Response response = client.send(request)) { + // assertEquals(200, response.getStatusCode()); + // TestUtils.assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); + // } + // } finally { + // if (client != null) { + // client.close(); + // } + // } + // }); + // clientThread.start(); + // + // try (Socket socket = serverSocket.accept()) { + // OutputStream out = socket.getOutputStream(); + // out.write(rawResponse.getBytes(StandardCharsets.UTF_8)); + // out.write(SHORT_BODY); + // out.flush(); + // } + // + // clientThread.join(10000); + // } catch (InterruptedException e) { + // Thread.currentThread().interrupt(); + // throw new RuntimeException(e); + // } + // } @Test public void nonPooledClientSendsRequestSuccessfully() { diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java index a4ee27afe26c..ffe9466ad0d5 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java @@ -57,6 +57,7 @@ public void setup() { testChannel = new TestMockChannel(new MockEventLoop()); testChannel.attr(AttributeKey.valueOf("channel-lock")).set(new ReentrantLock()); testChannel.attr(AttributeKey.valueOf("pipeline-owner-token")).set(OBJECT); + testChannel.config.setAutoRead(false); errorReference = new AtomicReference<>(); latch = new CountDownLatch(1); } @@ -72,8 +73,8 @@ public void cleanupWhenPooledAndActiveReleasesChannel() { handler.cleanup(ctx, false); - //verify(connectionPool).release(testChannel); - //assertEquals(0, testChannel.getCloseCallCount()); + verify(connectionPool).release(testChannel); + assertEquals(0, testChannel.getCloseCallCount()); assertNull(testChannel.pipeline().get(HTTP_CODEC)); assertFalse(testChannel.config().isAutoRead()); } @@ -131,7 +132,7 @@ public void cleanupWhenHttp2PreservesHttpCodec() { assertNotNull(testChannel.pipeline().get(HTTP_CODEC)); assertNull(testChannel.pipeline().get(HTTP_RESPONSE)); - //verify(connectionPool).release(testChannel); + verify(connectionPool).release(testChannel); } @Test From 9e65d7816f98c4551c56b6ff761403440494dd8d Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 16 Jul 2025 13:00:20 +0300 Subject: [PATCH 46/57] check connection --- .../http/netty4/NettyHttpClientBuilder.java | 3 - .../implementation/Netty4ConnectionPool.java | 68 ++++++++++++++++++- .../http/HttpClientTestsServer.java | 6 +- 3 files changed, 70 insertions(+), 7 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 30699759b3d8..2935bc8ed0b6 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -14,7 +14,6 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; -import io.netty.channel.WriteBufferWaterMark; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioSocketChannel; @@ -397,8 +396,6 @@ public HttpClient build() { // Disable auto-read as we want to control when and how data is read from the channel. bootstrap.option(ChannelOption.AUTO_READ, false); bootstrap.option(ChannelOption.SO_KEEPALIVE, true); - bootstrap.option(ChannelOption.WRITE_BUFFER_WATER_MARK, - new WriteBufferWaterMark(8 * 1024 * 1024, 16 * 1024 * 1024)); Configuration buildConfiguration = (configuration == null) ? Configuration.getGlobalConfiguration() : configuration; diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index d39e2d8b56f9..246fb3d15991 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -16,6 +16,12 @@ import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; +import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.HttpHeaderNames; +import io.netty.handler.codec.http.HttpMethod; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpVersion; import io.netty.handler.codec.http2.Http2GoAwayFrame; import io.netty.handler.proxy.HttpProxyHandler; import io.netty.handler.proxy.ProxyHandler; @@ -24,6 +30,7 @@ import io.netty.handler.ssl.SslContextBuilder; import io.netty.handler.ssl.SslHandler; import io.netty.util.AttributeKey; +import io.netty.util.ReferenceCountUtil; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; @@ -341,15 +348,70 @@ private PooledConnection pollIdleConnection() { return null; } - if (isHealthy(connection)) { - connection.idleSince = null; // Mark as active - return connection; + boolean isConnectionAlive = isConnectionAlive(connection.channel); + System.out.println("ISCONNECTIONALIVE: " + isConnectionAlive); + if (isConnectionAlive) { + if (isHealthy(connection)) { + connection.idleSince = null; // Mark as active + return connection; + } } connection.close(); // The close listener will handle decrementing the counter. } } + private boolean isConnectionAlive(Channel channel) { + PooledConnection pooledConnection = channel.attr(POOLED_CONNECTION_KEY).get(); + if (pooledConnection == null) { + return false; + } + + SocketAddress address = pooledConnection.key.getFinalDestination(); + if (!(address instanceof InetSocketAddress)) { + return false; + } + + String host = ((InetSocketAddress) address).getHostString(); + int port = ((InetSocketAddress) address).getPort(); + String hostHeader = host + ":" + port; + + try { + Promise validationPromise = channel.eventLoop().newPromise(); + + ChannelHandler validationHandler = new ChannelInboundHandlerAdapter() { + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof io.netty.handler.codec.http.HttpResponse) { + validationPromise.trySuccess(true); + io.netty.util.ReferenceCountUtil.release(msg); + ctx.pipeline().remove(this); + } else { + super.channelRead(ctx, msg); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + validationPromise.trySuccess(false); + ctx.pipeline().remove(this); + } + }; + + channel.pipeline().addLast("validationHandler", validationHandler); + + FullHttpRequest pingRequest = new DefaultFullHttpRequest( + HttpVersion.HTTP_1_1, HttpMethod.OPTIONS, "*"); + pingRequest.headers().set(HttpHeaderNames.HOST, hostHeader); + + channel.writeAndFlush(pingRequest); + + return validationPromise.get(2, TimeUnit.SECONDS); + } catch (Exception e) { + return false; + } + } + /** * Queues a new promise for a connection. * This is called when the pool is at max capacity. diff --git a/sdk/core/azure-core/src/test/java/com/azure/core/validation/http/HttpClientTestsServer.java b/sdk/core/azure-core/src/test/java/com/azure/core/validation/http/HttpClientTestsServer.java index bd20025b8e89..3955b4c0804d 100644 --- a/sdk/core/azure-core/src/test/java/com/azure/core/validation/http/HttpClientTestsServer.java +++ b/sdk/core/azure-core/src/test/java/com/azure/core/validation/http/HttpClientTestsServer.java @@ -82,8 +82,12 @@ public static LocalTestServer getHttpClientTestsServer() { boolean head = "HEAD".equalsIgnoreCase(req.getMethod()); boolean delete = "DELETE".equalsIgnoreCase(req.getMethod()); boolean patch = "PATCH".equalsIgnoreCase(req.getMethod()); + boolean options = "OPTIONS".equalsIgnoreCase(req.getMethod()); - if (get && path.startsWith("/bytes")) { + if (options && "*".equals(path)) { + resp.setStatus(200); + resp.flushBuffer(); + } else if (get && path.startsWith("/bytes")) { // Stub that will return a response with a body containing the passed number of bytes. sendBytesResponse(path, resp); } else if (get && path.startsWith("/status")) { From 84be996ddcf9a2c0dfda407094e8afe0f64c3d52 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 16 Jul 2025 13:35:33 +0300 Subject: [PATCH 47/57] Revert "check connection" This reverts commit 9e65d7816f98c4551c56b6ff761403440494dd8d. --- .../implementation/Netty4ConnectionPool.java | 68 +------------------ .../http/HttpClientTestsServer.java | 6 +- 2 files changed, 4 insertions(+), 70 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 246fb3d15991..d39e2d8b56f9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -16,12 +16,6 @@ import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; -import io.netty.handler.codec.http.DefaultFullHttpRequest; -import io.netty.handler.codec.http.FullHttpRequest; -import io.netty.handler.codec.http.HttpHeaderNames; -import io.netty.handler.codec.http.HttpMethod; -import io.netty.handler.codec.http.HttpResponse; -import io.netty.handler.codec.http.HttpVersion; import io.netty.handler.codec.http2.Http2GoAwayFrame; import io.netty.handler.proxy.HttpProxyHandler; import io.netty.handler.proxy.ProxyHandler; @@ -30,7 +24,6 @@ import io.netty.handler.ssl.SslContextBuilder; import io.netty.handler.ssl.SslHandler; import io.netty.util.AttributeKey; -import io.netty.util.ReferenceCountUtil; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; @@ -348,70 +341,15 @@ private PooledConnection pollIdleConnection() { return null; } - boolean isConnectionAlive = isConnectionAlive(connection.channel); - System.out.println("ISCONNECTIONALIVE: " + isConnectionAlive); - if (isConnectionAlive) { - if (isHealthy(connection)) { - connection.idleSince = null; // Mark as active - return connection; - } + if (isHealthy(connection)) { + connection.idleSince = null; // Mark as active + return connection; } connection.close(); // The close listener will handle decrementing the counter. } } - private boolean isConnectionAlive(Channel channel) { - PooledConnection pooledConnection = channel.attr(POOLED_CONNECTION_KEY).get(); - if (pooledConnection == null) { - return false; - } - - SocketAddress address = pooledConnection.key.getFinalDestination(); - if (!(address instanceof InetSocketAddress)) { - return false; - } - - String host = ((InetSocketAddress) address).getHostString(); - int port = ((InetSocketAddress) address).getPort(); - String hostHeader = host + ":" + port; - - try { - Promise validationPromise = channel.eventLoop().newPromise(); - - ChannelHandler validationHandler = new ChannelInboundHandlerAdapter() { - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof io.netty.handler.codec.http.HttpResponse) { - validationPromise.trySuccess(true); - io.netty.util.ReferenceCountUtil.release(msg); - ctx.pipeline().remove(this); - } else { - super.channelRead(ctx, msg); - } - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - validationPromise.trySuccess(false); - ctx.pipeline().remove(this); - } - }; - - channel.pipeline().addLast("validationHandler", validationHandler); - - FullHttpRequest pingRequest = new DefaultFullHttpRequest( - HttpVersion.HTTP_1_1, HttpMethod.OPTIONS, "*"); - pingRequest.headers().set(HttpHeaderNames.HOST, hostHeader); - - channel.writeAndFlush(pingRequest); - - return validationPromise.get(2, TimeUnit.SECONDS); - } catch (Exception e) { - return false; - } - } - /** * Queues a new promise for a connection. * This is called when the pool is at max capacity. diff --git a/sdk/core/azure-core/src/test/java/com/azure/core/validation/http/HttpClientTestsServer.java b/sdk/core/azure-core/src/test/java/com/azure/core/validation/http/HttpClientTestsServer.java index 3955b4c0804d..bd20025b8e89 100644 --- a/sdk/core/azure-core/src/test/java/com/azure/core/validation/http/HttpClientTestsServer.java +++ b/sdk/core/azure-core/src/test/java/com/azure/core/validation/http/HttpClientTestsServer.java @@ -82,12 +82,8 @@ public static LocalTestServer getHttpClientTestsServer() { boolean head = "HEAD".equalsIgnoreCase(req.getMethod()); boolean delete = "DELETE".equalsIgnoreCase(req.getMethod()); boolean patch = "PATCH".equalsIgnoreCase(req.getMethod()); - boolean options = "OPTIONS".equalsIgnoreCase(req.getMethod()); - if (options && "*".equals(path)) { - resp.setStatus(200); - resp.flushBuffer(); - } else if (get && path.startsWith("/bytes")) { + if (get && path.startsWith("/bytes")) { // Stub that will return a response with a body containing the passed number of bytes. sendBytesResponse(path, resp); } else if (get && path.startsWith("/status")) { From e092c9c5622133496449ccb22343d0cd93d32b2d Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 16 Jul 2025 14:31:13 +0300 Subject: [PATCH 48/57] try to centralize waiting queue logic --- .../implementation/Netty4ConnectionPool.java | 193 ++++++++---------- 1 file changed, 80 insertions(+), 113 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index d39e2d8b56f9..fbede0e8c43c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -275,6 +275,7 @@ private class PerRoutePool { private final Deque> pendingAcquirers = new ConcurrentLinkedDeque<>(); // Counter for all connections for a specific route (active and idle). private final AtomicInteger totalConnections = new AtomicInteger(0); + private final AtomicBoolean dispatching = new AtomicBoolean(false); private final Netty4ConnectionPoolKey key; private final SocketAddress route; private final boolean isHttps; @@ -310,137 +311,103 @@ Future acquire() { return connection.channel.eventLoop().newSucceededFuture(connection.channel); } - // No idle connections, we need to either create a new one or queue. - int currentTotal = totalConnections.getAndIncrement(); - if (currentTotal < maxConnectionsPerRoute) { - return createNewConnection(); + Promise promise = bootstrap.config().group().next().newPromise(); + if (pendingAcquirers.size() >= maxPendingAcquires) { + promise.setFailure(CoreException.from("Pending acquisition queue is full.")); + return promise; } - // Pool is full, decrement the counter back and queue the request. - totalConnections.getAndDecrement(); - return queueAcquireRequest(); + pendingAcquirers.offer(promise); + dispatch(); + + if (pendingAcquireTimeout != null) { + promise.addListener(future -> { + if (future.isCancelled()) { + pendingAcquirers.remove(promise); + } + }); + + bootstrap.config().group().schedule(() -> { + if (promise.tryFailure( + CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout))) { + pendingAcquirers.remove(promise); + } + }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); + } + return promise; } void release(PooledConnection connection) { if (!isHealthy(connection)) { - connection.close(); // The close listener will handle decrementing the counter. + connection.close(); return; } - connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); - - // Offer to the idle queue and then try to satisfy pending waiters. idleConnections.offer(connection); - satisfyWaiter(); + dispatch(); } - private PooledConnection pollIdleConnection() { - while (true) { - PooledConnection connection = idleConnections.poll(); - if (connection == null) { - return null; - } - - if (isHealthy(connection)) { - connection.idleSince = null; // Mark as active - return connection; - } - - connection.close(); // The close listener will handle decrementing the counter. - } - } - - /** - * Queues a new promise for a connection. - * This is called when the pool is at max capacity. - * - * @return A Future that will be completed later. - */ - private Future queueAcquireRequest() { - if (pendingAcquirers.size() >= maxPendingAcquires) { - return bootstrap.config() - .group() - .next() - .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); + private void dispatch() { + if (!dispatching.compareAndSet(false, true)) { + return; } - Promise promise = bootstrap.config().group().next().newPromise(); - promise.addListener(future -> { - if (future.isCancelled()) { - pendingAcquirers.remove(promise); - } - }); - pendingAcquirers.offer(promise); - if (pendingAcquireTimeout != null) { - bootstrap.config().group().schedule(() -> { - if (!promise.isDone()) { - promise.tryFailure( - CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout)); + try { + while (!pendingAcquirers.isEmpty()) { + PooledConnection idleConnection = pollIdleConnection(); + if (idleConnection != null) { + Promise waiter = pollNextWaiter(); + if (waiter != null) { + if (!waiter.trySuccess(idleConnection.channel)) { + release(idleConnection); + } + } else { + idleConnections.addFirst(idleConnection); + break; + } + continue; } - }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); - } - satisfyWaiter(); - return promise; - } - /** - * This is the core logic that matches pending waiters with available resources. - * It can be triggered when a connection is released, or when a slot opens up. - */ - private void satisfyWaiter() { - if (pendingAcquirers.isEmpty()) { - return; - } + if (totalConnections.get() < maxConnectionsPerRoute) { + if (totalConnections.incrementAndGet() > maxConnectionsPerRoute) { + totalConnections.decrementAndGet(); + break; + } - // First, try to get a ready-to-use idle connection. - PooledConnection idleConnection = pollIdleConnection(); - if (idleConnection != null) { - Promise waiter = pollNextWaiter(); - if (waiter != null) { - if (!waiter.trySuccess(idleConnection.channel)) { - // Waiter was canceled, release the connection back. - release(idleConnection); + Promise waiter = pollNextWaiter(); + if (waiter != null) { + createNewConnection().addListener(future -> { + if (future.isSuccess()) { + if (!waiter.trySuccess((Channel) future.getNow())) { + release(((Channel) future.getNow()).attr(POOLED_CONNECTION_KEY).get()); + } + } else { + waiter.tryFailure(future.cause()); + } + }); + } else { + totalConnections.decrementAndGet(); + } + } else { + break; } - } else { - // No waiter, put the connection back in the idle queue. - idleConnections.addFirst(idleConnection); } - return; + } finally { + dispatching.set(false); } + } - // No idle connections, try to create a new one if there is capacity. + private PooledConnection pollIdleConnection() { while (true) { - int currentTotal = totalConnections.get(); - if (currentTotal >= maxConnectionsPerRoute) { - // No capacity, can't create a new connection. - return; + PooledConnection connection = idleConnections.poll(); + if (connection == null) { + return null; } - - if (totalConnections.compareAndSet(currentTotal, currentTotal + 1)) { - // We successfully reserved a slot for a new connection. - Promise waiter = pollNextWaiter(); - if (waiter != null) { - // Create a new connection for this specific waiter. - createNewConnection().addListener(future -> { - if (future.isSuccess()) { - if (!waiter.trySuccess((Channel) future.getNow())) { - // The Waiter was canceled while we were connecting. - // Release the new connection. - release(((Channel) future.getNow()).attr(POOLED_CONNECTION_KEY).get()); - } - } else { - // Connection failed. The close listener on the (failed) channel - // will decrement totalConnections and trigger another satisfyWaiter call. - waiter.tryFailure(future.cause()); - } - }); - } else { - // We reserved a slot, but there's no waiter. Release the slot. - totalConnections.decrementAndGet(); - } - return; + if (isHealthy(connection)) { + connection.idleSince = null; // Mark as active + return connection; } - // CAS failed, another thread acted. Loop to retry. + connection.close(); } } @@ -448,7 +415,7 @@ private Promise pollNextWaiter() { while (true) { Promise waiter = pendingAcquirers.poll(); if (waiter == null) { - return null; // Queue is empty + return null; } if (!waiter.isCancelled()) { return waiter; @@ -493,16 +460,16 @@ public void initChannel(Channel channel) throws SSLException { newConnectionBootstrap.connect(route).addListener(future -> { if (!future.isSuccess()) { LOGGER.atError().setThrowable(future.cause()).log("Failed connection."); - totalConnections.getAndDecrement(); - satisfyWaiter(); + totalConnections.decrementAndGet(); + dispatch(); promise.setFailure(future.cause()); return; } Channel newChannel = ((ChannelFuture) future).channel(); newChannel.closeFuture().addListener(closeFuture -> { - totalConnections.getAndDecrement(); - satisfyWaiter(); + totalConnections.decrementAndGet(); + dispatch(); }); Runnable connectionReadyRunner = () -> { @@ -539,7 +506,7 @@ public void initChannel(Channel channel) throws SSLException { } }); } else { - promise.setSuccess(newChannel); + connectionReadyRunner.run(); } }); return promise; From df003e0dd4c64900e3eae84723a430fa8871a9d1 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 16 Jul 2025 14:51:54 +0300 Subject: [PATCH 49/57] Revert "try to centralize waiting queue logic" This reverts commit e092c9c5622133496449ccb22343d0cd93d32b2d. --- .../implementation/Netty4ConnectionPool.java | 193 ++++++++++-------- 1 file changed, 113 insertions(+), 80 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index fbede0e8c43c..d39e2d8b56f9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -275,7 +275,6 @@ private class PerRoutePool { private final Deque> pendingAcquirers = new ConcurrentLinkedDeque<>(); // Counter for all connections for a specific route (active and idle). private final AtomicInteger totalConnections = new AtomicInteger(0); - private final AtomicBoolean dispatching = new AtomicBoolean(false); private final Netty4ConnectionPoolKey key; private final SocketAddress route; private final boolean isHttps; @@ -311,103 +310,137 @@ Future acquire() { return connection.channel.eventLoop().newSucceededFuture(connection.channel); } - Promise promise = bootstrap.config().group().next().newPromise(); - if (pendingAcquirers.size() >= maxPendingAcquires) { - promise.setFailure(CoreException.from("Pending acquisition queue is full.")); - return promise; + // No idle connections, we need to either create a new one or queue. + int currentTotal = totalConnections.getAndIncrement(); + if (currentTotal < maxConnectionsPerRoute) { + return createNewConnection(); } - pendingAcquirers.offer(promise); - dispatch(); - - if (pendingAcquireTimeout != null) { - promise.addListener(future -> { - if (future.isCancelled()) { - pendingAcquirers.remove(promise); - } - }); - - bootstrap.config().group().schedule(() -> { - if (promise.tryFailure( - CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout))) { - pendingAcquirers.remove(promise); - } - }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); - } - return promise; + // Pool is full, decrement the counter back and queue the request. + totalConnections.getAndDecrement(); + return queueAcquireRequest(); } void release(PooledConnection connection) { if (!isHealthy(connection)) { - connection.close(); + connection.close(); // The close listener will handle decrementing the counter. return; } + connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); + + // Offer to the idle queue and then try to satisfy pending waiters. idleConnections.offer(connection); - dispatch(); + satisfyWaiter(); } - private void dispatch() { - if (!dispatching.compareAndSet(false, true)) { - return; + private PooledConnection pollIdleConnection() { + while (true) { + PooledConnection connection = idleConnections.poll(); + if (connection == null) { + return null; + } + + if (isHealthy(connection)) { + connection.idleSince = null; // Mark as active + return connection; + } + + connection.close(); // The close listener will handle decrementing the counter. } + } - try { - while (!pendingAcquirers.isEmpty()) { - PooledConnection idleConnection = pollIdleConnection(); - if (idleConnection != null) { - Promise waiter = pollNextWaiter(); - if (waiter != null) { - if (!waiter.trySuccess(idleConnection.channel)) { - release(idleConnection); - } - } else { - idleConnections.addFirst(idleConnection); - break; - } - continue; + /** + * Queues a new promise for a connection. + * This is called when the pool is at max capacity. + * + * @return A Future that will be completed later. + */ + private Future queueAcquireRequest() { + if (pendingAcquirers.size() >= maxPendingAcquires) { + return bootstrap.config() + .group() + .next() + .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); + } + + Promise promise = bootstrap.config().group().next().newPromise(); + promise.addListener(future -> { + if (future.isCancelled()) { + pendingAcquirers.remove(promise); + } + }); + pendingAcquirers.offer(promise); + if (pendingAcquireTimeout != null) { + bootstrap.config().group().schedule(() -> { + if (!promise.isDone()) { + promise.tryFailure( + CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout)); } + }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); + } + satisfyWaiter(); + return promise; + } - if (totalConnections.get() < maxConnectionsPerRoute) { - if (totalConnections.incrementAndGet() > maxConnectionsPerRoute) { - totalConnections.decrementAndGet(); - break; - } + /** + * This is the core logic that matches pending waiters with available resources. + * It can be triggered when a connection is released, or when a slot opens up. + */ + private void satisfyWaiter() { + if (pendingAcquirers.isEmpty()) { + return; + } - Promise waiter = pollNextWaiter(); - if (waiter != null) { - createNewConnection().addListener(future -> { - if (future.isSuccess()) { - if (!waiter.trySuccess((Channel) future.getNow())) { - release(((Channel) future.getNow()).attr(POOLED_CONNECTION_KEY).get()); - } - } else { - waiter.tryFailure(future.cause()); - } - }); - } else { - totalConnections.decrementAndGet(); - } - } else { - break; + // First, try to get a ready-to-use idle connection. + PooledConnection idleConnection = pollIdleConnection(); + if (idleConnection != null) { + Promise waiter = pollNextWaiter(); + if (waiter != null) { + if (!waiter.trySuccess(idleConnection.channel)) { + // Waiter was canceled, release the connection back. + release(idleConnection); } + } else { + // No waiter, put the connection back in the idle queue. + idleConnections.addFirst(idleConnection); } - } finally { - dispatching.set(false); + return; } - } - private PooledConnection pollIdleConnection() { + // No idle connections, try to create a new one if there is capacity. while (true) { - PooledConnection connection = idleConnections.poll(); - if (connection == null) { - return null; + int currentTotal = totalConnections.get(); + if (currentTotal >= maxConnectionsPerRoute) { + // No capacity, can't create a new connection. + return; } - if (isHealthy(connection)) { - connection.idleSince = null; // Mark as active - return connection; + + if (totalConnections.compareAndSet(currentTotal, currentTotal + 1)) { + // We successfully reserved a slot for a new connection. + Promise waiter = pollNextWaiter(); + if (waiter != null) { + // Create a new connection for this specific waiter. + createNewConnection().addListener(future -> { + if (future.isSuccess()) { + if (!waiter.trySuccess((Channel) future.getNow())) { + // The Waiter was canceled while we were connecting. + // Release the new connection. + release(((Channel) future.getNow()).attr(POOLED_CONNECTION_KEY).get()); + } + } else { + // Connection failed. The close listener on the (failed) channel + // will decrement totalConnections and trigger another satisfyWaiter call. + waiter.tryFailure(future.cause()); + } + }); + } else { + // We reserved a slot, but there's no waiter. Release the slot. + totalConnections.decrementAndGet(); + } + return; } - connection.close(); + // CAS failed, another thread acted. Loop to retry. } } @@ -415,7 +448,7 @@ private Promise pollNextWaiter() { while (true) { Promise waiter = pendingAcquirers.poll(); if (waiter == null) { - return null; + return null; // Queue is empty } if (!waiter.isCancelled()) { return waiter; @@ -460,16 +493,16 @@ public void initChannel(Channel channel) throws SSLException { newConnectionBootstrap.connect(route).addListener(future -> { if (!future.isSuccess()) { LOGGER.atError().setThrowable(future.cause()).log("Failed connection."); - totalConnections.decrementAndGet(); - dispatch(); + totalConnections.getAndDecrement(); + satisfyWaiter(); promise.setFailure(future.cause()); return; } Channel newChannel = ((ChannelFuture) future).channel(); newChannel.closeFuture().addListener(closeFuture -> { - totalConnections.decrementAndGet(); - dispatch(); + totalConnections.getAndDecrement(); + satisfyWaiter(); }); Runnable connectionReadyRunner = () -> { @@ -506,7 +539,7 @@ public void initChannel(Channel channel) throws SSLException { } }); } else { - connectionReadyRunner.run(); + promise.setSuccess(newChannel); } }); return promise; From dbf1c69e16dc89932c087b5c108c4092cfba531a Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 16 Jul 2025 15:35:45 +0300 Subject: [PATCH 50/57] enable half closure --- .../http/netty4/NettyHttpClientBuilder.java | 1 + .../implementation/Netty4ConnectionPool.java | 37 +++++++++ .../http/netty4/NettyHttpClientTests.java | 78 +++++++++---------- 3 files changed, 77 insertions(+), 39 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 2935bc8ed0b6..8fe13c372930 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -396,6 +396,7 @@ public HttpClient build() { // Disable auto-read as we want to control when and how data is read from the channel. bootstrap.option(ChannelOption.AUTO_READ, false); bootstrap.option(ChannelOption.SO_KEEPALIVE, true); + bootstrap.option(ChannelOption.ALLOW_HALF_CLOSURE, true); Configuration buildConfiguration = (configuration == null) ? Configuration.getGlobalConfiguration() : configuration; diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index d39e2d8b56f9..e67e52e49a1d 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -16,6 +16,7 @@ import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; +import io.netty.channel.socket.ChannelInputShutdownEvent; import io.netty.handler.codec.http2.Http2GoAwayFrame; import io.netty.handler.proxy.HttpProxyHandler; import io.netty.handler.proxy.ProxyHandler; @@ -130,6 +131,36 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc } } + @ChannelHandler.Sharable + private static class PoolConnectionHealthHandler extends ChannelInboundHandlerAdapter { + private static final AttributeKey CONNECTION_INVALIDATED + = AttributeKey.valueOf("connection-invalidated"); + + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + // This event is fired when the server closes its side of the connection. + if (evt instanceof ChannelInputShutdownEvent) { + invalidateAndClose(ctx.channel()); + } + super.userEventTriggered(ctx, evt); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + // This is a fallback for when the connection is fully closed for any reason. + invalidateAndClose(ctx.channel()); + super.channelInactive(ctx); + } + + private void invalidateAndClose(Channel channel) { + System.out.println("CONNECTION INVALIDATED"); + // Mark the channel as invalid. The 'isHealthy' check can use this attribute + // to immediately reject the channel without waiting for it to be fully closed. + channel.attr(CONNECTION_INVALIDATED).set(true); + channel.close(); + } + } + private static final SslGracefulShutdownHandler SSL_GRACEFUL_SHUTDOWN_HANDLER = new SslGracefulShutdownHandler(); public Netty4ConnectionPool(Bootstrap bootstrap, @@ -468,6 +499,8 @@ public void initChannel(Channel channel) throws SSLException { new PooledConnection(channel, key); ChannelPipeline pipeline = channel.pipeline(); + pipeline.addLast("poolHealthHandler", new PoolConnectionHealthHandler()); + // Test whether proxying should be applied to this Channel. If so, add it. // Proxy detection MUST use the final destination address from the key. boolean hasProxy = channelInitializationProxyHandler.test(key.getFinalDestination()); @@ -548,6 +581,10 @@ public void initChannel(Channel channel) throws SSLException { private boolean isHealthy(PooledConnection connection) { Channel channel = connection.channel; + if (Boolean.TRUE.equals(channel.attr(PoolConnectionHealthHandler.CONNECTION_INVALIDATED).get())) { + return false; + } + if (!connection.isActiveAndWriteable() || channel.config().isAutoRead()) { return false; } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java index 4704a5553dba..fdf4ff5b7668 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java @@ -444,45 +444,45 @@ public void sendWithServerSentEventsAndNoListenerThrows() { } } - // @Test - // public void malformedContentLengthIsIgnored() throws IOException { - // String rawResponse = "HTTP/1.1 200 OK\r\n" + "Content-Type: application/octet-stream\r\n" - // + "Content-Length: not-a-number\r\n" + "\r\n"; - // - // try (ServerSocket serverSocket = new ServerSocket(0)) { - // int port = serverSocket.getLocalPort(); - // URI url = URI.create("http://localhost:" + port); - // - // Thread clientThread = new Thread(() -> { - // NettyHttpClient client = null; - // try { - // client = (NettyHttpClient) new NettyHttpClientBuilder().build(); - // HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(url); - // try (Response response = client.send(request)) { - // assertEquals(200, response.getStatusCode()); - // TestUtils.assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); - // } - // } finally { - // if (client != null) { - // client.close(); - // } - // } - // }); - // clientThread.start(); - // - // try (Socket socket = serverSocket.accept()) { - // OutputStream out = socket.getOutputStream(); - // out.write(rawResponse.getBytes(StandardCharsets.UTF_8)); - // out.write(SHORT_BODY); - // out.flush(); - // } - // - // clientThread.join(10000); - // } catch (InterruptedException e) { - // Thread.currentThread().interrupt(); - // throw new RuntimeException(e); - // } - // } + @Test + public void malformedContentLengthIsIgnored() throws IOException { + String rawResponse = "HTTP/1.1 200 OK\r\n" + "Content-Type: application/octet-stream\r\n" + + "Content-Length: not-a-number\r\n" + "\r\n"; + + try (ServerSocket serverSocket = new ServerSocket(0)) { + int port = serverSocket.getLocalPort(); + URI url = URI.create("http://localhost:" + port); + + Thread clientThread = new Thread(() -> { + NettyHttpClient client = null; + try { + client = (NettyHttpClient) new NettyHttpClientBuilder().build(); + HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(url); + try (Response response = client.send(request)) { + assertEquals(200, response.getStatusCode()); + TestUtils.assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); + } + } finally { + if (client != null) { + client.close(); + } + } + }); + clientThread.start(); + + try (Socket socket = serverSocket.accept()) { + OutputStream out = socket.getOutputStream(); + out.write(rawResponse.getBytes(StandardCharsets.UTF_8)); + out.write(SHORT_BODY); + out.flush(); + } + + clientThread.join(10000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } @Test public void nonPooledClientSendsRequestSuccessfully() { From 9a3477bc104753f42db391ed730a603cb71f32ac Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 16 Jul 2025 17:06:23 +0300 Subject: [PATCH 51/57] add lock on acquire/release --- .../http-netty4/spotbugs-exclude.xml | 3 + .../http/netty4/NettyHttpClient.java | 151 +++++---- .../implementation/Netty4AlpnHandler.java | 31 -- .../Netty4ChannelBinaryData.java | 1 - .../implementation/Netty4ConnectionPool.java | 288 ++++++++++-------- .../Netty4PipelineCleanupHandler.java | 3 +- .../implementation/Netty4ResponseHandler.java | 34 ++- .../netty4/implementation/Netty4Utility.java | 9 +- .../netty4/NettyHttp2HttpClientTests.java | 3 + .../NettyHttpClientHttpClientTests.java | 7 +- ...ttyHttpClientHttpClientWithHttpsTests.java | 3 + .../http/netty4/NettyHttpClientTests.java | 43 --- .../Netty4ConnectionPoolTests.java | 76 +++++ .../Netty4Http11ChannelBinaryDataTests.java | 18 +- 14 files changed, 366 insertions(+), 304 deletions(-) diff --git a/sdk/clientcore/http-netty4/spotbugs-exclude.xml b/sdk/clientcore/http-netty4/spotbugs-exclude.xml index b29b42c590e0..5eee4b5d1967 100644 --- a/sdk/clientcore/http-netty4/spotbugs-exclude.xml +++ b/sdk/clientcore/http-netty4/spotbugs-exclude.xml @@ -64,7 +64,10 @@ + + + diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index 0373905ef357..a9a5fcfc425c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -313,90 +313,89 @@ private void configurePooledRequestPipeline(Channel channel, HttpRequest request AtomicReference responseReference, AtomicReference errorReference, CountDownLatch latch, boolean isHttps) { - channel.eventLoop().execute(() -> { - ReentrantLock lock = channel.attr(Netty4ConnectionPool.CHANNEL_LOCK).get(); - lock.lock(); - try { - channel.config().setAutoRead(false); - - // It's possible that the channel was closed between the time it was acquired and now. - // This check ensures that we don't try to add handlers to a closed channel. - // Read handlers are responsible after this check for not being added in a closed channel. - if (!channel.isActive()) { - LOGGER.atWarning().log("Channel acquired from the pool is inactive, failing the request."); - setOrSuppressError(errorReference, new ClosedChannelException()); - latch.countDown(); - return; - } + ReentrantLock lock = channel.attr(Netty4ConnectionPool.CHANNEL_LOCK).get(); + lock.lock(); + try { + channel.config().setAutoRead(false); + + // It's possible that the channel was closed between the time it was acquired and now. + // This check ensures that we don't try to add handlers to a closed channel. + // Read handlers are responsible after this check for not being added in a closed channel. + if (!channel.isActive()) { + LOGGER.atWarning().log("Channel acquired from the pool is inactive, failing the request."); + setOrSuppressError(errorReference, new ClosedChannelException()); + latch.countDown(); + return; + } - final Object pipelineOwnerToken = new Object(); - channel.attr(Netty4ConnectionPool.PIPELINE_OWNER_TOKEN).set(pipelineOwnerToken); - - ProgressReporter progressReporter = (request.getContext() == null) - ? null - : (ProgressReporter) request.getContext().getMetadata("progressReporter"); - boolean addProgressAndTimeoutHandler = progressReporter != null - || writeTimeoutMillis > 0 - || responseTimeoutMillis > 0 - || readTimeoutMillis > 0; - - ChannelPipeline pipeline = channel.pipeline(); - - // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is - // applying timeouts. - // This is done to keep the ChannelPipeline shorter, therefore more performant if this - // effectively is a no-op. - if (addProgressAndTimeoutHandler) { - pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, - writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); - } + channel.config().setAutoRead(false); - // The SslHandler is already in the pipeline if this is an HTTPS request, as it's added - // by the connection pool during the initial connection setup. The SSL handshake is also - // guaranteed to be complete by the time we get the channel because the Netty4AlpnHandler - // reacts to the result of the ALPN negotiation that happened during the SSL handshake. - if (isHttps) { - HttpProtocolVersion protocolVersion - = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); - if (protocolVersion != null) { - // The Connection is being reused, ALPN is already done. - // Manually configure the pipeline based on the stored protocol. - boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; - pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); - - if (!isHttp2 && pipeline.get(HTTP_CODEC) == null) { - pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); - } + final Object pipelineOwnerToken = new Object(); + channel.attr(Netty4ConnectionPool.PIPELINE_OWNER_TOKEN).set(pipelineOwnerToken); - pipeline.addLast(PIPELINE_CLEANUP, new Netty4PipelineCleanupHandler(connectionPool, - errorReference, latch, pipelineOwnerToken)); - if (isHttp2) { - sendHttp2Request(request, channel, errorReference, latch); - } else { - send(request, channel, errorReference, latch); - } + ProgressReporter progressReporter = (request.getContext() == null) + ? null + : (ProgressReporter) request.getContext().getMetadata("progressReporter"); + boolean addProgressAndTimeoutHandler = progressReporter != null + || writeTimeoutMillis > 0 + || responseTimeoutMillis > 0 + || readTimeoutMillis > 0; + + ChannelPipeline pipeline = channel.pipeline(); + + pipeline.addLast(PIPELINE_CLEANUP, + new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, pipelineOwnerToken)); + + // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is + // applying timeouts. + // This is done to keep the ChannelPipeline shorter, therefore more performant if this + // effectively is a no-op. + if (addProgressAndTimeoutHandler) { + pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, + writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); + } + + // The SslHandler is already in the pipeline if this is an HTTPS request, as it's added + // by the connection pool during the initial connection setup. The SSL handshake is also + // guaranteed to be complete by the time we get the channel because the Netty4AlpnHandler + // reacts to the result of the ALPN negotiation that happened during the SSL handshake. + if (isHttps) { + HttpProtocolVersion protocolVersion = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); + if (protocolVersion != null) { + // The Connection is being reused, ALPN is already done. + // Manually configure the pipeline based on the stored protocol. + boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; + pipeline.addLast(HTTP_RESPONSE, + new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + + if (!isHttp2 && pipeline.get(HTTP_CODEC) == null) { + pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); + } + + if (isHttp2) { + sendHttp2Request(request, channel, errorReference, latch); } else { - // This is a new connection, let ALPN do the work. - // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. - pipeline.addAfter(SSL, ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, - latch, connectionPool, pipelineOwnerToken)); + send(request, channel, errorReference, latch); } } else { - // If there isn't an SslHandler, we can send the request immediately. - // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. - pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); - String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; - pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); - pipeline.addLast(PIPELINE_CLEANUP, - new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, pipelineOwnerToken)); - send(request, channel, errorReference, latch); + // This is a new connection, let ALPN do the work. + // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. + pipeline.addAfter(SSL, ALPN, + new Netty4AlpnHandler(request, responseReference, errorReference, latch)); + channel.writeAndFlush(Unpooled.EMPTY_BUFFER); } - } finally { - lock.unlock(); + } else { + // If there isn't an SslHandler, we can send the request immediately. + // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. + pipeline.addLast(HTTP_RESPONSE, + new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; + pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); + send(request, channel, errorReference, latch); } - }); + } finally { + lock.unlock(); + } } private void send(HttpRequest request, Channel channel, AtomicReference errorReference, diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index df01024d8980..8edf5185bd8b 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -15,7 +15,6 @@ import java.util.concurrent.atomic.AtomicReference; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.ALPN; -import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PIPELINE_CLEANUP; import static io.clientcore.http.netty4.implementation.Netty4Utility.configureHttpsPipeline; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp2Request; @@ -41,28 +40,6 @@ public final class Netty4AlpnHandler extends ApplicationProtocolNegotiationHandl private final AtomicReference responseReference; private final AtomicReference errorReference; private final CountDownLatch latch; - private final Netty4ConnectionPool connectionPool; - private final Object pipelineOwnerToken; - - /** - * Creates a new instance of {@link Netty4AlpnHandler} with a fallback to using HTTP/1.1. - * - * @param request The request to send once ALPN negotiation completes. - * @param errorReference An AtomicReference keeping track of errors during the request lifecycle. - * @param latch A CountDownLatch that will be released once the request completes. - * @param connectionPool The connection pool. - */ - public Netty4AlpnHandler(HttpRequest request, AtomicReference responseReference, - AtomicReference errorReference, CountDownLatch latch, Netty4ConnectionPool connectionPool, - Object pipelineOwnerToken) { - super(ApplicationProtocolNames.HTTP_1_1); - this.request = request; - this.responseReference = responseReference; - this.errorReference = errorReference; - this.latch = latch; - this.connectionPool = connectionPool; - this.pipelineOwnerToken = pipelineOwnerToken; - } /** * Creates a new instance of {@link Netty4AlpnHandler} with a fallback to using HTTP/1.1. @@ -78,8 +55,6 @@ public Netty4AlpnHandler(HttpRequest request, AtomicReference this.responseReference = responseReference; this.errorReference = errorReference; this.latch = latch; - this.connectionPool = null; - this.pipelineOwnerToken = null; } @Override @@ -108,12 +83,6 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { ctx.pipeline().addLast(GO_AWAY_HANDLER); } - if (connectionPool != null) { - ctx.pipeline() - .addLast(PIPELINE_CLEANUP, - new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, pipelineOwnerToken)); - } - if (protocolVersion == HttpProtocolVersion.HTTP_2) { sendHttp2Request(request, ctx.channel(), errorReference, latch); } else { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 8932fe3e327a..66affc648ed9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -212,7 +212,6 @@ private void drainStream() { } CountDownLatch ioLatch = new CountDownLatch(1); - // This handler is responsible for triggering the cleanup when it completes. Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(ioLatch, buf -> buf.readBytes(eagerContent, buf.readableBytes()), isHttp2); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index e67e52e49a1d..c44f4e8f71af 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -27,6 +27,7 @@ import io.netty.util.AttributeKey; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; +import io.netty.util.concurrent.ScheduledFuture; import javax.net.ssl.SSLException; import java.io.Closeable; @@ -153,8 +154,7 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } private void invalidateAndClose(Channel channel) { - System.out.println("CONNECTION INVALIDATED"); - // Mark the channel as invalid. The 'isHealthy' check can use this attribute + // Mark the channel as invalid. The 'isHealthy' check uses this attribute // to immediately reject the channel without waiting for it to be fully closed. channel.attr(CONNECTION_INVALIDATED).set(true); channel.close(); @@ -301,7 +301,7 @@ private void close() { /** * Manages connections and pending acquirers for a single route. */ - private class PerRoutePool { + class PerRoutePool { private final Deque idleConnections = new ConcurrentLinkedDeque<>(); private final Deque> pendingAcquirers = new ConcurrentLinkedDeque<>(); // Counter for all connections for a specific route (active and idle). @@ -310,6 +310,9 @@ private class PerRoutePool { private final SocketAddress route; private final boolean isHttps; + // A lock to protect the pool's internal state during acquire/release decisions. + private final ReentrantLock poolLock = new ReentrantLock(); + PerRoutePool(Netty4ConnectionPoolKey key, boolean isHttps) { this.key = key; this.route = key.getConnectionTarget(); @@ -335,156 +338,184 @@ Future acquire() { .newFailedFuture(new IllegalStateException(CLOSED_POOL_ERROR_MESSAGE)); } - // First, try the optimistic fast-path. - PooledConnection connection = pollIdleConnection(); - if (connection != null) { - return connection.channel.eventLoop().newSucceededFuture(connection.channel); - } - - // No idle connections, we need to either create a new one or queue. - int currentTotal = totalConnections.getAndIncrement(); - if (currentTotal < maxConnectionsPerRoute) { - return createNewConnection(); - } - - // Pool is full, decrement the counter back and queue the request. - totalConnections.getAndDecrement(); - return queueAcquireRequest(); - } + poolLock.lock(); + try { + // First, check for an available idle connection. + PooledConnection connection = pollIdleAndCheckHealth(); + if (connection != null) { + // Found a valid, idle connection. Return it immediately. + return connection.channel.eventLoop().newSucceededFuture(connection.channel); + } - void release(PooledConnection connection) { - if (!isHealthy(connection)) { - connection.close(); // The close listener will handle decrementing the counter. - return; - } + // No idle connections. Check if we can create a new one. + if (totalConnections.get() < maxConnectionsPerRoute) { + // Increment count and create a new connection outside the lock. + totalConnections.getAndIncrement(); + return createNewConnection(); + } - connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); + // Pool is full. Queue the acquisition request. + if (pendingAcquirers.size() >= maxPendingAcquires) { + return bootstrap.config() + .group() + .next() + .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); + } - // Offer to the idle queue and then try to satisfy pending waiters. - idleConnections.offer(connection); - satisfyWaiter(); - } + Promise promise = bootstrap.config().group().next().newPromise(); + + if (pendingAcquireTimeout != null && pendingAcquireTimeout.toMillis() > 0) { + final ScheduledFuture timeoutFuture = bootstrap.config().group().schedule(() -> { + if (promise.tryFailure( + CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout))) { + poolLock.lock(); + try { + pendingAcquirers.remove(promise); + } finally { + poolLock.unlock(); + } + } + }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); - private PooledConnection pollIdleConnection() { - while (true) { - PooledConnection connection = idleConnections.poll(); - if (connection == null) { - return null; + promise.addListener(f -> { + if (f.isDone()) { + timeoutFuture.cancel(false); + } + }); } - if (isHealthy(connection)) { - connection.idleSince = null; // Mark as active - return connection; - } + promise.addListener(future -> { + if (future.isCancelled()) { + poolLock.lock(); + try { + pendingAcquirers.remove(promise); + } finally { + poolLock.unlock(); + } + } + }); - connection.close(); // The close listener will handle decrementing the counter. + pendingAcquirers.offer(promise); + return promise; + } finally { + poolLock.unlock(); } } - /** - * Queues a new promise for a connection. - * This is called when the pool is at max capacity. - * - * @return A Future that will be completed later. - */ - private Future queueAcquireRequest() { - if (pendingAcquirers.size() >= maxPendingAcquires) { - return bootstrap.config() - .group() - .next() - .newFailedFuture(CoreException.from("Pending acquisition queue is full.")); - } + void release(PooledConnection connection) { + poolLock.lock(); + try { + if (!isHealthy(connection)) { + // If the connection is unhealthy, it cannot be used by any waiter. + connection.close(); - Promise promise = bootstrap.config().group().next().newPromise(); - promise.addListener(future -> { - if (future.isCancelled()) { - pendingAcquirers.remove(promise); - } - }); - pendingAcquirers.offer(promise); - if (pendingAcquireTimeout != null) { - bootstrap.config().group().schedule(() -> { - if (!promise.isDone()) { - promise.tryFailure( - CoreException.from("Connection acquisition timed out after " + pendingAcquireTimeout)); + // Now, fail all pending waiters because this release operation cannot satisfy them. + // The close listener on the connection will attempt to create a new connection for *one* + // subsequent waiter, but we should proactively fail all current waiters to avoid hangs. + Promise waiter; + while ((waiter = pendingAcquirers.poll()) != null) { + if (!waiter.isCancelled()) { + waiter.tryFailure(new IOException("Released connection was unhealthy.")); + } } - }, pendingAcquireTimeout.toMillis(), TimeUnit.MILLISECONDS); - } - satisfyWaiter(); - return promise; - } + return; // Exit after handling the unhealthy connection. + } - /** - * This is the core logic that matches pending waiters with available resources. - * It can be triggered when a connection is released, or when a slot opens up. - */ - private void satisfyWaiter() { - if (pendingAcquirers.isEmpty()) { - return; - } + // The connection is healthy. Offer it to the waiters. + while (!pendingAcquirers.isEmpty()) { + Promise waiter = pendingAcquirers.poll(); + if (waiter.isCancelled()) { + continue; + } - // First, try to get a ready-to-use idle connection. - PooledConnection idleConnection = pollIdleConnection(); - if (idleConnection != null) { - Promise waiter = pollNextWaiter(); - if (waiter != null) { - if (!waiter.trySuccess(idleConnection.channel)) { - // Waiter was canceled, release the connection back. - release(idleConnection); + if (waiter.trySuccess(connection.channel)) { + // The waiter accepted the connection, so we are done. + return; } - } else { - // No waiter, put the connection back in the idle queue. - idleConnections.addFirst(idleConnection); + // If the waiter didn't accept it (e.g., timed out), the loop will + // offer the connection to the next waiter. } - return; + + // There are no pending waiters, so add the healthy connection to the idle queue. + connection.idleSince = OffsetDateTime.now(ZoneOffset.UTC); + idleConnections.offer(connection); + } finally { + poolLock.unlock(); } + } - // No idle connections, try to create a new one if there is capacity. - while (true) { - int currentTotal = totalConnections.get(); - if (currentTotal >= maxConnectionsPerRoute) { - // No capacity, can't create a new connection. - return; - } + private void handleConnectionClosure() { + poolLock.lock(); + try { + totalConnections.getAndDecrement(); - if (totalConnections.compareAndSet(currentTotal, currentTotal + 1)) { - // We successfully reserved a slot for a new connection. + // A slot has opened up. Loop and create new connections + // for as long as there are waiters, and we have capacity. + while (totalConnections.get() < maxConnectionsPerRoute && !pendingAcquirers.isEmpty()) { Promise waiter = pollNextWaiter(); - if (waiter != null) { - // Create a new connection for this specific waiter. - createNewConnection().addListener(future -> { - if (future.isSuccess()) { - if (!waiter.trySuccess((Channel) future.getNow())) { - // The Waiter was canceled while we were connecting. - // Release the new connection. - release(((Channel) future.getNow()).attr(POOLED_CONNECTION_KEY).get()); - } - } else { - // Connection failed. The close listener on the (failed) channel - // will decrement totalConnections and trigger another satisfyWaiter call. - waiter.tryFailure(future.cause()); - } - }); - } else { - // We reserved a slot, but there's no waiter. Release the slot. - totalConnections.decrementAndGet(); + if (waiter == null) { + break; } - return; + + totalConnections.getAndIncrement(); + createNewConnection().addListener(future -> { + if (future.isSuccess()) { + // Try to give the new channel to the waiter. + // If it fails (e.g., the waiter timed out in the meantime), + // release the brand new channel back to the pool. + if (!waiter.trySuccess((Channel) future.getNow())) { + release(((Channel) future.getNow()).attr(POOLED_CONNECTION_KEY).get()); + } + } else { + // The connection failed, so notify the waiter. + // The connection's own close handler will decrement totalConnections again. + waiter.tryFailure(future.cause()); + } + }); } - // CAS failed, another thread acted. Loop to retry. + } finally { + poolLock.unlock(); } } + // private void createNewConnectionForWaiter() { + // Promise waiter = pollNextWaiter(); + // if (waiter == null) { + // totalConnections.getAndDecrement(); + // return; + // } + // + // createNewConnection().addListener(future -> { + // if (future.isSuccess()) { + // if (!waiter.trySuccess((Channel) future.getNow())) { + // release(((Channel) future.getNow()).attr(POOLED_CONNECTION_KEY).get()); + // } + // } else { + // waiter.tryFailure(future.cause()); + // } + // }); + // } + private Promise pollNextWaiter() { - while (true) { + while (!pendingAcquirers.isEmpty()) { Promise waiter = pendingAcquirers.poll(); - if (waiter == null) { - return null; // Queue is empty - } if (!waiter.isCancelled()) { return waiter; } } + return null; + } + + private PooledConnection pollIdleAndCheckHealth() { + while (!idleConnections.isEmpty()) { + PooledConnection connection = idleConnections.poll(); + if (isHealthy(connection)) { + connection.idleSince = null; // Mark as active + return connection; + } + connection.close(); + } + return null; } private Future createNewConnection() { @@ -498,9 +529,9 @@ public void initChannel(Channel channel) throws SSLException { // Create the connection wrapper and attach it to the channel. new PooledConnection(channel, key); - ChannelPipeline pipeline = channel.pipeline(); + ChannelPipeline pipeline = channel.pipeline(); //TODO: fix handler names pipeline.addLast("poolHealthHandler", new PoolConnectionHealthHandler()); - + // Test whether proxying should be applied to this Channel. If so, add it. // Proxy detection MUST use the final destination address from the key. boolean hasProxy = channelInitializationProxyHandler.test(key.getFinalDestination()); @@ -526,17 +557,13 @@ public void initChannel(Channel channel) throws SSLException { newConnectionBootstrap.connect(route).addListener(future -> { if (!future.isSuccess()) { LOGGER.atError().setThrowable(future.cause()).log("Failed connection."); - totalConnections.getAndDecrement(); - satisfyWaiter(); + handleConnectionClosure(); promise.setFailure(future.cause()); return; } Channel newChannel = ((ChannelFuture) future).channel(); - newChannel.closeFuture().addListener(closeFuture -> { - totalConnections.getAndDecrement(); - satisfyWaiter(); - }); + newChannel.closeFuture().addListener(closeFuture -> handleConnectionClosure()); Runnable connectionReadyRunner = () -> { SslHandler sslHandler = newChannel.pipeline().get(SslHandler.class); @@ -555,7 +582,6 @@ public void initChannel(Channel channel) throws SSLException { }; ProxyHandler proxyHandler = (ProxyHandler) newChannel.pipeline().get(PROXY); - if (proxyHandler != null) { // Wait for the proxy handshake to complete if proxy is being used. proxyHandler.connectFuture().addListener(proxyFuture -> { diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index 97d6273bbd64..adb305a20c29 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -74,7 +74,8 @@ public void channelInactive(ChannelHandlerContext ctx) { if (latch != null) { latch.countDown(); } - ctx.channel().eventLoop().execute(() -> cleanup(ctx, true)); + cleanup(ctx, true); + ctx.fireChannelInactive(); } @Override diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 4b915a53052d..776b5b778642 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -6,7 +6,6 @@ import io.clientcore.core.http.models.HttpHeaders; import io.clientcore.core.http.models.HttpRequest; import io.clientcore.core.http.models.Response; -import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandler; import io.netty.channel.ChannelInboundHandlerAdapter; @@ -16,6 +15,7 @@ import io.netty.handler.codec.http.HttpObject; import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.LastHttpContent; +import io.netty.util.ReferenceCountUtil; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -142,28 +142,41 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception if (msg instanceof FullHttpResponse) { complete = true; - ByteBuf content = ((FullHttpResponse) msg).content(); - readByteBufIntoOutputStream(content, eagerContent); + FullHttpResponse fullHttpResponse = (FullHttpResponse) msg; + try { + readByteBufIntoOutputStream(fullHttpResponse.content(), eagerContent); + } finally { + fullHttpResponse.release(); + } } - return; } if (msg instanceof LastHttpContent) { complete = true; - ByteBuf content = ((LastHttpContent) msg).content(); - readByteBufIntoOutputStream(content, eagerContent); + LastHttpContent lastHttpContent = (LastHttpContent) msg; + try { + readByteBufIntoOutputStream(lastHttpContent.content(), eagerContent); + } finally { + lastHttpContent.release(); + } return; } if (!started) { - // Haven't received the HttpResponse, discard this message. + // This is an HttpContent that arrived before the HttpResponse. + // It's unexpected, so we release it and discard it. + ReferenceCountUtil.release(msg); return; } if (msg instanceof HttpContent) { - ByteBuf content = ((HttpContent) msg).content(); - readByteBufIntoOutputStream(content, eagerContent); + HttpContent httpContent = (HttpContent) msg; + try { + readByteBufIntoOutputStream(httpContent.content(), eagerContent); + } finally { + httpContent.release(); + } } } @@ -175,12 +188,11 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { ctx.fireChannelReadComplete(); return; } + ctx.fireChannelReadComplete(); responseReference.set(new ResponseStateInfo(ctx.channel(), complete, statusCode, headers, eagerContent, ResponseBodyHandling.getBodyHandling(request, headers), isHttp2)); latch.countDown(); - - ctx.fireChannelReadComplete(); } @Override diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java index 771c7092e4f7..70d2c50def02 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java @@ -148,10 +148,10 @@ static void readByteBufIntoOutputStream(ByteBuf byteBuf, OutputStream stream) th } byteBuf.readBytes(stream, byteBuf.readableBytes()); - if (byteBuf.refCnt() > 0) { - // Release the ByteBuf as we've consumed it. - byteBuf.release(); - } + // if (byteBuf.refCnt() > 0) { + // // Release the ByteBuf as we've consumed it. + // byteBuf.release(); + // } } /** @@ -558,7 +558,6 @@ private static io.netty.handler.codec.http.HttpRequest toNettyHttpRequest(HttpRe return new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, nettyMethod, uri, bodyBytes, nettyHeaders, trailersFactory().newHeaders()); } else { - nettyHeaders.getCoreHeaders().set(HttpHeaderName.TRANSFER_ENCODING, "chunked"); return new DefaultHttpRequest(HttpVersion.HTTP_1_1, nettyMethod, uri, nettyHeaders); } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java index 9fb189ccd741..42bef6550f82 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java @@ -47,6 +47,9 @@ public static void startTestServer() { @AfterAll public static void stopTestServer() { + if (HTTP_CLIENT_INSTANCE instanceof NettyHttpClient) { + ((NettyHttpClient) HTTP_CLIENT_INSTANCE).close(); + } if (server != null) { server.stop(); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java index e31381e1edaa..82c53989ce5a 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java @@ -20,15 +20,20 @@ @Timeout(value = 3, unit = TimeUnit.MINUTES) public class NettyHttpClientHttpClientTests extends HttpClientTests { private static LocalTestServer server; + private static HttpClient client; @BeforeAll public static void startTestServer() { server = HttpClientTestsServer.getHttpClientTestsServer(HttpProtocolVersion.HTTP_1_1, false); server.start(); + client = new NettyHttpClientBuilder().build(); } @AfterAll public static void stopTestServer() { + if (client instanceof NettyHttpClient) { + ((NettyHttpClient) client).close(); + } if (server != null) { server.stop(); } @@ -47,6 +52,6 @@ protected String getServerUri(boolean secure) { @Override protected HttpClient getHttpClient() { - return new NettyHttpClientBuilder().build(); + return client; } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java index 38a00845ce1a..9fb62774c137 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java @@ -40,6 +40,9 @@ public static void startTestServer() { @AfterAll public static void stopTestServer() { + if (HTTP_CLIENT_INSTANCE instanceof NettyHttpClient) { + ((NettyHttpClient) HTTP_CLIENT_INSTANCE).close(); + } if (server != null) { server.stop(); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java index fdf4ff5b7668..b78d706dba61 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java @@ -42,11 +42,8 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; -import java.io.OutputStream; import java.io.PrintWriter; import java.io.StringWriter; -import java.net.ServerSocket; -import java.net.Socket; import java.net.URI; import java.nio.channels.Channels; import java.nio.channels.ClosedChannelException; @@ -444,46 +441,6 @@ public void sendWithServerSentEventsAndNoListenerThrows() { } } - @Test - public void malformedContentLengthIsIgnored() throws IOException { - String rawResponse = "HTTP/1.1 200 OK\r\n" + "Content-Type: application/octet-stream\r\n" - + "Content-Length: not-a-number\r\n" + "\r\n"; - - try (ServerSocket serverSocket = new ServerSocket(0)) { - int port = serverSocket.getLocalPort(); - URI url = URI.create("http://localhost:" + port); - - Thread clientThread = new Thread(() -> { - NettyHttpClient client = null; - try { - client = (NettyHttpClient) new NettyHttpClientBuilder().build(); - HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(url); - try (Response response = client.send(request)) { - assertEquals(200, response.getStatusCode()); - TestUtils.assertArraysEqual(SHORT_BODY, response.getValue().toBytes()); - } - } finally { - if (client != null) { - client.close(); - } - } - }); - clientThread.start(); - - try (Socket socket = serverSocket.accept()) { - OutputStream out = socket.getOutputStream(); - out.write(rawResponse.getBytes(StandardCharsets.UTF_8)); - out.write(SHORT_BODY); - out.flush(); - } - - clientThread.join(10000); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - @Test public void nonPooledClientSendsRequestSuccessfully() { HttpClient client = new NettyHttpClientBuilder().connectionPoolSize(0).build(); diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java index 0826e73d67f9..7cd1d05ef7a5 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java @@ -24,8 +24,18 @@ import java.net.SocketAddress; import java.time.Duration; import java.util.ArrayList; +import java.util.Deque; import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -37,6 +47,7 @@ import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; /** * Tests for {@link Netty4ConnectionPool}. @@ -319,4 +330,69 @@ public void testSeparatePoolsForSeparateRemoteAddresses() throws IOException { route2Server.stop(); } } + + @Test + public void poolDoesNotDeadlockAndRecoversCleanlyUnderSaturation() throws InterruptedException, IOException { + final int poolSize = 10; + final int numThreads = 20; + final int numTasks = 100; + + final CountDownLatch latch = new CountDownLatch(numTasks); + final AtomicInteger successCounter = new AtomicInteger(0); + final Queue exceptions = new ConcurrentLinkedQueue<>(); + final ExecutorService executor = Executors.newFixedThreadPool(numThreads); + + try (Netty4ConnectionPool pool + = createPool(poolSize, Duration.ofSeconds(10), null, Duration.ofSeconds(2), numTasks)) { + for (int i = 0; i < numTasks; i++) { + executor.submit(() -> { + try { + Channel channel = pool.acquire(connectionPoolKey, false).awaitUninterruptibly().getNow(); + + // Hold the connection for a short, random time to simulate work. + Thread.sleep(ThreadLocalRandom.current().nextInt(10, 50)); + + pool.release(channel); + successCounter.incrementAndGet(); + } catch (Throwable t) { + exceptions.add(t); + } finally { + latch.countDown(); + } + }); + } + + assertTrue(latch.await(30, TimeUnit.SECONDS), "Test deadlocked, not all tasks completed."); + + executor.shutdown(); + assertTrue(executor.awaitTermination(5, TimeUnit.SECONDS)); + + if (!exceptions.isEmpty()) { + fail("Test tasks threw exceptions: " + + exceptions.stream().map(Throwable::getMessage).collect(Collectors.joining(", "))); + } + assertEquals(numTasks, successCounter.get(), "Mismatch in the number of successful tasks."); + + // Use reflection to check the final state of the pool's queues. + assertDoesNotThrow(() -> { + Field poolField = Netty4ConnectionPool.class.getDeclaredField("pool"); + poolField.setAccessible(true); + @SuppressWarnings("unchecked") + ConcurrentMap routePools + = (ConcurrentMap) poolField.get(pool); + Netty4ConnectionPool.PerRoutePool perRoutePool = routePools.get(connectionPoolKey); + + Field idleField = Netty4ConnectionPool.PerRoutePool.class.getDeclaredField("idleConnections"); + idleField.setAccessible(true); + Deque idleConnections = (Deque) idleField.get(perRoutePool); + + Field pendingField = Netty4ConnectionPool.PerRoutePool.class.getDeclaredField("pendingAcquirers"); + pendingField.setAccessible(true); + Deque pendingAcquirers = (Deque) pendingField.get(perRoutePool); + + assertEquals(poolSize, idleConnections.size(), "Pool should be full of idle connections."); + assertTrue(pendingAcquirers.isEmpty(), "Pending acquirers queue should be empty."); + }); + } + } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java index 202172f95eef..3b273f9eee43 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java @@ -12,6 +12,8 @@ import io.netty.channel.AbstractChannel; import io.netty.channel.Channel; import io.netty.channel.ChannelConfig; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelMetadata; import io.netty.channel.ChannelOutboundBuffer; import io.netty.channel.ChannelPromise; @@ -241,8 +243,10 @@ public void toBytesDrainsFromLiveChannel() throws IOException { @Test public void toBytesThrowsIfChannelErrors() { IOException testException = new IOException("test error"); - Channel channel - = createChannelWithReadHandling((ignored, ch) -> ch.pipeline().fireExceptionCaught(testException)); + Channel channel = createChannelWithReadHandling((ignored, ch) -> { + ch.pipeline().addLast(new ExceptionSuppressingHandler()); + ch.pipeline().fireExceptionCaught(testException); + }); Netty4ChannelBinaryData binaryData = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), channel, 10L, false); @@ -418,12 +422,12 @@ public ChannelConfig config() { @Override public boolean isOpen() { - return true; + return !closeCalled.get(); } @Override public boolean isActive() { - return true; + return !closeCalled.get(); } @Override @@ -432,6 +436,12 @@ public ChannelMetadata metadata() { } } + private static final class ExceptionSuppressingHandler extends ChannelInboundHandlerAdapter { + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + } + } + private static Channel channelWithNoData() { return createChannelWithReadHandling((ignored, channel) -> { Netty4EagerConsumeChannelHandler handler = channel.pipeline().get(Netty4EagerConsumeChannelHandler.class); From 6855d45120cb443da4a894a2f5cb1c1b10041b84 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Sat, 19 Jul 2025 13:33:56 +0300 Subject: [PATCH 52/57] documentation fix & pr comments --- .../http/netty4/NettyHttpClient.java | 95 +++++----- .../http/netty4/NettyHttpClientBuilder.java | 10 +- .../implementation/Netty4AlpnHandler.java | 14 -- .../Netty4ChannelBinaryData.java | 58 +++++- .../implementation/Netty4ConnectionPool.java | 174 ++++++++++-------- .../Netty4EagerConsumeChannelHandler.java | 1 - .../Netty4PipelineCleanupHandler.java | 15 +- .../implementation/Netty4ResponseHandler.java | 2 +- .../netty4/implementation/Netty4Utility.java | 46 ++--- .../http/netty4/NettyHttpClientTests.java | 83 +++++++++ .../HttpResponseDrainsBufferTests.java | 11 +- .../Netty4ConnectionPoolTests.java | 9 +- ...Netty4EagerConsumeChannelHandlerTests.java | 12 +- .../Netty4Http11ChannelBinaryDataTests.java | 99 ++++++++++ .../Netty4PipelineCleanupHandlerTests.java | 31 +--- 15 files changed, 449 insertions(+), 211 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index a9a5fcfc425c..fa55e5ac56c9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -79,6 +79,8 @@ */ class NettyHttpClient implements HttpClient { private static final ClientLogger LOGGER = new ClientLogger(NettyHttpClient.class); + private static final Netty4ConnectionPool.Http2GoAwayHandler HTTP_2_GO_AWAY_HANDLER + = new Netty4ConnectionPool.Http2GoAwayHandler(); /** * Error message for when no {@link ServerSentEventListener} is attached to the {@link HttpRequest}. @@ -147,7 +149,7 @@ private Response sendWithConnectionPool(HttpRequest request) { final Channel channel = future.getNow(); try { - configurePooledRequestPipeline(channel, request, responseReference, errorReference, latch, isHttps); + configurePooledRequestPipeline(channel, request, responseReference, errorReference, latch, isHttps, uri.getHost(), port); } catch (Exception e) { // An exception occurred during the pipeline setup. // We fire the exception through the pipeline to trigger the cleanup handler, @@ -162,16 +164,18 @@ private Response sendWithConnectionPool(HttpRequest request) { awaitLatch(latch); - if (errorReference.get() != null) { - throw LOGGER.throwableAtError().log(errorReference.get(), CoreException::from); + ResponseStateInfo info = responseReference.get(); + if (info != null) { + return createResponse(request, info); } - ResponseStateInfo info = responseReference.get(); - if (info == null) { + if (errorReference.get() != null) { throw LOGGER.throwableAtError().log(errorReference.get(), CoreException::from); + } else { + throw LOGGER.throwableAtError() + .log("The request latch was released without a response or an error being set.", + IllegalStateException::new); } - - return createResponse(request, info); } private Response sendWithoutConnectionPool(HttpRequest request) { @@ -311,7 +315,7 @@ protected void initChannel(Channel channel) throws SSLException { private void configurePooledRequestPipeline(Channel channel, HttpRequest request, AtomicReference responseReference, AtomicReference errorReference, - CountDownLatch latch, boolean isHttps) { + CountDownLatch latch, boolean isHttps, String host, int port) { ReentrantLock lock = channel.attr(Netty4ConnectionPool.CHANNEL_LOCK).get(); lock.lock(); @@ -344,55 +348,52 @@ private void configurePooledRequestPipeline(Channel channel, HttpRequest request ChannelPipeline pipeline = channel.pipeline(); pipeline.addLast(PIPELINE_CLEANUP, - new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, pipelineOwnerToken)); + new Netty4PipelineCleanupHandler(connectionPool, errorReference, pipelineOwnerToken)); - // Only add CoreProgressAndTimeoutHandler if it will do anything, ex it is reporting progress or is - // applying timeouts. - // This is done to keep the ChannelPipeline shorter, therefore more performant if this - // effectively is a no-op. - if (addProgressAndTimeoutHandler) { - pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, - writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); - } + HttpProtocolVersion protocol = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); - // The SslHandler is already in the pipeline if this is an HTTPS request, as it's added - // by the connection pool during the initial connection setup. The SSL handshake is also - // guaranteed to be complete by the time we get the channel because the Netty4AlpnHandler - // reacts to the result of the ALPN negotiation that happened during the SSL handshake. - if (isHttps) { - HttpProtocolVersion protocolVersion = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); - if (protocolVersion != null) { - // The Connection is being reused, ALPN is already done. - // Manually configure the pipeline based on the stored protocol. - boolean isHttp2 = protocolVersion == HttpProtocolVersion.HTTP_2; - pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); - - if (!isHttp2 && pipeline.get(HTTP_CODEC) == null) { - pipeline.addBefore(HTTP_RESPONSE, HTTP_CODEC, createCodec()); - } + if (protocol != null) { + if (addProgressAndTimeoutHandler) { + pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); + } + pipeline.addLast(HTTP_RESPONSE, new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + + if (protocol == HttpProtocolVersion.HTTP_1_1) { + String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; + pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); + } - if (isHttp2) { + channel.eventLoop().execute(() -> { + if (protocol == HttpProtocolVersion.HTTP_2) { sendHttp2Request(request, channel, errorReference, latch); } else { send(request, channel, errorReference, latch); } - } else { - // This is a new connection, let ALPN do the work. - // For HTTPS, we delegate the addition of the response handler and codec to the ALPN handler. - pipeline.addAfter(SSL, ALPN, - new Netty4AlpnHandler(request, responseReference, errorReference, latch)); + }); + } else { + if (addProgressAndTimeoutHandler) { + pipeline.addLast(PROGRESS_AND_TIMEOUT, + new Netty4ProgressAndTimeoutHandler(progressReporter, writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); + } + + if (isHttps) { + SslContext sslContext = buildSslContext(maximumHttpVersion, sslContextModifier); + SslHandler sslHandler = sslContext.newHandler(channel.alloc(), host, port); + pipeline.addFirst(SSL, sslHandler); + pipeline.addAfter(SSL, ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, latch)); + channel.writeAndFlush(Unpooled.EMPTY_BUFFER); + } else { + Netty4ResponseHandler responseHandler = new Netty4ResponseHandler(request, responseReference, errorReference, latch); + pipeline.addLast(HTTP_RESPONSE, responseHandler); + String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; + pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); + + send(request, channel, errorReference, latch); } - } else { - // If there isn't an SslHandler, we can send the request immediately. - // Add the HTTP/1.1 codec, as we only support HTTP/2 when using SSL ALPN. - pipeline.addLast(HTTP_RESPONSE, - new Netty4ResponseHandler(request, responseReference, errorReference, latch)); - String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; - pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); - send(request, channel, errorReference, latch); } + } catch (SSLException e) { + throw new RuntimeException(e); } finally { lock.unlock(); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 8fe13c372930..96c3d432c874 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -138,7 +138,7 @@ private static Class getChannelClass(String className) // --- Connection Pool Configuration --- private int connectionPoolSize = 1000; - private Duration connectionIdleTimeout = Duration.ofSeconds(50); + private Duration connectionIdleTimeout = Duration.ofSeconds(60); private Duration maxConnectionLifetime; private Duration pendingAcquireTimeout = Duration.ofSeconds(60); // Default wait time for a connection private int maxPendingAcquires = 10_000; // Default pending queue size @@ -326,9 +326,10 @@ public NettyHttpClientBuilder connectionIdleTimeout(Duration connectionIdleTimeo /** * Sets the maximum time a connection is allowed to exist. *

- * After this time, the connection will be closed upon release. + * By default, connections have no lifetime limit and can be used indefinitely. *

- * A {@link Duration} of zero or less, or a null value, will result in connections having no lifetime limit. + * After this time is met or exceeded, the connection will be closed upon release. A {@link Duration} of zero or + * less, or a null value, will also result in connections having no lifetime limit. * * @param maxConnectionLifetime The maximum connection lifetime. * @return The updated builder. @@ -340,7 +341,8 @@ public NettyHttpClientBuilder maxConnectionLifetime(Duration maxConnectionLifeti /** * Sets the maximum time to wait for a connection from the pool. - * + *

+ * If not set, a default value of 60 seconds is used. * @param pendingAcquireTimeout The timeout for pending acquires. * @return The updated builder. */ diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java index 8edf5185bd8b..0a869b84f761 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4AlpnHandler.java @@ -33,8 +33,6 @@ public final class Netty4AlpnHandler extends ApplicationProtocolNegotiationHandl */ public static final AttributeKey HTTP_PROTOCOL_VERSION_KEY = AttributeKey.valueOf("http-protocol-version"); - private static final Netty4ConnectionPool.Http2GoAwayHandler GO_AWAY_HANDLER - = new Netty4ConnectionPool.Http2GoAwayHandler(); private final HttpRequest request; private final AtomicReference responseReference; @@ -74,15 +72,10 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { return; } - // Store the negotiated protocol for connection reuse. ctx.channel().attr(HTTP_PROTOCOL_VERSION_KEY).set(protocolVersion); configureHttpsPipeline(ctx.pipeline(), request, protocolVersion, responseReference, errorReference, latch); - if (protocolVersion == HttpProtocolVersion.HTTP_2) { - ctx.pipeline().addLast(GO_AWAY_HANDLER); - } - if (protocolVersion == HttpProtocolVersion.HTTP_2) { sendHttp2Request(request, ctx.channel(), errorReference, latch); } else { @@ -102,11 +95,4 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { ctx.pipeline().remove(this); } } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - setOrSuppressError(errorReference, cause); - ctx.fireExceptionCaught(cause); - latch.countDown(); - } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java index 66affc648ed9..f1c8500dbb39 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ChannelBinaryData.java @@ -123,10 +123,50 @@ public void writeTo(JsonWriter jsonWriter) { @Override public void writeTo(OutputStream outputStream) { Objects.requireNonNull(outputStream, "'outputStream' cannot be null."); + try { - outputStream.write(toBytes()); + if (bytes != null) { + outputStream.write(bytes); + return; + } + + if (streamDrained.compareAndSet(false, true)) { + try { + // Channel hasn't been read yet, don't buffer it, just write it to the OutputStream as it's being read. + if (eagerContent != null && eagerContent.size() > 0) { + eagerContent.writeTo(outputStream); + } + + CountDownLatch latch = new CountDownLatch(1); + Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, + buf -> buf.readBytes(outputStream, buf.readableBytes()), isHttp2); + channel.pipeline().addLast(Netty4HandlerNames.EAGER_CONSUME, handler); + channel.config().setAutoRead(true); + + awaitLatch(latch); + + Throwable exception = handler.channelException(); + if (exception != null) { + if (exception instanceof Error) { + throw (Error) exception; + } else { + throw CoreException.from(exception); + } + } + } finally { + eagerContent = null; + drainLatch.countDown(); + + if (onClose != null) { + onClose.run(); + } + } + } else { + throw LOGGER.throwableAtError() + .log("The stream has already been consumed and is not replayable.", IllegalStateException::new); + } } catch (IOException e) { - throw LOGGER.throwableAtError().log(e, CoreException::from); + throw CoreException.from(e); } } @@ -207,8 +247,15 @@ private void drainStream() { } if (!channel.isActive()) { - bytes = (eagerContent == null) ? new byte[0] : eagerContent.toByteArray(); - return; + // The connection was closed before we could read the full body. + // Check if, by chance, the eager content we already have satisfies the full length. + // This can happen if the body was very small and the server closed immediately. + if (length != null && eagerContent != null && length == eagerContent.size()) { + bytes = eagerContent.toByteArray(); + return; + } + throw LOGGER.throwableAtError() + .log("Connection closed prematurely while reading response body.", IOException::new); } CountDownLatch ioLatch = new CountDownLatch(1); @@ -219,7 +266,6 @@ private void drainStream() { channel.config().setAutoRead(true); awaitLatch(ioLatch); - bytes = eagerContent.toByteArray(); Throwable exception = handler.channelException(); if (exception != null) { @@ -231,6 +277,8 @@ private void drainStream() { } else { bytes = eagerContent.toByteArray(); } + } catch (IOException e) { + throw LOGGER.throwableAtError().log(e, CoreException::from); } finally { eagerContent = null; drainLatch.countDown(); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index c44f4e8f71af..021cc3da143a 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -8,6 +8,7 @@ import io.clientcore.core.models.CoreException; import io.clientcore.core.utils.AuthenticateChallenge; import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.Unpooled; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelHandler; @@ -20,6 +21,8 @@ import io.netty.handler.codec.http2.Http2GoAwayFrame; import io.netty.handler.proxy.HttpProxyHandler; import io.netty.handler.proxy.ProxyHandler; +import io.netty.handler.ssl.ApplicationProtocolNames; +import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler; import io.netty.handler.ssl.SslCloseCompletionEvent; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslContextBuilder; @@ -51,10 +54,12 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_CODEC; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL_INITIALIZER; import static io.clientcore.http.netty4.implementation.Netty4Utility.buildSslContext; +import static io.clientcore.http.netty4.implementation.Netty4Utility.createHttp2Codec; /** * A pool of Netty channels that can be reused for requests to the same remote address. @@ -112,6 +117,9 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception @ChannelHandler.Sharable public static final class SuppressProxyConnectExceptionWarningHandler extends ChannelInboundHandlerAdapter { + private static final SuppressProxyConnectExceptionWarningHandler INSTANCE + = new SuppressProxyConnectExceptionWarningHandler(); + @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { if (cause instanceof HttpProxyHandler.HttpProxyConnectException) { @@ -123,6 +131,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { @ChannelHandler.Sharable private static class SslGracefulShutdownHandler extends ChannelInboundHandlerAdapter { + private static final SslGracefulShutdownHandler INSTANCE = new SslGracefulShutdownHandler(); + @Override public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { if (evt instanceof SslCloseCompletionEvent) { @@ -134,6 +144,8 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc @ChannelHandler.Sharable private static class PoolConnectionHealthHandler extends ChannelInboundHandlerAdapter { + private static final PoolConnectionHealthHandler INSTANCE = new PoolConnectionHealthHandler(); + private static final AttributeKey CONNECTION_INVALIDATED = AttributeKey.valueOf("connection-invalidated"); @@ -161,7 +173,33 @@ private void invalidateAndClose(Channel channel) { } } - private static final SslGracefulShutdownHandler SSL_GRACEFUL_SHUTDOWN_HANDLER = new SslGracefulShutdownHandler(); + @ChannelHandler.Sharable + private static class PoolAlpnConfigurator extends ApplicationProtocolNegotiationHandler { + private static final PoolAlpnConfigurator INSTANCE = new PoolAlpnConfigurator(); + + PoolAlpnConfigurator() { + super(ApplicationProtocolNames.HTTP_1_1); + } + + @Override + protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { + HttpProtocolVersion protocolVersion; + if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { + protocolVersion = HttpProtocolVersion.HTTP_2; + } else { + protocolVersion = HttpProtocolVersion.HTTP_1_1; + } + + ctx.channel().attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).set(protocolVersion); + + ctx.pipeline().remove(this); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + ctx.fireExceptionCaught(cause); + } + } public Netty4ConnectionPool(Bootstrap bootstrap, ChannelInitializationProxyHandler channelInitializationProxyHandler, @@ -203,7 +241,10 @@ public Netty4ConnectionPool(Bootstrap bootstrap, */ public Future acquire(Netty4ConnectionPoolKey key, boolean isHttps) { if (closed.get()) { - throw LOGGER.throwableAtError().log(CLOSED_POOL_ERROR_MESSAGE, IllegalStateException::new); + return bootstrap.config() + .group() + .next() + .newFailedFuture(new IllegalStateException(CLOSED_POOL_ERROR_MESSAGE)); } PerRoutePool perRoutePool = pool.computeIfAbsent(key, k -> new PerRoutePool(k, isHttps)); @@ -406,26 +447,19 @@ void release(PooledConnection connection) { poolLock.lock(); try { if (!isHealthy(connection)) { - // If the connection is unhealthy, it cannot be used by any waiter. + // The connection is unhealthy. Close it. + // The asynchronous closeFuture listener ('handleConnectionClosure') will eventually + // decrement the connection count and create a new connections for available waiters. connection.close(); - - // Now, fail all pending waiters because this release operation cannot satisfy them. - // The close listener on the connection will attempt to create a new connection for *one* - // subsequent waiter, but we should proactively fail all current waiters to avoid hangs. - Promise waiter; - while ((waiter = pendingAcquirers.poll()) != null) { - if (!waiter.isCancelled()) { - waiter.tryFailure(new IOException("Released connection was unhealthy.")); - } - } - return; // Exit after handling the unhealthy connection. + return; } // The connection is healthy. Offer it to the waiters. while (!pendingAcquirers.isEmpty()) { - Promise waiter = pendingAcquirers.poll(); - if (waiter.isCancelled()) { - continue; + Promise waiter = pollNextWaiter(); + if (waiter == null) { + // All remaining waiters were cancelled. + break; } if (waiter.trySuccess(connection.channel)) { @@ -478,24 +512,6 @@ private void handleConnectionClosure() { } } - // private void createNewConnectionForWaiter() { - // Promise waiter = pollNextWaiter(); - // if (waiter == null) { - // totalConnections.getAndDecrement(); - // return; - // } - // - // createNewConnection().addListener(future -> { - // if (future.isSuccess()) { - // if (!waiter.trySuccess((Channel) future.getNow())) { - // release(((Channel) future.getNow()).attr(POOLED_CONNECTION_KEY).get()); - // } - // } else { - // waiter.tryFailure(future.cause()); - // } - // }); - // } - private Promise pollNextWaiter() { while (!pendingAcquirers.isEmpty()) { Promise waiter = pendingAcquirers.poll(); @@ -530,7 +546,7 @@ public void initChannel(Channel channel) throws SSLException { new PooledConnection(channel, key); ChannelPipeline pipeline = channel.pipeline(); //TODO: fix handler names - pipeline.addLast("poolHealthHandler", new PoolConnectionHealthHandler()); + pipeline.addLast("poolHealthHandler", PoolConnectionHealthHandler.INSTANCE); // Test whether proxying should be applied to this Channel. If so, add it. // Proxy detection MUST use the final destination address from the key. @@ -539,18 +555,19 @@ public void initChannel(Channel channel) throws SSLException { ProxyHandler proxyHandler = channelInitializationProxyHandler.createProxy(proxyChallenges); pipeline.addFirst(PROXY, proxyHandler); pipeline.addAfter(PROXY, "clientcore.suppressproxyexception", - new SuppressProxyConnectExceptionWarningHandler()); + SuppressProxyConnectExceptionWarningHandler.INSTANCE); } // Add SSL handling if the request is HTTPS. - if (isHttps) { - InetSocketAddress inetSocketAddress = (InetSocketAddress) key.getFinalDestination(); - SslContext ssl = buildSslContext(maximumHttpVersion, sslContextModifier); - pipeline.addLast(SSL, ssl.newHandler(channel.alloc(), inetSocketAddress.getHostString(), - inetSocketAddress.getPort())); - pipeline.addAfter(SSL, "clientcore.sslshutdown", SSL_GRACEFUL_SHUTDOWN_HANDLER); - pipeline.addLast(SSL_INITIALIZER, new Netty4SslInitializationHandler()); - } +// if (isHttps) { +// InetSocketAddress inetSocketAddress = (InetSocketAddress) key.getFinalDestination(); +// SslContext ssl = buildSslContext(maximumHttpVersion, sslContextModifier); +// pipeline.addLast(SSL, ssl.newHandler(channel.alloc(), inetSocketAddress.getHostString(), +// inetSocketAddress.getPort())); +// pipeline.addAfter(SSL, "clientcore.sslshutdown", SslGracefulShutdownHandler.INSTANCE); +// //pipeline.addLast(SSL_INITIALIZER, new Netty4SslInitializationHandler()); +// pipeline.addLast("pool-alpn-configurator", PoolAlpnConfigurator.INSTANCE); +// } } }); @@ -565,40 +582,53 @@ public void initChannel(Channel channel) throws SSLException { Channel newChannel = ((ChannelFuture) future).channel(); newChannel.closeFuture().addListener(closeFuture -> handleConnectionClosure()); - Runnable connectionReadyRunner = () -> { - SslHandler sslHandler = newChannel.pipeline().get(SslHandler.class); - if (sslHandler != null) { - sslHandler.handshakeFuture().addListener(sslFuture -> { - if (sslFuture.isSuccess()) { - promise.setSuccess(newChannel); - } else { - promise.setFailure(sslFuture.cause()); - newChannel.close(); - } - }); - } else { - promise.setSuccess(newChannel); - } - }; - +// Runnable connectionReadyRunner = () -> { +// SslHandler sslHandler = newChannel.pipeline().get(SslHandler.class); +// if (sslHandler != null) { +// sslHandler.handshakeFuture().addListener(sslFuture -> { +// if (sslFuture.isSuccess()) { +// promise.setSuccess(newChannel); +// } else { +// promise.setFailure(sslFuture.cause()); +// newChannel.close(); +// } +// }); +// newChannel.writeAndFlush(Unpooled.EMPTY_BUFFER); +// } else { +// promise.setSuccess(newChannel); +// } +// }; + +// ProxyHandler proxyHandler = (ProxyHandler) newChannel.pipeline().get(PROXY); +// if (proxyHandler != null){ +// // Wait for the proxy handshake to complete if proxy is being used. +// proxyHandler.connectFuture().addListener(proxyFuture -> { +// if (proxyFuture.isSuccess()) { +// if (!newChannel.isActive()) { +// promise.setFailure(new ClosedChannelException()); +// newChannel.close(); +// return; +// } +// connectionReadyRunner.run(); +// } else { +// promise.setFailure(proxyFuture.cause()); +// newChannel.close(); +// } +// }); +// } else { +// connectionReadyRunner.run(); +// } ProxyHandler proxyHandler = (ProxyHandler) newChannel.pipeline().get(PROXY); if (proxyHandler != null) { - // Wait for the proxy handshake to complete if proxy is being used. proxyHandler.connectFuture().addListener(proxyFuture -> { - if (proxyFuture.isSuccess()) { - if (!newChannel.isActive()) { - promise.setFailure(new ClosedChannelException()); - newChannel.close(); - return; - } - connectionReadyRunner.run(); - } else { - promise.setFailure(proxyFuture.cause()); + if (proxyFuture.isSuccess()) promise.trySuccess(newChannel); + else { + promise.tryFailure(proxyFuture.cause()); newChannel.close(); } }); } else { - promise.setSuccess(newChannel); + promise.trySuccess(newChannel); } }); return promise; diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java index 60b969aac13e..061e66b65dee 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandler.java @@ -99,7 +99,6 @@ public void channelReadComplete(ChannelHandlerContext ctx) { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { this.exception = cause; - ctx.fireExceptionCaught(cause); signalComplete(ctx); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java index adb305a20c29..ece4bd9e77c0 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandler.java @@ -11,7 +11,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; @@ -33,7 +32,6 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { private final Netty4ConnectionPool connectionPool; private final AtomicReference errorReference; - private final CountDownLatch latch; private final AtomicBoolean cleanedUp = new AtomicBoolean(false); private final Object pipelineOwnerToken; @@ -52,28 +50,21 @@ public class Netty4PipelineCleanupHandler extends ChannelDuplexHandler { } public Netty4PipelineCleanupHandler(Netty4ConnectionPool connectionPool, AtomicReference errorReference, - CountDownLatch latch, Object pipelineOwnerToken) { + Object pipelineOwnerToken) { this.connectionPool = connectionPool; this.errorReference = errorReference; - this.latch = latch; this.pipelineOwnerToken = pipelineOwnerToken; } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - if (errorReference != null && latch != null) { - setOrSuppressError(errorReference, cause); - latch.countDown(); - } + setOrSuppressError(errorReference, cause); cleanup(ctx, true); + ctx.fireExceptionCaught(cause); } @Override public void channelInactive(ChannelHandlerContext ctx) { - ctx.fireChannelInactive(); - if (latch != null) { - latch.countDown(); - } cleanup(ctx, true); ctx.fireChannelInactive(); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index 776b5b778642..dae3b8457f16 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -77,7 +77,7 @@ public Netty4ResponseHandler(HttpRequest request, AtomicReference * Content will only be written to the {@link OutputStream} if the {@link ByteBuf} is non-null and is * {@link ByteBuf#isReadable()}. The entire {@link ByteBuf} will be consumed. + *

+ * This method does NOT release the {@link ByteBuf} if it was consumed. * * @param byteBuf The Netty {@link ByteBuf} to read from. * @param stream The {@link OutputStream} to write to. @@ -148,10 +150,6 @@ static void readByteBufIntoOutputStream(ByteBuf byteBuf, OutputStream stream) th } byteBuf.readBytes(stream, byteBuf.readableBytes()); - // if (byteBuf.refCnt() > 0) { - // // Release the ByteBuf as we've consumed it. - // byteBuf.release(); - // } } /** @@ -480,24 +478,7 @@ public static void configureHttpsPipeline(ChannelPipeline pipeline, HttpRequest AtomicReference errorReference, CountDownLatch latch) { final ChannelHandler httpCodec; if (HttpProtocolVersion.HTTP_2 == protocol) { - // TODO (alzimmer): InboundHttp2ToHttpAdapter buffers the entire response into a FullHttpResponse. Need to - // create a streaming version of this to support huge response payloads. - Http2Connection http2Connection = new DefaultHttp2Connection(false); - Http2Settings settings = new Http2Settings().headerTableSize(4096) - .maxHeaderListSize(TWO_FIFTY_SIX_KB) - .pushEnabled(false) - .initialWindowSize(TWO_FIFTY_SIX_KB); - Http2FrameListener frameListener = new DelegatingDecompressorFrameListener(http2Connection, - new InboundHttp2ToHttpAdapterBuilder(http2Connection).maxContentLength(Integer.MAX_VALUE) - .propagateSettings(true) - .validateHttpHeaders(true) - .build()); - - httpCodec = new HttpToHttp2ConnectionHandlerBuilder().initialSettings(settings) - .frameListener(frameListener) - .connection(http2Connection) - .validateHeaders(true) - .build(); + httpCodec = createHttp2Codec(); } else { // HTTP/1.1 httpCodec = createCodec(); } @@ -515,6 +496,27 @@ public static void configureHttpsPipeline(ChannelPipeline pipeline, HttpRequest } } + public static ChannelHandler createHttp2Codec() { + // TODO (alzimmer): InboundHttp2ToHttpAdapter buffers the entire response into a FullHttpResponse. Need to + // create a streaming version of this to support huge response payloads. + Http2Connection http2Connection = new DefaultHttp2Connection(false); + Http2Settings settings = new Http2Settings().headerTableSize(4096) + .maxHeaderListSize(TWO_FIFTY_SIX_KB) + .pushEnabled(false) + .initialWindowSize(TWO_FIFTY_SIX_KB); + Http2FrameListener frameListener = new DelegatingDecompressorFrameListener(http2Connection, + new InboundHttp2ToHttpAdapterBuilder(http2Connection).maxContentLength(Integer.MAX_VALUE) + .propagateSettings(true) + .validateHttpHeaders(true) + .build()); + + return new HttpToHttp2ConnectionHandlerBuilder().initialSettings(settings) + .frameListener(frameListener) + .connection(http2Connection) + .validateHeaders(true) + .build(); + } + public static void sendHttp2Request(HttpRequest request, Channel channel, AtomicReference errorReference, CountDownLatch latch) { io.netty.handler.codec.http.HttpRequest nettyRequest = toNettyHttpRequest(request); diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java index b78d706dba61..0798dbd90f74 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientTests.java @@ -28,6 +28,8 @@ import io.clientcore.http.netty4.implementation.NettyHttpClientLocalTestServer; import io.netty.channel.ChannelPipeline; import io.netty.handler.proxy.ProxyConnectException; +import org.eclipse.jetty.io.EndPoint; +import org.eclipse.jetty.server.HttpConnection; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.RepeatedTest; @@ -37,6 +39,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import javax.net.ssl.SSLException; import javax.servlet.http.HttpServletResponse; import java.io.ByteArrayOutputStream; import java.io.FileOutputStream; @@ -44,9 +47,11 @@ import java.io.InputStream; import java.io.PrintWriter; import java.io.StringWriter; +import java.net.SocketException; import java.net.URI; import java.nio.channels.Channels; import java.nio.channels.ClosedChannelException; +import java.nio.channels.SocketChannel; import java.nio.channels.WritableByteChannel; import java.nio.charset.StandardCharsets; import java.nio.file.Files; @@ -452,6 +457,84 @@ public void nonPooledClientSendsRequestSuccessfully() { } } + @Test + public void nonPooledConnectionFails() { + HttpClient client = null; + try { + client = new NettyHttpClientBuilder().connectionPoolSize(0).build(); + HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri("http://localhost:1"); + + HttpClient finalClient = client; + assertThrows(CoreException.class, () -> finalClient.send(request)); + } finally { + if (client != null) { + ((NettyHttpClient) client).close(); + } + } + } + + @Test + public void sslHandshakeFails() { + LocalTestServer server = new LocalTestServer(HttpProtocolVersion.HTTP_1_1, false, + (req, res, body) -> res.setStatus(HttpServletResponse.SC_OK)); + HttpClient client = null; + try { + server.start(); + client = new NettyHttpClientBuilder().connectionPoolSize(0).build(); + + URI httpsUri = URI.create("https://localhost:" + server.getPort()); + + HttpRequest request = new HttpRequest().setMethod(HttpMethod.GET).setUri(httpsUri); + + HttpClient finalClient = client; + CoreException exception = assertThrows(CoreException.class, () -> finalClient.send(request)); + assertInstanceOf(SSLException.class, exception.getCause()); + } finally { + if (client != null) { + ((NettyHttpClient) client).close(); + } + server.stop(); + } + } + + @Test + public void requestWriteFailsWhenServerClosesConnection() { + LocalTestServer server = new LocalTestServer(HttpProtocolVersion.HTTP_1_1, false, (req, res, body) -> { + try { + // Get the underlying java.nio.SocketChannel from the Jetty connection + EndPoint endPoint = HttpConnection.getCurrentConnection().getEndPoint(); + SocketChannel channel = (SocketChannel) endPoint.getTransport(); + + // Set SO_LINGER to true with a timeout of 0 seconds. + // This forces the OS to send a TCP RST packet on close() instead of the normal FIN sequence. + channel.socket().setSoLinger(true, 0); + } catch (SocketException e) { + throw new RuntimeException(e); + } + + // Now, close the connection. This will trigger the RST. + HttpConnection.getCurrentConnection().getEndPoint().close(); + }); + + HttpClient client = null; + try { + server.start(); + client = new NettyHttpClientBuilder().connectionPoolSize(0).build(); + HttpRequest request = new HttpRequest().setMethod(HttpMethod.POST) + .setUri(URI.create(server.getUri())) + .setBody(BinaryData.fromString("test data")); + + HttpClient finalClient = client; + CoreException exception = assertThrows(CoreException.class, () -> finalClient.send(request)); + assertInstanceOf(IOException.class, exception.getCause()); + } finally { + if (client != null) { + ((NettyHttpClient) client).close(); + } + server.stop(); + } + } + private static Stream requestHeaderSupplier() { return Stream.of(Arguments.of(null, NULL_REPLACEMENT), Arguments.of("", ""), Arguments.of("aValue", "aValue")); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/HttpResponseDrainsBufferTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/HttpResponseDrainsBufferTests.java index 06959e521553..10a9e3769338 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/HttpResponseDrainsBufferTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/HttpResponseDrainsBufferTests.java @@ -8,6 +8,7 @@ import io.clientcore.core.http.models.HttpMethod; import io.clientcore.core.http.models.HttpRequest; import io.clientcore.core.http.models.Response; +import io.clientcore.core.models.CoreException; import io.clientcore.core.models.binarydata.BinaryData; import io.clientcore.core.shared.LocalTestServer; import io.clientcore.core.utils.IOExceptionCheckedConsumer; @@ -47,6 +48,8 @@ import static io.clientcore.http.netty4.implementation.NettyHttpClientLocalTestServer.LONG_BODY; import static io.clientcore.http.netty4.implementation.NettyHttpClientLocalTestServer.LONG_BODY_PATH; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertThrows; /** * Tests that closing the {@link Response} drains the network buffers. @@ -128,7 +131,11 @@ public void closeHttpResponseWithConsumingPartialBody() { @Test public void closeHttpResponseWithConsumingPartialWrite() { - runScenario(response -> response.getValue().writeTo(new ThrowingWritableByteChannel())); + RuntimeException ex = assertThrows(RuntimeException.class, + () -> runScenario(response -> response.getValue().writeTo(new ThrowingWritableByteChannel()))); + assertInstanceOf(ExecutionException.class, ex.getCause()); + assertInstanceOf(CoreException.class, ex.getCause().getCause()); + assertEquals(0, testResourceLeakDetectorFactory.getTotalReportedLeakCount()); } private static final class ThrowingWritableByteChannel implements WritableByteChannel { @@ -213,8 +220,6 @@ private void runScenario(IOExceptionCheckedConsumer> respon } catch (InterruptedException | ExecutionException ex) { throw new RuntimeException(ex); } - - assertEquals(0, testResourceLeakDetectorFactory.getTotalReportedLeakCount()); } @Test diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java index 7cd1d05ef7a5..ba84eb3e9035 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPoolTests.java @@ -45,7 +45,6 @@ import static org.junit.jupiter.api.Assertions.assertNotSame; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; @@ -293,10 +292,14 @@ public void testUnhealthyConnectionIsDiscarded() throws IOException { } @Test - public void testAcquireOnClosedPoolFails() throws IOException { + public void testAcquireOnClosedPoolFails() throws IOException, InterruptedException { Netty4ConnectionPool pool = createPool(1, Duration.ofSeconds(10), null, Duration.ofSeconds(10), 1); pool.close(); - assertThrows(IllegalStateException.class, () -> pool.acquire(connectionPoolKey, false)); + Future future = pool.acquire(connectionPoolKey, false); + future.await(); + + assertFalse(future.isSuccess()); + assertInstanceOf(IllegalStateException.class, future.cause()); } @Test diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandlerTests.java index 3442eef4d529..6eeb35a06ca0 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandlerTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4EagerConsumeChannelHandlerTests.java @@ -87,7 +87,7 @@ public void asyncDrainCallsOnComplete() { } @Test - public void consumerExceptionIsPropagated() { + public void consumerExceptionIsCapturedByHandler() { IOException testException = new IOException("test"); CountDownLatch latch = new CountDownLatch(1); Netty4EagerConsumeChannelHandler handler = new Netty4EagerConsumeChannelHandler(latch, buf -> { @@ -97,12 +97,12 @@ public void consumerExceptionIsPropagated() { EmbeddedChannel channel = new EmbeddedChannel(handler); ByteBuf content = Unpooled.wrappedBuffer(HELLO_BYTES); - IOException thrown = assertThrows(IOException.class, () -> { - channel.writeInbound(content); - channel.checkException(); - }); + channel.writeInbound(content); - assertEquals(testException, thrown); + Throwable capturedException = handler.channelException(); + + assertNotNull(capturedException); + assertEquals(testException, capturedException); assertNull(channel.pipeline().get(Netty4EagerConsumeChannelHandler.class)); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java index 3b273f9eee43..02ccf7538c9b 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4Http11ChannelBinaryDataTests.java @@ -37,6 +37,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import static io.clientcore.http.netty4.TestUtils.assertArraysEqual; import static io.clientcore.http.netty4.TestUtils.createChannelWithReadHandling; @@ -360,6 +361,104 @@ public void closeBeforeDrainingEventuallyCleansUp() throws InterruptedException "Cleanup task was not called after the channel became inactive."); } + @Test + public void testBinaryDataWithoutOnClose() { + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), new EmbeddedChannel(), 10L, false); + assertEquals(10L, binaryData.getLength()); + } + + @Test + public void writeToAlreadyDrainedStreamThrowsException() { + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), channelWithNoData(), 0L, false); + + binaryData.writeTo(new ByteArrayOutputStream()); + + assertThrows(IllegalStateException.class, () -> binaryData.writeTo(new ByteArrayOutputStream())); + } + + @Test + public void writeToThrowsWhenChannelErrors() { + IOException testException = new IOException("test writeTo error"); + Channel channel = createChannelWithReadHandling((ignored, ch) -> { + ch.pipeline().addLast(new ExceptionSuppressingHandler()); + ch.pipeline().fireExceptionCaught(testException); + }); + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), channel, 10L, false); + + CoreException exception + = assertThrows(CoreException.class, () -> binaryData.writeTo(new ByteArrayOutputStream())); + assertEquals(testException, exception.getCause()); + } + + @Test + public void writeToThrowsWhenChannelThrowsError() { + // This test covers the 'instanceof Error' branch in writeTo(OutputStream). + AssertionError testError = new AssertionError("test writeTo error"); + Channel channel = createChannelWithReadHandling((ignored, ch) -> { + ch.pipeline().addLast(new ExceptionSuppressingHandler()); + ch.pipeline().fireExceptionCaught(testError); + }); + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), channel, 10L, false); + + AssertionError error + = assertThrows(AssertionError.class, () -> binaryData.writeTo(new ByteArrayOutputStream())); + assertEquals(testError, error); + } + + @Test + public void closeIsIdempotent() { + AtomicInteger closed = new AtomicInteger(0); + Runnable cleanupTask = closed::getAndIncrement; + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), channelWithNoData(), 0L, false, cleanupTask); + + binaryData.toBytes(); + + binaryData.close(); + binaryData.close(); + + assertEquals(1, closed.get(), "Close should have been called only once"); + } + + @Test + public void toBytesThrowsOnInactiveChannelWithIncompleteBody() throws IOException { + // This test covers the case where the channel is closed but the eager content is insufficient. + byte[] eagerBytes = "eager".getBytes(StandardCharsets.UTF_8); + ByteArrayOutputStream eagerContent = new ByteArrayOutputStream(); + eagerContent.write(eagerBytes); + + TestMockChannel channel = new TestMockChannel(); + new DefaultEventLoop().register(channel); + + // The Expected length is 10, but we only have 5 bytes. + Netty4ChannelBinaryData binaryData = new Netty4ChannelBinaryData(eagerContent, channel, 10L, false); + + channel.close().awaitUninterruptibly(); + + CoreException exception = assertThrows(CoreException.class, binaryData::toBytes); + + assertInstanceOf(IOException.class, exception.getCause()); + } + + @Test + public void toBytesThrowsWhenChannelThrowsError() { + // This test covers the 'instanceof Error' branch in drainStream() used by toBytes(). + AssertionError testError = new AssertionError("test toBytes error"); + Channel channel = createChannelWithReadHandling((ignored, ch) -> { + ch.pipeline().addLast(new ExceptionSuppressingHandler()); + ch.pipeline().fireExceptionCaught(testError); + }); + Netty4ChannelBinaryData binaryData + = new Netty4ChannelBinaryData(new ByteArrayOutputStream(), channel, 10L, false); + + AssertionError error = assertThrows(AssertionError.class, binaryData::toBytes); + assertEquals(testError, error); + } + private static class TestMockChannel extends AbstractChannel { private final AtomicBoolean disconnectCalled = new AtomicBoolean(false); private final AtomicBoolean closeCalled = new AtomicBoolean(false); diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java index ffe9466ad0d5..ebb0666b6607 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4PipelineCleanupHandlerTests.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.net.SocketAddress; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; @@ -49,7 +48,6 @@ public class Netty4PipelineCleanupHandlerTests { private static final Object OBJECT = new Object(); private TestMockChannel testChannel; private AtomicReference errorReference; - private CountDownLatch latch; @BeforeEach public void setup() { @@ -59,15 +57,13 @@ public void setup() { testChannel.attr(AttributeKey.valueOf("pipeline-owner-token")).set(OBJECT); testChannel.config.setAutoRead(false); errorReference = new AtomicReference<>(); - latch = new CountDownLatch(1); } @Test public void cleanupWhenPooledAndActiveReleasesChannel() { testChannel.setActive(true); testChannel.pipeline().addLast(HTTP_CODEC, new MockChannelHandler()); - Netty4PipelineCleanupHandler handler - = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -82,8 +78,7 @@ public void cleanupWhenPooledAndActiveReleasesChannel() { @Test public void cleanupWhenForceCloseClosesChannel() { testChannel.setActive(true); - Netty4PipelineCleanupHandler handler - = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -96,7 +91,7 @@ public void cleanupWhenForceCloseClosesChannel() { @Test public void cleanupWhenNonPooledClosesChannel() { testChannel.setActive(true); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(null, errorReference, latch, OBJECT); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(null, errorReference, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -108,8 +103,7 @@ public void cleanupWhenNonPooledClosesChannel() { @Test public void cleanupWhenChannelInactiveClosesChannel() { testChannel.setActive(false); - Netty4PipelineCleanupHandler handler - = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -123,8 +117,7 @@ public void cleanupWhenChannelInactiveClosesChannel() { public void cleanupWhenHttp2PreservesHttpCodec() { testChannel.setActive(true); testChannel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).set(HttpProtocolVersion.HTTP_2); - Netty4PipelineCleanupHandler handler - = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, OBJECT); populatePipelineWithStandardHandlers(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -138,8 +131,7 @@ public void cleanupWhenHttp2PreservesHttpCodec() { @Test public void cleanupIsIdempotent() { testChannel.setActive(true); - Netty4PipelineCleanupHandler handler - = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); @@ -152,8 +144,7 @@ public void cleanupIsIdempotent() { @Test public void exceptionCaughtSetsErrorAndClosesChannel() { testChannel.setActive(true); - Netty4PipelineCleanupHandler handler - = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); Throwable testException = new IOException("Test Exception"); @@ -161,15 +152,14 @@ public void exceptionCaughtSetsErrorAndClosesChannel() { handler.exceptionCaught(ctx, testException); assertEquals(testException, errorReference.get()); - assertEquals(0, latch.getCount()); assertEquals(1, testChannel.getCloseCallCount()); verify(connectionPool, never()).release(testChannel); } @Test - public void exceptionCaughtWithNullsStillClosesChannel() { + public void exceptionCaughtStillClosesChannel() { testChannel.setActive(true); - Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, null, null, OBJECT); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, new AtomicReference<>(), OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); Throwable testException = new IOException("Test Exception"); @@ -184,8 +174,7 @@ public void exceptionCaughtWithNullsStillClosesChannel() { public void channelInactiveSchedulesAndExecutesCleanup() { testChannel.setActive(true); assertTrue(testChannel.isActive()); - Netty4PipelineCleanupHandler handler - = new Netty4PipelineCleanupHandler(connectionPool, errorReference, latch, OBJECT); + Netty4PipelineCleanupHandler handler = new Netty4PipelineCleanupHandler(connectionPool, errorReference, OBJECT); testChannel.pipeline().addLast(handler); testChannel.close(); From 814285d42495a75f9da49ea14e14f0418434e274 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Mon, 21 Jul 2025 12:51:40 +0300 Subject: [PATCH 53/57] fix ssl/alpn on pooled connections --- .../http/netty4/NettyHttpClient.java | 103 ++++----- .../http/netty4/NettyHttpClientBuilder.java | 5 + .../implementation/Netty4ConnectionPool.java | 198 +++++++++--------- .../implementation/Netty4HandlerNames.java | 25 +++ .../implementation/Netty4ResponseHandler.java | 2 +- .../Netty4PipelineCleanupHandlerTests.java | 3 +- .../Netty4ResponseHandlerTests.java | 2 +- 7 files changed, 191 insertions(+), 147 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index fa55e5ac56c9..54d5ac144721 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -60,16 +60,20 @@ import static io.clientcore.core.utils.ServerSentEventUtils.attemptRetry; import static io.clientcore.core.utils.ServerSentEventUtils.processTextEventStream; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.ALPN; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP2_GOAWAY; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_CODEC; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_RESPONSE; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PIPELINE_CLEANUP; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.POOL_CONNECTION_HEALTH; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROGRESS_AND_TIMEOUT; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY_EXCEPTION_WARNING_SUPPRESSION; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL_INITIALIZER; import static io.clientcore.http.netty4.implementation.Netty4Utility.awaitLatch; import static io.clientcore.http.netty4.implementation.Netty4Utility.buildSslContext; import static io.clientcore.http.netty4.implementation.Netty4Utility.createCodec; +import static io.clientcore.http.netty4.implementation.Netty4Utility.createHttp2Codec; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp11Request; import static io.clientcore.http.netty4.implementation.Netty4Utility.sendHttp2Request; import static io.clientcore.http.netty4.implementation.Netty4Utility.setOrSuppressError; @@ -79,8 +83,6 @@ */ class NettyHttpClient implements HttpClient { private static final ClientLogger LOGGER = new ClientLogger(NettyHttpClient.class); - private static final Netty4ConnectionPool.Http2GoAwayHandler HTTP_2_GO_AWAY_HANDLER - = new Netty4ConnectionPool.Http2GoAwayHandler(); /** * Error message for when no {@link ServerSentEventListener} is attached to the {@link HttpRequest}. @@ -149,7 +151,7 @@ private Response sendWithConnectionPool(HttpRequest request) { final Channel channel = future.getNow(); try { - configurePooledRequestPipeline(channel, request, responseReference, errorReference, latch, isHttps, uri.getHost(), port); + configurePooledRequestPipeline(channel, request, responseReference, errorReference, latch, isHttps); } catch (Exception e) { // An exception occurred during the pipeline setup. // We fire the exception through the pipeline to trigger the cleanup handler, @@ -173,7 +175,7 @@ private Response sendWithConnectionPool(HttpRequest request) { throw LOGGER.throwableAtError().log(errorReference.get(), CoreException::from); } else { throw LOGGER.throwableAtError() - .log("The request latch was released without a response or an error being set.", + .log("The request pipeline completed without producing a response or an error.", IllegalStateException::new); } } @@ -209,7 +211,10 @@ protected void initChannel(Channel channel) throws SSLException { } }); - channel.pipeline().addFirst(PROXY, proxyHandler); + ChannelPipeline pipeline = channel.pipeline(); + pipeline.addFirst(PROXY, proxyHandler); + pipeline.addAfter(PROXY, PROXY_EXCEPTION_WARNING_SUPPRESSION, + Netty4ConnectionPool.SuppressProxyConnectExceptionWarningHandler.INSTANCE); } // Add SSL handling if the request is HTTPS. @@ -315,7 +320,7 @@ protected void initChannel(Channel channel) throws SSLException { private void configurePooledRequestPipeline(Channel channel, HttpRequest request, AtomicReference responseReference, AtomicReference errorReference, - CountDownLatch latch, boolean isHttps, String host, int port) { + CountDownLatch latch, boolean isHttps) { ReentrantLock lock = channel.attr(Netty4ConnectionPool.CHANNEL_LOCK).get(); lock.lock(); @@ -336,64 +341,64 @@ private void configurePooledRequestPipeline(Channel channel, HttpRequest request final Object pipelineOwnerToken = new Object(); channel.attr(Netty4ConnectionPool.PIPELINE_OWNER_TOKEN).set(pipelineOwnerToken); + ChannelPipeline pipeline = channel.pipeline(); + + HttpProtocolVersion protocol = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); + boolean isHttp2 = protocol == HttpProtocolVersion.HTTP_2; + + if (protocol == null) { + // Ideally, this should never happen, but as a safeguard. + setOrSuppressError(errorReference, new IllegalStateException("Channel from pool is missing protocol.")); + latch.countDown(); + return; + } - ProgressReporter progressReporter = (request.getContext() == null) + if (isHttp2) { + // For H2 (which is always HTTPS), the codec is persistent. + // Add it only if it's not already there (first request). + if (pipeline.get(HTTP_CODEC) == null) { + pipeline.addAfter(SSL, HTTP_CODEC, createHttp2Codec()); + pipeline.addAfter(HTTP_CODEC, HTTP2_GOAWAY, new Netty4ConnectionPool.Http2GoAwayHandler()); + } + } else { // HTTP/1.1 (can be HTTP or HTTPS) + // For H1, the codec is transient and must be added for every request. + // The cleanup handler is responsible for removing it. + String after = isHttps ? SSL : POOL_CONNECTION_HEALTH; + pipeline.addAfter(after, HTTP_CODEC, createCodec()); + } + + ProgressReporter progressReporter = request.getContext() == null ? null : (ProgressReporter) request.getContext().getMetadata("progressReporter"); + boolean addProgressAndTimeoutHandler = progressReporter != null || writeTimeoutMillis > 0 || responseTimeoutMillis > 0 || readTimeoutMillis > 0; - ChannelPipeline pipeline = channel.pipeline(); - - pipeline.addLast(PIPELINE_CLEANUP, - new Netty4PipelineCleanupHandler(connectionPool, errorReference, pipelineOwnerToken)); - - HttpProtocolVersion protocol = channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).get(); - - if (protocol != null) { - if (addProgressAndTimeoutHandler) { - pipeline.addLast(PROGRESS_AND_TIMEOUT, new Netty4ProgressAndTimeoutHandler(progressReporter, writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); - } - pipeline.addLast(HTTP_RESPONSE, new Netty4ResponseHandler(request, responseReference, errorReference, latch)); + Netty4ResponseHandler responseHandler + = new Netty4ResponseHandler(request, responseReference, errorReference, latch); - if (protocol == HttpProtocolVersion.HTTP_1_1) { - String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; - pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); - } + if (addProgressAndTimeoutHandler) { + Netty4ProgressAndTimeoutHandler progressAndTimeoutHandler = new Netty4ProgressAndTimeoutHandler( + progressReporter, writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis); - channel.eventLoop().execute(() -> { - if (protocol == HttpProtocolVersion.HTTP_2) { - sendHttp2Request(request, channel, errorReference, latch); - } else { - send(request, channel, errorReference, latch); - } - }); + pipeline.addAfter(HTTP_CODEC, PROGRESS_AND_TIMEOUT, progressAndTimeoutHandler); + pipeline.addAfter(PROGRESS_AND_TIMEOUT, HTTP_RESPONSE, responseHandler); } else { - if (addProgressAndTimeoutHandler) { - pipeline.addLast(PROGRESS_AND_TIMEOUT, - new Netty4ProgressAndTimeoutHandler(progressReporter, writeTimeoutMillis, responseTimeoutMillis, readTimeoutMillis)); - } + pipeline.addAfter(HTTP_CODEC, HTTP_RESPONSE, responseHandler); + } - if (isHttps) { - SslContext sslContext = buildSslContext(maximumHttpVersion, sslContextModifier); - SslHandler sslHandler = sslContext.newHandler(channel.alloc(), host, port); - pipeline.addFirst(SSL, sslHandler); - pipeline.addAfter(SSL, ALPN, new Netty4AlpnHandler(request, responseReference, errorReference, latch)); - - channel.writeAndFlush(Unpooled.EMPTY_BUFFER); - } else { - Netty4ResponseHandler responseHandler = new Netty4ResponseHandler(request, responseReference, errorReference, latch); - pipeline.addLast(HTTP_RESPONSE, responseHandler); - String addBefore = addProgressAndTimeoutHandler ? PROGRESS_AND_TIMEOUT : HTTP_RESPONSE; - pipeline.addBefore(addBefore, HTTP_CODEC, createCodec()); + pipeline.addLast(PIPELINE_CLEANUP, + new Netty4PipelineCleanupHandler(connectionPool, errorReference, pipelineOwnerToken)); + channel.eventLoop().execute(() -> { + if (isHttp2) { + sendHttp2Request(request, channel, errorReference, latch); + } else { // HTTP/1.1 send(request, channel, errorReference, latch); } - } - } catch (SSLException e) { - throw new RuntimeException(e); + }); } finally { lock.unlock(); } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 96c3d432c874..90644bd1765c 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -355,6 +355,7 @@ public NettyHttpClientBuilder pendingAcquireTimeout(Duration pendingAcquireTimeo * Sets the maximum number of requests that can be queued waiting for a connection. *

* This limit is applied on a per-route (per-host) basis. + * If not set, a default value of 10_000 is used. * * @param maxPendingAcquires The maximum number of pending acquires. * @return The updated builder. @@ -397,7 +398,11 @@ public HttpClient build() { .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, (int) getTimeoutMillis(connectTimeout, 10_000)); // Disable auto-read as we want to control when and how data is read from the channel. bootstrap.option(ChannelOption.AUTO_READ, false); + // Enable TCP keep-alive to proactively detect and clean up stale connections in the pool. This helps evict + // connections that have been silently dropped by network intermediaries. bootstrap.option(ChannelOption.SO_KEEPALIVE, true); + // Allow the channel to remain open for writing even after the server has closed its sending side. + // This helps detect half-closures with a ChannelInputShutdownEvent in the PoolConnectionHealthHandler. bootstrap.option(ChannelOption.ALLOW_HALF_CLOSURE, true); Configuration buildConfiguration diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java index 021cc3da143a..f5645652f5bb 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ConnectionPool.java @@ -8,7 +8,6 @@ import io.clientcore.core.models.CoreException; import io.clientcore.core.utils.AuthenticateChallenge; import io.netty.bootstrap.Bootstrap; -import io.netty.buffer.Unpooled; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelHandler; @@ -26,7 +25,6 @@ import io.netty.handler.ssl.SslCloseCompletionEvent; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslContextBuilder; -import io.netty.handler.ssl.SslHandler; import io.netty.util.AttributeKey; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; @@ -37,7 +35,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; -import java.nio.channels.ClosedChannelException; import java.time.Duration; import java.time.OffsetDateTime; import java.time.ZoneOffset; @@ -54,15 +51,20 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; -import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.HTTP_CODEC; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.CONNECTION_POOL_ALPN; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.POOL_CONNECTION_HEALTH; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.PROXY_EXCEPTION_WARNING_SUPPRESSION; import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL; -import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL_INITIALIZER; +import static io.clientcore.http.netty4.implementation.Netty4HandlerNames.SSL_GRACEFUL_SHUTDOWN; import static io.clientcore.http.netty4.implementation.Netty4Utility.buildSslContext; -import static io.clientcore.http.netty4.implementation.Netty4Utility.createHttp2Codec; /** - * A pool of Netty channels that can be reused for requests to the same remote address. + * A thread-safe pool of Netty {@link Channel}s that are reused for requests to the same route. + *

+ * This connection pool manages the entire connection lifecycle, including TCP connected, proxy handshakes, and the + * asynchronous SSL/ALPN negotiation. It is designed to return fully configured channels that are + * ready for immediate use, thereby eliminating per-request handshake latency. */ public class Netty4ConnectionPool implements Closeable { @@ -75,11 +77,17 @@ public class Netty4ConnectionPool implements Closeable { * lock instance, making the lock contention extremely low. */ public static final AttributeKey CHANNEL_LOCK = AttributeKey.valueOf("channel-lock"); - public static final AttributeKey HTTP2_GOAWAY_RECEIVED = AttributeKey.valueOf("http2-goaway-received"); - // A unique token to identify the current owner of a channel pipeline + /** + * A unique token created for each request to identify the current owner of a channel pipeline. + *

+ * It protects against stale cleanup handlers from previous, timed-out, or failed requests, + * ensuring that only the {@link Netty4PipelineCleanupHandler} that belongs to the current, + * active request is allowed to modify the pipeline. + */ public static final AttributeKey PIPELINE_OWNER_TOKEN = AttributeKey.valueOf("pipeline-owner-token"); + private static final AttributeKey HTTP2_GOAWAY_RECEIVED = AttributeKey.valueOf("http2-goaway-received"); private static final AttributeKey POOLED_CONNECTION_KEY = AttributeKey.valueOf("pooled-connection-key"); @@ -107,8 +115,8 @@ public static class Http2GoAwayHandler extends ChannelInboundHandlerAdapter { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { if (msg instanceof Http2GoAwayFrame) { - // A GOAWAY frame was received. Mark the channel so the pool knows - // not to reuse it for new requests. + // A GOAWAY frame was received. + // Mark the channel so the pool knows not to reuse it for new requests. ctx.channel().attr(HTTP2_GOAWAY_RECEIVED).set(true); } super.channelRead(ctx, msg); @@ -117,7 +125,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception @ChannelHandler.Sharable public static final class SuppressProxyConnectExceptionWarningHandler extends ChannelInboundHandlerAdapter { - private static final SuppressProxyConnectExceptionWarningHandler INSTANCE + public static final SuppressProxyConnectExceptionWarningHandler INSTANCE = new SuppressProxyConnectExceptionWarningHandler(); @Override @@ -129,10 +137,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { } } - @ChannelHandler.Sharable - private static class SslGracefulShutdownHandler extends ChannelInboundHandlerAdapter { - private static final SslGracefulShutdownHandler INSTANCE = new SslGracefulShutdownHandler(); - + public static class SslGracefulShutdownHandler extends ChannelInboundHandlerAdapter { @Override public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { if (evt instanceof SslCloseCompletionEvent) { @@ -143,7 +148,7 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc } @ChannelHandler.Sharable - private static class PoolConnectionHealthHandler extends ChannelInboundHandlerAdapter { + public static class PoolConnectionHealthHandler extends ChannelInboundHandlerAdapter { private static final PoolConnectionHealthHandler INSTANCE = new PoolConnectionHealthHandler(); private static final AttributeKey CONNECTION_INVALIDATED @@ -151,7 +156,8 @@ private static class PoolConnectionHealthHandler extends ChannelInboundHandlerAd @Override public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - // This event is fired when the server closes its side of the connection. + // This event signals the server has closed its sending side (TCP half-closure). + // The channel is now considered unusable and must be closed. if (evt instanceof ChannelInputShutdownEvent) { invalidateAndClose(ctx.channel()); } @@ -173,12 +179,15 @@ private void invalidateAndClose(Channel channel) { } } - @ChannelHandler.Sharable - private static class PoolAlpnConfigurator extends ApplicationProtocolNegotiationHandler { - private static final PoolAlpnConfigurator INSTANCE = new PoolAlpnConfigurator(); + /** + * A specialized ALPN handler that signals when a channel is fully negotiated and ready for use by the pool. + */ + public static class ConnectionPoolAlpnHandler extends ApplicationProtocolNegotiationHandler { + private final Promise promise; - PoolAlpnConfigurator() { + ConnectionPoolAlpnHandler(Promise promise) { super(ApplicationProtocolNames.HTTP_1_1); + this.promise = promise; } @Override @@ -192,11 +201,16 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { ctx.channel().attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).set(protocolVersion); + // After setting the protocol, fulfill the promise to signal the channel is ready. + promise.setSuccess(ctx.channel()); + ctx.pipeline().remove(this); } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + // If an error happens during negotiation, fail the promise. + promise.setFailure(cause); ctx.fireExceptionCaught(cause); } } @@ -232,12 +246,17 @@ public Netty4ConnectionPool(Bootstrap bootstrap, } /** - * Acquires a channel for the given composite key from the pool. + * Acquires a channel for the given route from the pool. + *

+ * The returned {@link Future} will be notified with a channel that is fully connected, authenticated by any + * proxy, and has completed its SSL/ALPN handshake (for HTTPS). This method will first attempt to reuse an + * available idle channel. If none are available and the pool has not reached its maximum capacity, a new + * channel will be created. If the pool is at maximum capacity, the acquisition request will be queued. * * @param key The composite key representing the connection route. - * @param isHttps Flag indicating whether connections for this route should be secured using TLS/SSL. - * @return A {@link Future} that will be notified when a channel is acquired. - * @throws IllegalStateException if the connection pool has been closed. + * @param isHttps Flag indicating if the connection should be secured. + * @return A {@link Future} that will complete with a ready-to-use {@link Channel}, or a failed {@link Future} + * in case the connection pool has been closed. */ public Future acquire(Netty4ConnectionPoolKey key, boolean isHttps) { if (closed.get()) { @@ -252,9 +271,10 @@ public Future acquire(Netty4ConnectionPoolKey key, boolean isHttps) { } /** - * Releases a channel back to the connection pool. - * The channel pipeline must be cleaned of request-specific handlers before releasing. - * This method is not responsible for that. + * Releases a healthy channel back to the connection pool to be reused for future requests. + *

+ * The channel's pipeline must be cleaned of all request-specific handlers before being released. + * Unhealthy channels (e.g., those that are inactive or have received a GOAWAY frame) will be closed and discarded. * * @param channel The channel to release back to the connection pool. */ @@ -361,17 +381,16 @@ class PerRoutePool { } /** - * Acquires a connection. - * - *

- * This method is the entry point for getting a connection. It will first try to poll from the idle queue. - * If it can't, it will attempt to create a new one if pool capacity is not reached. If capacity is reached, - * it will queue the request. - *

+ * Acquires a connection for this specific route, following the pool's logic flow: + *
    + *
  1. Attempt to poll a healthy, idle connection from the queue.
  2. + *
  3. If none is available, attempt to create a new connection if capacity allows.
  4. + *
  5. If at capacity, queue the acquisition request.
  6. + *
* * @return A {@link Future} that completes with a {@link Channel}. */ - Future acquire() { + private Future acquire() { if (closed.get()) { return bootstrap.config() .group() @@ -395,7 +414,7 @@ Future acquire() { return createNewConnection(); } - // Pool is full. Queue the acquisition request. + // The Pool is full. Queue the acquisition request. if (pendingAcquirers.size() >= maxPendingAcquires) { return bootstrap.config() .group() @@ -443,7 +462,14 @@ Future acquire() { } } - void release(PooledConnection connection) { + /** + * Releases a connection back to this route's pool. + *

+ * First offers the connection to any pending acquirers before adding it to the idle queue. + * + * @param connection The connection to release. + */ + private void release(PooledConnection connection) { poolLock.lock(); try { if (!isHealthy(connection)) { @@ -458,7 +484,7 @@ void release(PooledConnection connection) { while (!pendingAcquirers.isEmpty()) { Promise waiter = pollNextWaiter(); if (waiter == null) { - // All remaining waiters were cancelled. + // All remaining waiters were canceled. break; } @@ -496,7 +522,7 @@ private void handleConnectionClosure() { if (future.isSuccess()) { // Try to give the new channel to the waiter. // If it fails (e.g., the waiter timed out in the meantime), - // release the brand new channel back to the pool. + // release the brand-new channel back to the pool. if (!waiter.trySuccess((Channel) future.getNow())) { release(((Channel) future.getNow()).attr(POOLED_CONNECTION_KEY).get()); } @@ -534,6 +560,15 @@ private PooledConnection pollIdleAndCheckHealth() { return null; } + /** + * Creates a new channel and asynchronously orchestrates its full setup. + *

+ * This method configures a {@link ChannelInitializer} to set up the base pipeline with handlers for health, + * proxying, and SSL. The readiness of the channel is signaled by a {@link Promise}, which is only fulfilled + * after all asynchronous setup stages (TCP connect, proxy handshake, and SSL/ALPN negotiation) are complete. + * + * @return A {@link Future} that will complete with a new, fully configured channel. + */ private Future createNewConnection() { Bootstrap newConnectionBootstrap = bootstrap.clone(); Promise promise = newConnectionBootstrap.config().group().next().newPromise(); @@ -545,8 +580,8 @@ public void initChannel(Channel channel) throws SSLException { // Create the connection wrapper and attach it to the channel. new PooledConnection(channel, key); - ChannelPipeline pipeline = channel.pipeline(); //TODO: fix handler names - pipeline.addLast("poolHealthHandler", PoolConnectionHealthHandler.INSTANCE); + ChannelPipeline pipeline = channel.pipeline(); + pipeline.addLast(POOL_CONNECTION_HEALTH, PoolConnectionHealthHandler.INSTANCE); // Test whether proxying should be applied to this Channel. If so, add it. // Proxy detection MUST use the final destination address from the key. @@ -554,26 +589,27 @@ public void initChannel(Channel channel) throws SSLException { if (hasProxy) { ProxyHandler proxyHandler = channelInitializationProxyHandler.createProxy(proxyChallenges); pipeline.addFirst(PROXY, proxyHandler); - pipeline.addAfter(PROXY, "clientcore.suppressproxyexception", + pipeline.addAfter(PROXY, PROXY_EXCEPTION_WARNING_SUPPRESSION, SuppressProxyConnectExceptionWarningHandler.INSTANCE); } // Add SSL handling if the request is HTTPS. -// if (isHttps) { -// InetSocketAddress inetSocketAddress = (InetSocketAddress) key.getFinalDestination(); -// SslContext ssl = buildSslContext(maximumHttpVersion, sslContextModifier); -// pipeline.addLast(SSL, ssl.newHandler(channel.alloc(), inetSocketAddress.getHostString(), -// inetSocketAddress.getPort())); -// pipeline.addAfter(SSL, "clientcore.sslshutdown", SslGracefulShutdownHandler.INSTANCE); -// //pipeline.addLast(SSL_INITIALIZER, new Netty4SslInitializationHandler()); -// pipeline.addLast("pool-alpn-configurator", PoolAlpnConfigurator.INSTANCE); -// } + if (isHttps) { + InetSocketAddress inetSocketAddress = (InetSocketAddress) key.getFinalDestination(); + SslContext ssl = buildSslContext(maximumHttpVersion, sslContextModifier); + pipeline.addLast(SSL, ssl.newHandler(channel.alloc(), inetSocketAddress.getHostString(), + inetSocketAddress.getPort())); + pipeline.addAfter(SSL, SSL_GRACEFUL_SHUTDOWN, new SslGracefulShutdownHandler()); + pipeline.addLast(CONNECTION_POOL_ALPN, new ConnectionPoolAlpnHandler(promise)); + } else { + channel.attr(Netty4AlpnHandler.HTTP_PROTOCOL_VERSION_KEY).set(HttpProtocolVersion.HTTP_1_1); + } } }); newConnectionBootstrap.connect(route).addListener(future -> { if (!future.isSuccess()) { - LOGGER.atError().setThrowable(future.cause()).log("Failed connection."); + LOGGER.atError().setThrowable(future.cause()).log("Failed to connect to the route."); handleConnectionClosure(); promise.setFailure(future.cause()); return; @@ -582,53 +618,25 @@ public void initChannel(Channel channel) throws SSLException { Channel newChannel = ((ChannelFuture) future).channel(); newChannel.closeFuture().addListener(closeFuture -> handleConnectionClosure()); -// Runnable connectionReadyRunner = () -> { -// SslHandler sslHandler = newChannel.pipeline().get(SslHandler.class); -// if (sslHandler != null) { -// sslHandler.handshakeFuture().addListener(sslFuture -> { -// if (sslFuture.isSuccess()) { -// promise.setSuccess(newChannel); -// } else { -// promise.setFailure(sslFuture.cause()); -// newChannel.close(); -// } -// }); -// newChannel.writeAndFlush(Unpooled.EMPTY_BUFFER); -// } else { -// promise.setSuccess(newChannel); -// } -// }; - -// ProxyHandler proxyHandler = (ProxyHandler) newChannel.pipeline().get(PROXY); -// if (proxyHandler != null){ -// // Wait for the proxy handshake to complete if proxy is being used. -// proxyHandler.connectFuture().addListener(proxyFuture -> { -// if (proxyFuture.isSuccess()) { -// if (!newChannel.isActive()) { -// promise.setFailure(new ClosedChannelException()); -// newChannel.close(); -// return; -// } -// connectionReadyRunner.run(); -// } else { -// promise.setFailure(proxyFuture.cause()); -// newChannel.close(); -// } -// }); -// } else { -// connectionReadyRunner.run(); -// } + Runnable connectionSuccessRunner = () -> { + if (!isHttps) { + promise.trySuccess(newChannel); + } + // If it IS https, we do nothing. The ConnectionPoolAlpnHandler is in charge. + }; + ProxyHandler proxyHandler = (ProxyHandler) newChannel.pipeline().get(PROXY); if (proxyHandler != null) { proxyHandler.connectFuture().addListener(proxyFuture -> { - if (proxyFuture.isSuccess()) promise.trySuccess(newChannel); - else { + if (proxyFuture.isSuccess()) { + connectionSuccessRunner.run(); + } else { promise.tryFailure(proxyFuture.cause()); newChannel.close(); } }); } else { - promise.trySuccess(newChannel); + connectionSuccessRunner.run(); } }); return promise; @@ -671,7 +679,7 @@ private boolean isHealthy(PooledConnection connection) { return true; } - void cleanup() { + private void cleanup() { if (idleConnections.isEmpty()) { return; } @@ -687,7 +695,7 @@ void cleanup() { } } - void close() { + private void close() { PooledConnection connection; while ((connection = idleConnections.poll()) != null) { connection.close(); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java index 8cae50f4a624..94983260bc9d 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4HandlerNames.java @@ -39,6 +39,11 @@ public final class Netty4HandlerNames { */ public static final String ALPN = "clientcore.alpn"; + /** + * Name for the {@link Netty4ConnectionPool.ConnectionPoolAlpnHandler}. + */ + public static final String CONNECTION_POOL_ALPN = "clientcore.connectionpoolalpn"; + /** * Name for the HTTP/1.1 {@link HttpClientCodec} */ @@ -74,6 +79,26 @@ public final class Netty4HandlerNames { */ public static final String PIPELINE_CLEANUP = "clientcore.pipelinecleanup"; + /** + * Name for the {@link Netty4ConnectionPool.Http2GoAwayHandler} + */ + public static final String HTTP2_GOAWAY = "clientcore.http2goaway"; + + /** + * Name for the {@link Netty4ConnectionPool.SslGracefulShutdownHandler} + */ + public static final String SSL_GRACEFUL_SHUTDOWN = "clientcore.sslgracefulshutdown"; + + /** + * Name for the {@link Netty4ConnectionPool.PoolConnectionHealthHandler} + */ + public static final String POOL_CONNECTION_HEALTH = "clientcore.poolconnectionhealth"; + + /** + * Name for the {@link Netty4ConnectionPool.SuppressProxyConnectExceptionWarningHandler} + */ + public static final String PROXY_EXCEPTION_WARNING_SUPPRESSION = "clientcore.suppressproxyexceptionwarning"; + private Netty4HandlerNames() { } } diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java index dae3b8457f16..776b5b778642 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandler.java @@ -77,7 +77,7 @@ public Netty4ResponseHandler(HttpRequest request, AtomicReference(), OBJECT); + Netty4PipelineCleanupHandler handler + = new Netty4PipelineCleanupHandler(connectionPool, new AtomicReference<>(), OBJECT); testChannel.pipeline().addLast(handler); ChannelHandlerContext ctx = testChannel.pipeline().context(handler); Throwable testException = new IOException("Test Exception"); diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java index 091c96b033b7..d1808e26490a 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/implementation/Netty4ResponseHandlerTests.java @@ -30,7 +30,7 @@ /** * Tests {@link Netty4ResponseHandler}. */ -@Timeout(value = 1, unit = TimeUnit.MINUTES) +@Timeout(value = 3, unit = TimeUnit.MINUTES) public class Netty4ResponseHandlerTests { @Test public void firstReadIsFullHttpResponse() throws Exception { From 879c6386e19b7be2b78ebbd4b0cccf7ff5fe06e1 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Tue, 22 Jul 2025 13:33:59 +0300 Subject: [PATCH 54/57] remove duplicate line & javadoc fix --- .../main/java/io/clientcore/http/netty4/NettyHttpClient.java | 2 -- .../clientcore/http/netty4/implementation/Netty4Utility.java | 4 +++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java index 54d5ac144721..66dea873d2c1 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClient.java @@ -337,8 +337,6 @@ private void configurePooledRequestPipeline(Channel channel, HttpRequest request return; } - channel.config().setAutoRead(false); - final Object pipelineOwnerToken = new Object(); channel.attr(Netty4ConnectionPool.PIPELINE_OWNER_TOKEN).set(pipelineOwnerToken); ChannelPipeline pipeline = channel.pipeline(); diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java index 7d5e6bb2bb18..3a8e3cee2cc1 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/implementation/Netty4Utility.java @@ -136,7 +136,9 @@ public static void awaitLatch(CountDownLatch latch) { * Content will only be written to the {@link OutputStream} if the {@link ByteBuf} is non-null and is * {@link ByteBuf#isReadable()}. The entire {@link ByteBuf} will be consumed. *

- * This method does NOT release the {@link ByteBuf} if it was consumed. + *

Warning: This is a helper method and does NOT release the {@link ByteBuf} + * after it is consumed, and it must be manually released to avoid memory leaks (either the {@link ByteBuf} + * or the container holding the {@link ByteBuf}). * * @param byteBuf The Netty {@link ByteBuf} to read from. * @param stream The {@link OutputStream} to write to. From d4694d8bbdf0fc371c5d81492428d6815047180e Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 23 Jul 2025 09:05:59 +0300 Subject: [PATCH 55/57] add javadoc on netty provider --- .../http/netty4/NettyHttpClientProvider.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientProvider.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientProvider.java index 9151620ee25c..5541317fb0f9 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientProvider.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientProvider.java @@ -30,6 +30,20 @@ public HttpClient getHttpClient() { public NettyHttpClientProvider() { } + /** + * Creates a new {@link HttpClient} instance with a default, shared connection pool. + *

+ * For more advanced customization, such as disabling pooling entirely, use the {@link NettyHttpClientBuilder}. + *

+ * Example: Creating a client without a connection pool + *

{@code
+     * HttpClient client = new NettyHttpClientBuilder()
+     * .connectionPoolSize(0)
+     * .build();
+     * }
+ * + * @return A new {@link HttpClient} instance. + */ @Override public HttpClient getNewInstance() { return new NettyHttpClientBuilder().build(); From 61be5794cc1374d25c33f8d5b6c215613b5ad819 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 23 Jul 2025 17:15:46 +0300 Subject: [PATCH 56/57] add HTTP client tests for pooled connections --- .../netty4/NettyHttp2HttpClientTests.java | 2 +- .../NettyHttpClientHttpClientTests.java | 15 +-- ...ttyHttpClientHttpClientWithHttpsTests.java | 3 +- ...yHttpClientWithPooledConnectionsTests.java | 57 ++++++++++++ ...ClientWithPooledHttp2ConnectionsTests.java | 93 +++++++++++++++++++ ...ClientWithPooledHttpsConnectionsTests.java | 66 +++++++++++++ 6 files changed, 228 insertions(+), 8 deletions(-) create mode 100644 sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledConnectionsTests.java create mode 100644 sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledHttp2ConnectionsTests.java create mode 100644 sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledHttpsConnectionsTests.java diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java index 42bef6550f82..dba6aa10bbea 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttp2HttpClientTests.java @@ -31,7 +31,7 @@ public class NettyHttp2HttpClientTests extends HttpClientTests { private static final HttpClient HTTP_CLIENT_INSTANCE; static { - HTTP_CLIENT_INSTANCE = new NettyHttpClientBuilder() + HTTP_CLIENT_INSTANCE = new NettyHttpClientBuilder().connectionPoolSize(0) .sslContextModifier( builder -> builder.trustManager(new InsecureTrustManager()).secureRandom(new SecureRandom())) .maximumHttpVersion(HttpProtocolVersion.HTTP_2) diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java index 82c53989ce5a..b9380d3b7f7b 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientTests.java @@ -15,24 +15,27 @@ import java.util.concurrent.TimeUnit; /** - * Reactor Netty {@link HttpClientTests}. + * Netty {@link HttpClientTests}. */ @Timeout(value = 3, unit = TimeUnit.MINUTES) public class NettyHttpClientHttpClientTests extends HttpClientTests { private static LocalTestServer server; - private static HttpClient client; + private static final HttpClient HTTP_CLIENT_INSTANCE; + + static { + HTTP_CLIENT_INSTANCE = new NettyHttpClientBuilder().connectionPoolSize(0).build(); + } @BeforeAll public static void startTestServer() { server = HttpClientTestsServer.getHttpClientTestsServer(HttpProtocolVersion.HTTP_1_1, false); server.start(); - client = new NettyHttpClientBuilder().build(); } @AfterAll public static void stopTestServer() { - if (client instanceof NettyHttpClient) { - ((NettyHttpClient) client).close(); + if (HTTP_CLIENT_INSTANCE instanceof NettyHttpClient) { + ((NettyHttpClient) HTTP_CLIENT_INSTANCE).close(); } if (server != null) { server.stop(); @@ -52,6 +55,6 @@ protected String getServerUri(boolean secure) { @Override protected HttpClient getHttpClient() { - return client; + return HTTP_CLIENT_INSTANCE; } } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java index 9fb62774c137..5904ada0299d 100644 --- a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientHttpClientWithHttpsTests.java @@ -17,7 +17,7 @@ import java.util.concurrent.TimeUnit; /** - * Reactor Netty {@link HttpClientTests} with https. + * Netty {@link HttpClientTests} with https. * Some request logic branches out if it's https like file uploads. */ @Timeout(value = 3, unit = TimeUnit.MINUTES) @@ -28,6 +28,7 @@ public class NettyHttpClientHttpClientWithHttpsTests extends HttpClientTests { static { HTTP_CLIENT_INSTANCE = new NettyHttpClientBuilder() //.maximumHttpVersion(HttpProtocolVersion.HTTP_1_1) + .connectionPoolSize(0) .sslContextModifier(ssl -> ssl.trustManager(new InsecureTrustManager()).secureRandom(new SecureRandom())) .build(); } diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledConnectionsTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledConnectionsTests.java new file mode 100644 index 000000000000..604e2ad47587 --- /dev/null +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledConnectionsTests.java @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package io.clientcore.http.netty4; + +import io.clientcore.core.http.client.HttpClient; +import io.clientcore.core.http.client.HttpProtocolVersion; +import io.clientcore.core.shared.HttpClientTests; +import io.clientcore.core.shared.HttpClientTestsServer; +import io.clientcore.core.shared.LocalTestServer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Timeout; + +import java.util.concurrent.TimeUnit; + +/** + * Tests for {@link NettyHttpClient} with a connection pool over plain HTTP. + */ +@Timeout(value = 3, unit = TimeUnit.MINUTES) +public class NettyHttpClientWithPooledConnectionsTests extends HttpClientTests { + private static LocalTestServer server; + private static HttpClient client; + + @BeforeAll + public static void startTestServer() { + server = HttpClientTestsServer.getHttpClientTestsServer(HttpProtocolVersion.HTTP_1_1, false); + server.start(); + + client = new NettyHttpClientBuilder().build(); + } + + @AfterAll + public static void stopTestServer() { + if (client instanceof NettyHttpClient) { + ((NettyHttpClient) client).close(); + } + if (server != null) { + server.stop(); + } + } + + @Override + protected int getPort() { + return server.getPort(); + } + + @Override + protected String getServerUri(boolean secure) { + return server.getUri(); + } + + @Override + protected HttpClient getHttpClient() { + return client; + } +} diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledHttp2ConnectionsTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledHttp2ConnectionsTests.java new file mode 100644 index 000000000000..8e2eefc312f0 --- /dev/null +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledHttp2ConnectionsTests.java @@ -0,0 +1,93 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package io.clientcore.http.netty4; + +import io.clientcore.core.http.client.HttpClient; +import io.clientcore.core.http.client.HttpProtocolVersion; +import io.clientcore.core.http.models.HttpMethod; +import io.clientcore.core.http.models.HttpRequest; +import io.clientcore.core.http.models.Response; +import io.clientcore.core.models.binarydata.BinaryData; +import io.clientcore.core.shared.HttpClientTests; +import io.clientcore.core.shared.HttpClientTestsServer; +import io.clientcore.core.shared.InsecureTrustManager; +import io.clientcore.core.shared.LocalTestServer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.security.SecureRandom; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; + +/** + * Tests for {@link NettyHttpClient} with a connection pool using HTTP/2. + */ +@Timeout(value = 3, unit = TimeUnit.MINUTES) +public class NettyHttpClientWithPooledHttp2ConnectionsTests extends HttpClientTests { + private static LocalTestServer server; + private static HttpClient client; + + @BeforeAll + public static void startTestServer() { + server = HttpClientTestsServer.getHttpClientTestsServer(HttpProtocolVersion.HTTP_2, true); + server.start(); + + client = new NettyHttpClientBuilder().maximumHttpVersion(HttpProtocolVersion.HTTP_2) + .sslContextModifier( + builder -> builder.trustManager(new InsecureTrustManager()).secureRandom(new SecureRandom())) + .build(); + } + + @AfterAll + public static void stopTestServer() { + if (client instanceof NettyHttpClient) { + ((NettyHttpClient) client).close(); + } + if (server != null) { + server.stop(); + } + } + + @Override + protected boolean isHttp2() { + return true; + } + + @Override + protected boolean isSecure() { + return true; + } + + @Override + protected int getPort() { + return server.getPort(); + } + + @Override + protected String getServerUri(boolean secure) { + return server.getHttpsUri(); + } + + @Override + protected HttpClient getHttpClient() { + return client; + } + + @Test + public void canSendBinaryDataDebug() { + byte[] expectedBytes = new byte[1024 * 1024]; + ThreadLocalRandom.current().nextBytes(expectedBytes); + HttpRequest request = new HttpRequest().setMethod(HttpMethod.PUT) + .setUri(getRequestUri("echo")) + .setBody(BinaryData.fromBytes(expectedBytes)); + + try (Response response = getHttpClient().send(request)) { + assertArrayEquals(expectedBytes, response.getValue().toBytes()); + } + } +} diff --git a/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledHttpsConnectionsTests.java b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledHttpsConnectionsTests.java new file mode 100644 index 000000000000..6bf1571b228f --- /dev/null +++ b/sdk/clientcore/http-netty4/src/test/java/io/clientcore/http/netty4/NettyHttpClientWithPooledHttpsConnectionsTests.java @@ -0,0 +1,66 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package io.clientcore.http.netty4; + +import io.clientcore.core.http.client.HttpClient; +import io.clientcore.core.http.client.HttpProtocolVersion; +import io.clientcore.core.shared.HttpClientTests; +import io.clientcore.core.shared.HttpClientTestsServer; +import io.clientcore.core.shared.InsecureTrustManager; +import io.clientcore.core.shared.LocalTestServer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Timeout; + +import java.security.SecureRandom; +import java.util.concurrent.TimeUnit; + +/** + * Tests for {@link NettyHttpClient} with a connection pool over HTTPS. + */ +@Timeout(value = 3, unit = TimeUnit.MINUTES) +public class NettyHttpClientWithPooledHttpsConnectionsTests extends HttpClientTests { + private static LocalTestServer server; + private static HttpClient client; + + @BeforeAll + public static void startTestServer() { + server = HttpClientTestsServer.getHttpClientTestsServer(HttpProtocolVersion.HTTP_1_1, true); + server.start(); + + client = new NettyHttpClientBuilder() + .sslContextModifier(ssl -> ssl.trustManager(new InsecureTrustManager()).secureRandom(new SecureRandom())) + .build(); + } + + @AfterAll + public static void stopTestServer() { + if (client instanceof NettyHttpClient) { + ((NettyHttpClient) client).close(); + } + if (server != null) { + server.stop(); + } + } + + @Override + protected int getPort() { + return server.getPort(); + } + + @Override + protected String getServerUri(boolean secure) { + return secure ? server.getHttpsUri() : server.getUri(); + } + + @Override + protected boolean isSecure() { + return true; + } + + @Override + protected HttpClient getHttpClient() { + return client; + } +} From 5551d477f0900b2543eb4f7fe34c4a7858fbe399 Mon Sep 17 00:00:00 2001 From: George Banasios Date: Wed, 23 Jul 2025 17:45:38 +0300 Subject: [PATCH 57/57] fix builder logging configuration logic --- .../http/netty4/NettyHttpClientBuilder.java | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java index 90644bd1765c..68c163abd001 100644 --- a/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java +++ b/sdk/clientcore/http-netty4/src/main/java/io/clientcore/http/netty4/NettyHttpClientBuilder.java @@ -7,6 +7,7 @@ import io.clientcore.core.http.client.HttpProtocolVersion; import io.clientcore.core.http.models.ProxyOptions; import io.clientcore.core.instrumentation.logging.ClientLogger; +import io.clientcore.core.instrumentation.logging.LoggingEvent; import io.clientcore.core.utils.configuration.Configuration; import io.clientcore.http.netty4.implementation.ChannelInitializationProxyHandler; import io.clientcore.http.netty4.implementation.Netty4ConnectionPool; @@ -381,17 +382,21 @@ public HttpClient build() { = getChannelClass(this.channelClass, group.getClass(), IS_EPOLL_AVAILABLE, IS_KQUEUE_AVAILABLE); // Leave breadcrumbs about the NettyHttpClient configuration, in case troubleshooting is needed. - LOGGER.atVerbose() + LoggingEvent loggingEvent = LOGGER.atVerbose() .addKeyValue("customEventLoopGroup", eventLoopGroup != null) .addKeyValue("eventLoopGroupClass", group.getClass()) .addKeyValue("customChannelClass", this.channelClass != null) - .addKeyValue("channelClass", channelClass) - .addKeyValue("connectionPoolSize", this.connectionPoolSize) - .addKeyValue("connectionIdleTimeout", this.connectionIdleTimeout) - .addKeyValue("maxConnectionLifetime", this.maxConnectionLifetime) - .addKeyValue("pendingAcquireTimeout", this.pendingAcquireTimeout) - .addKeyValue("maxPendingAcquires", this.maxPendingAcquires) - .log("NettyHttpClient was built with these configurations."); + .addKeyValue("channelClass", channelClass); + + if (connectionPoolSize > 0) { + loggingEvent.addKeyValue("connectionPoolSize", this.connectionPoolSize) + .addKeyValue("connectionIdleTimeout", this.connectionIdleTimeout) + .addKeyValue("maxConnectionLifetime", this.maxConnectionLifetime) + .addKeyValue("pendingAcquireTimeout", this.pendingAcquireTimeout) + .addKeyValue("maxPendingAcquires", this.maxPendingAcquires); + } + + loggingEvent.log("NettyHttpClient was built with these configurations."); Bootstrap bootstrap = new Bootstrap().group(group) .channel(channelClass)