From 25c1d35320d0636cee0f513f391eb140d969fac7 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 19 Sep 2022 11:05:10 +0200 Subject: [PATCH 01/82] new http2 pool --- .../http/client/netty/ConnectionManager.java | 431 ++++++++++++++++-- .../http/client/netty/DefaultHttpClient.java | 4 +- .../client/netty/ConnectionManagerSpec.groovy | 109 +++++ .../http/client/netty/DummyChannelId.groovy | 26 ++ .../http/client/netty/EmbeddedTestUtil.groovy | 99 ++++ 5 files changed, 624 insertions(+), 45 deletions(-) create mode 100644 http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy create mode 100644 http-client/src/test/groovy/io/micronaut/http/client/netty/DummyChannelId.groovy create mode 100644 http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index fe4f9769767..59e4c36a1ac 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -57,7 +57,6 @@ import io.netty.channel.pool.ChannelPoolMap; import io.netty.channel.pool.FixedChannelPool; import io.netty.channel.pool.SimpleChannelPool; -import io.netty.channel.socket.SocketChannel; import io.netty.handler.codec.LineBasedFrameDecoder; import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.DefaultHttpContent; @@ -77,10 +76,16 @@ import io.netty.handler.codec.http2.DelegatingDecompressorFrameListener; import io.netty.handler.codec.http2.Http2ClientUpgradeCodec; import io.netty.handler.codec.http2.Http2Connection; +import io.netty.handler.codec.http2.Http2FrameCodecBuilder; import io.netty.handler.codec.http2.Http2FrameListener; import io.netty.handler.codec.http2.Http2FrameLogger; +import io.netty.handler.codec.http2.Http2MultiplexHandler; import io.netty.handler.codec.http2.Http2Settings; +import io.netty.handler.codec.http2.Http2SettingsFrame; import io.netty.handler.codec.http2.Http2Stream; +import io.netty.handler.codec.http2.Http2StreamChannel; +import io.netty.handler.codec.http2.Http2StreamChannelBootstrap; +import io.netty.handler.codec.http2.Http2StreamFrameToHttpObjectCodec; import io.netty.handler.codec.http2.HttpConversionUtil; import io.netty.handler.codec.http2.HttpToHttp2ConnectionHandler; import io.netty.handler.codec.http2.HttpToHttp2ConnectionHandlerBuilder; @@ -115,11 +120,17 @@ import java.net.SocketAddress; import java.time.Duration; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.OptionalInt; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; /** @@ -127,7 +138,7 @@ * channels (wrapped in {@link PoolHandle}s), including pooling and timeouts. */ @Internal -final class ConnectionManager { +class ConnectionManager { private static final AttributeKey CHANNEL_CUSTOMIZER_KEY = AttributeKey.valueOf("micronaut.http.customizer"); /** @@ -144,6 +155,7 @@ final class ConnectionManager { final HttpVersion httpVersion; private final Logger log; + private final Map pools = new ConcurrentHashMap<>(); private EventLoopGroup group; private final boolean shutdownGroup; private final ThreadFactory threadFactory; @@ -158,10 +170,28 @@ final class ConnectionManager { private final Collection pipelineListeners; private final String informationalServiceId; + ConnectionManager(ConnectionManager from) { + this.poolMap = from.poolMap; + this.instrumenter = from.instrumenter; + this.httpVersion = from.httpVersion; + this.log = from.log; + this.group = from.group; + this.shutdownGroup = from.shutdownGroup; + this.threadFactory = from.threadFactory; + this.bootstrap = from.bootstrap; + this.configuration = from.configuration; + this.readTimeoutMillis = from.readTimeoutMillis; + this.connectionTimeAliveMillis = from.connectionTimeAliveMillis; + this.sslContext = from.sslContext; + this.clientCustomizer = from.clientCustomizer; + this.pipelineListeners = from.pipelineListeners; + this.informationalServiceId = from.informationalServiceId; + } + ConnectionManager( Logger log, @Nullable EventLoopGroup eventLoopGroup, - ThreadFactory threadFactory, + @Nullable ThreadFactory threadFactory, HttpClientConfiguration configuration, HttpVersion httpVersion, InvocationInstrumenter instrumenter, @@ -190,7 +220,7 @@ final class ConnectionManager { this.readTimeoutMillis = configuration.getReadTimeout() .map(duration -> !duration.isNegative() ? duration.toMillis() : null) .orElse(null); - this.sslContext = nettyClientSslBuilder.build(configuration.getSslConfiguration(), httpVersion).orElse(null); + this.sslContext = nettyClientSslBuilder.build(configuration.getSslConfiguration(), HttpVersion.HTTP_2_0).orElse(null); // TODO: alpn config if (eventLoopGroup != null) { group = eventLoopGroup; @@ -313,6 +343,7 @@ public void start() { * @see DefaultHttpClient#stop() */ public void shutdown() { + // todo: shutdown pools if (poolMap instanceof Iterable) { Iterable> i = (Iterable) poolMap; for (Map.Entry entry : i) { @@ -389,21 +420,23 @@ private ChannelFuture doConnect( boolean isProxy, boolean acceptsEvents, Consumer contextConsumer) throws HttpClientException { + return doConnect(requestKey, new HttpClientInitializer( + buildSslContext(requestKey), + requestKey.getHost(), + requestKey.getPort(), + isStream, + isProxy, + acceptsEvents, + contextConsumer)); + } - SslContext sslCtx = buildSslContext(requestKey); + // for testing + protected ChannelFuture doConnect(DefaultHttpClient.RequestKey requestKey, ChannelInitializer channelInitializer) { String host = requestKey.getHost(); int port = requestKey.getPort(); Bootstrap localBootstrap = bootstrap.clone(); - initBootstrapForProxy(localBootstrap, sslCtx != null, host, port); - localBootstrap.handler(new HttpClientInitializer( - sslCtx, - host, - port, - isStream, - isProxy, - acceptsEvents, - contextConsumer) - ); + initBootstrapForProxy(localBootstrap, requestKey.isSecure(), host, port); + localBootstrap.handler(channelInitializer); return localBootstrap.connect(host, port); } @@ -434,7 +467,7 @@ private SslContext buildSslContext(DefaultHttpClient.RequestKey requestKey) { } private PoolHandle mockPoolHandle(Channel channel) { - return new PoolHandle(null, channel); + return new OldPoolHandle(null, channel); } /** @@ -446,6 +479,10 @@ private PoolHandle mockPoolHandle(Channel channel) { * @return A mono that will complete once the channel is ready for transmission */ Mono connectForExchange(DefaultHttpClient.RequestKey requestKey, boolean multipart, boolean acceptEvents) { + if (true) { + Pool pool = pools.computeIfAbsent(requestKey, Pool::new); + return pool.acquire(); + } return Mono.create(emitter -> { if (poolMap != null && !multipart) { try { @@ -453,7 +490,7 @@ Mono connectForExchange(DefaultHttpClient.RequestKey requestKey, boo addInstrumentedListener(channelPool.acquire(), future -> { if (future.isSuccess()) { Channel channel = future.get(); - PoolHandle poolHandle = new PoolHandle(channelPool, channel); + PoolHandle poolHandle = new OldPoolHandle(channelPool, channel); Future initFuture = channel.attr(STREAM_CHANNEL_INITIALIZED).get(); if (initFuture == null) { emitter.success(poolHandle); @@ -720,7 +757,7 @@ public void channelAcquired(Channel ch) throws Exception { */ private void configureHttp2Ssl( HttpClientInitializer httpClientInitializer, - @NonNull SocketChannel ch, + @NonNull Channel ch, @NonNull SslContext sslCtx, String host, int port, @@ -787,7 +824,7 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { */ private void configureHttp2ClearText( HttpClientInitializer httpClientInitializer, - @NonNull SocketChannel ch, + @NonNull Channel ch, @NonNull HttpToHttp2ConnectionHandler connectionHandler) { HttpClientCodec sourceCodec = new HttpClientCodec(); Http2ClientUpgradeCodec upgradeCodec = new Http2ClientUpgradeCodec(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, connectionHandler); @@ -1032,7 +1069,7 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { /** * Initializes the HTTP client channel. */ - private class HttpClientInitializer extends ChannelInitializer { + private class HttpClientInitializer extends ChannelInitializer { final SslContext sslContext; final String host; @@ -1073,7 +1110,7 @@ protected HttpClientInitializer(SslContext sslContext, * @param ch The channel */ @Override - protected void initChannel(SocketChannel ch) { + protected void initChannel(Channel ch) { channelCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); ch.attr(CHANNEL_CUSTOMIZER_KEY).set(channelCustomizer); @@ -1259,7 +1296,7 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc } super.userEventTriggered(ctx, evt); } - }); + }); } private boolean acceptsEventStream() { @@ -1267,27 +1304,183 @@ private boolean acceptsEventStream() { } } - final class PoolHandle { + private class AdaptiveAlpnChannelInitializer extends ChannelInitializer { + private final Pool pool; + + final SslContext sslContext; + final String host; + final int port; + Http2SettingsHandler settingsHandler; + private NettyClientCustomizer channelCustomizer; + + AdaptiveAlpnChannelInitializer(Pool pool, + SslContext sslContext, + String host, + int port) { + this.pool = pool; + this.sslContext = sslContext; + this.host = host; + this.port = port; + } + + /** + * @param ch The channel + */ + @Override + protected void initChannel(Channel ch) { + channelCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); + ch.attr(CHANNEL_CUSTOMIZER_KEY).set(channelCustomizer); + + configureProxy(ch.pipeline(), true, host, port); + + InitialConnectionErrorHandler initialErrorHandler = new InitialConnectionErrorHandler(pool); + ch.pipeline() + .addLast(ChannelPipelineCustomizer.HANDLER_SSL, sslContext.newHandler(ch.alloc(), host, port)) + .addLast( + ChannelPipelineCustomizer.HANDLER_HTTP2_PROTOCOL_NEGOTIATOR, + new ApplicationProtocolNegotiationHandler(ApplicationProtocolNames.HTTP_1_1) { + @Override + protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { + if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { + ctx.pipeline() + .addLast(new Http2PriorKnowledgeInitializer(pool)) + .remove(initialErrorHandler); + } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { + // TODO + throw new UnsupportedOperationException(); + } else { + ctx.close(); + throw customizeException(new HttpClientException("Unknown Protocol: " + protocol)); + } + } + }) + .addLast(initialErrorHandler); + } + + /** + * Called when the stream pipeline is fully set up (all handshakes completed) and we can + * start processing requests. + */ + void onStreamPipelineBuilt() { + channelCustomizer.onStreamPipelineBuilt(); + } + } + + private class InitialConnectionErrorHandler extends ChannelInboundHandlerAdapter { + private final Pool pool; + + InitialConnectionErrorHandler(Pool pool) { + this.pool = pool; + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + log.error("Failed to open connection", cause); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + super.channelInactive(ctx); + pool.onNewConnectionFailure(); + } + } + + private class Http2PriorKnowledgeInitializer extends ChannelInitializer { + private final Pool pool; + + Http2PriorKnowledgeInitializer(Pool pool) { + this.pool = pool; + } + + @Override + protected void initChannel(Channel ch) throws Exception { + ch.pipeline() + .addLast(Http2FrameCodecBuilder.forClient() + .build()) + .addLast(new Http2MultiplexHandler(new ChannelInitializer() { + @Override + protected void initChannel(Http2StreamChannel ch1) throws Exception { + // todo: fail connection? + log.warn("Server opened HTTP2 stream {}, closing immediately", ch1.stream().id()); + ch1.close(); + } + }, new ChannelInboundHandlerAdapter() { + @Override + public void handlerAdded(ChannelHandlerContext ctx) throws Exception { + ctx.read(); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + // todo: log + ReferenceCountUtil.release(msg); + ctx.read(); + } + })) + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new InitialConnectionErrorHandler(pool) { + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof Http2SettingsFrame) { + ctx.pipeline().remove(this); + pool.new Http2ConnectionHolder(ch).init(); + return; + } else { + log.warn("Premature frame: {}", msg.getClass()); + } + + super.channelRead(ctx, msg); + } + }); + } + } + + abstract class PoolHandle { final Channel channel; + + /** + * Prevent this connection from being reused. + */ + abstract void taint(); + + private PoolHandle(Channel channel) { + this.channel = channel; + } + + /** + * Close this connection or release it back to the pool. + */ + abstract void release(); + + /** + * Whether this connection may be returned to a connection pool (i.e. should be kept + * keepalive). + * + * @return Whether this connection may be reused + */ + abstract boolean canReturn(); + + /** + * Notify any {@link NettyClientCustomizer} that the request pipeline has been built. + */ + void notifyRequestPipelineBuilt() { + channel.attr(CHANNEL_CUSTOMIZER_KEY).get().onRequestPipelineBuilt(); + } + } + + private final class OldPoolHandle extends PoolHandle { private final ChannelPool channelPool; private boolean canReturn; - private PoolHandle(ChannelPool channelPool, Channel channel) { - this.channel = channel; + private OldPoolHandle(ChannelPool channelPool, Channel channel) { + super(channel); this.channelPool = channelPool; this.canReturn = channelPool != null; } - /** - * Prevent this connection from being reused. - */ void taint() { canReturn = false; } - /** - * Close this connection or release it back to the pool. - */ void release() { if (channelPool != null) { removeReadTimeoutHandler(channel.pipeline()); @@ -1304,21 +1497,173 @@ void release() { } } - /** - * Whether this connection may be returned to a connection pool (i.e. should be kept - * keepalive). - * - * @return Whether this connection may be reused - */ - public boolean canReturn() { + boolean canReturn() { return canReturn; } - /** - * Notify any {@link NettyClientCustomizer} that the request pipeline has been built. - */ - void notifyRequestPipelineBuilt() { - channel.attr(CHANNEL_CUSTOMIZER_KEY).get().onRequestPipelineBuilt(); + } + + private static boolean incrementWithLimit(AtomicInteger variable, int limit) { + while (true) { + int old = variable.get(); + if (old >= limit) { + return false; + } + if (variable.compareAndSet(old, old + 1)) { + return true; + } + } + } + + private final class Pool { + private static final int MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION = 4; // TODO: config + private static final int MAX_PENDING_CONNECTIONS = 1; + + private final DefaultHttpClient.RequestKey requestKey; + + private final Queue> pendingRequests = new ConcurrentLinkedQueue<>(); + private final List http2Connections = new CopyOnWriteArrayList<>(); + private final AtomicInteger pendingConnectionCount = new AtomicInteger(0); + + Pool(DefaultHttpClient.RequestKey requestKey) { + this.requestKey = requestKey; + } + + Mono acquire() { + Sinks.One sink = Sinks.one(); + acquire(sink); + return sink.asMono(); + } + + private void acquire(Sinks.One sink) { + for (Http2ConnectionHolder http2Connection : http2Connections) { + if (http2Connection.satisfy(sink)) { + return; + } + } + // no http2 connection open that has room + pendingRequests.add(sink); + // todo: http1 + openNewConnection(); + } + + private void openNewConnection() { + if (!incrementWithLimit(pendingConnectionCount, MAX_PENDING_CONNECTIONS)) { + return; + } + // open a new connection + ChannelFuture connectFuture = doConnect(requestKey, new AdaptiveAlpnChannelInitializer( + this, + buildSslContext(requestKey), + requestKey.getHost(), + requestKey.getPort() + )); + addInstrumentedListener(connectFuture, future -> { + if (!future.isSuccess()) { + log.error("Failed to connect to remote", future.cause()); + onNewConnectionFailure(); + } + }); + } + + void onNewConnectionFailure() { + pendingConnectionCount.decrementAndGet(); + // todo: retry connection + } + + final class Http2ConnectionHolder { + private final Channel channel; + private final AtomicInteger liveRequests = new AtomicInteger(0); + + Http2ConnectionHolder(Channel channel) { + this.channel = channel; + } + + void init() { + channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + super.channelInactive(ctx); + http2Connections.remove(Http2ConnectionHolder.this); + } + }); + + // todo: notify customizer + + for (int i = 0; i < MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION; i++) { + Sinks.One pendingRequest = pendingRequests.poll(); + if (pendingRequest == null) { + break; + } + liveRequests.incrementAndGet(); + satisfy0(pendingRequest); + } + http2Connections.add(this); + pendingConnectionCount.decrementAndGet(); + } + + boolean satisfy(Sinks.One sink) { + if (!incrementWithLimit(liveRequests, MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION)) { + return false; + } + + if (channel.eventLoop().inEventLoop()) { + satisfy0(sink); + } else { + channel.eventLoop().execute(() -> satisfy0(sink)); + } + return true; + } + + private void satisfy0(Sinks.One sink) { + if (!channel.isActive()) { + returnPendingRequest(sink); + return; + } + addInstrumentedListener(new Http2StreamChannelBootstrap(channel).open(), (Future future) -> { + if (future.isSuccess()) { + Http2StreamChannel streamChannel = future.get(); + streamChannel.pipeline() + .addLast(new Http2StreamFrameToHttpObjectCodec(false)); + Sinks.EmitResult emitResult = sink.tryEmitValue(new PoolHandle(streamChannel) { + @Override + void taint() { + // todo + } + + @Override + void release() { + liveRequests.decrementAndGet(); + // todo: claim a new request + } + + @Override + boolean canReturn() { + return true; + } + + @Override + void notifyRequestPipelineBuilt() { + // TODO + } + }); + if (emitResult.isFailure()) { + streamChannel.close(); + liveRequests.decrementAndGet(); + // todo: claim a new request + } + } else { + log.debug("Failed to open http2 stream", future.cause()); + returnPendingRequest(sink); + } + }); + } + + private void returnPendingRequest(Sinks.One sink) { + // failed, but the pending request may still work on another connection. + pendingRequests.add(sink); + liveRequests.decrementAndGet(); + } } } } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index 8bf5d2456f7..f8347da1bc5 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -232,7 +232,7 @@ public class DefaultHttpClient implements protected MediaTypeCodecRegistry mediaTypeCodecRegistry; protected ByteBufferFactory byteBufferFactory = new NettyByteBufferFactory(); - final ConnectionManager connectionManager; + ConnectionManager connectionManager; private final List> clientFilterEntries; private final LoadBalancer loadBalancer; @@ -1085,7 +1085,7 @@ private Publisher> exchang final Duration rt = readTimeout.get(); if (!rt.isNegative()) { Duration duration = rt.plus(Duration.ofSeconds(1)); - finalReactiveSequence = finalReactiveSequence.timeout(duration) + finalReactiveSequence = finalReactiveSequence.timeout(duration) // todo: move to CM .onErrorResume(throwable -> { if (throwable instanceof TimeoutException) { return Flux.error(ReadTimeoutException.TIMEOUT_EXCEPTION); diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy new file mode 100644 index 00000000000..06d9c9df03e --- /dev/null +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -0,0 +1,109 @@ +package io.micronaut.http.client.netty + +import io.micronaut.context.ApplicationContext +import io.micronaut.http.HttpStatus +import io.micronaut.http.HttpVersion +import io.micronaut.http.server.netty.ssl.CertificateProvidedSslBuilder +import io.micronaut.http.ssl.SslConfiguration +import io.netty.buffer.ByteBufAllocator +import io.netty.channel.Channel +import io.netty.channel.ChannelFuture +import io.netty.channel.ChannelHandler +import io.netty.channel.ChannelHandlerContext +import io.netty.channel.ChannelInitializer +import io.netty.channel.embedded.EmbeddedChannel +import io.netty.handler.codec.http2.DefaultHttp2Headers +import io.netty.handler.codec.http2.DefaultHttp2HeadersFrame +import io.netty.handler.codec.http2.DefaultHttp2SettingsFrame +import io.netty.handler.codec.http2.Http2FrameCodecBuilder +import io.netty.handler.codec.http2.Http2Headers +import io.netty.handler.codec.http2.Http2HeadersFrame +import io.netty.handler.codec.http2.Http2Settings +import io.netty.handler.codec.http2.Http2SettingsAckFrame +import io.netty.handler.codec.http2.Http2SettingsFrame +import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler +import io.netty.handler.ssl.SslContextBuilder +import io.netty.handler.ssl.util.SelfSignedCertificate +import reactor.core.publisher.Mono +import spock.lang.Specification + +class ConnectionManagerSpec extends Specification { + private static void patch(DefaultHttpClient httpClient, EmbeddedChannel... channels) { + httpClient.connectionManager = new ConnectionManager(httpClient.connectionManager) { + int i = 0 + + @Override + protected ChannelFuture doConnect(DefaultHttpClient.RequestKey requestKey, ChannelInitializer channelInitializer) { + def channel = channels[i++] + channel.pipeline().addLast(channelInitializer) + def promise = channel.newPromise() + promise.setSuccess() + return promise + } + } + } + + private static ChannelHandler tlsHandler() { + def certificate = new SelfSignedCertificate() + def builder = SslContextBuilder.forServer(certificate.key(), certificate.cert()) + CertificateProvidedSslBuilder.setupSslBuilder(builder, new SslConfiguration(), HttpVersion.HTTP_2_0); + return builder.build().newHandler(ByteBufAllocator.DEFAULT) + } + + def 'simple get'() { + given: + def ctx = ApplicationContext.run([ + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + ]) + def client = ctx.getBean(DefaultHttpClient) + + def serverChannel = new EmbeddedChannel(new DummyChannelId('server')) + serverChannel.freezeTime() + serverChannel.config().setAutoRead(true) + serverChannel.pipeline() + .addLast(tlsHandler()) + .addLast(new ApplicationProtocolNegotiationHandler("h2") { + @Override + protected void configurePipeline(ChannelHandlerContext chtx, String protocol) throws Exception { + chtx.pipeline() + .addLast(Http2FrameCodecBuilder.forServer().build()) + } + }) + + def clientChannel = new EmbeddedChannel(new DummyChannelId('client')) + clientChannel.freezeTime() + EmbeddedTestUtil.connect(serverChannel, clientChannel) + patch(client, clientChannel) + + when: + def future = Mono.from(client.exchange('https://example.com/foo')).toFuture() + future.exceptionally(t -> t.printStackTrace()) + EmbeddedTestUtil.advance(serverChannel, clientChannel) + then: + serverChannel.readInbound() instanceof Http2SettingsFrame + + when: + serverChannel.writeOutbound(new DefaultHttp2SettingsFrame(Http2Settings.defaultSettings())) + EmbeddedTestUtil.advance(serverChannel, clientChannel) + then: + serverChannel.readInbound() instanceof Http2SettingsAckFrame + Http2HeadersFrame request = serverChannel.readInbound() + request.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/foo' + request.headers().get(Http2Headers.PseudoHeaderName.SCHEME.value()) == 'https' + request.headers().get(Http2Headers.PseudoHeaderName.AUTHORITY.value()) == 'example.com' + request.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' + + when: + def responseHeaders = new DefaultHttp2Headers() + responseHeaders.add(Http2Headers.PseudoHeaderName.STATUS.value(), "200") + serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, true).stream(request.stream())) + EmbeddedTestUtil.advance(serverChannel, clientChannel) + then: + def response = future.get() + response.status() == HttpStatus.OK + + cleanup: + client.close() + ctx.close() + } +} diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/DummyChannelId.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/DummyChannelId.groovy new file mode 100644 index 00000000000..c5d6af6d3ff --- /dev/null +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/DummyChannelId.groovy @@ -0,0 +1,26 @@ +package io.micronaut.http.client.netty + +import io.netty.channel.ChannelId + +class DummyChannelId implements ChannelId { + final String name + + DummyChannelId(String name) { + this.name = name + } + + @Override + String asShortText() { + return name + } + + @Override + String asLongText() { + return name + } + + @Override + int compareTo(ChannelId o) { + return asLongText() <=> o.asLongText() + } +} diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy new file mode 100644 index 00000000000..ce03ec4db70 --- /dev/null +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy @@ -0,0 +1,99 @@ +package io.micronaut.http.client.netty + +import io.netty.buffer.ByteBuf +import io.netty.channel.ChannelHandlerContext +import io.netty.channel.ChannelOutboundHandlerAdapter +import io.netty.channel.ChannelPromise +import io.netty.channel.embedded.EmbeddedChannel + +// todo: can we unify this with the util class in http-server-netty tests? +class EmbeddedTestUtil { + static void advance(EmbeddedChannel... channels) { + boolean advanced + do { + advanced = false + for (EmbeddedChannel channel : channels) { + if (channel.hasPendingTasks()) { + advanced = true + channel.runPendingTasks() + } + channel.checkException() + } + } while (advanced); + } + + static void connect(EmbeddedChannel server, EmbeddedChannel client) { + new ConnectionDirection(server, client).register() + new ConnectionDirection(client, server).register() + } + + private static class ConnectionDirection { + private static final Object FLUSH = new Object() + + final EmbeddedChannel source + final EmbeddedChannel dest + final Queue queue = new ArrayDeque<>() + boolean readPending + boolean newData = false + + ConnectionDirection(EmbeddedChannel source, EmbeddedChannel dest) { + this.source = source + this.dest = dest + } + + private void forwardLater(Object msg) { + if (readPending || dest.config().isAutoRead()) { + dest.eventLoop().execute(() -> forwardNow(msg)) + readPending = false + } else { + queue.add(msg) + } + } + + private void forwardNow(Object msg) { + if (msg == FLUSH) { + if (dest.isOpen() && newData) { + dest.pipeline().fireChannelReadComplete() + newData = false + } + } else { + newData = true + dest.writeOneInbound(msg) + } + } + + void register() { + source.pipeline().addFirst(new ChannelOutboundHandlerAdapter() { + @Override + void write(ChannelHandlerContext ctx_, Object msg, ChannelPromise promise) throws Exception { + if (!(msg instanceof ByteBuf)) { + throw new IllegalArgumentException("Can only forward bytes!") + } + if (!msg.isReadable()) { + // no data + msg.release() + promise.setSuccess() + return + } + forwardLater(msg) + promise.setSuccess() + } + + @Override + void flush(ChannelHandlerContext ctx_) throws Exception { + forwardLater(FLUSH) + } + }) + dest.pipeline().addFirst(new ChannelOutboundHandlerAdapter() { + @Override + void read(ChannelHandlerContext ctx) throws Exception { + if (queue.isEmpty()) { + readPending = true + } else { + ctx.fireChannelRead(queue.poll()) + } + } + }) + } + } +} From 434fd9e98e44bd04abf34cf76dafda8c8960a65d Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 19 Sep 2022 13:09:39 +0200 Subject: [PATCH 02/82] streaming get --- .../http/client/netty/ConnectionManager.java | 22 +++ .../client/netty/ConnectionManagerSpec.groovy | 143 +++++++++++++----- .../stream/HttpStreamsClientHandler.java | 6 + 3 files changed, 137 insertions(+), 34 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 59e4c36a1ac..83abc475fba 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -481,6 +481,7 @@ private PoolHandle mockPoolHandle(Channel channel) { Mono connectForExchange(DefaultHttpClient.RequestKey requestKey, boolean multipart, boolean acceptEvents) { if (true) { Pool pool = pools.computeIfAbsent(requestKey, Pool::new); + // todo: aggregator return pool.acquire(); } return Mono.create(emitter -> { @@ -554,6 +555,27 @@ private Publisher delayUntilHttp2Ready(PoolHandle poolHandle) { * @return A mono that will complete once the channel is ready for transmission */ Mono connectForStream(DefaultHttpClient.RequestKey requestKey, boolean isProxy, boolean acceptEvents) { + if (true) { + Pool pool = pools.computeIfAbsent(requestKey, Pool::new); + return pool.acquire() + .map(ph -> { + // TODO: this sucks + ph.channel.pipeline().addLast( + ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, + new HttpStreamsClientHandler() { + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof IdleStateEvent) { + // close the connection if it is idle for too long + ctx.close(); + } + super.userEventTriggered(ctx, evt); + } + } + ); + return ph; + }); + } return Mono.create(emitter -> { ChannelFuture channelFuture; try { diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 06d9c9df03e..f03111603ae 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -1,17 +1,19 @@ package io.micronaut.http.client.netty import io.micronaut.context.ApplicationContext +import io.micronaut.http.HttpRequest import io.micronaut.http.HttpStatus import io.micronaut.http.HttpVersion import io.micronaut.http.server.netty.ssl.CertificateProvidedSslBuilder import io.micronaut.http.ssl.SslConfiguration import io.netty.buffer.ByteBufAllocator +import io.netty.buffer.Unpooled import io.netty.channel.Channel import io.netty.channel.ChannelFuture -import io.netty.channel.ChannelHandler import io.netty.channel.ChannelHandlerContext import io.netty.channel.ChannelInitializer import io.netty.channel.embedded.EmbeddedChannel +import io.netty.handler.codec.http2.DefaultHttp2DataFrame import io.netty.handler.codec.http2.DefaultHttp2Headers import io.netty.handler.codec.http2.DefaultHttp2HeadersFrame import io.netty.handler.codec.http2.DefaultHttp2SettingsFrame @@ -24,9 +26,12 @@ import io.netty.handler.codec.http2.Http2SettingsFrame import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler import io.netty.handler.ssl.SslContextBuilder import io.netty.handler.ssl.util.SelfSignedCertificate +import reactor.core.publisher.Flux import reactor.core.publisher.Mono import spock.lang.Specification +import java.nio.charset.StandardCharsets + class ConnectionManagerSpec extends Specification { private static void patch(DefaultHttpClient httpClient, EmbeddedChannel... channels) { httpClient.connectionManager = new ConnectionManager(httpClient.connectionManager) { @@ -43,51 +48,30 @@ class ConnectionManagerSpec extends Specification { } } - private static ChannelHandler tlsHandler() { - def certificate = new SelfSignedCertificate() - def builder = SslContextBuilder.forServer(certificate.key(), certificate.cert()) - CertificateProvidedSslBuilder.setupSslBuilder(builder, new SslConfiguration(), HttpVersion.HTTP_2_0); - return builder.build().newHandler(ByteBufAllocator.DEFAULT) - } - - def 'simple get'() { + def 'simple http2 get'() { given: def ctx = ApplicationContext.run([ 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, ]) def client = ctx.getBean(DefaultHttpClient) - def serverChannel = new EmbeddedChannel(new DummyChannelId('server')) - serverChannel.freezeTime() - serverChannel.config().setAutoRead(true) - serverChannel.pipeline() - .addLast(tlsHandler()) - .addLast(new ApplicationProtocolNegotiationHandler("h2") { - @Override - protected void configurePipeline(ChannelHandlerContext chtx, String protocol) throws Exception { - chtx.pipeline() - .addLast(Http2FrameCodecBuilder.forServer().build()) - } - }) - - def clientChannel = new EmbeddedChannel(new DummyChannelId('client')) - clientChannel.freezeTime() - EmbeddedTestUtil.connect(serverChannel, clientChannel) - patch(client, clientChannel) + def conn = new EmbeddedTestConnection(ctx) + conn.setupHttp2Tls() + patch(client, conn.clientChannel) when: def future = Mono.from(client.exchange('https://example.com/foo')).toFuture() future.exceptionally(t -> t.printStackTrace()) - EmbeddedTestUtil.advance(serverChannel, clientChannel) + conn.advance() then: - serverChannel.readInbound() instanceof Http2SettingsFrame + conn.serverChannel.readInbound() instanceof Http2SettingsFrame when: - serverChannel.writeOutbound(new DefaultHttp2SettingsFrame(Http2Settings.defaultSettings())) - EmbeddedTestUtil.advance(serverChannel, clientChannel) + conn.serverChannel.writeOutbound(new DefaultHttp2SettingsFrame(Http2Settings.defaultSettings())) + conn.advance() then: - serverChannel.readInbound() instanceof Http2SettingsAckFrame - Http2HeadersFrame request = serverChannel.readInbound() + conn.serverChannel.readInbound() instanceof Http2SettingsAckFrame + Http2HeadersFrame request = conn.serverChannel.readInbound() request.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/foo' request.headers().get(Http2Headers.PseudoHeaderName.SCHEME.value()) == 'https' request.headers().get(Http2Headers.PseudoHeaderName.AUTHORITY.value()) == 'example.com' @@ -96,8 +80,8 @@ class ConnectionManagerSpec extends Specification { when: def responseHeaders = new DefaultHttp2Headers() responseHeaders.add(Http2Headers.PseudoHeaderName.STATUS.value(), "200") - serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, true).stream(request.stream())) - EmbeddedTestUtil.advance(serverChannel, clientChannel) + conn.serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, true).stream(request.stream())) + conn.advance() then: def response = future.get() response.status() == HttpStatus.OK @@ -106,4 +90,95 @@ class ConnectionManagerSpec extends Specification { client.close() ctx.close() } + + def 'http2 streaming get'() { + given: + def ctx = ApplicationContext.run([ + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnection(ctx) + conn.setupHttp2Tls() + patch(client, conn.clientChannel) + + when: + def responseData = new ArrayDeque() + def responseComplete = false + Flux.from(client.dataStream(HttpRequest.GET('https://example.com/foo'))) + .doOnError(t -> t.printStackTrace()) + .doOnComplete(() -> responseComplete = true) + .subscribe(b -> responseData.add(b.toString(StandardCharsets.UTF_8))) + conn.advance() + then: + conn.serverChannel.readInbound() instanceof Http2SettingsFrame + + when: + conn.serverChannel.writeOutbound(new DefaultHttp2SettingsFrame(Http2Settings.defaultSettings())) + conn.advance() + then: + conn.serverChannel.readInbound() instanceof Http2SettingsAckFrame + Http2HeadersFrame request = conn.serverChannel.readInbound() + request.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/foo' + request.headers().get(Http2Headers.PseudoHeaderName.SCHEME.value()) == 'https' + request.headers().get(Http2Headers.PseudoHeaderName.AUTHORITY.value()) == 'example.com' + request.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' + + when: + def responseHeaders = new DefaultHttp2Headers() + responseHeaders.add(Http2Headers.PseudoHeaderName.STATUS.value(), "200") + conn.serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, false).stream(request.stream())) + conn.serverChannel.writeOutbound(new DefaultHttp2DataFrame(Unpooled.wrappedBuffer('foo'.bytes)).stream(request.stream())) + conn.advance() + then: + responseData.poll() == 'foo' + !responseComplete + + when: + conn.serverChannel.writeOutbound(new DefaultHttp2DataFrame(Unpooled.wrappedBuffer('bar'.bytes), true).stream(request.stream())) + conn.advance() + then: + responseData.poll() == 'bar' + responseComplete + + cleanup: + client.close() + ctx.close() + } + + static class EmbeddedTestConnection { + final EmbeddedChannel serverChannel + final EmbeddedChannel clientChannel + + EmbeddedTestConnection(ApplicationContext ctx) { + serverChannel = new EmbeddedChannel(new DummyChannelId('server')) + serverChannel.freezeTime() + serverChannel.config().setAutoRead(true) + + clientChannel = new EmbeddedChannel(new DummyChannelId('client')) + clientChannel.freezeTime() + EmbeddedTestUtil.connect(serverChannel, clientChannel) + } + + void setupHttp2Tls() { + def certificate = new SelfSignedCertificate() + def builder = SslContextBuilder.forServer(certificate.key(), certificate.cert()) + CertificateProvidedSslBuilder.setupSslBuilder(builder, new SslConfiguration(), HttpVersion.HTTP_2_0); + def tlsHandler = builder.build().newHandler(ByteBufAllocator.DEFAULT) + + serverChannel.pipeline() + .addLast(tlsHandler) + .addLast(new ApplicationProtocolNegotiationHandler("h2") { + @Override + protected void configurePipeline(ChannelHandlerContext chtx, String protocol) throws Exception { + chtx.pipeline() + .addLast(Http2FrameCodecBuilder.forServer().build()) + } + }) + } + + void advance() { + EmbeddedTestUtil.advance(serverChannel, clientChannel) + } + } } diff --git a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java index f7702f69059..fecd63000fc 100644 --- a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java +++ b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java @@ -181,6 +181,12 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } } + @Override + protected int getStreamId(Object msg) { + // TODO + return -1; + } + @Override public void write(final ChannelHandlerContext ctx, Object msg, final ChannelPromise promise) throws Exception { if (ctx.channel().attr(AttributeKey.valueOf(ChannelPipelineCustomizer.HANDLER_HTTP_CHUNK)).get() == Boolean.TRUE) { From 2b620de2aeb51b9195019f7f8f959adbe9a82071 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 19 Sep 2022 16:59:24 +0200 Subject: [PATCH 03/82] http1 plaintext support --- .../http/client/netty/ConnectionManager.java | 172 +++++++++++++++--- .../http/client/netty/DefaultHttpClient.java | 3 +- .../client/netty/ConnectionManagerSpec.groovy | 98 +++++++++- 3 files changed, 248 insertions(+), 25 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 83abc475fba..7215cc7788c 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -130,6 +130,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; @@ -482,7 +483,19 @@ Mono connectForExchange(DefaultHttpClient.RequestKey requestKey, boo if (true) { Pool pool = pools.computeIfAbsent(requestKey, Pool::new); // todo: aggregator - return pool.acquire(); + return pool.acquire().map(ph -> { + ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR, new HttpObjectAggregator(configuration.getMaxContentLength()) { + @Override + protected void finishAggregation(FullHttpMessage aggregated) throws Exception { + if (!HttpUtil.isContentLengthSet(aggregated)) { + if (aggregated.content().readableBytes() > 0) { + super.finishAggregation(aggregated); + } + } + } + }); + return ph; + }); } return Mono.create(emitter -> { if (poolMap != null && !multipart) { @@ -1226,10 +1239,8 @@ void addHttp1Handlers(ChannelPipeline p) { p.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); - int maxContentLength = configuration.getMaxContentLength(); - if (!stream) { - p.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR, new HttpObjectAggregator(maxContentLength) { + p.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR, new HttpObjectAggregator(configuration.getMaxContentLength()) { @Override protected void finishAggregation(FullHttpMessage aggregated) throws Exception { if (!HttpUtil.isContentLengthSet(aggregated)) { @@ -1332,7 +1343,6 @@ private class AdaptiveAlpnChannelInitializer extends ChannelInitializer final SslContext sslContext; final String host; final int port; - Http2SettingsHandler settingsHandler; private NettyClientCustomizer channelCustomizer; AdaptiveAlpnChannelInitializer(Pool pool, @@ -1378,14 +1388,6 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { }) .addLast(initialErrorHandler); } - - /** - * Called when the stream pipeline is fully set up (all handshakes completed) and we can - * start processing requests. - */ - void onStreamPipelineBuilt() { - channelCustomizer.onStreamPipelineBuilt(); - } } private class InitialConnectionErrorHandler extends ChannelInboundHandlerAdapter { @@ -1407,6 +1409,37 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } } + private class Http1Initializer extends ChannelInitializer { + final Pool pool; + final boolean secure; + final String host; + final int port; + + Http1Initializer(Pool pool, + boolean secure, + String host, + int port) { + this.pool = pool; + this.secure = secure; + this.host = host; + this.port = port; + } + + /** + * @param ch The channel + */ + @Override + protected void initChannel(Channel ch) { + configureProxy(ch.pipeline(), secure, host, port); // todo + + ch.pipeline() + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()) + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); + + pool.new Http1ConnectionHolder(ch).init(); + } + } + private class Http2PriorKnowledgeInitializer extends ChannelInitializer { private final Pool pool; @@ -1544,6 +1577,7 @@ private final class Pool { private final DefaultHttpClient.RequestKey requestKey; private final Queue> pendingRequests = new ConcurrentLinkedQueue<>(); + private final List http1Connections = new CopyOnWriteArrayList<>(); private final List http2Connections = new CopyOnWriteArrayList<>(); private final AtomicInteger pendingConnectionCount = new AtomicInteger(0); @@ -1563,7 +1597,12 @@ private void acquire(Sinks.One sink) { return; } } - // no http2 connection open that has room + for (Http1ConnectionHolder http1Connection : http1Connections) { + if (http1Connection.satisfy(sink)) { + return; + } + } + // no connection open that has room pendingRequests.add(sink); // todo: http1 openNewConnection(); @@ -1574,13 +1613,23 @@ private void openNewConnection() { return; } // open a new connection - ChannelFuture connectFuture = doConnect(requestKey, new AdaptiveAlpnChannelInitializer( - this, - buildSslContext(requestKey), - requestKey.getHost(), - requestKey.getPort() - )); - addInstrumentedListener(connectFuture, future -> { + ChannelInitializer initializer; + if (requestKey.isSecure()) { + initializer = new AdaptiveAlpnChannelInitializer( + this, + buildSslContext(requestKey), + requestKey.getHost(), + requestKey.getPort() + ); + } else { + initializer = new Http1Initializer( + this, + false, + requestKey.getHost(), + requestKey.getPort() + ); + } + addInstrumentedListener(doConnect(requestKey, initializer), future -> { if (!future.isSuccess()) { log.error("Failed to connect to remote", future.cause()); onNewConnectionFailure(); @@ -1593,6 +1642,87 @@ void onNewConnectionFailure() { // todo: retry connection } + final class Http1ConnectionHolder { + private final Channel channel; + private final AtomicBoolean hasLiveRequest = new AtomicBoolean(false); + private volatile boolean canReturn = true; + + Http1ConnectionHolder(Channel channel) { + this.channel = channel; + } + + void init() { + channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + super.channelInactive(ctx); + canReturn = false; + } + }); + + // todo: notify customizer + + Sinks.One pendingRequest = pendingRequests.poll(); + if (pendingRequest != null) { + hasLiveRequest.set(true); + satisfy0(pendingRequest); + } + pendingConnectionCount.decrementAndGet(); + } + + boolean satisfy(Sinks.One sink) { + if (!hasLiveRequest.compareAndSet(false, true)) { + return false; + } + + if (channel.eventLoop().inEventLoop()) { + satisfy0(sink); + } else { + channel.eventLoop().execute(() -> satisfy0(sink)); + } + return true; + } + + private void satisfy0(Sinks.One sink) { + if (!channel.isActive()) { + returnPendingRequest(sink); + return; + } + Sinks.EmitResult emitResult = sink.tryEmitValue(new PoolHandle(channel) { + @Override + void taint() { + canReturn = false; + } + + @Override + void release() { + hasLiveRequest.set(false); + // todo: claim a new request + } + + @Override + boolean canReturn() { + return canReturn; + } + + @Override + void notifyRequestPipelineBuilt() { + // TODO + } + }); + if (emitResult.isFailure()) { + hasLiveRequest.set(false); + // todo: claim a new request + } + } + + private void returnPendingRequest(Sinks.One sink) { + // failed, but the pending request may still work on another connection. + pendingRequests.add(sink); + hasLiveRequest.set(false); + } + } + final class Http2ConnectionHolder { private final Channel channel; private final AtomicInteger liveRequests = new AtomicInteger(0); diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index f8347da1bc5..fee9a8dac1a 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -968,7 +968,8 @@ private Publisher> buildStreamExchange( applyFilterToResponsePublisher(parentRequest, request, requestURI, requestWrapper, streamResponsePublisher) ); - return streamResponsePublisher.subscribeOn(connectionManager.getEventLoopScheduler()); + //return streamResponsePublisher.subscribeOn(connectionManager.getEventLoopScheduler()); + return streamResponsePublisher; // todo } @Override diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index f03111603ae..51ec0bc1bd8 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -13,6 +13,12 @@ import io.netty.channel.ChannelFuture import io.netty.channel.ChannelHandlerContext import io.netty.channel.ChannelInitializer import io.netty.channel.embedded.EmbeddedChannel +import io.netty.handler.codec.http.DefaultHttpContent +import io.netty.handler.codec.http.DefaultHttpResponse +import io.netty.handler.codec.http.DefaultLastHttpContent +import io.netty.handler.codec.http.HttpMethod +import io.netty.handler.codec.http.HttpResponseStatus +import io.netty.handler.codec.http.HttpServerCodec import io.netty.handler.codec.http2.DefaultHttp2DataFrame import io.netty.handler.codec.http2.DefaultHttp2Headers import io.netty.handler.codec.http2.DefaultHttp2HeadersFrame @@ -23,6 +29,8 @@ import io.netty.handler.codec.http2.Http2HeadersFrame import io.netty.handler.codec.http2.Http2Settings import io.netty.handler.codec.http2.Http2SettingsAckFrame import io.netty.handler.codec.http2.Http2SettingsFrame +import io.netty.handler.logging.LogLevel +import io.netty.handler.logging.LoggingHandler import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler import io.netty.handler.ssl.SslContextBuilder import io.netty.handler.ssl.util.SelfSignedCertificate @@ -55,7 +63,7 @@ class ConnectionManagerSpec extends Specification { ]) def client = ctx.getBean(DefaultHttpClient) - def conn = new EmbeddedTestConnection(ctx) + def conn = new EmbeddedTestConnection() conn.setupHttp2Tls() patch(client, conn.clientChannel) @@ -98,7 +106,7 @@ class ConnectionManagerSpec extends Specification { ]) def client = ctx.getBean(DefaultHttpClient) - def conn = new EmbeddedTestConnection(ctx) + def conn = new EmbeddedTestConnection() conn.setupHttp2Tls() patch(client, conn.clientChannel) @@ -146,11 +154,89 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'simple http1 get'() { + given: + def ctx = ApplicationContext.run() + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnection() + conn.setupHttp1() + patch(client, conn.clientChannel) + + when: + def future = Mono.from(client.exchange('http://example.com/foo')).toFuture() + future.exceptionally(t -> t.printStackTrace()) + conn.advance() + then: + io.netty.handler.codec.http.HttpRequest request = conn.serverChannel.readInbound() + request.uri() == '/foo' + request.method() == HttpMethod.GET + request.headers().get('host') == 'example.com' + + when: + def response = new DefaultHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) + response.headers().add('content-length', 0) + conn.serverChannel.writeOutbound(response) + conn.advance() + then: + def mnResponse = future.get() + mnResponse.status() == HttpStatus.OK + + cleanup: + client.close() + ctx.close() + } + + def 'http1 streaming get'() { + given: + def ctx = ApplicationContext.run() + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnection() + conn.setupHttp1() + patch(client, conn.clientChannel) + + when: + def responseData = new ArrayDeque() + def responseComplete = false + Flux.from(client.dataStream(HttpRequest.GET('http://example.com/foo'))) + .doOnError(t -> t.printStackTrace()) + .doOnComplete(() -> responseComplete = true) + .subscribe(b -> responseData.add(b.toString(StandardCharsets.UTF_8))) + conn.advance() + then: + io.netty.handler.codec.http.HttpRequest request = conn.serverChannel.readInbound() + request.uri() == '/foo' + request.method() == HttpMethod.GET + request.headers().get('host') == 'example.com' + + when: + def response = new DefaultHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) + response.headers().add('content-length', 6) + conn.serverChannel.writeOutbound(response) + conn.serverChannel.writeOutbound(new DefaultHttpContent(Unpooled.wrappedBuffer('foo'.bytes))) + conn.advance() + then: + responseData.poll() == 'foo' + !responseComplete + + when: + conn.serverChannel.writeOutbound(new DefaultLastHttpContent(Unpooled.wrappedBuffer('bar'.bytes))) + conn.advance() + then: + responseData.poll() == 'bar' + responseComplete + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnection { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel - EmbeddedTestConnection(ApplicationContext ctx) { + EmbeddedTestConnection() { serverChannel = new EmbeddedChannel(new DummyChannelId('server')) serverChannel.freezeTime() serverChannel.config().setAutoRead(true) @@ -177,6 +263,12 @@ class ConnectionManagerSpec extends Specification { }) } + void setupHttp1() { + serverChannel.pipeline() + .addLast(new LoggingHandler(LogLevel.INFO)) + .addLast(new HttpServerCodec()) + } + void advance() { EmbeddedTestUtil.advance(serverChannel, clientChannel) } From 8f0e4b4ddc13df5954cfab8400b25c10d18b589a Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 19 Sep 2022 19:20:08 +0200 Subject: [PATCH 04/82] test http2 connection reuse --- .../client/netty/ConnectionManagerSpec.groovy | 125 ++++++++++++++---- 1 file changed, 97 insertions(+), 28 deletions(-) diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 51ec0bc1bd8..0d8843c27e4 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -22,15 +22,12 @@ import io.netty.handler.codec.http.HttpServerCodec import io.netty.handler.codec.http2.DefaultHttp2DataFrame import io.netty.handler.codec.http2.DefaultHttp2Headers import io.netty.handler.codec.http2.DefaultHttp2HeadersFrame -import io.netty.handler.codec.http2.DefaultHttp2SettingsFrame import io.netty.handler.codec.http2.Http2FrameCodecBuilder +import io.netty.handler.codec.http2.Http2FrameStream import io.netty.handler.codec.http2.Http2Headers import io.netty.handler.codec.http2.Http2HeadersFrame -import io.netty.handler.codec.http2.Http2Settings import io.netty.handler.codec.http2.Http2SettingsAckFrame import io.netty.handler.codec.http2.Http2SettingsFrame -import io.netty.handler.logging.LogLevel -import io.netty.handler.logging.LoggingHandler import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler import io.netty.handler.ssl.SslContextBuilder import io.netty.handler.ssl.util.SelfSignedCertificate @@ -70,15 +67,8 @@ class ConnectionManagerSpec extends Specification { when: def future = Mono.from(client.exchange('https://example.com/foo')).toFuture() future.exceptionally(t -> t.printStackTrace()) - conn.advance() + conn.exchangeSettings() then: - conn.serverChannel.readInbound() instanceof Http2SettingsFrame - - when: - conn.serverChannel.writeOutbound(new DefaultHttp2SettingsFrame(Http2Settings.defaultSettings())) - conn.advance() - then: - conn.serverChannel.readInbound() instanceof Http2SettingsAckFrame Http2HeadersFrame request = conn.serverChannel.readInbound() request.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/foo' request.headers().get(Http2Headers.PseudoHeaderName.SCHEME.value()) == 'https' @@ -86,9 +76,7 @@ class ConnectionManagerSpec extends Specification { request.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' when: - def responseHeaders = new DefaultHttp2Headers() - responseHeaders.add(Http2Headers.PseudoHeaderName.STATUS.value(), "200") - conn.serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, true).stream(request.stream())) + conn.respondOk(request.stream()) conn.advance() then: def response = future.get() @@ -117,15 +105,8 @@ class ConnectionManagerSpec extends Specification { .doOnError(t -> t.printStackTrace()) .doOnComplete(() -> responseComplete = true) .subscribe(b -> responseData.add(b.toString(StandardCharsets.UTF_8))) - conn.advance() - then: - conn.serverChannel.readInbound() instanceof Http2SettingsFrame - - when: - conn.serverChannel.writeOutbound(new DefaultHttp2SettingsFrame(Http2Settings.defaultSettings())) - conn.advance() + conn.exchangeSettings() then: - conn.serverChannel.readInbound() instanceof Http2SettingsAckFrame Http2HeadersFrame request = conn.serverChannel.readInbound() request.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/foo' request.headers().get(Http2Headers.PseudoHeaderName.SCHEME.value()) == 'https' @@ -174,9 +155,7 @@ class ConnectionManagerSpec extends Specification { request.headers().get('host') == 'example.com' when: - def response = new DefaultHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) - response.headers().add('content-length', 0) - conn.serverChannel.writeOutbound(response) + conn.respondOk() conn.advance() then: def mnResponse = future.get() @@ -232,6 +211,73 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'http2 concurrent stream'() { + given: + def ctx = ApplicationContext.run([ + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn1 = new EmbeddedTestConnection() + conn1.setupHttp2Tls() + def conn2 = new EmbeddedTestConnection() + conn2.setupHttp2Tls() + patch(client, conn1.clientChannel, conn2.clientChannel) + + when: + // start two requests. this will open two connections + def f1 = Mono.from(client.exchange('https://example.com/r1')).toFuture() + f1.exceptionally(t -> t.printStackTrace()) + def f2 = Mono.from(client.exchange('https://example.com/r2')).toFuture() + f2.exceptionally(t -> t.printStackTrace()) + + then: + // no data yet, haven't finished the handshake + conn1.serverChannel.readInbound() == null + + when: + // finish handshake for first connection + conn1.exchangeSettings() + then: + // both requests immediately go to the first connection + def req1 = conn1.serverChannel. readInbound() + req1.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/r1' + def req2 = conn1.serverChannel. readInbound() + req2.stream().id() != req1.stream().id() + req2.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/r2' + + when: + // start a third request, this should reuse the existing connection + def f3 = Mono.from(client.exchange('https://example.com/r3')).toFuture() + f3.exceptionally(t -> t.printStackTrace()) + conn1.advance() + then: + def req3 = conn1.serverChannel. readInbound() + req3.stream().id() != req1.stream().id() + req3.stream().id() != req2.stream().id() + req3.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/r3' + + // finish up the third request + when: + conn1.respondOk(req3.stream()) + conn1.advance() + then: + f3.get().status() == HttpStatus.OK + + // finish up the second and first request + when: + conn1.respondOk(req2.stream()) + conn1.respondOk(req1.stream()) + conn1.advance() + then: + f1.get().status() == HttpStatus.OK + f2.get().status() == HttpStatus.OK + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnection { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel @@ -258,19 +304,42 @@ class ConnectionManagerSpec extends Specification { @Override protected void configurePipeline(ChannelHandlerContext chtx, String protocol) throws Exception { chtx.pipeline() - .addLast(Http2FrameCodecBuilder.forServer().build()) + .addLast(Http2FrameCodecBuilder.forServer() + .autoAckSettingsFrame(false) + .build()) } }) } void setupHttp1() { serverChannel.pipeline() - .addLast(new LoggingHandler(LogLevel.INFO)) .addLast(new HttpServerCodec()) } void advance() { EmbeddedTestUtil.advance(serverChannel, clientChannel) } + + void exchangeSettings() { + advance() + if (!(serverChannel.readInbound() instanceof Http2SettingsFrame)) { + throw new AssertionError() + } + if (!(serverChannel.readInbound() instanceof Http2SettingsAckFrame)) { + throw new AssertionError() + } + } + + void respondOk(Http2FrameStream stream) { + def responseHeaders = new DefaultHttp2Headers() + responseHeaders.add(Http2Headers.PseudoHeaderName.STATUS.value(), "200") + serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, true).stream(stream)) + } + + void respondOk() { + def response = new DefaultHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) + response.headers().add('content-length', 0) + serverChannel.writeOutbound(response) + } } } From b19ef74f28aa20860751fed6e6197167650dabdb Mon Sep 17 00:00:00 2001 From: yawkat Date: Tue, 20 Sep 2022 10:43:12 +0200 Subject: [PATCH 05/82] http1 connection reuse --- .../http/client/netty/ConnectionManager.java | 2 + .../http/client/netty/DefaultHttpClient.java | 1 + .../client/netty/ConnectionManagerSpec.groovy | 47 ++++++++++++++++++- 3 files changed, 49 insertions(+), 1 deletion(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 7215cc7788c..9df8a7a8f31 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -1656,6 +1656,7 @@ void init() { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { super.channelInactive(ctx); + http1Connections.remove(Http1ConnectionHolder.this); canReturn = false; } }); @@ -1667,6 +1668,7 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { hasLiveRequest.set(true); satisfy0(pendingRequest); } + http1Connections.add(this); pendingConnectionCount.decrementAndGet(); } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index fee9a8dac1a..d8ceeb8699e 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -2290,6 +2290,7 @@ public Argument getErrorType(MediaType mediaType) { @Override public void handlerRemoved(ChannelHandlerContext ctx) { + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR); poolHandle.release(); } diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 0d8843c27e4..77b85a16617 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -13,10 +13,12 @@ import io.netty.channel.ChannelFuture import io.netty.channel.ChannelHandlerContext import io.netty.channel.ChannelInitializer import io.netty.channel.embedded.EmbeddedChannel +import io.netty.handler.codec.http.DefaultFullHttpResponse import io.netty.handler.codec.http.DefaultHttpContent import io.netty.handler.codec.http.DefaultHttpResponse import io.netty.handler.codec.http.DefaultLastHttpContent import io.netty.handler.codec.http.HttpMethod +import io.netty.handler.codec.http.HttpObjectAggregator import io.netty.handler.codec.http.HttpResponseStatus import io.netty.handler.codec.http.HttpServerCodec import io.netty.handler.codec.http2.DefaultHttp2DataFrame @@ -278,6 +280,49 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'http1 reuse'() { + given: + def ctx = ApplicationContext.run() + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnection() + conn.setupHttp1() + conn.serverChannel.pipeline().addLast(new HttpObjectAggregator(1024)) + patch(client, conn.clientChannel) + + when: + def f1 = Mono.from(client.exchange('http://example.com/r1')).toFuture() + f1.exceptionally(t -> t.printStackTrace()) + conn.advance() + then: + io.netty.handler.codec.http.HttpRequest req1 = conn.serverChannel.readInbound() + req1.headers().get("connection") == "keep-alive" + + when: + conn.respondOk() + conn.advance() + then: + f1.get().status() == HttpStatus.OK + + when: + def f2 = Mono.from(client.exchange('http://example.com/r1')).toFuture() + f2.exceptionally(t -> t.printStackTrace()) + conn.advance() + then: + io.netty.handler.codec.http.HttpRequest req2 = conn.serverChannel.readInbound() + req2.headers().get("connection") == "keep-alive" + + when: + conn.respondOk() + conn.advance() + then: + f1.get().status() == HttpStatus.OK + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnection { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel @@ -337,7 +382,7 @@ class ConnectionManagerSpec extends Specification { } void respondOk() { - def response = new DefaultHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) + def response = new DefaultFullHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) response.headers().add('content-length', 0) serverChannel.writeOutbound(response) } From 5704549df289c71115e52e0507e0c092efb51f3d Mon Sep 17 00:00:00 2001 From: yawkat Date: Tue, 20 Sep 2022 10:57:31 +0200 Subject: [PATCH 06/82] restructure tests a bit --- .../client/netty/ConnectionManagerSpec.groovy | 188 ++++++++---------- 1 file changed, 86 insertions(+), 102 deletions(-) diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 77b85a16617..facacc88131 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -4,6 +4,8 @@ import io.micronaut.context.ApplicationContext import io.micronaut.http.HttpRequest import io.micronaut.http.HttpStatus import io.micronaut.http.HttpVersion +import io.micronaut.http.client.HttpClient +import io.micronaut.http.client.StreamingHttpClient import io.micronaut.http.server.netty.ssl.CertificateProvidedSslBuilder import io.micronaut.http.ssl.SslConfiguration import io.netty.buffer.ByteBufAllocator @@ -18,9 +20,9 @@ import io.netty.handler.codec.http.DefaultHttpContent import io.netty.handler.codec.http.DefaultHttpResponse import io.netty.handler.codec.http.DefaultLastHttpContent import io.netty.handler.codec.http.HttpMethod -import io.netty.handler.codec.http.HttpObjectAggregator import io.netty.handler.codec.http.HttpResponseStatus import io.netty.handler.codec.http.HttpServerCodec +import io.netty.handler.codec.http.LastHttpContent import io.netty.handler.codec.http2.DefaultHttp2DataFrame import io.netty.handler.codec.http2.DefaultHttp2Headers import io.netty.handler.codec.http2.DefaultHttp2HeadersFrame @@ -62,7 +64,7 @@ class ConnectionManagerSpec extends Specification { ]) def client = ctx.getBean(DefaultHttpClient) - def conn = new EmbeddedTestConnection() + def conn = new EmbeddedTestConnectionHttp2() conn.setupHttp2Tls() patch(client, conn.clientChannel) @@ -96,7 +98,7 @@ class ConnectionManagerSpec extends Specification { ]) def client = ctx.getBean(DefaultHttpClient) - def conn = new EmbeddedTestConnection() + def conn = new EmbeddedTestConnectionHttp2() conn.setupHttp2Tls() patch(client, conn.clientChannel) @@ -138,30 +140,14 @@ class ConnectionManagerSpec extends Specification { } def 'simple http1 get'() { - given: def ctx = ApplicationContext.run() def client = ctx.getBean(DefaultHttpClient) - def conn = new EmbeddedTestConnection() + def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() patch(client, conn.clientChannel) - when: - def future = Mono.from(client.exchange('http://example.com/foo')).toFuture() - future.exceptionally(t -> t.printStackTrace()) - conn.advance() - then: - io.netty.handler.codec.http.HttpRequest request = conn.serverChannel.readInbound() - request.uri() == '/foo' - request.method() == HttpMethod.GET - request.headers().get('host') == 'example.com' - - when: - conn.respondOk() - conn.advance() - then: - def mnResponse = future.get() - mnResponse.status() == HttpStatus.OK + conn.testExchange(client) cleanup: client.close() @@ -169,44 +155,14 @@ class ConnectionManagerSpec extends Specification { } def 'http1 streaming get'() { - given: def ctx = ApplicationContext.run() def client = ctx.getBean(DefaultHttpClient) - def conn = new EmbeddedTestConnection() + def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() patch(client, conn.clientChannel) - when: - def responseData = new ArrayDeque() - def responseComplete = false - Flux.from(client.dataStream(HttpRequest.GET('http://example.com/foo'))) - .doOnError(t -> t.printStackTrace()) - .doOnComplete(() -> responseComplete = true) - .subscribe(b -> responseData.add(b.toString(StandardCharsets.UTF_8))) - conn.advance() - then: - io.netty.handler.codec.http.HttpRequest request = conn.serverChannel.readInbound() - request.uri() == '/foo' - request.method() == HttpMethod.GET - request.headers().get('host') == 'example.com' - - when: - def response = new DefaultHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) - response.headers().add('content-length', 6) - conn.serverChannel.writeOutbound(response) - conn.serverChannel.writeOutbound(new DefaultHttpContent(Unpooled.wrappedBuffer('foo'.bytes))) - conn.advance() - then: - responseData.poll() == 'foo' - !responseComplete - - when: - conn.serverChannel.writeOutbound(new DefaultLastHttpContent(Unpooled.wrappedBuffer('bar'.bytes))) - conn.advance() - then: - responseData.poll() == 'bar' - responseComplete + conn.testStreaming(client) cleanup: client.close() @@ -220,9 +176,9 @@ class ConnectionManagerSpec extends Specification { ]) def client = ctx.getBean(DefaultHttpClient) - def conn1 = new EmbeddedTestConnection() + def conn1 = new EmbeddedTestConnectionHttp2() conn1.setupHttp2Tls() - def conn2 = new EmbeddedTestConnection() + def conn2 = new EmbeddedTestConnectionHttp2() conn2.setupHttp2Tls() patch(client, conn1.clientChannel, conn2.clientChannel) @@ -281,53 +237,26 @@ class ConnectionManagerSpec extends Specification { } def 'http1 reuse'() { - given: def ctx = ApplicationContext.run() def client = ctx.getBean(DefaultHttpClient) - def conn = new EmbeddedTestConnection() + def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() - conn.serverChannel.pipeline().addLast(new HttpObjectAggregator(1024)) patch(client, conn.clientChannel) - when: - def f1 = Mono.from(client.exchange('http://example.com/r1')).toFuture() - f1.exceptionally(t -> t.printStackTrace()) - conn.advance() - then: - io.netty.handler.codec.http.HttpRequest req1 = conn.serverChannel.readInbound() - req1.headers().get("connection") == "keep-alive" - - when: - conn.respondOk() - conn.advance() - then: - f1.get().status() == HttpStatus.OK - - when: - def f2 = Mono.from(client.exchange('http://example.com/r1')).toFuture() - f2.exceptionally(t -> t.printStackTrace()) - conn.advance() - then: - io.netty.handler.codec.http.HttpRequest req2 = conn.serverChannel.readInbound() - req2.headers().get("connection") == "keep-alive" - - when: - conn.respondOk() - conn.advance() - then: - f1.get().status() == HttpStatus.OK + conn.testExchange(client) + conn.testExchange(client) cleanup: client.close() ctx.close() } - static class EmbeddedTestConnection { + static class EmbeddedTestConnectionBase { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel - EmbeddedTestConnection() { + EmbeddedTestConnectionBase() { serverChannel = new EmbeddedChannel(new DummyChannelId('server')) serverChannel.freezeTime() serverChannel.config().setAutoRead(true) @@ -337,6 +266,76 @@ class ConnectionManagerSpec extends Specification { EmbeddedTestUtil.connect(serverChannel, clientChannel) } + void advance() { + EmbeddedTestUtil.advance(serverChannel, clientChannel) + } + } + + static class EmbeddedTestConnectionHttp1 extends EmbeddedTestConnectionBase { + void setupHttp1() { + serverChannel.pipeline() + .addLast(new HttpServerCodec()) + } + + void respondOk() { + def response = new DefaultFullHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) + response.headers().add('content-length', 0) + serverChannel.writeOutbound(response) + } + + void testExchange(HttpClient client) { + def future = Mono.from(client.exchange('http://example.com/foo')).toFuture() + future.exceptionally(t -> t.printStackTrace()) + advance() + + io.netty.handler.codec.http.HttpRequest request = serverChannel.readInbound() + assert request.uri() == '/foo' + assert request.method() == HttpMethod.GET + assert request.headers().get('host') == 'example.com' + assert request.headers().get("connection") == "keep-alive" + + def tail = serverChannel.readInbound() + assert tail == null || tail instanceof LastHttpContent + + respondOk() + advance() + + assert future.get().status() == HttpStatus.OK + } + + void testStreaming(StreamingHttpClient client) { + def responseData = new ArrayDeque() + def responseComplete = false + Flux.from(client.dataStream(HttpRequest.GET('http://example.com/foo'))) + .doOnError(t -> t.printStackTrace()) + .doOnComplete(() -> responseComplete = true) + .subscribe(b -> responseData.add(b.toString(StandardCharsets.UTF_8))) + advance() + + io.netty.handler.codec.http.HttpRequest request = serverChannel.readInbound() + assert request.uri() == '/foo' + assert request.method() == HttpMethod.GET + assert request.headers().get('host') == 'example.com' + //assert request.headers().get("connection") == "keep-alive" + + def response = new DefaultHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) + response.headers().add('content-length', 6) + serverChannel.writeOutbound(response) + serverChannel.writeOutbound(new DefaultHttpContent(Unpooled.wrappedBuffer('foo'.bytes))) + advance() + + assert responseData.poll() == 'foo' + assert !responseComplete + + serverChannel.writeOutbound(new DefaultLastHttpContent(Unpooled.wrappedBuffer('bar'.bytes))) + advance() + + assert responseData.poll() == 'bar' + assert responseComplete + } + } + + static class EmbeddedTestConnectionHttp2 extends EmbeddedTestConnectionBase { void setupHttp2Tls() { def certificate = new SelfSignedCertificate() def builder = SslContextBuilder.forServer(certificate.key(), certificate.cert()) @@ -356,15 +355,6 @@ class ConnectionManagerSpec extends Specification { }) } - void setupHttp1() { - serverChannel.pipeline() - .addLast(new HttpServerCodec()) - } - - void advance() { - EmbeddedTestUtil.advance(serverChannel, clientChannel) - } - void exchangeSettings() { advance() if (!(serverChannel.readInbound() instanceof Http2SettingsFrame)) { @@ -380,11 +370,5 @@ class ConnectionManagerSpec extends Specification { responseHeaders.add(Http2Headers.PseudoHeaderName.STATUS.value(), "200") serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, true).stream(stream)) } - - void respondOk() { - def response = new DefaultFullHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) - response.headers().add('content-length', 0) - serverChannel.writeOutbound(response) - } } } From 3aa122690de6bd1c8dac5fa812a4c1ef16d57ed1 Mon Sep 17 00:00:00 2001 From: yawkat Date: Tue, 20 Sep 2022 12:17:50 +0200 Subject: [PATCH 07/82] reuse streaming http1 connections --- .../http/client/netty/ConnectionManager.java | 46 +++++- .../http/client/netty/DefaultHttpClient.java | 145 ++++++++---------- .../client/netty/ConnectionManagerSpec.groovy | 7 +- 3 files changed, 117 insertions(+), 81 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 9df8a7a8f31..0a100b4ac2e 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -69,6 +69,8 @@ import io.netty.handler.codec.http.HttpMessage; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.LastHttpContent; import io.netty.handler.codec.http.websocketx.extensions.compression.WebSocketClientCompressionHandler; @@ -573,6 +575,30 @@ Mono connectForStream(DefaultHttpClient.RequestKey requestKey, boole return pool.acquire() .map(ph -> { // TODO: this sucks + ph.channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { + boolean ignoreOneLast = false; + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof HttpResponse && + ((HttpResponse) msg).status().equals(HttpResponseStatus.CONTINUE)) { + ignoreOneLast = true; + } + + super.channelRead(ctx, msg); + + if (msg instanceof LastHttpContent) { + if (ignoreOneLast) { + ignoreOneLast = false; + } else { + ctx.pipeline() + .remove(this) + .remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); + ph.release(); + } + } + } + }); ph.channel.pipeline().addLast( ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, new HttpStreamsClientHandler() { @@ -580,7 +606,8 @@ Mono connectForStream(DefaultHttpClient.RequestKey requestKey, boole public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { if (evt instanceof IdleStateEvent) { // close the connection if it is idle for too long - ctx.close(); + ph.taint(); + ph.release(); } super.userEventTriggered(ctx, evt); } @@ -1691,6 +1718,8 @@ private void satisfy0(Sinks.One sink) { return; } Sinks.EmitResult emitResult = sink.tryEmitValue(new PoolHandle(channel) { + final ChannelHandlerContext lastContext = channel.pipeline().lastContext(); + @Override void taint() { canReturn = false; @@ -1698,8 +1727,19 @@ void taint() { @Override void release() { - hasLiveRequest.set(false); - // todo: claim a new request + if (canReturn) { + ChannelHandlerContext newLast = channel.pipeline().lastContext(); + if (lastContext != newLast) { + log.warn("BUG - Handler not removed: {}", newLast); + taint(); + } + } + if (canReturn) { + hasLiveRequest.set(false); + // todo: claim a new request + } else { + channel.close(); + } } @Override diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index d8ceeb8699e..0947b388104 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -867,13 +867,7 @@ private Flux>> exchangeStreamImpl(io.micronaut.ht return (HttpResponse>) new HttpResponseWrapper<>(thisResponse); }); }).doOnTerminate(() -> { - final Object o = request.getAttribute(NettyClientHttpRequest.CHANNEL).orElse(null); - if (o instanceof Channel) { - final Channel c = (Channel) o; - if (c.isOpen()) { - c.close(); - } - } + terminateRequestChannel(request); }); } @@ -911,13 +905,7 @@ private Flux jsonStreamImpl(io.micronaut.http.HttpRequest parentReq return Flux.from(jsonProcessor) .map(jsonNode -> mediaTypeCodec.decode(type, jsonNode)); }).doOnTerminate(() -> { - final Object o = request.getAttribute(NettyClientHttpRequest.CHANNEL).orElse(null); - if (o instanceof Channel) { - final Channel c = (Channel) o; - if (c.isOpen()) { - c.close(); - } - } + terminateRequestChannel(request); }); } @@ -938,16 +926,20 @@ private Flux> dataStreamImpl(io.micronaut.http.HttpRequest .map(contentMapper); }) .doOnTerminate(() -> { - final Object o = request.getAttribute(NettyClientHttpRequest.CHANNEL).orElse(null); - if (o instanceof Channel) { - final Channel c = (Channel) o; - if (c.isOpen()) { - c.close(); - } - } + terminateRequestChannel(request); }); } + private static void terminateRequestChannel(io.micronaut.http.HttpRequest req) { + final Object o = req.getAttribute(NettyClientHttpRequest.CHANNEL).orElse(null); + if (o instanceof Channel) { + final Channel c = (Channel) o; + if (c.isOpen()) { + //c.close(); TODO + } + } + } + /** * Implementation of {@link #jsonStream}, {@link #dataStream}, {@link #exchangeStream}. */ @@ -1060,7 +1052,6 @@ private Publisher> exchang bodyType, errorType, emitter, - poolHandle.channel, requestKey.isSecure(), poolHandle ); @@ -1445,7 +1436,6 @@ private void sendRequestThroughChannel( Argument bodyType, Argument errorType, FluxSink> emitter, - Channel channel, boolean secure, ConnectionManager.PoolHandle poolHandle) throws HttpPostRequestEncoder.ErrorDataEncoderException { URI requestURI = finalRequest.getUri(); @@ -1475,7 +1465,7 @@ private void sendRequestThroughChannel( finalRequest, nettyRequest, permitsBody, - !poolHandle.canReturn() + poolHandle.canReturn() ); if (log.isDebugEnabled()) { @@ -1486,8 +1476,8 @@ private void sendRequestThroughChannel( traceRequest(finalRequest, nettyRequest); } - Promise> responsePromise = channel.eventLoop().newPromise(); - channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_FULL_HTTP_RESPONSE, + Promise> responsePromise = poolHandle.channel.eventLoop().newPromise(); + poolHandle.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_FULL_HTTP_RESPONSE, new FullHttpResponseHandler<>(responsePromise, poolHandle, secure, finalRequest, bodyType, errorType)); poolHandle.notifyRequestPipelineBuilt(); Publisher> publisher = new NettyFuturePublisher<>(responsePromise, true); @@ -1497,7 +1487,7 @@ private void sendRequestThroughChannel( } publisher.subscribe(new ForwardingSubscriber<>(emitter)); - requestWriter.write(channel, secure, emitter); + requestWriter.write(poolHandle, secure, emitter); } private Flux> streamRequestThroughChannel( @@ -1533,11 +1523,27 @@ private void streamRequestThroughChannel0( FluxSink emitter, ConnectionManager.PoolHandle poolHandle, boolean secure) throws HttpPostRequestEncoder.ErrorDataEncoderException { - NettyRequestWriter requestWriter = prepareRequest( - finalRequest, - finalRequest.getUri(), - emitter + if (!(finalRequest instanceof MutableHttpRequest)) { + throw new IllegalArgumentException("A MutableHttpRequest is required"); + } + URI requestURI = finalRequest.getUri(); + boolean permitsBody = io.micronaut.http.HttpMethod.permitsRequestBody(finalRequest.getMethod()); + NettyRequestWriter requestWriter = buildNettyRequest( + (MutableHttpRequest) finalRequest, + requestURI, + finalRequest + .getContentType() + .orElse(MediaType.APPLICATION_JSON_TYPE), + permitsBody, + null, + throwable -> { + if (!((FluxSink>) emitter).isCancelled()) { + ((FluxSink>) emitter).error(throwable); + } + } ); + prepareHttpHeaders(requestURI, finalRequest, requestWriter.getNettyRequest(), permitsBody, poolHandle.canReturn()); + HttpRequest nettyRequest = requestWriter.getNettyRequest(); Promise> responsePromise = poolHandle.channel.eventLoop().newPromise(); ChannelPipeline pipeline = poolHandle.channel.pipeline(); @@ -1553,7 +1559,7 @@ private void streamRequestThroughChannel0( traceRequest(finalRequest, nettyRequest); } - requestWriter.write(poolHandle.channel, secure, emitter); + requestWriter.write(poolHandle, secure, emitter); responsePromise.addListener(future -> { if (future.isSuccess()) { emitter.next(future.getNow()); @@ -1584,11 +1590,11 @@ private String getHostHeader(URI requestURI) { } private void prepareHttpHeaders( - URI requestURI, - io.micronaut.http.HttpRequest request, - io.netty.handler.codec.http.HttpRequest nettyRequest, - boolean permitsBody, - boolean closeConnection) { + URI requestURI, + io.micronaut.http.HttpRequest request, + io.netty.handler.codec.http.HttpRequest nettyRequest, + boolean permitsBody, + boolean keepAlive) { HttpHeaders headers = nettyRequest.headers(); if (!headers.contains(HttpHeaderNames.HOST)) { @@ -1597,10 +1603,10 @@ private void prepareHttpHeaders( // HTTP/2 assumes keep-alive connections if (connectionManager.httpVersion != io.micronaut.http.HttpVersion.HTTP_2_0) { - if (closeConnection) { - headers.set(HttpHeaderNames.CONNECTION, HttpHeaderValues.CLOSE); - } else { + if (keepAlive) { headers.set(HttpHeaderNames.CONNECTION, HttpHeaderValues.KEEP_ALIVE); + } else { + headers.set(HttpHeaderNames.CONNECTION, HttpHeaderValues.CLOSE); } } @@ -1788,37 +1794,6 @@ private static MediaTypeCodecRegistry createDefaultMediaTypeRegistry() { ); } - private NettyRequestWriter prepareRequest( - io.micronaut.http.HttpRequest request, - URI requestURI, - FluxSink> emitter) throws HttpPostRequestEncoder.ErrorDataEncoderException { - MediaType requestContentType = request - .getContentType() - .orElse(MediaType.APPLICATION_JSON_TYPE); - - boolean permitsBody = io.micronaut.http.HttpMethod.permitsRequestBody(request.getMethod()); - - if (!(request instanceof MutableHttpRequest)) { - throw new IllegalArgumentException("A MutableHttpRequest is required"); - } - MutableHttpRequest clientHttpRequest = (MutableHttpRequest) request; - NettyRequestWriter requestWriter = buildNettyRequest( - clientHttpRequest, - requestURI, - requestContentType, - permitsBody, - null, - throwable -> { - if (!emitter.isCancelled()) { - emitter.error(throwable); - } - } - ); - io.netty.handler.codec.http.HttpRequest nettyRequest = requestWriter.getNettyRequest(); - prepareHttpHeaders(requestURI, request, nettyRequest, permitsBody, true); - return requestWriter; - } - private @NonNull InvocationInstrumenter combineFactories() { if (CollectionUtils.isEmpty(invocationInstrumenterFactories)) { return NOOP; @@ -1953,23 +1928,21 @@ private class NettyRequestWriter { * @param channelPool The channel pool * @param emitter The emitter */ - protected void write(Channel channel, boolean isSecure, FluxSink emitter) { - final ChannelPipeline pipeline = channel.pipeline(); + protected void write(ConnectionManager.PoolHandle poolHandle, boolean isSecure, FluxSink emitter) { if (connectionManager.httpVersion == io.micronaut.http.HttpVersion.HTTP_2_0) { + // todo: move to CM if (isSecure) { nettyRequest.headers().add(AbstractNettyHttpRequest.HTTP2_SCHEME, HttpScheme.HTTPS); } else { nettyRequest.headers().add(AbstractNettyHttpRequest.HTTP2_SCHEME, HttpScheme.HTTP); } } - processRequestWrite(channel, emitter, pipeline); - } - private void processRequestWrite(Channel channel, FluxSink emitter, ChannelPipeline pipeline) { + Channel channel = poolHandle.channel; ChannelFuture writeFuture; if (encoder != null && encoder.isChunked()) { channel.attr(AttributeKey.valueOf(ChannelPipelineCustomizer.HANDLER_HTTP_CHUNK)).set(true); - pipeline.addAfter(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, ChannelPipelineCustomizer.HANDLER_HTTP_CHUNK, new ChunkedWriteHandler()); + channel.pipeline().addAfter(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, ChannelPipelineCustomizer.HANDLER_HTTP_CHUNK, new ChunkedWriteHandler()); channel.write(nettyRequest); writeFuture = channel.writeAndFlush(encoder); } else { @@ -1979,6 +1952,7 @@ private void processRequestWrite(Channel channel, FluxSink emitter, ChannelPi connectionManager.addInstrumentedListener(writeFuture, f -> { try { if (!f.isSuccess()) { + poolHandle.taint(); if (!emitter.isCancelled()) { emitter.error(f.cause()); } @@ -2096,6 +2070,7 @@ protected void channelReadInstrumented(ChannelHandlerContext ctx, R msg) throws traceHeaders(headers); } buildResponse(responsePromise, msg, httpStatus); + removeHandler(ctx); } private void setRedirectHeaders(@Nullable io.micronaut.http.HttpRequest request, MutableHttpRequest redirectRequest) { @@ -2115,6 +2090,10 @@ private void setRedirectHeaders(@Nullable io.micronaut.http.HttpRequest reque } } + protected void removeHandler(ChannelHandlerContext ctx) { + ctx.pipeline().remove(this); + } + protected abstract Function> makeRedirectHandler(io.micronaut.http.HttpRequest parentRequest, MutableHttpRequest redirectRequest); protected abstract void buildResponse(Promise promise, R msg, HttpStatus httpStatus); @@ -2312,6 +2291,12 @@ public boolean acceptInboundMessage(Object msg) { return msg instanceof FullHttpResponse; } + @Override + protected void removeHandler(ChannelHandlerContext ctx) { + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_FULL); + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_STREAM); + } + @Override protected void buildResponse(Promise> promise, FullHttpResponse msg, HttpStatus httpStatus) { Publisher bodyPublisher; @@ -2345,6 +2330,12 @@ public boolean acceptInboundMessage(Object msg) { return msg instanceof StreamedHttpResponse; } + @Override + protected void removeHandler(ChannelHandlerContext ctx) { + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_FULL); + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_STREAM); + } + @Override protected void buildResponse(Promise> promise, StreamedHttpResponse msg, HttpStatus httpStatus) { promise.trySuccess(new NettyStreamedHttpResponse<>(msg, httpStatus)); diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index facacc88131..a23d6185640 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -245,7 +245,9 @@ class ConnectionManagerSpec extends Specification { patch(client, conn.clientChannel) conn.testExchange(client) + conn.testStreaming(client) conn.testExchange(client) + conn.testStreaming(client) cleanup: client.close() @@ -316,7 +318,10 @@ class ConnectionManagerSpec extends Specification { assert request.uri() == '/foo' assert request.method() == HttpMethod.GET assert request.headers().get('host') == 'example.com' - //assert request.headers().get("connection") == "keep-alive" + assert request.headers().get("connection") == "keep-alive" + + def tail = serverChannel.readInbound() + assert tail == null || tail instanceof LastHttpContent def response = new DefaultHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK) response.headers().add('content-length', 6) From f7bb5c0dddf2a45b91b46fa335aabef29d43a885 Mon Sep 17 00:00:00 2001 From: yawkat Date: Tue, 20 Sep 2022 12:27:19 +0200 Subject: [PATCH 08/82] http1 tls --- .../http/client/netty/ConnectionManager.java | 5 +-- .../client/netty/ConnectionManagerSpec.groovy | 36 +++++++++++++++++-- 2 files changed, 37 insertions(+), 4 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 0a100b4ac2e..2d7fa2f056a 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -1405,8 +1405,9 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { .addLast(new Http2PriorKnowledgeInitializer(pool)) .remove(initialErrorHandler); } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { - // TODO - throw new UnsupportedOperationException(); + ctx.pipeline() + .addLast(new Http1Initializer(pool, true, host, port)) + .remove(initialErrorHandler); } else { ctx.close(); throw customizeException(new HttpClientException("Unknown Protocol: " + protocol)); diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index a23d6185640..f5767c143d1 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -154,6 +154,23 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'simple http1 tls get'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp1() + conn.setupHttp1Tls() + patch(client, conn.clientChannel) + + conn.testExchange(client) + + cleanup: + client.close() + ctx.close() + } + def 'http1 streaming get'() { def ctx = ApplicationContext.run() def client = ctx.getBean(DefaultHttpClient) @@ -274,8 +291,23 @@ class ConnectionManagerSpec extends Specification { } static class EmbeddedTestConnectionHttp1 extends EmbeddedTestConnectionBase { + private String scheme + void setupHttp1() { + scheme = 'http' + serverChannel.pipeline() + .addLast(new HttpServerCodec()) + } + + void setupHttp1Tls() { + def certificate = new SelfSignedCertificate() + def builder = SslContextBuilder.forServer(certificate.key(), certificate.cert()) + CertificateProvidedSslBuilder.setupSslBuilder(builder, new SslConfiguration(), HttpVersion.HTTP_1_1); + def tlsHandler = builder.build().newHandler(ByteBufAllocator.DEFAULT) + + scheme = 'https' serverChannel.pipeline() + .addLast(tlsHandler) .addLast(new HttpServerCodec()) } @@ -286,7 +318,7 @@ class ConnectionManagerSpec extends Specification { } void testExchange(HttpClient client) { - def future = Mono.from(client.exchange('http://example.com/foo')).toFuture() + def future = Mono.from(client.exchange(scheme + '://example.com/foo')).toFuture() future.exceptionally(t -> t.printStackTrace()) advance() @@ -308,7 +340,7 @@ class ConnectionManagerSpec extends Specification { void testStreaming(StreamingHttpClient client) { def responseData = new ArrayDeque() def responseComplete = false - Flux.from(client.dataStream(HttpRequest.GET('http://example.com/foo'))) + Flux.from(client.dataStream(HttpRequest.GET(scheme + '://example.com/foo'))) .doOnError(t -> t.printStackTrace()) .doOnComplete(() -> responseComplete = true) .subscribe(b -> responseData.add(b.toString(StandardCharsets.UTF_8))) From f82bfcae680618af74f905381eb9e6f895420208 Mon Sep 17 00:00:00 2001 From: yawkat Date: Tue, 20 Sep 2022 16:51:21 +0200 Subject: [PATCH 09/82] h2c --- .../http/client/HttpClientConfiguration.java | 16 ++ .../http/client/netty/ConnectionManager.java | 141 ++++++++++++----- .../client/netty/ConnectionManagerSpec.groovy | 145 +++++++++++++++--- 3 files changed, 236 insertions(+), 66 deletions(-) diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java index 16119133eae..09cf6f71371 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java @@ -147,6 +147,8 @@ public abstract class HttpClientConfiguration { private HttpVersion httpVersion = HttpVersion.HTTP_1_1; + private PlaintextMode plaintextMode = PlaintextMode.HTTP_1; + private LogLevel logLevel; /** @@ -637,6 +639,15 @@ public Proxy resolveProxy(boolean isSsl, String host, int port) { } } + // TODO: docs + public PlaintextMode getPlaintextMode() { + return plaintextMode; + } + + public void setPlaintextMode(PlaintextMode plaintextMode) { + this.plaintextMode = plaintextMode; + } + /** * Configuration for the HTTP client connnection pool. */ @@ -739,4 +750,9 @@ public void setAcquireTimeout(@Nullable Duration acquireTimeout) { this.acquireTimeout = acquireTimeout; } } + + public enum PlaintextMode { + HTTP_1, + H2C, + } } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 2d7fa2f056a..fac3d1dc46b 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -78,6 +78,7 @@ import io.netty.handler.codec.http2.DelegatingDecompressorFrameListener; import io.netty.handler.codec.http2.Http2ClientUpgradeCodec; import io.netty.handler.codec.http2.Http2Connection; +import io.netty.handler.codec.http2.Http2FrameCodec; import io.netty.handler.codec.http2.Http2FrameCodecBuilder; import io.netty.handler.codec.http2.Http2FrameListener; import io.netty.handler.codec.http2.Http2FrameLogger; @@ -1470,50 +1471,98 @@ protected void initChannel(Channel ch) { private class Http2PriorKnowledgeInitializer extends ChannelInitializer { private final Pool pool; + private final Http2FrameCodec frameCodec; Http2PriorKnowledgeInitializer(Pool pool) { + this(pool, Http2FrameCodecBuilder.forClient().build()); + } + + Http2PriorKnowledgeInitializer(Pool pool, Http2FrameCodec frameCodec) { this.pool = pool; + this.frameCodec = frameCodec; } @Override protected void initChannel(Channel ch) throws Exception { - ch.pipeline() - .addLast(Http2FrameCodecBuilder.forClient() - .build()) - .addLast(new Http2MultiplexHandler(new ChannelInitializer() { - @Override - protected void initChannel(Http2StreamChannel ch1) throws Exception { - // todo: fail connection? - log.warn("Server opened HTTP2 stream {}, closing immediately", ch1.stream().id()); - ch1.close(); - } - }, new ChannelInboundHandlerAdapter() { - @Override - public void handlerAdded(ChannelHandlerContext ctx) throws Exception { - ctx.read(); + ch.pipeline().addLast(frameCodec); + ch.pipeline().addLast(new Http2MultiplexHandler(new ChannelInitializer() { + @Override + protected void initChannel(Http2StreamChannel ch) throws Exception { + // todo: fail connection? + log.warn("Server opened HTTP2 stream {}, closing immediately", ch.stream().id()); + ch.close(); + } + }, new ChannelInitializer() { + @Override + protected void initChannel(Http2StreamChannel ch) throws Exception { + // discard any response data for the upgrade request + ch.close(); + } + })); + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new InitialConnectionErrorHandler(pool) { + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof Http2SettingsFrame) { + ctx.pipeline().remove(this); + pool.new Http2ConnectionHolder(ch).init(); + return; + } else { + log.warn("Premature frame: {}", msg.getClass()); } - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - // todo: log - ReferenceCountUtil.release(msg); - ctx.read(); - } - })) - .addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new InitialConnectionErrorHandler(pool) { - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof Http2SettingsFrame) { - ctx.pipeline().remove(this); - pool.new Http2ConnectionHolder(ch).init(); - return; - } else { - log.warn("Premature frame: {}", msg.getClass()); - } + super.channelRead(ctx, msg); + } + }); + // stream frames should be handled by the multiplexer + ch.pipeline().addLast(new ChannelInboundHandlerAdapter() { + @Override + public void handlerAdded(ChannelHandlerContext ctx) throws Exception { + ctx.read(); + } - super.channelRead(ctx, msg); - } - }); + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + // todo: log + ReferenceCountUtil.release(msg); + ctx.read(); + } + }); + } + } + + private class Http2UpgradeInitializer extends ChannelInitializer { + private final Pool pool; + + Http2UpgradeInitializer(Pool pool) { + this.pool = pool; + } + + @Override + protected void initChannel(Channel ch) throws Exception { + Http2FrameCodec frameCodec = Http2FrameCodecBuilder.forClient().build(); + + HttpClientCodec sourceCodec = new HttpClientCodec(); + Http2ClientUpgradeCodec upgradeCodec = new Http2ClientUpgradeCodec(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, frameCodec, + new Http2PriorKnowledgeInitializer(pool, frameCodec)); + HttpClientUpgradeHandler upgradeHandler = new HttpClientUpgradeHandler(sourceCodec, upgradeCodec, 65536); + + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, sourceCodec); + ch.pipeline().addLast(upgradeHandler); + + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_UPGRADE_REQUEST, new ChannelInboundHandlerAdapter() { + @Override + public void channelActive(ChannelHandlerContext ctx) throws Exception { + DefaultFullHttpRequest upgradeRequest = + new DefaultFullHttpRequest(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpMethod.GET, "/", Unpooled.EMPTY_BUFFER); + + // Set HOST header as the remote peer may require it. + upgradeRequest.headers().set(HttpHeaderNames.HOST, pool.requestKey.getHost() + ':' + pool.requestKey.getPort()); + ctx.writeAndFlush(upgradeRequest); + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP2_UPGRADE_REQUEST); + + super.channelActive(ctx); + } + }); } } @@ -1632,7 +1681,6 @@ private void acquire(Sinks.One sink) { } // no connection open that has room pendingRequests.add(sink); - // todo: http1 openNewConnection(); } @@ -1650,12 +1698,23 @@ private void openNewConnection() { requestKey.getPort() ); } else { - initializer = new Http1Initializer( - this, - false, - requestKey.getHost(), - requestKey.getPort() - ); + switch (configuration.getPlaintextMode()) { + case HTTP_1: + initializer = new Http1Initializer( + this, + false, + requestKey.getHost(), + requestKey.getPort() + ); + break; + case H2C: + initializer = new Http2UpgradeInitializer( + this + ); + break; + default: + throw new AssertionError("Unknown plaintext mode"); + } } addInstrumentedListener(doConnect(requestKey, initializer), future -> { if (!future.isSuccess()) { diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index f5767c143d1..590f2fc6630 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -2,6 +2,7 @@ package io.micronaut.http.client.netty import io.micronaut.context.ApplicationContext import io.micronaut.http.HttpRequest +import io.micronaut.http.HttpResponse import io.micronaut.http.HttpStatus import io.micronaut.http.HttpVersion import io.micronaut.http.client.HttpClient @@ -12,8 +13,12 @@ import io.netty.buffer.ByteBufAllocator import io.netty.buffer.Unpooled import io.netty.channel.Channel import io.netty.channel.ChannelFuture +import io.netty.channel.ChannelHandler import io.netty.channel.ChannelHandlerContext +import io.netty.channel.ChannelId +import io.netty.channel.ChannelInboundHandlerAdapter import io.netty.channel.ChannelInitializer +import io.netty.channel.ServerChannel import io.netty.channel.embedded.EmbeddedChannel import io.netty.handler.codec.http.DefaultFullHttpResponse import io.netty.handler.codec.http.DefaultHttpContent @@ -22,24 +27,34 @@ import io.netty.handler.codec.http.DefaultLastHttpContent import io.netty.handler.codec.http.HttpMethod import io.netty.handler.codec.http.HttpResponseStatus import io.netty.handler.codec.http.HttpServerCodec +import io.netty.handler.codec.http.HttpServerUpgradeHandler import io.netty.handler.codec.http.LastHttpContent import io.netty.handler.codec.http2.DefaultHttp2DataFrame import io.netty.handler.codec.http2.DefaultHttp2Headers import io.netty.handler.codec.http2.DefaultHttp2HeadersFrame +import io.netty.handler.codec.http2.Http2FrameCodec import io.netty.handler.codec.http2.Http2FrameCodecBuilder import io.netty.handler.codec.http2.Http2FrameStream +import io.netty.handler.codec.http2.Http2FrameStreamEvent import io.netty.handler.codec.http2.Http2Headers import io.netty.handler.codec.http2.Http2HeadersFrame +import io.netty.handler.codec.http2.Http2ResetFrame +import io.netty.handler.codec.http2.Http2ServerUpgradeCodec import io.netty.handler.codec.http2.Http2SettingsAckFrame import io.netty.handler.codec.http2.Http2SettingsFrame +import io.netty.handler.codec.http2.Http2Stream +import io.netty.handler.logging.LogLevel +import io.netty.handler.logging.LoggingHandler import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler import io.netty.handler.ssl.SslContextBuilder import io.netty.handler.ssl.util.SelfSignedCertificate +import io.netty.util.AsciiString import reactor.core.publisher.Flux import reactor.core.publisher.Mono import spock.lang.Specification import java.nio.charset.StandardCharsets +import java.util.concurrent.Future class ConnectionManagerSpec extends Specification { private static void patch(DefaultHttpClient httpClient, EmbeddedChannel... channels) { @@ -58,7 +73,6 @@ class ConnectionManagerSpec extends Specification { } def 'simple http2 get'() { - given: def ctx = ApplicationContext.run([ 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, ]) @@ -68,23 +82,9 @@ class ConnectionManagerSpec extends Specification { conn.setupHttp2Tls() patch(client, conn.clientChannel) - when: - def future = Mono.from(client.exchange('https://example.com/foo')).toFuture() - future.exceptionally(t -> t.printStackTrace()) + def future = conn.testExchangeRequest(client) conn.exchangeSettings() - then: - Http2HeadersFrame request = conn.serverChannel.readInbound() - request.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/foo' - request.headers().get(Http2Headers.PseudoHeaderName.SCHEME.value()) == 'https' - request.headers().get(Http2Headers.PseudoHeaderName.AUTHORITY.value()) == 'example.com' - request.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' - - when: - conn.respondOk(request.stream()) - conn.advance() - then: - def response = future.get() - response.status() == HttpStatus.OK + conn.testExchangeResponse(future) cleanup: client.close() @@ -171,6 +171,39 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'simple h2c get'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.plaintext-mode': 'h2c', + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp2() + conn.setupH2c() + patch(client, conn.clientChannel) + + def future = conn.testExchangeRequest(client) + conn.clientChannel.pipeline().fireChannelActive() + conn.advance() + + Http2HeadersFrame upgradeRequest = conn.serverChannel.readInbound() + assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' + assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/' + assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.AUTHORITY.value()) == 'example.com:80' + assert upgradeRequest.headers().get('content-length') == '0' + // client closes the stream immediately + assert upgradeRequest.stream().state() == Http2Stream.State.CLOSED + + assert conn.serverChannel.readInbound() instanceof Http2SettingsFrame + assert conn.serverChannel.readInbound() instanceof Http2ResetFrame + assert conn.serverChannel.readInbound() instanceof Http2SettingsAckFrame + + conn.testExchangeResponse(future) + + cleanup: + client.close() + ctx.close() + } + def 'http1 streaming get'() { def ctx = ApplicationContext.run() def client = ctx.getBean(DefaultHttpClient) @@ -276,7 +309,7 @@ class ConnectionManagerSpec extends Specification { final EmbeddedChannel clientChannel EmbeddedTestConnectionBase() { - serverChannel = new EmbeddedChannel(new DummyChannelId('server')) + serverChannel = new EmbeddedServerChannel(new DummyChannelId('server')) serverChannel.freezeTime() serverChannel.config().setAutoRead(true) @@ -290,6 +323,12 @@ class ConnectionManagerSpec extends Specification { } } + static class EmbeddedServerChannel extends EmbeddedChannel implements ServerChannel { + EmbeddedServerChannel(ChannelId channelId) { + super(channelId) + } + } + static class EmbeddedTestConnectionHttp1 extends EmbeddedTestConnectionBase { private String scheme @@ -373,7 +412,12 @@ class ConnectionManagerSpec extends Specification { } static class EmbeddedTestConnectionHttp2 extends EmbeddedTestConnectionBase { + private String scheme + Http2FrameStream h2cResponseStream + void setupHttp2Tls() { + scheme = 'https' + def certificate = new SelfSignedCertificate() def builder = SslContextBuilder.forServer(certificate.key(), certificate.cert()) CertificateProvidedSslBuilder.setupSslBuilder(builder, new SslConfiguration(), HttpVersion.HTTP_2_0); @@ -385,20 +429,51 @@ class ConnectionManagerSpec extends Specification { @Override protected void configurePipeline(ChannelHandlerContext chtx, String protocol) throws Exception { chtx.pipeline() - .addLast(Http2FrameCodecBuilder.forServer() - .autoAckSettingsFrame(false) - .build()) + .addLast(Http2FrameCodecBuilder.forServer().build()) } }) } + void setupH2c() { + scheme = 'http' + + ChannelHandler responseStreamHandler = new ChannelInboundHandlerAdapter() { + @Override + void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof Http2FrameStreamEvent && evt.stream().id() == 1) { + h2cResponseStream = evt.stream() + } + + super.userEventTriggered(ctx, evt) + } + } + Http2FrameCodec frameCodec = Http2FrameCodecBuilder.forServer() + .build() + HttpServerUpgradeHandler.UpgradeCodecFactory upgradeCodecFactory = protocol -> { + if (AsciiString.contentEquals("h2c", protocol)) { + return new Http2ServerUpgradeCodec(frameCodec, responseStreamHandler) + } else { + return null + } + } + + HttpServerCodec sourceCodec = new HttpServerCodec() + serverChannel.pipeline() + .addLast(new LoggingHandler(LogLevel.INFO)) + .addLast(sourceCodec) + .addLast(new HttpServerUpgradeHandler(sourceCodec, upgradeCodecFactory, 1024)) + } + void exchangeSettings() { advance() - if (!(serverChannel.readInbound() instanceof Http2SettingsFrame)) { - throw new AssertionError() + + def settings = serverChannel.readInbound() + if (!(settings instanceof Http2SettingsFrame)) { + throw new AssertionError(settings) } - if (!(serverChannel.readInbound() instanceof Http2SettingsAckFrame)) { - throw new AssertionError() + def ack = serverChannel.readInbound() + if (!(ack instanceof Http2SettingsAckFrame)) { + throw new AssertionError(ack) } } @@ -407,5 +482,25 @@ class ConnectionManagerSpec extends Specification { responseHeaders.add(Http2Headers.PseudoHeaderName.STATUS.value(), "200") serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, true).stream(stream)) } + + Future> testExchangeRequest(HttpClient client) { + def future = Mono.from(client.exchange(scheme + '://example.com/foo')).toFuture() + future.exceptionally(t -> t.printStackTrace()) + return future + } + + void testExchangeResponse(Future> future) { + Http2HeadersFrame request = serverChannel.readInbound() + assert request.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/foo' + assert request.headers().get(Http2Headers.PseudoHeaderName.SCHEME.value()) == scheme + assert request.headers().get(Http2Headers.PseudoHeaderName.AUTHORITY.value()) == 'example.com' + assert request.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' + + respondOk(request.stream()) + advance() + + def response = future.get() + assert response.status() == HttpStatus.OK + } } } From 47fbed177751931d371c8aee815180df1b27dd49 Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 21 Sep 2022 11:19:02 +0200 Subject: [PATCH 10/82] channel customization --- .../http/client/netty/ConnectionManager.java | 192 +++++++-------- .../http/client/netty/DefaultHttpClient.java | 9 +- .../client/netty/NettyClientCustomizer.java | 4 + .../client/netty/ConnectionManagerSpec.groovy | 220 +++++++++++++++--- 4 files changed, 286 insertions(+), 139 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index fac3d1dc46b..1e18afedaae 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -1402,13 +1402,13 @@ protected void initChannel(Channel ch) { @Override protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { - ctx.pipeline() - .addLast(new Http2PriorKnowledgeInitializer(pool)) - .remove(initialErrorHandler); + ctx.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, Http2FrameCodecBuilder.forClient().build()); + initHttp2(pool, ctx.channel(), channelCustomizer); + ctx.pipeline().remove(initialErrorHandler); } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { - ctx.pipeline() - .addLast(new Http1Initializer(pool, true, host, port)) - .remove(initialErrorHandler); + initHttp1(ctx.channel()); + pool.new Http1ConnectionHolder(ch, channelCustomizer).init(false); + ctx.pipeline().remove(initialErrorHandler); } else { ctx.close(); throw customizeException(new HttpClientException("Unknown Protocol: " + protocol)); @@ -1416,6 +1416,8 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { } }) .addLast(initialErrorHandler); + + channelCustomizer.onInitialPipelineBuilt(); } } @@ -1438,96 +1440,55 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } } - private class Http1Initializer extends ChannelInitializer { - final Pool pool; - final boolean secure; - final String host; - final int port; - - Http1Initializer(Pool pool, - boolean secure, - String host, - int port) { - this.pool = pool; - this.secure = secure; - this.host = host; - this.port = port; - } - - /** - * @param ch The channel - */ - @Override - protected void initChannel(Channel ch) { - configureProxy(ch.pipeline(), secure, host, port); // todo - - ch.pipeline() - .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()) - .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); - - pool.new Http1ConnectionHolder(ch).init(); - } + private void initHttp1(Channel ch) { + ch.pipeline() + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()) + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); } - private class Http2PriorKnowledgeInitializer extends ChannelInitializer { - private final Pool pool; - private final Http2FrameCodec frameCodec; - - Http2PriorKnowledgeInitializer(Pool pool) { - this(pool, Http2FrameCodecBuilder.forClient().build()); - } - - Http2PriorKnowledgeInitializer(Pool pool, Http2FrameCodec frameCodec) { - this.pool = pool; - this.frameCodec = frameCodec; - } - - @Override - protected void initChannel(Channel ch) throws Exception { - ch.pipeline().addLast(frameCodec); - ch.pipeline().addLast(new Http2MultiplexHandler(new ChannelInitializer() { - @Override - protected void initChannel(Http2StreamChannel ch) throws Exception { - // todo: fail connection? - log.warn("Server opened HTTP2 stream {}, closing immediately", ch.stream().id()); - ch.close(); - } - }, new ChannelInitializer() { - @Override - protected void initChannel(Http2StreamChannel ch) throws Exception { - // discard any response data for the upgrade request - ch.close(); + private void initHttp2(Pool pool, Channel ch, NettyClientCustomizer connectionCustomizer) { + ch.pipeline().addLast(new Http2MultiplexHandler(new ChannelInitializer() { + @Override + protected void initChannel(Http2StreamChannel ch) throws Exception { + // todo: fail connection? + log.warn("Server opened HTTP2 stream {}, closing immediately", ch.stream().id()); + ch.close(); + } + }, new ChannelInitializer() { + @Override + protected void initChannel(Http2StreamChannel ch) throws Exception { + // discard any response data for the upgrade request + ch.close(); + } + })); + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new InitialConnectionErrorHandler(pool) { + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof Http2SettingsFrame) { + ctx.pipeline().remove(this); + pool.new Http2ConnectionHolder(ch, connectionCustomizer).init(); + return; + } else { + log.warn("Premature frame: {}", msg.getClass()); } - })); - ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new InitialConnectionErrorHandler(pool) { - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof Http2SettingsFrame) { - ctx.pipeline().remove(this); - pool.new Http2ConnectionHolder(ch).init(); - return; - } else { - log.warn("Premature frame: {}", msg.getClass()); - } - super.channelRead(ctx, msg); - } - }); - // stream frames should be handled by the multiplexer - ch.pipeline().addLast(new ChannelInboundHandlerAdapter() { - @Override - public void handlerAdded(ChannelHandlerContext ctx) throws Exception { - ctx.read(); - } + super.channelRead(ctx, msg); + } + }); + // stream frames should be handled by the multiplexer + ch.pipeline().addLast(new ChannelInboundHandlerAdapter() { + @Override + public void handlerAdded(ChannelHandlerContext ctx) throws Exception { + ctx.read(); + } - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - // todo: log - ReferenceCountUtil.release(msg); - ctx.read(); - } - }); - } + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + // todo: log + ReferenceCountUtil.release(msg); + ctx.read(); + } + }); } private class Http2UpgradeInitializer extends ChannelInitializer { @@ -1539,11 +1500,19 @@ private class Http2UpgradeInitializer extends ChannelInitializer { @Override protected void initChannel(Channel ch) throws Exception { + NettyClientCustomizer connectionCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); + Http2FrameCodec frameCodec = Http2FrameCodecBuilder.forClient().build(); HttpClientCodec sourceCodec = new HttpClientCodec(); Http2ClientUpgradeCodec upgradeCodec = new Http2ClientUpgradeCodec(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, frameCodec, - new Http2PriorKnowledgeInitializer(pool, frameCodec)); + new ChannelInitializer() { + @Override + protected void initChannel(Channel ch) throws Exception { + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, frameCodec); + initHttp2(pool, ch, connectionCustomizer); + } + }); HttpClientUpgradeHandler upgradeHandler = new HttpClientUpgradeHandler(sourceCodec, upgradeCodec, 65536); ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, sourceCodec); @@ -1563,6 +1532,8 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { super.channelActive(ctx); } }); + + connectionCustomizer.onInitialPipelineBuilt(); } } @@ -1700,12 +1671,15 @@ private void openNewConnection() { } else { switch (configuration.getPlaintextMode()) { case HTTP_1: - initializer = new Http1Initializer( - this, - false, - requestKey.getHost(), - requestKey.getPort() - ); + initializer = new ChannelInitializer() { + @Override + protected void initChannel(Channel ch) throws Exception { + NettyClientCustomizer channelCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); + configureProxy(ch.pipeline(), false, requestKey.getHost(), requestKey.getPort()); + initHttp1(ch); + new Http1ConnectionHolder(ch, channelCustomizer).init(true); + } + }; break; case H2C: initializer = new Http2UpgradeInitializer( @@ -1731,14 +1705,16 @@ void onNewConnectionFailure() { final class Http1ConnectionHolder { private final Channel channel; + private final NettyClientCustomizer connectionCustomizer; private final AtomicBoolean hasLiveRequest = new AtomicBoolean(false); private volatile boolean canReturn = true; - Http1ConnectionHolder(Channel channel) { + Http1ConnectionHolder(Channel channel, NettyClientCustomizer connectionCustomizer) { this.channel = channel; + this.connectionCustomizer = connectionCustomizer; } - void init() { + void init(boolean fireInitialPipelineBuilt) { channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { @@ -1748,7 +1724,10 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } }); - // todo: notify customizer + if (fireInitialPipelineBuilt) { + connectionCustomizer.onInitialPipelineBuilt(); + } + connectionCustomizer.onStreamPipelineBuilt(); Sinks.One pendingRequest = pendingRequests.poll(); if (pendingRequest != null) { @@ -1809,7 +1788,7 @@ boolean canReturn() { @Override void notifyRequestPipelineBuilt() { - // TODO + connectionCustomizer.onRequestPipelineBuilt(); } }); if (emitResult.isFailure()) { @@ -1828,9 +1807,11 @@ private void returnPendingRequest(Sinks.One sink) { final class Http2ConnectionHolder { private final Channel channel; private final AtomicInteger liveRequests = new AtomicInteger(0); + private final NettyClientCustomizer customizer; - Http2ConnectionHolder(Channel channel) { + Http2ConnectionHolder(Channel channel, NettyClientCustomizer customizer) { this.channel = channel; + this.customizer = customizer; } void init() { @@ -1842,7 +1823,7 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } }); - // todo: notify customizer + customizer.onStreamPipelineBuilt(); for (int i = 0; i < MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION; i++) { Sinks.One pendingRequest = pendingRequests.poll(); @@ -1879,6 +1860,7 @@ private void satisfy0(Sinks.One sink) { Http2StreamChannel streamChannel = future.get(); streamChannel.pipeline() .addLast(new Http2StreamFrameToHttpObjectCodec(false)); + NettyClientCustomizer streamCustomizer = customizer.specializeForChannel(streamChannel, NettyClientCustomizer.ChannelRole.HTTP2_STREAM); Sinks.EmitResult emitResult = sink.tryEmitValue(new PoolHandle(streamChannel) { @Override void taint() { @@ -1898,7 +1880,7 @@ boolean canReturn() { @Override void notifyRequestPipelineBuilt() { - // TODO + streamCustomizer.onRequestPipelineBuilt(); } }); if (emitResult.isFailure()) { diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index 0947b388104..e82519ac474 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -2090,9 +2090,7 @@ private void setRedirectHeaders(@Nullable io.micronaut.http.HttpRequest reque } } - protected void removeHandler(ChannelHandlerContext ctx) { - ctx.pipeline().remove(this); - } + protected abstract void removeHandler(ChannelHandlerContext ctx); protected abstract Function> makeRedirectHandler(io.micronaut.http.HttpRequest parentRequest, MutableHttpRequest redirectRequest); @@ -2154,6 +2152,11 @@ protected void channelReadInstrumented(ChannelHandlerContext channelHandlerConte } } + @Override + protected void removeHandler(ChannelHandlerContext ctx) { + // done in channelReadInstrumented + } + @Override protected void buildResponse(Promise> promise, FullHttpResponse msg, HttpStatus httpStatus) { try { diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/NettyClientCustomizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/NettyClientCustomizer.java index cfe383afbea..8963996aa66 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/NettyClientCustomizer.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/NettyClientCustomizer.java @@ -79,6 +79,10 @@ enum ChannelRole { * {@link io.netty.channel.socket.SocketChannel}, representing an HTTP connection. */ CONNECTION, + /** + * The channel is a HTTP2 stream channel. + */ + HTTP2_STREAM, } /** diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 590f2fc6630..5605ac3334d 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -1,12 +1,15 @@ package io.micronaut.http.client.netty import io.micronaut.context.ApplicationContext +import io.micronaut.context.event.BeanCreatedEvent +import io.micronaut.context.event.BeanCreatedEventListener import io.micronaut.http.HttpRequest import io.micronaut.http.HttpResponse import io.micronaut.http.HttpStatus import io.micronaut.http.HttpVersion import io.micronaut.http.client.HttpClient import io.micronaut.http.client.StreamingHttpClient +import io.micronaut.http.netty.channel.ChannelPipelineCustomizer import io.micronaut.http.server.netty.ssl.CertificateProvidedSslBuilder import io.micronaut.http.ssl.SslConfiguration import io.netty.buffer.ByteBufAllocator @@ -49,6 +52,7 @@ import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler import io.netty.handler.ssl.SslContextBuilder import io.netty.handler.ssl.util.SelfSignedCertificate import io.netty.util.AsciiString +import jakarta.inject.Singleton import reactor.core.publisher.Flux import reactor.core.publisher.Mono import spock.lang.Specification @@ -92,7 +96,6 @@ class ConnectionManagerSpec extends Specification { } def 'http2 streaming get'() { - given: def ctx = ApplicationContext.run([ 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, ]) @@ -102,37 +105,9 @@ class ConnectionManagerSpec extends Specification { conn.setupHttp2Tls() patch(client, conn.clientChannel) - when: - def responseData = new ArrayDeque() - def responseComplete = false - Flux.from(client.dataStream(HttpRequest.GET('https://example.com/foo'))) - .doOnError(t -> t.printStackTrace()) - .doOnComplete(() -> responseComplete = true) - .subscribe(b -> responseData.add(b.toString(StandardCharsets.UTF_8))) + def r1 = conn.testStreamingRequest(client) conn.exchangeSettings() - then: - Http2HeadersFrame request = conn.serverChannel.readInbound() - request.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/foo' - request.headers().get(Http2Headers.PseudoHeaderName.SCHEME.value()) == 'https' - request.headers().get(Http2Headers.PseudoHeaderName.AUTHORITY.value()) == 'example.com' - request.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' - - when: - def responseHeaders = new DefaultHttp2Headers() - responseHeaders.add(Http2Headers.PseudoHeaderName.STATUS.value(), "200") - conn.serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, false).stream(request.stream())) - conn.serverChannel.writeOutbound(new DefaultHttp2DataFrame(Unpooled.wrappedBuffer('foo'.bytes)).stream(request.stream())) - conn.advance() - then: - responseData.poll() == 'foo' - !responseComplete - - when: - conn.serverChannel.writeOutbound(new DefaultHttp2DataFrame(Unpooled.wrappedBuffer('bar'.bytes), true).stream(request.stream())) - conn.advance() - then: - responseData.poll() == 'bar' - responseComplete + conn.testStreamingResponse(r1) cleanup: client.close() @@ -304,6 +279,102 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'http1 plain text customization'() { + given: + def ctx = ApplicationContext.run() + def client = ctx.getBean(DefaultHttpClient) + def tracker = ctx.getBean(CustomizerTracker) + + def conn = new EmbeddedTestConnectionHttp1() + conn.setupHttp1() + patch(client, conn.clientChannel) + + when: + conn.testExchange(client) + conn.testStreaming(client) + + then: + def outerChannel = tracker.initialPipelineBuilt.poll() + outerChannel.channel == conn.clientChannel + outerChannel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC) + outerChannel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER) + !outerChannel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR) + !outerChannel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM) + tracker.initialPipelineBuilt.isEmpty() + + def innerChannel = tracker.streamPipelineBuilt.poll() + innerChannel.channel == conn.clientChannel + innerChannel.handlerNames == outerChannel.handlerNames + tracker.streamPipelineBuilt.isEmpty() + + def req1Channel = tracker.requestPipelineBuilt.poll() + req1Channel.channel == conn.clientChannel + req1Channel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR) + req1Channel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_MICRONAUT_FULL_HTTP_RESPONSE) + + def req2Channel = tracker.requestPipelineBuilt.poll() + req2Channel.channel == conn.clientChannel + req2Channel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_FULL) + req2Channel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_STREAM) + + tracker.requestPipelineBuilt.isEmpty() + + cleanup: + client.close() + ctx.close() + } + + def 'http2 alpn customization'() { + given: + def ctx = ApplicationContext.run([ + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + ]) + def client = ctx.getBean(DefaultHttpClient) + def tracker = ctx.getBean(CustomizerTracker) + + def conn = new EmbeddedTestConnectionHttp2() + conn.setupHttp2Tls() + patch(client, conn.clientChannel) + + when: + def r1 = conn.testExchangeRequest(client) + conn.exchangeSettings() + conn.testExchangeResponse(r1) + + def r2 = conn.testStreamingRequest(client) + conn.testStreamingResponse(r2) + + then: + def outerChannel = tracker.initialPipelineBuilt.poll() + outerChannel.channel == conn.clientChannel + outerChannel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_SSL) + !outerChannel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION) + tracker.initialPipelineBuilt.isEmpty() + + def innerChannel = tracker.streamPipelineBuilt.poll() + innerChannel.channel == conn.clientChannel + innerChannel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION) + tracker.streamPipelineBuilt.isEmpty() + + def req1Channel = tracker.requestPipelineBuilt.poll() + req1Channel.role == NettyClientCustomizer.ChannelRole.HTTP2_STREAM + req1Channel.channel !== conn.clientChannel + req1Channel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR) + req1Channel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_MICRONAUT_FULL_HTTP_RESPONSE) + + def req2Channel = tracker.requestPipelineBuilt.poll() + req2Channel.role == NettyClientCustomizer.ChannelRole.HTTP2_STREAM + req2Channel.channel !== conn.clientChannel + req2Channel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_FULL) + req2Channel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_STREAM) + + tracker.requestPipelineBuilt.isEmpty() + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnectionBase { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel @@ -502,5 +573,92 @@ class ConnectionManagerSpec extends Specification { def response = future.get() assert response.status() == HttpStatus.OK } + + Queue testStreamingRequest(StreamingHttpClient client) { + def responseData = new ArrayDeque() + Flux.from(client.dataStream(HttpRequest.GET(scheme + '://example.com/foo'))) + .doOnError(t -> t.printStackTrace()) + .doOnComplete(() -> responseData.add("END")) + .subscribe(b -> responseData.add(b.toString(StandardCharsets.UTF_8))) + return responseData + } + + void testStreamingResponse(Queue responseData) { + advance() + Http2HeadersFrame request = serverChannel.readInbound() + assert request.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/foo' + assert request.headers().get(Http2Headers.PseudoHeaderName.SCHEME.value()) == scheme + assert request.headers().get(Http2Headers.PseudoHeaderName.AUTHORITY.value()) == 'example.com' + assert request.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' + + def responseHeaders = new DefaultHttp2Headers() + responseHeaders.add(Http2Headers.PseudoHeaderName.STATUS.value(), "200") + serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, false).stream(request.stream())) + serverChannel.writeOutbound(new DefaultHttp2DataFrame(Unpooled.wrappedBuffer('foo'.bytes)).stream(request.stream())) + advance() + + assert responseData.poll() == 'foo' + assert responseData.isEmpty() + + serverChannel.writeOutbound(new DefaultHttp2DataFrame(Unpooled.wrappedBuffer('bar'.bytes), true).stream(request.stream())) + advance() + + assert responseData.poll() == 'bar' + assert responseData.poll() == 'END' + } + } + + @Singleton + static class CustomizerTracker implements NettyClientCustomizer, BeanCreatedEventListener { + final Queue initialPipelineBuilt = new ArrayDeque<>() + final Queue streamPipelineBuilt = new ArrayDeque<>() + final Queue requestPipelineBuilt = new ArrayDeque<>() + + @Override + NettyClientCustomizer specializeForChannel(Channel channel, ChannelRole role) { + return new NettyClientCustomizer() { + @Override + NettyClientCustomizer specializeForChannel(Channel channel_, ChannelRole role_) { + return CustomizerTracker.this.specializeForChannel(channel_, role_) + } + + Snapshot snap() { + return new Snapshot(channel, role, channel.pipeline().names()) + } + + @Override + void onInitialPipelineBuilt() { + initialPipelineBuilt.add(snap()) + } + + @Override + void onStreamPipelineBuilt() { + streamPipelineBuilt.add(snap()) + } + + @Override + void onRequestPipelineBuilt() { + requestPipelineBuilt.add(snap()) + } + } + } + + @Override + Registry onCreated(BeanCreatedEvent event) { + event.getBean().register(this) + return event.getBean() + } + + static class Snapshot { + final Channel channel + final ChannelRole role + final List handlerNames + + Snapshot(Channel channel, ChannelRole role, List handlerNames) { + this.channel = channel + this.role = role + this.handlerNames = handlerNames + } + } } } From b06fb158affa25025258fc44fb77efc32bde4bc5 Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 21 Sep 2022 11:38:54 +0200 Subject: [PATCH 11/82] test h2c customization --- .../http/client/netty/ConnectionManager.java | 2 +- .../client/netty/ConnectionManagerSpec.groovy | 63 +++++++++++-------- 2 files changed, 37 insertions(+), 28 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 1e18afedaae..79d9827e08a 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -1505,7 +1505,7 @@ protected void initChannel(Channel ch) throws Exception { Http2FrameCodec frameCodec = Http2FrameCodecBuilder.forClient().build(); HttpClientCodec sourceCodec = new HttpClientCodec(); - Http2ClientUpgradeCodec upgradeCodec = new Http2ClientUpgradeCodec(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, frameCodec, + Http2ClientUpgradeCodec upgradeCodec = new Http2ClientUpgradeCodec(frameCodec, new ChannelInitializer() { @Override protected void initChannel(Channel ch) throws Exception { diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 5605ac3334d..21d7cd7bde5 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -157,21 +157,7 @@ class ConnectionManagerSpec extends Specification { patch(client, conn.clientChannel) def future = conn.testExchangeRequest(client) - conn.clientChannel.pipeline().fireChannelActive() - conn.advance() - - Http2HeadersFrame upgradeRequest = conn.serverChannel.readInbound() - assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' - assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/' - assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.AUTHORITY.value()) == 'example.com:80' - assert upgradeRequest.headers().get('content-length') == '0' - // client closes the stream immediately - assert upgradeRequest.stream().state() == Http2Stream.State.CLOSED - - assert conn.serverChannel.readInbound() instanceof Http2SettingsFrame - assert conn.serverChannel.readInbound() instanceof Http2ResetFrame - assert conn.serverChannel.readInbound() instanceof Http2SettingsAckFrame - + conn.exchangeH2c() conn.testExchangeResponse(future) cleanup: @@ -324,21 +310,30 @@ class ConnectionManagerSpec extends Specification { ctx.close() } - def 'http2 alpn customization'() { + def 'http2 customization'(boolean secure) { given: def ctx = ApplicationContext.run([ 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + 'micronaut.http.client.plaintext-mode': 'h2c', ]) def client = ctx.getBean(DefaultHttpClient) def tracker = ctx.getBean(CustomizerTracker) def conn = new EmbeddedTestConnectionHttp2() - conn.setupHttp2Tls() + if (secure) { + conn.setupHttp2Tls() + } else { + conn.setupH2c() + } patch(client, conn.clientChannel) when: def r1 = conn.testExchangeRequest(client) - conn.exchangeSettings() + if (secure) { + conn.exchangeSettings() + } else { + conn.exchangeH2c() + } conn.testExchangeResponse(r1) def r2 = conn.testStreamingRequest(client) @@ -347,7 +342,7 @@ class ConnectionManagerSpec extends Specification { then: def outerChannel = tracker.initialPipelineBuilt.poll() outerChannel.channel == conn.clientChannel - outerChannel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_SSL) + outerChannel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_SSL) == secure !outerChannel.handlerNames.contains(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION) tracker.initialPipelineBuilt.isEmpty() @@ -373,6 +368,9 @@ class ConnectionManagerSpec extends Specification { cleanup: client.close() ctx.close() + + where: + secure << [true, false] } static class EmbeddedTestConnectionBase { @@ -538,14 +536,25 @@ class ConnectionManagerSpec extends Specification { void exchangeSettings() { advance() - def settings = serverChannel.readInbound() - if (!(settings instanceof Http2SettingsFrame)) { - throw new AssertionError(settings) - } - def ack = serverChannel.readInbound() - if (!(ack instanceof Http2SettingsAckFrame)) { - throw new AssertionError(ack) - } + assert serverChannel.readInbound() instanceof Http2SettingsFrame + assert serverChannel.readInbound() instanceof Http2SettingsAckFrame + } + + void exchangeH2c() { + clientChannel.pipeline().fireChannelActive() + advance() + + Http2HeadersFrame upgradeRequest = serverChannel.readInbound() + assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' + assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.PATH.value()) == '/' + assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.AUTHORITY.value()) == 'example.com:80' + assert upgradeRequest.headers().get('content-length') == '0' + // client closes the stream immediately + assert upgradeRequest.stream().state() == Http2Stream.State.CLOSED + + assert serverChannel.readInbound() instanceof Http2SettingsFrame + assert serverChannel.readInbound() instanceof Http2ResetFrame + assert serverChannel.readInbound() instanceof Http2SettingsAckFrame } void respondOk(Http2FrameStream stream) { From c966519f196551decb1d63e468c7f245709916f1 Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 21 Sep 2022 11:59:30 +0200 Subject: [PATCH 12/82] test http1 compression --- .../client/netty/ConnectionManagerSpec.groovy | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 21d7cd7bde5..50091e3045a 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -27,6 +27,7 @@ import io.netty.handler.codec.http.DefaultFullHttpResponse import io.netty.handler.codec.http.DefaultHttpContent import io.netty.handler.codec.http.DefaultHttpResponse import io.netty.handler.codec.http.DefaultLastHttpContent +import io.netty.handler.codec.http.HttpContentCompressor import io.netty.handler.codec.http.HttpMethod import io.netty.handler.codec.http.HttpResponseStatus import io.netty.handler.codec.http.HttpServerCodec @@ -129,6 +130,35 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'http1 get with compression'() { + def ctx = ApplicationContext.run() + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp1() + conn.setupHttp1() + conn.serverChannel.pipeline().addLast(new HttpContentCompressor()) + patch(client, conn.clientChannel) + + def future = Mono.from(client.exchange( + HttpRequest.GET('http://example.com/foo').header('accept-encoding', 'gzip'), String)).toFuture() + future.exceptionally(t -> t.printStackTrace()) + conn.advance() + + assert conn.serverChannel.readInbound() instanceof io.netty.handler.codec.http.HttpRequest + + def response = new DefaultFullHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer("foo".bytes)) + response.headers().add('content-length', 3) + conn.serverChannel.writeOutbound(response) + + conn.advance() + assert future.get().status() == HttpStatus.OK + assert future.get().body() == 'foo' + + cleanup: + client.close() + ctx.close() + } + def 'simple http1 tls get'() { def ctx = ApplicationContext.run([ 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, From c0aa9f3c490a5a0043700edcddabcecc6635331c Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 21 Sep 2022 12:16:19 +0200 Subject: [PATCH 13/82] http2 compression --- .../http/client/netty/ConnectionManager.java | 3 +- .../client/netty/ConnectionManagerSpec.groovy | 36 +++++++++++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 79d9827e08a..35331c6da2a 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -1859,7 +1859,8 @@ private void satisfy0(Sinks.One sink) { if (future.isSuccess()) { Http2StreamChannel streamChannel = future.get(); streamChannel.pipeline() - .addLast(new Http2StreamFrameToHttpObjectCodec(false)); + .addLast(new Http2StreamFrameToHttpObjectCodec(false)) + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); NettyClientCustomizer streamCustomizer = customizer.specializeForChannel(streamChannel, NettyClientCustomizer.ChannelRole.HTTP2_STREAM); Sinks.EmitResult emitResult = sink.tryEmitValue(new PoolHandle(streamChannel) { @Override diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 50091e3045a..48adc737495 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -60,6 +60,7 @@ import spock.lang.Specification import java.nio.charset.StandardCharsets import java.util.concurrent.Future +import java.util.zip.GZIPOutputStream class ConnectionManagerSpec extends Specification { private static void patch(DefaultHttpClient httpClient, EmbeddedChannel... channels) { @@ -159,6 +160,41 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'http2 get with compression'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp2() + conn.setupHttp2Tls() + patch(client, conn.clientChannel) + + def future = Mono.from(client.exchange('https://example.com/foo', String)).toFuture() + future.exceptionally(t -> t.printStackTrace()) + conn.exchangeSettings() + + Http2HeadersFrame request = conn.serverChannel.readInbound() + def responseHeaders = new DefaultHttp2Headers() + responseHeaders.add(Http2Headers.PseudoHeaderName.STATUS.value(), "200") + responseHeaders.add('content-encoding', "gzip") + conn.serverChannel.writeOutbound(new DefaultHttp2HeadersFrame(responseHeaders, false).stream(request.stream())) + def compressedOut = new ByteArrayOutputStream() + try (OutputStream os = new GZIPOutputStream(compressedOut)) { + os.write('foo'.bytes) + } + conn.serverChannel.writeOutbound(new DefaultHttp2DataFrame(Unpooled.wrappedBuffer(compressedOut.toByteArray()), true).stream(request.stream())) + + conn.advance() + def response = future.get() + assert response.status() == HttpStatus.OK + assert response.body() == 'foo' + + cleanup: + client.close() + ctx.close() + } + def 'simple http1 tls get'() { def ctx = ApplicationContext.run([ 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, From ff1c8971a712aea5601428808decee19b783c33b Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 21 Sep 2022 12:24:43 +0200 Subject: [PATCH 14/82] http2 frame logging --- .../http/client/netty/ConnectionManager.java | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 35331c6da2a..d5dd1972bd5 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -1019,6 +1019,21 @@ private void addReadTimeoutHandler(ChannelPipeline pipeline) { } } + private Http2FrameCodec makeFrameCodec() { + Http2FrameCodecBuilder builder = Http2FrameCodecBuilder.forClient(); + configuration.getLogLevel().ifPresent(logLevel -> { + try { + final io.netty.handler.logging.LogLevel nettyLevel = io.netty.handler.logging.LogLevel.valueOf( + logLevel.name() + ); + builder.frameLogger(new Http2FrameLogger(nettyLevel, DefaultHttpClient.class)); + } catch (IllegalArgumentException e) { + throw customizeException(new HttpClientException("Unsupported log level: " + logLevel)); + } + }); + return builder.build(); + } + private void removeReadTimeoutHandler(ChannelPipeline pipeline) { if (readTimeoutMillis != null && pipeline.context(ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT) != null) { pipeline.remove(ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT); @@ -1402,7 +1417,7 @@ protected void initChannel(Channel ch) { @Override protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { - ctx.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, Http2FrameCodecBuilder.forClient().build()); + ctx.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, makeFrameCodec()); initHttp2(pool, ctx.channel(), channelCustomizer); ctx.pipeline().remove(initialErrorHandler); } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { @@ -1502,7 +1517,7 @@ private class Http2UpgradeInitializer extends ChannelInitializer { protected void initChannel(Channel ch) throws Exception { NettyClientCustomizer connectionCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); - Http2FrameCodec frameCodec = Http2FrameCodecBuilder.forClient().build(); + Http2FrameCodec frameCodec = makeFrameCodec(); HttpClientCodec sourceCodec = new HttpClientCodec(); Http2ClientUpgradeCodec upgradeCodec = new Http2ClientUpgradeCodec(frameCodec, From c20542b5af5c570d0a833f5d94b919b6f8c9df88 Mon Sep 17 00:00:00 2001 From: yawkat Date: Thu, 22 Sep 2022 18:24:34 +0200 Subject: [PATCH 15/82] remove some code, implement connection read timeout --- .../http/client/netty/ConnectionManager.java | 332 ++++-------------- .../client/netty/ConnectionManagerSpec.groovy | 81 +++++ 2 files changed, 145 insertions(+), 268 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index d5dd1972bd5..2eea26b7f3b 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -16,7 +16,6 @@ package io.micronaut.http.client.netty; import io.micronaut.core.annotation.Internal; -import io.micronaut.core.annotation.NonNull; import io.micronaut.core.annotation.Nullable; import io.micronaut.core.reflect.InstantiationUtils; import io.micronaut.core.util.StringUtils; @@ -30,7 +29,6 @@ import io.micronaut.http.netty.stream.DefaultHttp2Content; import io.micronaut.http.netty.stream.Http2Content; import io.micronaut.http.netty.stream.HttpStreamsClientHandler; -import io.micronaut.http.netty.stream.StreamingInboundHttp2ToHttpAdapter; import io.micronaut.scheduling.instrument.Instrumentation; import io.micronaut.scheduling.instrument.InvocationInstrumenter; import io.micronaut.websocket.exceptions.WebSocketSessionException; @@ -40,7 +38,6 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelFactory; import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; @@ -66,7 +63,6 @@ import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.HttpContentDecompressor; import io.netty.handler.codec.http.HttpHeaderNames; -import io.netty.handler.codec.http.HttpMessage; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpObjectAggregator; import io.netty.handler.codec.http.HttpResponse; @@ -75,12 +71,10 @@ import io.netty.handler.codec.http.LastHttpContent; import io.netty.handler.codec.http.websocketx.extensions.compression.WebSocketClientCompressionHandler; import io.netty.handler.codec.http2.DefaultHttp2Connection; -import io.netty.handler.codec.http2.DelegatingDecompressorFrameListener; import io.netty.handler.codec.http2.Http2ClientUpgradeCodec; import io.netty.handler.codec.http2.Http2Connection; import io.netty.handler.codec.http2.Http2FrameCodec; import io.netty.handler.codec.http2.Http2FrameCodecBuilder; -import io.netty.handler.codec.http2.Http2FrameListener; import io.netty.handler.codec.http2.Http2FrameLogger; import io.netty.handler.codec.http2.Http2MultiplexHandler; import io.netty.handler.codec.http2.Http2Settings; @@ -89,10 +83,6 @@ import io.netty.handler.codec.http2.Http2StreamChannel; import io.netty.handler.codec.http2.Http2StreamChannelBootstrap; import io.netty.handler.codec.http2.Http2StreamFrameToHttpObjectCodec; -import io.netty.handler.codec.http2.HttpConversionUtil; -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.logging.LoggingHandler; import io.netty.handler.proxy.HttpProxyHandler; import io.netty.handler.proxy.Socks5ProxyHandler; @@ -102,6 +92,7 @@ import io.netty.handler.ssl.SslHandler; import io.netty.handler.timeout.IdleStateEvent; import io.netty.handler.timeout.IdleStateHandler; +import io.netty.handler.timeout.ReadTimeoutException; import io.netty.handler.timeout.ReadTimeoutHandler; import io.netty.resolver.NoopAddressResolverGroup; import io.netty.util.Attribute; @@ -123,11 +114,13 @@ import java.net.SocketAddress; import java.time.Duration; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.OptionalInt; import java.util.Queue; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CopyOnWriteArrayList; @@ -618,37 +611,6 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc }); } return Mono.create(emitter -> { - ChannelFuture channelFuture; - try { - if (httpVersion == HttpVersion.HTTP_2_0) { - - channelFuture = doConnect(requestKey, true, isProxy, acceptEvents, channelHandlerContext -> { - try { - final Channel channel = channelHandlerContext.channel(); - emitter.success(mockPoolHandle(channel)); - } catch (Exception e) { - emitter.error(e); - } - }); - } else { - channelFuture = doConnect(requestKey, true, isProxy, acceptEvents, null); - addInstrumentedListener(channelFuture, - (ChannelFutureListener) f -> { - if (f.isSuccess()) { - Channel channel = f.channel(); - emitter.success(mockPoolHandle(channel)); - } else { - Throwable cause = f.cause(); - emitter.error(customizeException(new HttpClientException("Connect error:" + cause.getMessage(), cause))); - } - }); - } - } catch (HttpClientException e) { - emitter.error(e); - return; - } - - // todo: on emitter dispose/cancel, close channel }) .delayUntil(this::delayUntilHttp2Ready) .map(poolHandle -> { @@ -808,144 +770,6 @@ public void channelAcquired(Channel ch) throws Exception { }; } - /** - * Configures HTTP/2 for the channel when SSL is enabled. - * - * @param httpClientInitializer The client initializer - * @param ch The channel - * @param sslCtx The SSL context - * @param host The host - * @param port The port - * @param connectionHandler The connection handler - */ - private void configureHttp2Ssl( - HttpClientInitializer httpClientInitializer, - @NonNull Channel ch, - @NonNull SslContext sslCtx, - String host, - int port, - HttpToHttp2ConnectionHandler connectionHandler) { - ChannelPipeline pipeline = ch.pipeline(); - // Specify Host in SSLContext New Handler to add TLS SNI Extension - pipeline.addLast(ChannelPipelineCustomizer.HANDLER_SSL, sslCtx.newHandler(ch.alloc(), host, port)); - // We must wait for the handshake to finish and the protocol to be negotiated before configuring - // the HTTP/2 components of the pipeline. - pipeline.addLast( - ChannelPipelineCustomizer.HANDLER_HTTP2_PROTOCOL_NEGOTIATOR, - new ApplicationProtocolNegotiationHandler(ApplicationProtocolNames.HTTP_2) { - - @Override - public void handlerRemoved(ChannelHandlerContext ctx) { - // the logic to send the request should only be executed once the HTTP/2 - // Connection Preface request has been sent. Once the Preface has been sent and - // removed then this handler is removed so we invoke the remaining logic once - // this handler removed - final Consumer contextConsumer = - httpClientInitializer.contextConsumer; - if (contextConsumer != null) { - contextConsumer.accept(ctx); - } - } - - @Override - protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { - if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { - ChannelPipeline p = ctx.pipeline(); - if (httpClientInitializer.stream) { - // stream consumer manages backpressure and reads - ctx.channel().config().setAutoRead(false); - } - p.addLast( - ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, - new Http2SettingsHandler(ch.newPromise()) - ); - httpClientInitializer.addEventStreamHandlerIfNecessary(p); - httpClientInitializer.addFinalHandler(p); - for (ChannelPipelineListener pipelineListener : pipelineListeners) { - pipelineListener.onConnect(p); - } - } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { - ChannelPipeline p = ctx.pipeline(); - httpClientInitializer.addHttp1Handlers(p); - } else { - ctx.close(); - throw customizeException(new HttpClientException("Unknown Protocol: " + protocol)); - } - httpClientInitializer.onStreamPipelineBuilt(); - } - }); - - pipeline.addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, connectionHandler); - } - - /** - * Configures HTTP/2 handling for plaintext (non-SSL) connections. - * - * @param httpClientInitializer The client initializer - * @param ch The channel - * @param connectionHandler The connection handler - */ - private void configureHttp2ClearText( - HttpClientInitializer httpClientInitializer, - @NonNull Channel ch, - @NonNull HttpToHttp2ConnectionHandler connectionHandler) { - HttpClientCodec sourceCodec = new HttpClientCodec(); - Http2ClientUpgradeCodec upgradeCodec = new Http2ClientUpgradeCodec(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, connectionHandler); - HttpClientUpgradeHandler upgradeHandler = new HttpClientUpgradeHandler(sourceCodec, upgradeCodec, 65536); - - final ChannelPipeline pipeline = ch.pipeline(); - pipeline.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, sourceCodec); - httpClientInitializer.settingsHandler = new Http2SettingsHandler(ch.newPromise()); - pipeline.addLast(upgradeHandler); - pipeline.addLast(new ChannelInboundHandlerAdapter() { - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - ctx.fireUserEventTriggered(evt); - if (evt instanceof HttpClientUpgradeHandler.UpgradeEvent) { - httpClientInitializer.onStreamPipelineBuilt(); - ctx.pipeline().remove(this); - } - } - }); - pipeline.addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_UPGRADE_REQUEST, new H2cUpgradeRequestHandler(httpClientInitializer)); - } - - /** - * Creates a new {@link HttpToHttp2ConnectionHandlerBuilder} for the given HTTP/2 connection object and config. - * - * @param connection The connection - * @param configuration The configuration - * @param stream Whether this is a stream request - * @return The {@link HttpToHttp2ConnectionHandlerBuilder} - */ - @NonNull - private static HttpToHttp2ConnectionHandlerBuilder newHttp2ConnectionHandlerBuilder( - @NonNull Http2Connection connection, @NonNull HttpClientConfiguration configuration, boolean stream) { - final HttpToHttp2ConnectionHandlerBuilder builder = new HttpToHttp2ConnectionHandlerBuilder(); - builder.validateHeaders(true); - final Http2FrameListener http2ToHttpAdapter; - - if (!stream) { - http2ToHttpAdapter = new InboundHttp2ToHttpAdapterBuilder(connection) - .maxContentLength(configuration.getMaxContentLength()) - .validateHttpHeaders(true) - .propagateSettings(true) - .build(); - - } else { - http2ToHttpAdapter = new StreamingInboundHttp2ToHttpAdapter( - connection, - configuration.getMaxContentLength() - ); - } - return builder - .connection(connection) - .frameListener(new DelegatingDecompressorFrameListener( - connection, - http2ToHttpAdapter)); - - } - private void configureProxy(ChannelPipeline pipeline, boolean secure, String host, int port) { Proxy proxy = configuration.resolveProxy(secure, host, port); if (Proxy.NO_PROXY.equals(proxy)) { @@ -1040,68 +864,6 @@ private void removeReadTimeoutHandler(ChannelPipeline pipeline) { } } - /** - * A handler that triggers the cleartext upgrade to HTTP/2 by sending an initial HTTP request. - */ - private class H2cUpgradeRequestHandler extends ChannelInboundHandlerAdapter { - - private final HttpClientInitializer initializer; - - /** - * Default constructor. - * - * @param initializer The initializer - */ - public H2cUpgradeRequestHandler(HttpClientInitializer initializer) { - this.initializer = initializer; - } - - @Override - public void channelActive(ChannelHandlerContext ctx) { - // Done with this handler, remove it from the pipeline. - final ChannelPipeline pipeline = ctx.pipeline(); - - pipeline.addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, initializer.settingsHandler); - DefaultFullHttpRequest upgradeRequest = - new DefaultFullHttpRequest(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpMethod.GET, "/", Unpooled.EMPTY_BUFFER); - - // Set HOST header as the remote peer may require it. - InetSocketAddress remote = (InetSocketAddress) ctx.channel().remoteAddress(); - String hostString = remote.getHostString(); - if (hostString == null) { - hostString = remote.getAddress().getHostAddress(); - } - upgradeRequest.headers().set(HttpHeaderNames.HOST, hostString + ':' + remote.getPort()); - ctx.writeAndFlush(upgradeRequest); - - ctx.fireChannelActive(); - if (initializer.contextConsumer != null) { - initializer.contextConsumer.accept(ctx); - } - initializer.addFinalHandler(pipeline); - } - - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof HttpMessage) { - int streamId = ((HttpMessage) msg).headers().getInt(HttpConversionUtil.ExtensionHeaderNames.STREAM_ID.text(), -1); - if (streamId == 1) { - // ignore this message - if (log.isDebugEnabled()) { - log.debug("Received response on HTTP2 stream 1, the stream used to respond to the initial upgrade request. Ignoring."); - } - ReferenceCountUtil.release(msg); - if (msg instanceof LastHttpContent) { - ctx.pipeline().remove(this); - } - return; - } - } - - super.channelRead(ctx, msg); - } - } - /** * Reads the first {@link Http2Settings} object and notifies a {@link io.netty.channel.ChannelPromise}. */ @@ -1198,25 +960,8 @@ protected void initChannel(Channel ch) { if (httpVersion == HttpVersion.HTTP_2_0) { final Http2Connection connection = new DefaultHttp2Connection(false); - final HttpToHttp2ConnectionHandlerBuilder builder = - newHttp2ConnectionHandlerBuilder(connection, configuration, stream); - - configuration.getLogLevel().ifPresent(logLevel -> { - try { - final io.netty.handler.logging.LogLevel nettyLevel = io.netty.handler.logging.LogLevel.valueOf( - logLevel.name() - ); - builder.frameLogger(new Http2FrameLogger(nettyLevel, DefaultHttpClient.class)); - } catch (IllegalArgumentException e) { - throw customizeException(new HttpClientException("Unsupported log level: " + logLevel)); - } - }); - HttpToHttp2ConnectionHandler connectionHandler = builder - .build(); if (sslContext != null) { - configureHttp2Ssl(this, ch, sslContext, host, port, connectionHandler); } else { - configureHttp2ClearText(this, ch, connectionHandler); } channelCustomizer.onInitialPipelineBuilt(); } else { @@ -1280,7 +1025,7 @@ void onStreamPipelineBuilt() { void addHttp1Handlers(ChannelPipeline p) { p.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()); - p.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); + p.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECOMPRESSOR, new HttpContentDecompressor()); if (!stream) { p.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR, new HttpObjectAggregator(configuration.getMaxContentLength()) { @@ -1462,7 +1207,7 @@ private void initHttp1(Channel ch) { } private void initHttp2(Pool pool, Channel ch, NettyClientCustomizer connectionCustomizer) { - ch.pipeline().addLast(new Http2MultiplexHandler(new ChannelInitializer() { + Http2MultiplexHandler multiplexHandler = new Http2MultiplexHandler(new ChannelInitializer() { @Override protected void initChannel(Http2StreamChannel ch) throws Exception { // todo: fail connection? @@ -1475,7 +1220,8 @@ protected void initChannel(Http2StreamChannel ch) throws Exception { // discard any response data for the upgrade request ch.close(); } - })); + }); + ch.pipeline().addLast(multiplexHandler); ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new InitialConnectionErrorHandler(pool) { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { @@ -1552,7 +1298,7 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { } } - abstract class PoolHandle { + static abstract class PoolHandle { final Channel channel; /** @@ -1633,6 +1379,26 @@ private static boolean incrementWithLimit(AtomicInteger variable, int limit) { } } + private void addReadTimeoutHandler(ChannelPipeline pipeline, String before, Consumer fireTimeout) { + configuration.getReadTimeout() + .map(dur -> new ConditionalReadTimeoutHandler(dur.toNanos(), TimeUnit.NANOSECONDS, fireTimeout)) + .ifPresent(ch -> pipeline.addBefore(before, ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT, ch)); + } + + private static class ConditionalReadTimeoutHandler extends ReadTimeoutHandler { + private final Consumer fireTimeout; + + ConditionalReadTimeoutHandler(long timeout, TimeUnit unit, Consumer fireTimeout) { + super(timeout, unit); + this.fireTimeout = fireTimeout; + } + + @Override + protected void readTimedOut(ChannelHandlerContext ctx) throws Exception { + fireTimeout.accept(ctx); + } + } + private final class Pool { private static final int MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION = 4; // TODO: config private static final int MAX_PENDING_CONNECTIONS = 1; @@ -1651,6 +1417,7 @@ private final class Pool { Mono acquire() { Sinks.One sink = Sinks.one(); acquire(sink); + // todo: if the subscriber cancels before the connection is acquired, what happens? return sink.asMono(); } @@ -1730,6 +1497,18 @@ final class Http1ConnectionHolder { } void init(boolean fireInitialPipelineBuilt) { + addReadTimeoutHandler( + channel.pipeline(), + requestKey.isSecure() ? + ChannelPipelineCustomizer.HANDLER_SSL : + ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, + ctx -> { + if (hasLiveRequest.get()) { + ctx.fireExceptionCaught(ReadTimeoutException.INSTANCE); + ctx.close(); + } + } + ); channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { @@ -1823,6 +1602,7 @@ final class Http2ConnectionHolder { private final Channel channel; private final AtomicInteger liveRequests = new AtomicInteger(0); private final NettyClientCustomizer customizer; + private final Set liveStreamChannels = new HashSet<>(); // todo: https://github.com/netty/netty/pull/12830 Http2ConnectionHolder(Channel channel, NettyClientCustomizer customizer) { this.channel = channel; @@ -1830,6 +1610,20 @@ final class Http2ConnectionHolder { } void init() { + addReadTimeoutHandler( + channel.pipeline(), + requestKey.isSecure() ? + ChannelPipelineCustomizer.HANDLER_SSL : + ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, + ctx -> { + if (liveRequests.get() != 0) { + for (Channel sc : liveStreamChannels) { + sc.pipeline().fireExceptionCaught(ReadTimeoutException.INSTANCE); + } + ctx.close(); + } + } + ); channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { @@ -1875,9 +1669,9 @@ private void satisfy0(Sinks.One sink) { Http2StreamChannel streamChannel = future.get(); streamChannel.pipeline() .addLast(new Http2StreamFrameToHttpObjectCodec(false)) - .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECOMPRESSOR, new HttpContentDecompressor()); NettyClientCustomizer streamCustomizer = customizer.specializeForChannel(streamChannel, NettyClientCustomizer.ChannelRole.HTTP2_STREAM); - Sinks.EmitResult emitResult = sink.tryEmitValue(new PoolHandle(streamChannel) { + PoolHandle ph = new PoolHandle(streamChannel) { @Override void taint() { // todo @@ -1885,6 +1679,8 @@ void taint() { @Override void release() { + liveStreamChannels.remove(streamChannel); + streamChannel.close(); liveRequests.decrementAndGet(); // todo: claim a new request } @@ -1898,11 +1694,11 @@ boolean canReturn() { void notifyRequestPipelineBuilt() { streamCustomizer.onRequestPipelineBuilt(); } - }); + }; + liveStreamChannels.add(streamChannel); + Sinks.EmitResult emitResult = sink.tryEmitValue(ph); if (emitResult.isFailure()) { - streamChannel.close(); - liveRequests.decrementAndGet(); - // todo: claim a new request + ph.release(); } } else { log.debug("Failed to open http2 stream", future.cause()); diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 48adc737495..482df1c0edb 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -9,6 +9,7 @@ import io.micronaut.http.HttpStatus import io.micronaut.http.HttpVersion import io.micronaut.http.client.HttpClient import io.micronaut.http.client.StreamingHttpClient +import io.micronaut.http.client.exceptions.ReadTimeoutException import io.micronaut.http.netty.channel.ChannelPipelineCustomizer import io.micronaut.http.server.netty.ssl.CertificateProvidedSslBuilder import io.micronaut.http.ssl.SslConfiguration @@ -59,7 +60,9 @@ import reactor.core.publisher.Mono import spock.lang.Specification import java.nio.charset.StandardCharsets +import java.util.concurrent.ExecutionException import java.util.concurrent.Future +import java.util.concurrent.TimeUnit import java.util.zip.GZIPOutputStream class ConnectionManagerSpec extends Specification { @@ -439,6 +442,84 @@ class ConnectionManagerSpec extends Specification { secure << [true, false] } + def 'http1 exchange read timeout'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.read-timeout': '5s', + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp1() + conn.setupHttp1() + patch(client, conn.clientChannel) + + // do one request + conn.testExchange(client) + conn.clientChannel.unfreezeTime() + // connection is in reserve, should not time out + TimeUnit.SECONDS.sleep(10) + conn.advance() + + // second request + def future = Mono.from(client.exchange('http://example.com/foo', String)).toFuture() + conn.advance() + + // todo: move to advanceTime once IdleStateHandler supports it + TimeUnit.SECONDS.sleep(5) + conn.advance() + + assert future.isDone() + when: + future.get() + then: + def e = thrown ExecutionException + e.cause instanceof ReadTimeoutException + + cleanup: + client.close() + ctx.close() + } + + def 'http2 exchange read timeout'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + 'micronaut.http.client.read-timeout': '5s', + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp2() + conn.setupHttp2Tls() + patch(client, conn.clientChannel) + + // one request opens the connection + def r1 = conn.testExchangeRequest(client) + conn.exchangeSettings() + conn.testExchangeResponse(r1) + conn.clientChannel.unfreezeTime() + + // connection is in reserve, should not time out + TimeUnit.SECONDS.sleep(10) + conn.advance() + + // second request + def future = Mono.from(client.exchange('https://example.com/foo', String)).toFuture() + conn.advance() + + // todo: move to advanceTime once IdleStateHandler supports it + TimeUnit.SECONDS.sleep(5) + conn.advance() + + assert future.isDone() + when: + future.get() + then: + def e = thrown ExecutionException + e.cause instanceof ReadTimeoutException + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnectionBase { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel From d1b2e1342fa45f52100c3a98695a152659d9c355 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 23 Sep 2022 12:31:10 +0200 Subject: [PATCH 16/82] delete some code --- .../http/client/netty/ConnectionManager.java | 282 ++++-------------- 1 file changed, 58 insertions(+), 224 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 2eea26b7f3b..e35a6266fef 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -48,12 +48,8 @@ import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.pool.AbstractChannelPoolHandler; -import io.netty.channel.pool.AbstractChannelPoolMap; import io.netty.channel.pool.ChannelHealthChecker; import io.netty.channel.pool.ChannelPool; -import io.netty.channel.pool.ChannelPoolMap; -import io.netty.channel.pool.FixedChannelPool; -import io.netty.channel.pool.SimpleChannelPool; import io.netty.handler.codec.LineBasedFrameDecoder; import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.DefaultHttpContent; @@ -101,9 +97,7 @@ import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; import io.netty.util.concurrent.Promise; -import org.reactivestreams.Publisher; import org.slf4j.Logger; -import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import reactor.core.publisher.Sinks; import reactor.core.scheduler.Scheduler; @@ -147,7 +141,6 @@ class ConnectionManager { AttributeKey.valueOf("micronaut.http.streamChannelInitialized"); private static final AttributeKey STREAM_KEY = AttributeKey.valueOf("micronaut.http2.stream"); - final ChannelPoolMap poolMap; final InvocationInstrumenter instrumenter; final HttpVersion httpVersion; @@ -168,7 +161,6 @@ class ConnectionManager { private final String informationalServiceId; ConnectionManager(ConnectionManager from) { - this.poolMap = from.poolMap; this.instrumenter = from.instrumenter; this.httpVersion = from.httpVersion; this.log = from.log; @@ -235,51 +227,6 @@ class ConnectionManager { final ChannelHealthChecker channelHealthChecker = channel -> channel.eventLoop().newSucceededFuture(channel.isActive() && !ConnectTTLHandler.isChannelExpired(channel)); HttpClientConfiguration.ConnectionPoolConfiguration connectionPoolConfiguration = configuration.getConnectionPoolConfiguration(); - // HTTP/2 defaults to keep alive connections so should we should always use a pool - if (connectionPoolConfiguration.isEnabled() || httpVersion == io.micronaut.http.HttpVersion.HTTP_2_0) { - int maxConnections = connectionPoolConfiguration.getMaxConnections(); - if (maxConnections > -1) { - poolMap = new AbstractChannelPoolMap() { - @Override - protected ChannelPool newPool(DefaultHttpClient.RequestKey key) { - Bootstrap newBootstrap = bootstrap.clone(group); - initBootstrapForProxy(newBootstrap, key.isSecure(), key.getHost(), key.getPort()); - newBootstrap.remoteAddress(key.getRemoteAddress()); - - AbstractChannelPoolHandler channelPoolHandler = newPoolHandler(key); - final long acquireTimeoutMillis = connectionPoolConfiguration.getAcquireTimeout().map(Duration::toMillis).orElse(-1L); - return new FixedChannelPool( - newBootstrap, - channelPoolHandler, - channelHealthChecker, - acquireTimeoutMillis > -1 ? FixedChannelPool.AcquireTimeoutAction.FAIL : null, - acquireTimeoutMillis, - maxConnections, - connectionPoolConfiguration.getMaxPendingAcquires() - - ); - } - }; - } else { - poolMap = new AbstractChannelPoolMap() { - @Override - protected ChannelPool newPool(DefaultHttpClient.RequestKey key) { - Bootstrap newBootstrap = bootstrap.clone(group); - initBootstrapForProxy(newBootstrap, key.isSecure(), key.getHost(), key.getPort()); - newBootstrap.remoteAddress(key.getRemoteAddress()); - - AbstractChannelPoolHandler channelPoolHandler = newPoolHandler(key); - return new SimpleChannelPool( - newBootstrap, - channelPoolHandler, - channelHealthChecker - ); - } - }; - } - } else { - this.poolMap = null; - } Optional connectTimeout = configuration.getConnectTimeout(); connectTimeout.ifPresent(duration -> bootstrap.option( @@ -340,28 +287,8 @@ public void start() { * @see DefaultHttpClient#stop() */ public void shutdown() { - // todo: shutdown pools - if (poolMap instanceof Iterable) { - Iterable> i = (Iterable) poolMap; - for (Map.Entry entry : i) { - ChannelPool cp = entry.getValue(); - try { - if (cp instanceof SimpleChannelPool) { - addInstrumentedListener(((SimpleChannelPool) cp).closeAsync(), future -> { - if (!future.isSuccess()) { - final Throwable cause = future.cause(); - if (cause != null) { - log.error("Error shutting down HTTP client connection pool: " + cause.getMessage(), cause); - } - } - }); - } else { - cp.close(); - } - } catch (Exception cause) { - log.error("Error shutting down HTTP client connection pool: " + cause.getMessage(), cause); - } - } + for (Pool pool : pools.values()) { + pool.shutdown(); } if (shutdownGroup) { Duration shutdownTimeout = configuration.getShutdownTimeout() @@ -400,33 +327,6 @@ public Scheduler getEventLoopScheduler() { return Schedulers.fromExecutor(group); } - /** - * Creates an initial connection to the given remote host. - * - * @param requestKey The request key to connect to - * @param isStream Is the connection a stream connection - * @param isProxy Is this a streaming proxy - * @param acceptsEvents Whether the connection will accept events - * @param contextConsumer The logic to run once the channel is configured correctly - * @return A ChannelFuture - * @throws HttpClientException If the URI is invalid - */ - private ChannelFuture doConnect( - DefaultHttpClient.RequestKey requestKey, - boolean isStream, - boolean isProxy, - boolean acceptsEvents, - Consumer contextConsumer) throws HttpClientException { - return doConnect(requestKey, new HttpClientInitializer( - buildSslContext(requestKey), - requestKey.getHost(), - requestKey.getPort(), - isStream, - isProxy, - acceptsEvents, - contextConsumer)); - } - // for testing protected ChannelFuture doConnect(DefaultHttpClient.RequestKey requestKey, ChannelInitializer channelInitializer) { String host = requestKey.getHost(); @@ -463,10 +363,6 @@ private SslContext buildSslContext(DefaultHttpClient.RequestKey requestKey) { return sslCtx; } - private PoolHandle mockPoolHandle(Channel channel) { - return new OldPoolHandle(null, channel); - } - /** * Get a connection for exchange-like (non-streaming) http client methods. * @@ -476,83 +372,21 @@ private PoolHandle mockPoolHandle(Channel channel) { * @return A mono that will complete once the channel is ready for transmission */ Mono connectForExchange(DefaultHttpClient.RequestKey requestKey, boolean multipart, boolean acceptEvents) { - if (true) { - Pool pool = pools.computeIfAbsent(requestKey, Pool::new); - // todo: aggregator - return pool.acquire().map(ph -> { - ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR, new HttpObjectAggregator(configuration.getMaxContentLength()) { - @Override - protected void finishAggregation(FullHttpMessage aggregated) throws Exception { - if (!HttpUtil.isContentLengthSet(aggregated)) { - if (aggregated.content().readableBytes() > 0) { - super.finishAggregation(aggregated); - } + Pool pool = pools.computeIfAbsent(requestKey, Pool::new); + return pool.acquire().map(ph -> { + // TODO: this sucks + ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR, new HttpObjectAggregator(configuration.getMaxContentLength()) { + @Override + protected void finishAggregation(FullHttpMessage aggregated) throws Exception { + if (!HttpUtil.isContentLengthSet(aggregated)) { + if (aggregated.content().readableBytes() > 0) { + super.finishAggregation(aggregated); } } - }); - return ph; - }); - } - return Mono.create(emitter -> { - if (poolMap != null && !multipart) { - try { - ChannelPool channelPool = poolMap.get(requestKey); - addInstrumentedListener(channelPool.acquire(), future -> { - if (future.isSuccess()) { - Channel channel = future.get(); - PoolHandle poolHandle = new OldPoolHandle(channelPool, channel); - Future initFuture = channel.attr(STREAM_CHANNEL_INITIALIZED).get(); - if (initFuture == null) { - emitter.success(poolHandle); - } else { - // we should wait until the handshake completes - addInstrumentedListener(initFuture, f -> { - emitter.success(poolHandle); - }); - } - } else { - Throwable cause = future.cause(); - emitter.error(customizeException(new HttpClientException("Connect Error: " + cause.getMessage(), cause))); - } - }); - } catch (HttpClientException e) { - emitter.error(e); } - } else { - ChannelFuture connectionFuture = doConnect(requestKey, false, false, acceptEvents, null); - addInstrumentedListener(connectionFuture, future -> { - if (!future.isSuccess()) { - Throwable cause = future.cause(); - emitter.error(customizeException(new HttpClientException("Connect Error: " + cause.getMessage(), cause))); - } else { - emitter.success(mockPoolHandle(connectionFuture.channel())); - } - }); - } - }) - .delayUntil(this::delayUntilHttp2Ready) - .map(poolHandle -> { - addReadTimeoutHandler(poolHandle.channel.pipeline()); - return poolHandle; }); - } - - private Publisher delayUntilHttp2Ready(PoolHandle poolHandle) { - Http2SettingsHandler settingsHandler = (Http2SettingsHandler) poolHandle.channel.pipeline().get(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS); - if (settingsHandler == null) { - return Flux.empty(); - } - Sinks.Empty empty = Sinks.empty(); - addInstrumentedListener(settingsHandler.promise, future -> { - if (future.isSuccess()) { - empty.tryEmitEmpty(); - } else { - poolHandle.taint(); - poolHandle.release(); - empty.tryEmitError(future.cause()); - } + return ph; }); - return empty.asMono(); } /** @@ -564,58 +398,49 @@ private Publisher delayUntilHttp2Ready(PoolHandle poolHandle) { * @return A mono that will complete once the channel is ready for transmission */ Mono connectForStream(DefaultHttpClient.RequestKey requestKey, boolean isProxy, boolean acceptEvents) { - if (true) { - Pool pool = pools.computeIfAbsent(requestKey, Pool::new); - return pool.acquire() - .map(ph -> { - // TODO: this sucks - ph.channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { - boolean ignoreOneLast = false; + Pool pool = pools.computeIfAbsent(requestKey, Pool::new); + return pool.acquire() + .map(ph -> { + // TODO: this sucks + ph.channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { + boolean ignoreOneLast = false; - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof HttpResponse && - ((HttpResponse) msg).status().equals(HttpResponseStatus.CONTINUE)) { - ignoreOneLast = true; - } + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof HttpResponse && + ((HttpResponse) msg).status().equals(HttpResponseStatus.CONTINUE)) { + ignoreOneLast = true; + } - super.channelRead(ctx, msg); + super.channelRead(ctx, msg); - if (msg instanceof LastHttpContent) { - if (ignoreOneLast) { - ignoreOneLast = false; - } else { - ctx.pipeline() - .remove(this) - .remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); - ph.release(); - } + if (msg instanceof LastHttpContent) { + if (ignoreOneLast) { + ignoreOneLast = false; + } else { + ctx.pipeline() + .remove(this) + .remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); + ph.release(); } } - }); - ph.channel.pipeline().addLast( - ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, - new HttpStreamsClientHandler() { - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - if (evt instanceof IdleStateEvent) { - // close the connection if it is idle for too long - ph.taint(); - ph.release(); - } - super.userEventTriggered(ctx, evt); + } + }); + ph.channel.pipeline().addLast( + ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, + new HttpStreamsClientHandler() { + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof IdleStateEvent) { + // close the connection if it is idle for too long + ph.taint(); + ph.release(); } + super.userEventTriggered(ctx, evt); } - ); - return ph; - }); - } - return Mono.create(emitter -> { - }) - .delayUntil(this::delayUntilHttp2Ready) - .map(poolHandle -> { - addReadTimeoutHandler(poolHandle.channel.pipeline()); - return poolHandle; + } + ); + return ph; }); } @@ -989,7 +814,7 @@ protected void initChannel(Channel ch) { } // Pool connections require alternative timeout handling - if (poolMap == null) { + if (true/*poolMap == null*/) { // read timeout settings are not applied to streamed requests. // instead idle timeout settings are applied. if (stream) { @@ -1485,6 +1310,15 @@ void onNewConnectionFailure() { // todo: retry connection } + public void shutdown() { + for (Http1ConnectionHolder http1Connection : http1Connections) { + http1Connection.channel.close(); + } + for (Http2ConnectionHolder http2Connection : http2Connections) { + http2Connection.channel.close(); + } + } + final class Http1ConnectionHolder { private final Channel channel; private final NettyClientCustomizer connectionCustomizer; From 32b20252dc3a27976200cff3c6d1579832b2796c Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 23 Sep 2022 13:01:11 +0200 Subject: [PATCH 17/82] implement connection ttl --- .../http/client/netty/ConnectTTLHandler.java | 88 ------- .../http/client/netty/ConnectionManager.java | 217 +++++++++--------- .../http/client/ConnectTTLHandlerSpec.groovy | 41 ---- .../client/netty/ConnectionManagerSpec.groovy | 60 +++++ 4 files changed, 165 insertions(+), 241 deletions(-) delete mode 100644 http-client/src/main/java/io/micronaut/http/client/netty/ConnectTTLHandler.java delete mode 100644 http-client/src/test/groovy/io/micronaut/http/client/ConnectTTLHandlerSpec.groovy diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectTTLHandler.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectTTLHandler.java deleted file mode 100644 index 724611fb18a..00000000000 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectTTLHandler.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Copyright 2017-2020 original authors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.micronaut.http.client.netty; - -import io.netty.channel.Channel; -import io.netty.channel.ChannelDuplexHandler; -import io.netty.channel.ChannelHandlerContext; -import io.netty.util.AttributeKey; -import io.netty.util.concurrent.ScheduledFuture; - -import java.util.concurrent.TimeUnit; - -/** - * A handler that will close channels after they have reached their time-to-live, regardless of usage. - * - * channels that are in use will be closed when they are next - * released to the underlying connection pool. - */ -public class ConnectTTLHandler extends ChannelDuplexHandler { - - public static final AttributeKey RELEASE_CHANNEL = AttributeKey.newInstance("release_channel"); - - private final Long connectionTtlMillis; - private ScheduledFuture channelKiller; - - /** - * Construct ConnectTTLHandler for given arguments. - * @param connectionTtlMillis The configured connect-ttl - */ - public ConnectTTLHandler(Long connectionTtlMillis) { - if (connectionTtlMillis <= 0) { - throw new IllegalArgumentException("connectTTL must be positive"); - } - this.connectionTtlMillis = connectionTtlMillis; - } - - /** - * Will schedule a task when the handler added. - * @param ctx The context to use - * @throws Exception - */ - @Override - public void handlerAdded(ChannelHandlerContext ctx) throws Exception { - super.handlerAdded(ctx); - channelKiller = ctx.channel().eventLoop().schedule(() -> markChannelExpired(ctx), connectionTtlMillis, TimeUnit.MILLISECONDS); - } - - /** - * Will cancel the scheduled tasks when handler removed. - * @param ctx The context to use - */ - @Override - public void handlerRemoved(ChannelHandlerContext ctx) { - channelKiller.cancel(false); - } - - /** - * Will set RELEASE_CHANNEL as true for the channel attribute when connect-ttl is reached. - * @param ctx The context to use - */ - private void markChannelExpired(ChannelHandlerContext ctx) { - if (ctx.channel().isOpen()) { - ctx.channel().attr(RELEASE_CHANNEL).set(true); - } - } - - /** - * Indicates whether the channels connection ttl has expired. - * @param channel The channel to check - * @return true if the channels ttl has expired - */ - public static boolean isChannelExpired(Channel channel) { - return Boolean.TRUE.equals(channel.attr(ConnectTTLHandler.RELEASE_CHANNEL).get()); - } -} diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index e35a6266fef..87977e6d6ec 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -48,7 +48,6 @@ import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.pool.AbstractChannelPoolHandler; -import io.netty.channel.pool.ChannelHealthChecker; import io.netty.channel.pool.ChannelPool; import io.netty.handler.codec.LineBasedFrameDecoder; import io.netty.handler.codec.http.DefaultFullHttpRequest; @@ -97,6 +96,7 @@ import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; import io.netty.util.concurrent.Promise; +import io.netty.util.concurrent.ScheduledFuture; import org.slf4j.Logger; import reactor.core.publisher.Mono; import reactor.core.publisher.Sinks; @@ -151,10 +151,6 @@ class ConnectionManager { private final ThreadFactory threadFactory; private final Bootstrap bootstrap; private final HttpClientConfiguration configuration; - @Nullable - private final Long readTimeoutMillis; - @Nullable - private final Long connectionTimeAliveMillis; private final SslContext sslContext; private final NettyClientCustomizer clientCustomizer; private final Collection pipelineListeners; @@ -169,8 +165,6 @@ class ConnectionManager { this.threadFactory = from.threadFactory; this.bootstrap = from.bootstrap; this.configuration = from.configuration; - this.readTimeoutMillis = from.readTimeoutMillis; - this.connectionTimeAliveMillis = from.connectionTimeAliveMillis; this.sslContext = from.sslContext; this.clientCustomizer = from.clientCustomizer; this.pipelineListeners = from.pipelineListeners; @@ -203,12 +197,6 @@ class ConnectionManager { this.pipelineListeners = pipelineListeners; this.informationalServiceId = informationalServiceId; - this.connectionTimeAliveMillis = configuration.getConnectTtl() - .map(duration -> !duration.isNegative() ? duration.toMillis() : null) - .orElse(null); - this.readTimeoutMillis = configuration.getReadTimeout() - .map(duration -> !duration.isNegative() ? duration.toMillis() : null) - .orElse(null); this.sslContext = nettyClientSslBuilder.build(configuration.getSslConfiguration(), HttpVersion.HTTP_2_0).orElse(null); // TODO: alpn config if (eventLoopGroup != null) { @@ -224,10 +212,6 @@ class ConnectionManager { .channelFactory(socketChannelFactory) .option(ChannelOption.SO_KEEPALIVE, true); - final ChannelHealthChecker channelHealthChecker = channel -> channel.eventLoop().newSucceededFuture(channel.isActive() && !ConnectTTLHandler.isChannelExpired(channel)); - - HttpClientConfiguration.ConnectionPoolConfiguration connectionPoolConfiguration = configuration.getConnectionPoolConfiguration(); - Optional connectTimeout = configuration.getConnectTimeout(); connectTimeout.ifPresent(duration -> bootstrap.option( ChannelOption.CONNECT_TIMEOUT_MILLIS, @@ -550,13 +534,6 @@ void onStreamPipelineBuilt() { } }); - if (connectionTimeAliveMillis != null) { - ch.pipeline() - .addLast( - ChannelPipelineCustomizer.HANDLER_CONNECT_TTL, - new ConnectTTLHandler(connectionTimeAliveMillis) - ); - } } @Override @@ -572,10 +549,6 @@ public void channelReleased(Channel ch) { } } - if (ConnectTTLHandler.isChannelExpired(ch) && ch.isOpen() && !ch.eventLoop().isShuttingDown()) { - ch.close(); - } - removeReadTimeoutHandler(pipeline); } @@ -651,23 +624,6 @@ private E customizeException(E exc) { return exc; } - private void addReadTimeoutHandler(ChannelPipeline pipeline) { - if (readTimeoutMillis != null) { - if (httpVersion == HttpVersion.HTTP_2_0) { - pipeline.addBefore( - ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, - ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT, - new ReadTimeoutHandler(readTimeoutMillis, TimeUnit.MILLISECONDS) - ); - } else { - pipeline.addBefore( - ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, - ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT, - new ReadTimeoutHandler(readTimeoutMillis, TimeUnit.MILLISECONDS)); - } - } - } - private Http2FrameCodec makeFrameCodec() { Http2FrameCodecBuilder builder = Http2FrameCodecBuilder.forClient(); configuration.getLogLevel().ifPresent(logLevel -> { @@ -684,7 +640,7 @@ private Http2FrameCodec makeFrameCodec() { } private void removeReadTimeoutHandler(ChannelPipeline pipeline) { - if (readTimeoutMillis != null && pipeline.context(ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT) != null) { + if (pipeline.context(ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT) != null) { pipeline.remove(ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT); } } @@ -1204,12 +1160,6 @@ private static boolean incrementWithLimit(AtomicInteger variable, int limit) { } } - private void addReadTimeoutHandler(ChannelPipeline pipeline, String before, Consumer fireTimeout) { - configuration.getReadTimeout() - .map(dur -> new ConditionalReadTimeoutHandler(dur.toNanos(), TimeUnit.NANOSECONDS, fireTimeout)) - .ifPresent(ch -> pipeline.addBefore(before, ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT, ch)); - } - private static class ConditionalReadTimeoutHandler extends ReadTimeoutHandler { private final Consumer fireTimeout; @@ -1319,20 +1269,68 @@ public void shutdown() { } } - final class Http1ConnectionHolder { - private final Channel channel; - private final NettyClientCustomizer connectionCustomizer; - private final AtomicBoolean hasLiveRequest = new AtomicBoolean(false); - private volatile boolean canReturn = true; + abstract class ConnectionHolder { + final Channel channel; + final NettyClientCustomizer connectionCustomizer; + ScheduledFuture ttlFuture; + volatile boolean windDownConnection = false; - Http1ConnectionHolder(Channel channel, NettyClientCustomizer connectionCustomizer) { + ConnectionHolder(Channel channel, NettyClientCustomizer connectionCustomizer) { this.channel = channel; this.connectionCustomizer = connectionCustomizer; } + void addTimeoutHandlers(String before, Consumer fireTimeout) { + configuration.getReadTimeout() + .map(dur -> new ConditionalReadTimeoutHandler(dur.toNanos(), TimeUnit.NANOSECONDS, fireTimeout)) + .ifPresent(ch -> channel.pipeline().addBefore(before, ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT, ch)); + configuration.getConnectTtl().ifPresent(ttl -> + ttlFuture = channel.eventLoop().schedule(this::windDownConnection, ttl.toNanos(), TimeUnit.NANOSECONDS)); + channel.pipeline().addBefore(before, "connection-cleaner", new ChannelInboundHandlerAdapter() { + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + super.channelInactive(ctx); + onInactive(); + } + }); + } + + void windDownConnection() { + windDownConnection = true; + } + + boolean satisfy(Sinks.One sink) { + if (!tryEarmarkForRequest()) { + return false; + } + + if (channel.eventLoop().inEventLoop()) { + satisfy0(sink); + } else { + channel.eventLoop().execute(() -> satisfy0(sink)); + } + return true; + } + + abstract void satisfy0(Sinks.One sink); + + abstract boolean tryEarmarkForRequest(); + + void onInactive() { + ttlFuture.cancel(false); + windDownConnection = true; + } + } + + final class Http1ConnectionHolder extends ConnectionHolder { + private final AtomicBoolean hasLiveRequest = new AtomicBoolean(false); + + Http1ConnectionHolder(Channel channel, NettyClientCustomizer connectionCustomizer) { + super(channel, connectionCustomizer); + } + void init(boolean fireInitialPipelineBuilt) { - addReadTimeoutHandler( - channel.pipeline(), + addTimeoutHandlers( requestKey.isSecure() ? ChannelPipelineCustomizer.HANDLER_SSL : ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, @@ -1343,14 +1341,6 @@ void init(boolean fireInitialPipelineBuilt) { } } ); - channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - super.channelInactive(ctx); - http1Connections.remove(Http1ConnectionHolder.this); - canReturn = false; - } - }); if (fireInitialPipelineBuilt) { connectionCustomizer.onInitialPipelineBuilt(); @@ -1366,20 +1356,13 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { pendingConnectionCount.decrementAndGet(); } - boolean satisfy(Sinks.One sink) { - if (!hasLiveRequest.compareAndSet(false, true)) { - return false; - } - - if (channel.eventLoop().inEventLoop()) { - satisfy0(sink); - } else { - channel.eventLoop().execute(() -> satisfy0(sink)); - } - return true; + @Override + boolean tryEarmarkForRequest() { + return !windDownConnection && hasLiveRequest.compareAndSet(false, true); } - private void satisfy0(Sinks.One sink) { + @Override + void satisfy0(Sinks.One sink) { if (!channel.isActive()) { returnPendingRequest(sink); return; @@ -1389,19 +1372,19 @@ private void satisfy0(Sinks.One sink) { @Override void taint() { - canReturn = false; + windDownConnection = true; } @Override void release() { - if (canReturn) { + if (!windDownConnection) { ChannelHandlerContext newLast = channel.pipeline().lastContext(); if (lastContext != newLast) { log.warn("BUG - Handler not removed: {}", newLast); taint(); } } - if (canReturn) { + if (!windDownConnection) { hasLiveRequest.set(false); // todo: claim a new request } else { @@ -1411,7 +1394,7 @@ void release() { @Override boolean canReturn() { - return canReturn; + return !windDownConnection; } @Override @@ -1430,22 +1413,32 @@ private void returnPendingRequest(Sinks.One sink) { pendingRequests.add(sink); hasLiveRequest.set(false); } + + @Override + void windDownConnection() { + super.windDownConnection(); + if (!hasLiveRequest.get()) { + channel.close(); + } + } + + @Override + void onInactive() { + super.onInactive(); + http1Connections.remove(this); + } } - final class Http2ConnectionHolder { - private final Channel channel; + final class Http2ConnectionHolder extends ConnectionHolder { private final AtomicInteger liveRequests = new AtomicInteger(0); - private final NettyClientCustomizer customizer; private final Set liveStreamChannels = new HashSet<>(); // todo: https://github.com/netty/netty/pull/12830 Http2ConnectionHolder(Channel channel, NettyClientCustomizer customizer) { - this.channel = channel; - this.customizer = customizer; + super(channel, customizer); } void init() { - addReadTimeoutHandler( - channel.pipeline(), + addTimeoutHandlers( requestKey.isSecure() ? ChannelPipelineCustomizer.HANDLER_SSL : ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, @@ -1458,15 +1451,8 @@ void init() { } } ); - channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - super.channelInactive(ctx); - http2Connections.remove(Http2ConnectionHolder.this); - } - }); - customizer.onStreamPipelineBuilt(); + connectionCustomizer.onStreamPipelineBuilt(); for (int i = 0; i < MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION; i++) { Sinks.One pendingRequest = pendingRequests.poll(); @@ -1480,20 +1466,13 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { pendingConnectionCount.decrementAndGet(); } - boolean satisfy(Sinks.One sink) { - if (!incrementWithLimit(liveRequests, MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION)) { - return false; - } - - if (channel.eventLoop().inEventLoop()) { - satisfy0(sink); - } else { - channel.eventLoop().execute(() -> satisfy0(sink)); - } - return true; + @Override + boolean tryEarmarkForRequest() { + return !windDownConnection && incrementWithLimit(liveRequests, MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION); } - private void satisfy0(Sinks.One sink) { + @Override + void satisfy0(Sinks.One sink) { if (!channel.isActive()) { returnPendingRequest(sink); return; @@ -1504,7 +1483,7 @@ private void satisfy0(Sinks.One sink) { streamChannel.pipeline() .addLast(new Http2StreamFrameToHttpObjectCodec(false)) .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECOMPRESSOR, new HttpContentDecompressor()); - NettyClientCustomizer streamCustomizer = customizer.specializeForChannel(streamChannel, NettyClientCustomizer.ChannelRole.HTTP2_STREAM); + NettyClientCustomizer streamCustomizer = connectionCustomizer.specializeForChannel(streamChannel, NettyClientCustomizer.ChannelRole.HTTP2_STREAM); PoolHandle ph = new PoolHandle(streamChannel) { @Override void taint() { @@ -1546,6 +1525,20 @@ private void returnPendingRequest(Sinks.One sink) { pendingRequests.add(sink); liveRequests.decrementAndGet(); } + + @Override + void windDownConnection() { + super.windDownConnection(); + if (liveRequests.get() == 0) { + channel.close(); + } + } + + @Override + void onInactive() { + super.onInactive(); + http2Connections.remove(Http2ConnectionHolder.this); + } } } } diff --git a/http-client/src/test/groovy/io/micronaut/http/client/ConnectTTLHandlerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/ConnectTTLHandlerSpec.groovy deleted file mode 100644 index e684cc596cc..00000000000 --- a/http-client/src/test/groovy/io/micronaut/http/client/ConnectTTLHandlerSpec.groovy +++ /dev/null @@ -1,41 +0,0 @@ -package io.micronaut.http.client - -import io.micronaut.http.client.netty.ConnectTTLHandler -import io.netty.channel.ChannelHandlerContext -import io.netty.channel.embedded.EmbeddedChannel -import spock.lang.Specification - - -class ConnectTTLHandlerSpec extends Specification{ - - - def "RELEASE_CHANNEL should be true for those channels who's connect-ttl is reached"(){ - - setup: - MockChannel channel = new MockChannel(); - ChannelHandlerContext context = Mock() - - when: - new ConnectTTLHandler(1).handlerAdded(context) - channel.runAllPendingTasks() - - then: - _ * context.channel() >> channel - - channel.attr(ConnectTTLHandler.RELEASE_CHANNEL) - - } - - class MockChannel extends EmbeddedChannel { - MockChannel() throws Exception { - super.doRegister() - } - - void runAllPendingTasks() throws InterruptedException { - super.runPendingTasks() - while (runScheduledPendingTasks() != -1) { - Thread.sleep(1) - } - } - } -} diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 482df1c0edb..47e9e05b59c 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -60,6 +60,7 @@ import reactor.core.publisher.Mono import spock.lang.Specification import java.nio.charset.StandardCharsets +import java.util.concurrent.CompletableFuture import java.util.concurrent.ExecutionException import java.util.concurrent.Future import java.util.concurrent.TimeUnit @@ -520,6 +521,58 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'http1 ttl'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.connect-ttl': '100s', + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn1 = new EmbeddedTestConnectionHttp1() + conn1.setupHttp1() + def conn2 = new EmbeddedTestConnectionHttp1() + conn2.setupHttp1() + patch(client, conn1.clientChannel, conn2.clientChannel) + + def r1 = conn1.testExchangeRequest(client) + conn1.clientChannel.advanceTimeBy(101, TimeUnit.SECONDS) + conn1.testExchangeResponse(r1) + + // conn1 should expire now, conn2 will be the next connection + conn2.testExchange(client) + + cleanup: + client.close() + ctx.close() + } + + def 'http2 ttl'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + 'micronaut.http.client.connect-ttl': '100s', + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn1 = new EmbeddedTestConnectionHttp2() + conn1.setupHttp2Tls() + def conn2 = new EmbeddedTestConnectionHttp2() + conn2.setupHttp2Tls() + patch(client, conn1.clientChannel, conn2.clientChannel) + + def r1 = conn1.testExchangeRequest(client) + conn1.exchangeSettings() + conn1.clientChannel.advanceTimeBy(101, TimeUnit.SECONDS) + conn1.testExchangeResponse(r1) + + // conn1 should expire now, conn2 will be the next connection + def r2 = conn2.testExchangeRequest(client) + conn2.exchangeSettings() + conn2.testExchangeResponse(r2) + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnectionBase { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel @@ -573,10 +626,17 @@ class ConnectionManagerSpec extends Specification { } void testExchange(HttpClient client) { + testExchangeResponse(testExchangeRequest(client)) + } + + CompletableFuture> testExchangeRequest(HttpClient client) { def future = Mono.from(client.exchange(scheme + '://example.com/foo')).toFuture() future.exceptionally(t -> t.printStackTrace()) advance() + return future + } + void testExchangeResponse(CompletableFuture> future) { io.netty.handler.codec.http.HttpRequest request = serverChannel.readInbound() assert request.uri() == '/foo' assert request.method() == HttpMethod.GET From f780749b4ffae27fa250515bcf03f8dc5b69936f Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 23 Sep 2022 14:19:29 +0200 Subject: [PATCH 18/82] implement connection idle timeout --- .../http/client/netty/ConnectionManager.java | 103 +++++++++--------- .../http/client/netty/IdleTimeoutHandler.java | 50 --------- .../client/netty/IdlingConnectionHandler.java | 51 --------- .../client/netty/ConnectionManagerSpec.groovy | 25 +++++ 4 files changed, 77 insertions(+), 152 deletions(-) delete mode 100644 http-client/src/main/java/io/micronaut/http/client/netty/IdleTimeoutHandler.java delete mode 100644 http-client/src/main/java/io/micronaut/http/client/netty/IdlingConnectionHandler.java diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 87977e6d6ec..5e47006b710 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -540,14 +540,6 @@ void onStreamPipelineBuilt() { public void channelReleased(Channel ch) { Duration idleTimeout = configuration.getConnectionPoolIdleTimeout().orElse(Duration.ofNanos(0)); ChannelPipeline pipeline = ch.pipeline(); - if (ch.isOpen()) { - ch.config().setAutoRead(true); - pipeline.addLast(IdlingConnectionHandler.INSTANCE); - if (idleTimeout.toNanos() > 0) { - pipeline.addLast(ChannelPipelineCustomizer.HANDLER_IDLE_STATE, new IdleStateHandler(idleTimeout.toNanos(), idleTimeout.toNanos(), 0, TimeUnit.NANOSECONDS)); - pipeline.addLast(IdleTimeoutHandler.INSTANCE); - } - } removeReadTimeoutHandler(pipeline); } @@ -555,15 +547,6 @@ public void channelReleased(Channel ch) { @Override public void channelAcquired(Channel ch) throws Exception { ChannelPipeline pipeline = ch.pipeline(); - if (pipeline.context(IdlingConnectionHandler.INSTANCE) != null) { - pipeline.remove(IdlingConnectionHandler.INSTANCE); - } - if (pipeline.context(ChannelPipelineCustomizer.HANDLER_IDLE_STATE) != null) { - pipeline.remove(ChannelPipelineCustomizer.HANDLER_IDLE_STATE); - } - if (pipeline.context(IdleTimeoutHandler.INSTANCE) != null) { - pipeline.remove(IdleTimeoutHandler.INSTANCE); - } } }; } @@ -1160,20 +1143,6 @@ private static boolean incrementWithLimit(AtomicInteger variable, int limit) { } } - private static class ConditionalReadTimeoutHandler extends ReadTimeoutHandler { - private final Consumer fireTimeout; - - ConditionalReadTimeoutHandler(long timeout, TimeUnit unit, Consumer fireTimeout) { - super(timeout, unit); - this.fireTimeout = fireTimeout; - } - - @Override - protected void readTimedOut(ChannelHandlerContext ctx) throws Exception { - fireTimeout.accept(ctx); - } - } - private final class Pool { private static final int MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION = 4; // TODO: config private static final int MAX_PENDING_CONNECTIONS = 1; @@ -1280,10 +1249,28 @@ abstract class ConnectionHolder { this.connectionCustomizer = connectionCustomizer; } - void addTimeoutHandlers(String before, Consumer fireTimeout) { + final void addTimeoutHandlers(String before) { + // read timeout handles timeouts *during* a request configuration.getReadTimeout() - .map(dur -> new ConditionalReadTimeoutHandler(dur.toNanos(), TimeUnit.NANOSECONDS, fireTimeout)) - .ifPresent(ch -> channel.pipeline().addBefore(before, ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT, ch)); + .ifPresent(dur -> channel.pipeline().addBefore(before, ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT, new ReadTimeoutHandler(dur.toNanos(), TimeUnit.NANOSECONDS) { + @Override + protected void readTimedOut(ChannelHandlerContext ctx) throws Exception { + if (hasLiveRequests()) { + fireReadTimeout(ctx); + ctx.close(); + } + } + })); + // pool idle timeout happens *outside* a request + configuration.getConnectionPoolIdleTimeout() + .ifPresent(dur -> channel.pipeline().addBefore(before, ChannelPipelineCustomizer.HANDLER_IDLE_STATE, new ReadTimeoutHandler(dur.toNanos(), TimeUnit.NANOSECONDS) { + @Override + protected void readTimedOut(ChannelHandlerContext ctx) throws Exception { + if (!hasLiveRequests()) { + ctx.close(); + } + } + })); configuration.getConnectTtl().ifPresent(ttl -> ttlFuture = channel.eventLoop().schedule(this::windDownConnection, ttl.toNanos(), TimeUnit.NANOSECONDS)); channel.pipeline().addBefore(before, "connection-cleaner", new ChannelInboundHandlerAdapter() { @@ -1316,8 +1303,14 @@ boolean satisfy(Sinks.One sink) { abstract boolean tryEarmarkForRequest(); + abstract boolean hasLiveRequests(); + + abstract void fireReadTimeout(ChannelHandlerContext ctx); + void onInactive() { - ttlFuture.cancel(false); + if (ttlFuture != null) { + ttlFuture.cancel(false); + } windDownConnection = true; } } @@ -1333,13 +1326,7 @@ void init(boolean fireInitialPipelineBuilt) { addTimeoutHandlers( requestKey.isSecure() ? ChannelPipelineCustomizer.HANDLER_SSL : - ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, - ctx -> { - if (hasLiveRequest.get()) { - ctx.fireExceptionCaught(ReadTimeoutException.INSTANCE); - ctx.close(); - } - } + ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC ); if (fireInitialPipelineBuilt) { @@ -1361,6 +1348,16 @@ boolean tryEarmarkForRequest() { return !windDownConnection && hasLiveRequest.compareAndSet(false, true); } + @Override + boolean hasLiveRequests() { + return hasLiveRequest.get(); + } + + @Override + void fireReadTimeout(ChannelHandlerContext ctx) { + ctx.fireExceptionCaught(ReadTimeoutException.INSTANCE); + } + @Override void satisfy0(Sinks.One sink) { if (!channel.isActive()) { @@ -1441,15 +1438,7 @@ void init() { addTimeoutHandlers( requestKey.isSecure() ? ChannelPipelineCustomizer.HANDLER_SSL : - ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, - ctx -> { - if (liveRequests.get() != 0) { - for (Channel sc : liveStreamChannels) { - sc.pipeline().fireExceptionCaught(ReadTimeoutException.INSTANCE); - } - ctx.close(); - } - } + ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC ); connectionCustomizer.onStreamPipelineBuilt(); @@ -1471,6 +1460,18 @@ boolean tryEarmarkForRequest() { return !windDownConnection && incrementWithLimit(liveRequests, MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION); } + @Override + boolean hasLiveRequests() { + return liveRequests.get() > 0; + } + + @Override + void fireReadTimeout(ChannelHandlerContext ctx) { + for (Channel sc : liveStreamChannels) { + sc.pipeline().fireExceptionCaught(ReadTimeoutException.INSTANCE); + } + } + @Override void satisfy0(Sinks.One sink) { if (!channel.isActive()) { diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/IdleTimeoutHandler.java b/http-client/src/main/java/io/micronaut/http/client/netty/IdleTimeoutHandler.java deleted file mode 100644 index c2242e22535..00000000000 --- a/http-client/src/main/java/io/micronaut/http/client/netty/IdleTimeoutHandler.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright 2017-2020 original authors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.micronaut.http.client.netty; - -import io.micronaut.core.annotation.Internal; -import io.netty.channel.ChannelDuplexHandler; -import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundHandler; -import io.netty.handler.timeout.IdleState; -import io.netty.handler.timeout.IdleStateEvent; - -/** - * This class is responsible for detecting idle timeout events, upon which the channel in the pool is closed. - * - * @author Dan Maas - * @since 2.2.4 - */ -@ChannelHandler.Sharable -@Internal -final class IdleTimeoutHandler extends ChannelDuplexHandler { - - static final ChannelInboundHandler INSTANCE = new IdleTimeoutHandler(); - - private IdleTimeoutHandler() { - } - - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - if (evt instanceof IdleStateEvent) { - IdleStateEvent e = (IdleStateEvent) evt; - if (e.state() == IdleState.READER_IDLE || e.state() == IdleState.WRITER_IDLE) { - ctx.close(); - } - } - } -} diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/IdlingConnectionHandler.java b/http-client/src/main/java/io/micronaut/http/client/netty/IdlingConnectionHandler.java deleted file mode 100644 index d4e931566a2..00000000000 --- a/http-client/src/main/java/io/micronaut/http/client/netty/IdlingConnectionHandler.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Copyright 2017-2020 original authors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.micronaut.http.client.netty; - -import io.micronaut.core.annotation.Internal; -import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundHandler; -import io.netty.channel.ChannelInboundHandlerAdapter; -import io.netty.util.ReferenceCountUtil; - -/** - * This handler prevents reading a channel when it is not being used by the connection pool. - * - * @author Dan Maas - * @since 2.2.4 - */ -@ChannelHandler.Sharable -@Internal -final class IdlingConnectionHandler extends ChannelInboundHandlerAdapter { - - static final ChannelInboundHandler INSTANCE = new IdlingConnectionHandler(); - - private IdlingConnectionHandler() { - } - - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - ReferenceCountUtil.release(msg); - ctx.close(); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - ctx.close(); - } - -} diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 47e9e05b59c..9dedc1c8249 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -573,6 +573,31 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'http1 pool timeout'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.connection-pool-idle-timeout': '5s', + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn1 = new EmbeddedTestConnectionHttp1() + conn1.setupHttp1() + def conn2 = new EmbeddedTestConnectionHttp1() + conn2.setupHttp1() + patch(client, conn1.clientChannel, conn2.clientChannel) + + conn1.testExchange(client) + conn1.clientChannel.unfreezeTime() + // todo: move to advanceTime once IdleStateHandler supports it + TimeUnit.SECONDS.sleep(5) + conn1.advance() + // conn1 should expire now, conn2 will be the next connection + conn2.testExchange(client) + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnectionBase { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel From f7509c1b9eb0b3d24d975027dc7eb07f33ca3db4 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 23 Sep 2022 14:29:59 +0200 Subject: [PATCH 19/82] fix h2c --- .../java/io/micronaut/http/client/netty/ConnectionManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 5e47006b710..4f738ee3832 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -1438,7 +1438,7 @@ void init() { addTimeoutHandlers( requestKey.isSecure() ? ChannelPipelineCustomizer.HANDLER_SSL : - ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC + ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION ); connectionCustomizer.onStreamPipelineBuilt(); From 163a27a2f8fdef36c1ede6de420677869a38e5dc Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 23 Sep 2022 14:33:40 +0200 Subject: [PATCH 20/82] add log handler for http1 --- .../http/client/netty/ConnectionManager.java | 170 ++---------------- 1 file changed, 10 insertions(+), 160 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 4f738ee3832..fb726b763b7 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -44,11 +44,8 @@ import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelOption; import io.netty.channel.ChannelPipeline; -import io.netty.channel.ChannelPromise; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.pool.AbstractChannelPoolHandler; -import io.netty.channel.pool.ChannelPool; import io.netty.handler.codec.LineBasedFrameDecoder; import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.DefaultHttpContent; @@ -72,7 +69,6 @@ import io.netty.handler.codec.http2.Http2FrameCodecBuilder; import io.netty.handler.codec.http2.Http2FrameLogger; import io.netty.handler.codec.http2.Http2MultiplexHandler; -import io.netty.handler.codec.http2.Http2Settings; import io.netty.handler.codec.http2.Http2SettingsFrame; import io.netty.handler.codec.http2.Http2Stream; import io.netty.handler.codec.http2.Http2StreamChannel; @@ -95,7 +91,6 @@ import io.netty.util.ReferenceCountUtil; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; -import io.netty.util.concurrent.Promise; import io.netty.util.concurrent.ScheduledFuture; import org.slf4j.Logger; import reactor.core.publisher.Mono; @@ -132,13 +127,6 @@ class ConnectionManager { private static final AttributeKey CHANNEL_CUSTOMIZER_KEY = AttributeKey.valueOf("micronaut.http.customizer"); - /** - * Future on a pooled channel that will be completed when the channel has fully connected (e.g. - * TLS handshake has completed). If unset, then no handshake is needed or it has already - * completed. - */ - private static final AttributeKey> STREAM_CHANNEL_INITIALIZED = - AttributeKey.valueOf("micronaut.http.streamChannelInitialized"); private static final AttributeKey STREAM_KEY = AttributeKey.valueOf("micronaut.http2.stream"); final InvocationInstrumenter instrumenter; @@ -488,69 +476,6 @@ protected void addFinalHandler(ChannelPipeline pipeline) { return initial.asMono(); } - private AbstractChannelPoolHandler newPoolHandler(DefaultHttpClient.RequestKey key) { - return new AbstractChannelPoolHandler() { - @Override - public void channelCreated(Channel ch) { - Promise streamPipelineBuilt = ch.newPromise(); - ch.attr(STREAM_CHANNEL_INITIALIZED).set(streamPipelineBuilt); - - // make sure the future completes eventually - ChannelHandler failureHandler = new ChannelInboundHandlerAdapter() { - @Override - public void handlerRemoved(ChannelHandlerContext ctx) { - streamPipelineBuilt.trySuccess(null); - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) { - streamPipelineBuilt.trySuccess(null); - ctx.fireChannelInactive(); - } - }; - ch.pipeline().addLast(failureHandler); - - ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_INIT, new HttpClientInitializer( - key.isSecure() ? sslContext : null, - key.getHost(), - key.getPort(), - false, - false, - false, - null - ) { - @Override - protected void addFinalHandler(ChannelPipeline pipeline) { - // no-op, don't add the stream handler which is not supported - // in the connection pooled scenario - } - - @Override - void onStreamPipelineBuilt() { - super.onStreamPipelineBuilt(); - streamPipelineBuilt.trySuccess(null); - ch.pipeline().remove(failureHandler); - ch.attr(STREAM_CHANNEL_INITIALIZED).set(null); - } - }); - - } - - @Override - public void channelReleased(Channel ch) { - Duration idleTimeout = configuration.getConnectionPoolIdleTimeout().orElse(Duration.ofNanos(0)); - ChannelPipeline pipeline = ch.pipeline(); - - removeReadTimeoutHandler(pipeline); - } - - @Override - public void channelAcquired(Channel ch) throws Exception { - ChannelPipeline pipeline = ch.pipeline(); - } - }; - } - private void configureProxy(ChannelPipeline pipeline, boolean secure, String host, int port) { Proxy proxy = configuration.resolveProxy(secure, host, port); if (Proxy.NO_PROXY.equals(proxy)) { @@ -622,54 +547,6 @@ private Http2FrameCodec makeFrameCodec() { return builder.build(); } - private void removeReadTimeoutHandler(ChannelPipeline pipeline) { - if (pipeline.context(ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT) != null) { - pipeline.remove(ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT); - } - } - - /** - * Reads the first {@link Http2Settings} object and notifies a {@link io.netty.channel.ChannelPromise}. - */ - private class Http2SettingsHandler extends - SimpleChannelInboundHandlerInstrumented { - final ChannelPromise promise; - - /** - * Create new instance. - * - * @param promise Promise object used to notify when first settings are received - */ - Http2SettingsHandler(ChannelPromise promise) { - super(instrumenter); - this.promise = promise; - } - - @Override - protected void channelReadInstrumented(ChannelHandlerContext ctx, Http2Settings msg) { - promise.setSuccess(); - - // Only care about the first settings message - ctx.pipeline().remove(this); - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - super.channelInactive(ctx); - if (!promise.isDone()) { - promise.tryFailure(new HttpClientException("Channel became inactive before settings frame was received")); - } - } - - @Override - public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { - super.handlerRemoved(ctx); - if (!promise.isDone()) { - promise.tryFailure(new HttpClientException("Handler was removed before settings frame was received")); - } - } - } - /** * Initializes the HTTP client channel. */ @@ -681,7 +558,6 @@ private class HttpClientInitializer extends ChannelInitializer { final boolean stream; final boolean proxy; final boolean acceptsEvents; - Http2SettingsHandler settingsHandler; final Consumer contextConsumer; private NettyClientCustomizer channelCustomizer; @@ -965,6 +841,16 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } private void initHttp1(Channel ch) { + configuration.getLogLevel().ifPresent(logLevel -> { + try { + final io.netty.handler.logging.LogLevel nettyLevel = + io.netty.handler.logging.LogLevel.valueOf(logLevel.name()); + ch.pipeline().addLast(new LoggingHandler(DefaultHttpClient.class, nettyLevel)); + } catch (IllegalArgumentException e) { + throw customizeException(new HttpClientException("Unsupported log level: " + logLevel)); + } + }); + ch.pipeline() .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()) .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); @@ -1095,42 +981,6 @@ void notifyRequestPipelineBuilt() { } } - private final class OldPoolHandle extends PoolHandle { - private final ChannelPool channelPool; - private boolean canReturn; - - private OldPoolHandle(ChannelPool channelPool, Channel channel) { - super(channel); - this.channelPool = channelPool; - this.canReturn = channelPool != null; - } - - void taint() { - canReturn = false; - } - - void release() { - if (channelPool != null) { - removeReadTimeoutHandler(channel.pipeline()); - if (!canReturn) { - channel.closeFuture().addListener((future -> - channelPool.release(channel) - )); - } else { - channelPool.release(channel); - } - } else { - // just close it to prevent any future reads without a handler registered - channel.close(); - } - } - - boolean canReturn() { - return canReturn; - } - - } - private static boolean incrementWithLimit(AtomicInteger variable, int limit) { while (true) { int old = variable.get(); From e38e484ff9cea7b0a1d425bbda615593853df7a0 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 23 Sep 2022 14:34:26 +0200 Subject: [PATCH 21/82] ssl handshake timeout --- .../http/client/netty/ConnectionManager.java | 21 +++---------------- 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index fb726b763b7..f86cd41a156 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -611,23 +611,6 @@ protected void initChannel(Channel ch) { ch.config().setAutoRead(false); } - configuration.getLogLevel().ifPresent(logLevel -> { - try { - final io.netty.handler.logging.LogLevel nettyLevel = io.netty.handler.logging.LogLevel.valueOf( - logLevel.name() - ); - p.addLast(new LoggingHandler(DefaultHttpClient.class, nettyLevel)); - } catch (IllegalArgumentException e) { - throw customizeException(new HttpClientException("Unsupported log level: " + logLevel)); - } - }); - - if (sslContext != null) { - SslHandler sslHandler = sslContext.newHandler(ch.alloc(), host, port); - sslHandler.setHandshakeTimeoutMillis(configuration.getSslConfiguration().getHandshakeTimeout().toMillis()); - p.addLast(ChannelPipelineCustomizer.HANDLER_SSL, sslHandler); - } - // Pool connections require alternative timeout handling if (true/*poolMap == null*/) { // read timeout settings are not applied to streamed requests. @@ -794,8 +777,10 @@ protected void initChannel(Channel ch) { configureProxy(ch.pipeline(), true, host, port); InitialConnectionErrorHandler initialErrorHandler = new InitialConnectionErrorHandler(pool); + SslHandler sslHandler = sslContext.newHandler(ch.alloc(), host, port); + sslHandler.setHandshakeTimeoutMillis(configuration.getSslConfiguration().getHandshakeTimeout().toMillis()); ch.pipeline() - .addLast(ChannelPipelineCustomizer.HANDLER_SSL, sslContext.newHandler(ch.alloc(), host, port)) + .addLast(ChannelPipelineCustomizer.HANDLER_SSL, sslHandler) .addLast( ChannelPipelineCustomizer.HANDLER_HTTP2_PROTOCOL_NEGOTIATOR, new ApplicationProtocolNegotiationHandler(ApplicationProtocolNames.HTTP_1_1) { From aafe969fa248d2e66c3b83dcd61915d984fbe0d1 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 23 Sep 2022 15:11:04 +0200 Subject: [PATCH 22/82] sse support --- .../http/client/netty/ConnectionManager.java | 153 +++++++----------- 1 file changed, 61 insertions(+), 92 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index f86cd41a156..671269f41c2 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -374,6 +374,7 @@ Mono connectForStream(DefaultHttpClient.RequestKey requestKey, boole return pool.acquire() .map(ph -> { // TODO: this sucks + boolean sse = !isProxy && acceptEvents; ph.channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { boolean ignoreOneLast = false; @@ -390,6 +391,10 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception if (ignoreOneLast) { ignoreOneLast = false; } else { + if (sse) { + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_EVENT_STREAM); + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_CONTENT); + } ctx.pipeline() .remove(this) .remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); @@ -398,6 +403,53 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } } }); + if (sse) { + ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_EVENT_STREAM, new LineBasedFrameDecoder(configuration.getMaxContentLength(), true, true) { + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof HttpContent) { + if (msg instanceof LastHttpContent) { + super.channelRead(ctx, msg); + } else { + Attribute streamKey = ctx.channel().attr(STREAM_KEY); + if (msg instanceof Http2Content) { + streamKey.set(((Http2Content) msg).stream()); + } + try { + super.channelRead(ctx, ((HttpContent) msg).content()); + } finally { + streamKey.set(null); + } + } + } else { + super.channelRead(ctx, msg); + } + } + }); + ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_CONTENT, new SimpleChannelInboundHandlerInstrumented(instrumenter, false) { + + @Override + public boolean acceptInboundMessage(Object msg) { + return msg instanceof ByteBuf; + } + + @Override + protected void channelReadInstrumented(ChannelHandlerContext ctx, ByteBuf msg) { + try { + Attribute streamKey = ctx.channel().attr(STREAM_KEY); + Http2Stream http2Stream = streamKey.get(); + if (http2Stream != null) { + ctx.fireChannelRead(new DefaultHttp2Content(msg.copy(), http2Stream)); + } else { + ctx.fireChannelRead(new DefaultHttpContent(msg.copy())); + } + } finally { + msg.release(); + } + } + }); + } ph.channel.pipeline().addLast( ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, new HttpStreamsClientHandler() { @@ -611,27 +663,6 @@ protected void initChannel(Channel ch) { ch.config().setAutoRead(false); } - // Pool connections require alternative timeout handling - if (true/*poolMap == null*/) { - // read timeout settings are not applied to streamed requests. - // instead idle timeout settings are applied. - if (stream) { - Optional readIdleTime = configuration.getReadIdleTimeout(); - if (readIdleTime.isPresent()) { - Duration duration = readIdleTime.get(); - if (!duration.isNegative()) { - p.addLast(ChannelPipelineCustomizer.HANDLER_IDLE_STATE, new IdleStateHandler( - duration.toMillis(), - duration.toMillis(), - duration.toMillis(), - TimeUnit.MILLISECONDS - )); - } - } - } - } - - addHttp1Handlers(p); channelCustomizer.onInitialPipelineBuilt(); onStreamPipelineBuilt(); } @@ -645,80 +676,11 @@ void onStreamPipelineBuilt() { channelCustomizer.onStreamPipelineBuilt(); } - void addHttp1Handlers(ChannelPipeline p) { - p.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()); - - p.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECOMPRESSOR, new HttpContentDecompressor()); - - if (!stream) { - p.addLast(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR, new HttpObjectAggregator(configuration.getMaxContentLength()) { - @Override - protected void finishAggregation(FullHttpMessage aggregated) throws Exception { - if (!HttpUtil.isContentLengthSet(aggregated)) { - if (aggregated.content().readableBytes() > 0) { - super.finishAggregation(aggregated); - } - } - } - }); - } - addEventStreamHandlerIfNecessary(p); - addFinalHandler(p); - for (ChannelPipelineListener pipelineListener : pipelineListeners) { - pipelineListener.onConnect(p); - } - } - void addEventStreamHandlerIfNecessary(ChannelPipeline p) { // if the content type is a SSE event stream we add a decoder // to delimit the content by lines (unless we are proxying the stream) if (acceptsEventStream() && !proxy) { - p.addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_EVENT_STREAM, new LineBasedFrameDecoder(configuration.getMaxContentLength(), true, true) { - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof HttpContent) { - if (msg instanceof LastHttpContent) { - super.channelRead(ctx, msg); - } else { - Attribute streamKey = ctx.channel().attr(STREAM_KEY); - if (msg instanceof Http2Content) { - streamKey.set(((Http2Content) msg).stream()); - } - try { - super.channelRead(ctx, ((HttpContent) msg).content()); - } finally { - streamKey.set(null); - } - } - } else { - super.channelRead(ctx, msg); - } - } - }); - - p.addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_CONTENT, new SimpleChannelInboundHandlerInstrumented(instrumenter, false) { - - @Override - public boolean acceptInboundMessage(Object msg) { - return msg instanceof ByteBuf; - } - - @Override - protected void channelReadInstrumented(ChannelHandlerContext ctx, ByteBuf msg) { - try { - Attribute streamKey = ctx.channel().attr(STREAM_KEY); - Http2Stream http2Stream = streamKey.get(); - if (http2Stream != null) { - ctx.fireChannelRead(new DefaultHttp2Content(msg.copy(), http2Stream)); - } else { - ctx.fireChannelRead(new DefaultHttpContent(msg.copy())); - } - } finally { - msg.release(); - } - } - }); } } @@ -1035,10 +997,17 @@ private void openNewConnection() { initializer = new ChannelInitializer() { @Override protected void initChannel(Channel ch) throws Exception { - NettyClientCustomizer channelCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); configureProxy(ch.pipeline(), false, requestKey.getHost(), requestKey.getPort()); initHttp1(ch); - new Http1ConnectionHolder(ch, channelCustomizer).init(true); + ch.pipeline().addLast("activity-listener", new ChannelInboundHandlerAdapter() { + @Override + public void channelActive(ChannelHandlerContext ctx) throws Exception { + super.channelActive(ctx); + ctx.pipeline().remove(this); + NettyClientCustomizer channelCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); + new Http1ConnectionHolder(ch, channelCustomizer).init(true); + } + }); } }; break; From ca40d1af990c602d371adb6bf22e3a0f042aae60 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 23 Sep 2022 16:09:48 +0200 Subject: [PATCH 23/82] websocket support --- .../http/client/netty/ConnectionManager.java | 173 +++--------------- .../client/netty/ConnectionManagerSpec.groovy | 59 ++++++ .../http/client/netty/EmbeddedTestUtil.groovy | 2 +- 3 files changed, 90 insertions(+), 144 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 671269f41c2..1588b98ee6b 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -62,9 +62,7 @@ import io.netty.handler.codec.http.HttpUtil; import io.netty.handler.codec.http.LastHttpContent; import io.netty.handler.codec.http.websocketx.extensions.compression.WebSocketClientCompressionHandler; -import io.netty.handler.codec.http2.DefaultHttp2Connection; import io.netty.handler.codec.http2.Http2ClientUpgradeCodec; -import io.netty.handler.codec.http2.Http2Connection; import io.netty.handler.codec.http2.Http2FrameCodec; import io.netty.handler.codec.http2.Http2FrameCodecBuilder; import io.netty.handler.codec.http2.Http2FrameLogger; @@ -117,7 +115,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Consumer; /** * Connection manager for {@link DefaultHttpClient}. This class manages the lifecycle of netty @@ -479,47 +476,44 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc Mono connectForWebsocket(DefaultHttpClient.RequestKey requestKey, ChannelHandler handler) { Sinks.Empty initial = Sinks.empty(); - Bootstrap bootstrap = this.bootstrap.clone(); - SslContext sslContext = buildSslContext(requestKey); - - bootstrap.remoteAddress(requestKey.getHost(), requestKey.getPort()); - initBootstrapForProxy(bootstrap, sslContext != null, requestKey.getHost(), requestKey.getPort()); - bootstrap.handler(new HttpClientInitializer( - sslContext, - requestKey.getHost(), - requestKey.getPort(), - false, - false, - false, - null - ) { + ChannelFuture connectFuture = doConnect(requestKey, new ChannelInitializer() { @Override - protected void addFinalHandler(ChannelPipeline pipeline) { - pipeline.remove(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER); - ReadTimeoutHandler readTimeoutHandler = pipeline.get(ReadTimeoutHandler.class); - if (readTimeoutHandler != null) { - pipeline.remove(readTimeoutHandler); + protected void initChannel(Channel ch) throws Exception { + addLogHandler(ch); + + if (requestKey.isSecure()) { + SslHandler sslHandler = sslContext.newHandler(ch.alloc(), requestKey.getHost(), requestKey.getPort()); + sslHandler.setHandshakeTimeoutMillis(configuration.getSslConfiguration().getHandshakeTimeout().toMillis()); + ch.pipeline().addLast(sslHandler); } + ch.pipeline() + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()) + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR, new HttpObjectAggregator(configuration.getMaxContentLength())); + Optional readIdleTime = configuration.getReadIdleTimeout(); if (readIdleTime.isPresent()) { Duration duration = readIdleTime.get(); if (!duration.isNegative()) { - pipeline.addLast(ChannelPipelineCustomizer.HANDLER_IDLE_STATE, new IdleStateHandler(duration.toMillis(), duration.toMillis(), duration.toMillis(), TimeUnit.MILLISECONDS)); + ch.pipeline() + .addLast(ChannelPipelineCustomizer.HANDLER_IDLE_STATE, new IdleStateHandler(duration.toMillis(), duration.toMillis(), duration.toMillis(), TimeUnit.MILLISECONDS)); } } try { - pipeline.addLast(WebSocketClientCompressionHandler.INSTANCE); - pipeline.addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_WEBSOCKET_CLIENT, handler); - initial.tryEmitEmpty(); + ch.pipeline().addLast(WebSocketClientCompressionHandler.INSTANCE); + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_WEBSOCKET_CLIENT, handler); + if (initial.tryEmitEmpty().isSuccess()) { + return; + } } catch (Throwable e) { initial.tryEmitError(new WebSocketSessionException("Error opening WebSocket client session: " + e.getMessage(), e)); } + // failed + ch.close(); } }); - - addInstrumentedListener(bootstrap.connect(), future -> { + addInstrumentedListener(connectFuture, future -> { if (!future.isSuccess()) { initial.tryEmitError(future.cause()); } @@ -599,117 +593,6 @@ private Http2FrameCodec makeFrameCodec() { return builder.build(); } - /** - * Initializes the HTTP client channel. - */ - private class HttpClientInitializer extends ChannelInitializer { - - final SslContext sslContext; - final String host; - final int port; - final boolean stream; - final boolean proxy; - final boolean acceptsEvents; - final Consumer contextConsumer; - private NettyClientCustomizer channelCustomizer; - - /** - * @param sslContext The ssl context - * @param host The host - * @param port The port - * @param stream Whether is stream - * @param proxy Is this a streaming proxy - * @param acceptsEvents Whether an event stream is accepted - * @param contextConsumer The context consumer - */ - protected HttpClientInitializer(SslContext sslContext, - String host, - int port, - boolean stream, - boolean proxy, - boolean acceptsEvents, - Consumer contextConsumer) { - this.sslContext = sslContext; - this.stream = stream; - this.host = host; - this.port = port; - this.proxy = proxy; - this.acceptsEvents = acceptsEvents; - this.contextConsumer = contextConsumer; - } - - /** - * @param ch The channel - */ - @Override - protected void initChannel(Channel ch) { - channelCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); - ch.attr(CHANNEL_CUSTOMIZER_KEY).set(channelCustomizer); - - ChannelPipeline p = ch.pipeline(); - - configureProxy(p, sslContext != null, host, port); - - if (httpVersion == HttpVersion.HTTP_2_0) { - final Http2Connection connection = new DefaultHttp2Connection(false); - if (sslContext != null) { - } else { - } - channelCustomizer.onInitialPipelineBuilt(); - } else { - if (stream) { - // for streaming responses we disable auto read - // so that the consumer is in charge of back pressure - ch.config().setAutoRead(false); - } - - channelCustomizer.onInitialPipelineBuilt(); - onStreamPipelineBuilt(); - } - } - - /** - * Called when the stream pipeline is fully set up (all handshakes completed) and we can - * start processing requests. - */ - void onStreamPipelineBuilt() { - channelCustomizer.onStreamPipelineBuilt(); - } - - void addEventStreamHandlerIfNecessary(ChannelPipeline p) { - // if the content type is a SSE event stream we add a decoder - // to delimit the content by lines (unless we are proxying the stream) - if (acceptsEventStream() && !proxy) { - - - } - } - - /** - * Allows overriding the final handler added to the pipeline. - * - * @param pipeline The pipeline - */ - protected void addFinalHandler(ChannelPipeline pipeline) { - pipeline.addLast( - ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, - new HttpStreamsClientHandler() { - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - if (evt instanceof IdleStateEvent) { - // close the connection if it is idle for too long - ctx.close(); - } - super.userEventTriggered(ctx, evt); - } - }); - } - - private boolean acceptsEventStream() { - return this.acceptsEvents; - } - } - private class AdaptiveAlpnChannelInitializer extends ChannelInitializer { private final Pool pool; @@ -788,6 +671,14 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } private void initHttp1(Channel ch) { + addLogHandler(ch); + + ch.pipeline() + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()) + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); + } + + private void addLogHandler(Channel ch) { configuration.getLogLevel().ifPresent(logLevel -> { try { final io.netty.handler.logging.LogLevel nettyLevel = @@ -797,10 +688,6 @@ private void initHttp1(Channel ch) { throw customizeException(new HttpClientException("Unsupported log level: " + logLevel)); } }); - - ch.pipeline() - .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()) - .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); } private void initHttp2(Pool pool, Channel ch, NettyClientCustomizer connectionCustomizer) { diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 9dedc1c8249..677aa5a540a 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -13,6 +13,9 @@ import io.micronaut.http.client.exceptions.ReadTimeoutException import io.micronaut.http.netty.channel.ChannelPipelineCustomizer import io.micronaut.http.server.netty.ssl.CertificateProvidedSslBuilder import io.micronaut.http.ssl.SslConfiguration +import io.micronaut.websocket.WebSocketSession +import io.micronaut.websocket.annotation.ClientWebSocket +import io.micronaut.websocket.annotation.OnMessage import io.netty.buffer.ByteBufAllocator import io.netty.buffer.Unpooled import io.netty.channel.Channel @@ -30,10 +33,13 @@ import io.netty.handler.codec.http.DefaultHttpResponse import io.netty.handler.codec.http.DefaultLastHttpContent import io.netty.handler.codec.http.HttpContentCompressor import io.netty.handler.codec.http.HttpMethod +import io.netty.handler.codec.http.HttpObjectAggregator import io.netty.handler.codec.http.HttpResponseStatus import io.netty.handler.codec.http.HttpServerCodec import io.netty.handler.codec.http.HttpServerUpgradeHandler import io.netty.handler.codec.http.LastHttpContent +import io.netty.handler.codec.http.websocketx.TextWebSocketFrame +import io.netty.handler.codec.http.websocketx.WebSocketServerHandshakerFactory import io.netty.handler.codec.http2.DefaultHttp2DataFrame import io.netty.handler.codec.http2.DefaultHttp2Headers import io.netty.handler.codec.http2.DefaultHttp2HeadersFrame @@ -58,6 +64,7 @@ import jakarta.inject.Singleton import reactor.core.publisher.Flux import reactor.core.publisher.Mono import spock.lang.Specification +import spock.lang.Unroll import java.nio.charset.StandardCharsets import java.util.concurrent.CompletableFuture @@ -598,6 +605,58 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + @Unroll + def 'websocket ssl=#secure'(boolean secure) { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + 'micronaut.http.client.connect-ttl': '100s', + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp1() + if (secure) { + conn.setupHttp1Tls() + } else { + conn.setupHttp1() + } + conn.serverChannel.pipeline().addLast(new HttpObjectAggregator(1024)) + patch(client, conn.clientChannel) + + def uri = conn.scheme + "://example.com/foo" + Mono.from(client.connect(Ws, uri)).subscribe() + conn.advance() + conn.clientChannel.pipeline().fireChannelActive() + conn.advance() + io.netty.handler.codec.http.HttpRequest req = conn.serverChannel.readInbound() + def handshaker = new WebSocketServerHandshakerFactory(uri, null, false).newHandshaker(req) + handshaker.handshake(conn.serverChannel, req) + conn.advance() + + conn.serverChannel.writeOutbound(new TextWebSocketFrame('foo')) + conn.advance() + TextWebSocketFrame response = conn.serverChannel.readInbound() + assert response.text() == 'received: foo' + + cleanup: + client.close() + ctx.close() + + where: + secure << [true, false] + } + + @ClientWebSocket + static class Ws implements AutoCloseable { + @Override + void close() throws Exception { + } + + @OnMessage + def onMessage(String msg, WebSocketSession session) { + return session.send('received: ' + msg) + } + } + static class EmbeddedTestConnectionBase { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy index ce03ec4db70..bd46f086b74 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy @@ -67,7 +67,7 @@ class EmbeddedTestUtil { @Override void write(ChannelHandlerContext ctx_, Object msg, ChannelPromise promise) throws Exception { if (!(msg instanceof ByteBuf)) { - throw new IllegalArgumentException("Can only forward bytes!") + throw new IllegalArgumentException("Can only forward bytes, got " + msg) } if (!msg.isReadable()) { // no data From 4deebe0e4667f5c190fbd25c6dc38d438d205b00 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 26 Sep 2022 10:47:03 +0200 Subject: [PATCH 24/82] new http version infrastructure --- .../http/client/HttpClientConfiguration.java | 75 ++++++- .../http/client/HttpClientRegistry.java | 17 +- .../http/client/HttpVersionSelection.java | 195 ++++++++++++++++++ .../http/client/ServiceHttpClientFactory.java | 3 +- .../http/client/annotation/Client.java | 29 +++ .../http/client/netty/ConnectionManager.java | 20 +- .../http/client/netty/DefaultHttpClient.java | 30 +-- .../netty/DefaultNettyHttpClientRegistry.java | 29 ++- .../netty/ssl/NettyClientSslBuilder.java | 28 +-- .../client/netty/ConnectionManagerSpec.groovy | 72 ++++--- 10 files changed, 410 insertions(+), 88 deletions(-) create mode 100644 http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java index 09cf6f71371..5f9fb81ca51 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java @@ -36,8 +36,11 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.temporal.ChronoUnit; +import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.OptionalInt; import java.util.concurrent.ThreadFactory; @@ -145,9 +148,16 @@ public abstract class HttpClientConfiguration { private String eventLoopGroup = "default"; - private HttpVersion httpVersion = HttpVersion.HTTP_1_1; + @Deprecated + @Nullable + private HttpVersion httpVersion = null; - private PlaintextMode plaintextMode = PlaintextMode.HTTP_1; + private HttpVersionSelection.PlaintextMode plaintextMode = HttpVersionSelection.PlaintextMode.HTTP_1; + + private List alpnModes = Arrays.asList( + HttpVersionSelection.ALPN_HTTP_2, + HttpVersionSelection.ALPN_HTTP_1 + ); private LogLevel logLevel; @@ -203,7 +213,11 @@ public HttpClientConfiguration(HttpClientConfiguration copy) { /** * The HTTP version to use. Defaults to {@link HttpVersion#HTTP_1_1}. * @return The http version + * @deprecated There are now separate settings for HTTP and HTTPS connections. To configure + * HTTP connections (e.g. for h2c), use {@link #plaintextMode}. To configure ALPN, set + * {@link #alpnModes}. */ + @Deprecated public HttpVersion getHttpVersion() { return httpVersion; } @@ -211,7 +225,11 @@ public HttpVersion getHttpVersion() { /** * Sets the HTTP version to use. Defaults to {@link HttpVersion#HTTP_1_1}. * @param httpVersion The http version + * @deprecated There are now separate settings for HTTP and HTTPS connections. To configure + * HTTP connections (e.g. for h2c), use {@link #plaintextMode}. To configure ALPN, set + * {@link #alpnModes}. */ + @Deprecated public void setHttpVersion(HttpVersion httpVersion) { if (httpVersion != null) { this.httpVersion = httpVersion; @@ -639,13 +657,54 @@ public Proxy resolveProxy(boolean isSsl, String host, int port) { } } - // TODO: docs - public PlaintextMode getPlaintextMode() { + /** + * The connection mode to use for plaintext (http as opposed to https) connections. + *
+ * Note: If {@link #httpVersion} is set, this setting is ignored! + * + * @return The plaintext connection mode. + * @since 4.0.0 + */ + public HttpVersionSelection.PlaintextMode getPlaintextMode() { return plaintextMode; } - public void setPlaintextMode(PlaintextMode plaintextMode) { - this.plaintextMode = plaintextMode; + /** + * The connection mode to use for plaintext (http as opposed to https) connections. + *
+ * Note: If {@link #httpVersion} is set, this setting is ignored! + * + * @param plaintextMode The plaintext connection mode. + * @since 4.0.0 + */ + public void setPlaintextMode(HttpVersionSelection.PlaintextMode plaintextMode) { + this.plaintextMode = Objects.requireNonNull(plaintextMode, "plaintextMode"); + } + + /** + * The protocols to support for TLS ALPN. If HTTP 2 is included, this will also restrict the + * TLS cipher suites to those supported by the HTTP 2 standard. + *
+ * Note: If {@link #httpVersion} is set, this setting is ignored! + * + * @return The supported ALPN protocols. + * @since 4.0.0 + */ + public List getAlpnModes() { + return alpnModes; + } + + /** + * The protocols to support for TLS ALPN. If HTTP 2 is included, this will also restrict the + * TLS cipher suites to those supported by the HTTP 2 standard. + *
+ * Note: If {@link #httpVersion} is set, this setting is ignored! + * + * @param alpnModes The supported ALPN protocols. + * @since 4.0.0 + */ + public void setAlpnModes(List alpnModes) { + this.alpnModes = Objects.requireNonNull(alpnModes, "alpnModes"); } /** @@ -751,8 +810,4 @@ public void setAcquireTimeout(@Nullable Duration acquireTimeout) { } } - public enum PlaintextMode { - HTTP_1, - H2C, - } } diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientRegistry.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientRegistry.java index 8872c9927bf..cb84cb8a36c 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientRegistry.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientRegistry.java @@ -47,9 +47,24 @@ public interface HttpClientRegistry { * @param clientId The client ID * @param path The path (Optional) * @return The client + * @deprecated Use {@link #getClient(HttpVersionSelection, String, String)} instead */ + @Deprecated @NonNull - T getClient(HttpVersion httpVersion, @NonNull String clientId, @Nullable String path); + default T getClient(HttpVersion httpVersion, @NonNull String clientId, @Nullable String path) { + return getClient(HttpVersionSelection.forLegacyVersion(httpVersion), clientId, path); + } + + /** + * Return the client for the client ID and path. + * + * @param httpVersion The HTTP version + * @param clientId The client ID + * @param path The path (Optional) + * @return The client + */ + @NonNull + T getClient(HttpVersionSelection httpVersion, @NonNull String clientId, @Nullable String path); /** * Resolves a {@link HttpClient} for the given injection point. diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java new file mode 100644 index 00000000000..21f0f5d9b6c --- /dev/null +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java @@ -0,0 +1,195 @@ +/* + * Copyright 2017-2022 original authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.micronaut.http.client; + +import io.micronaut.core.annotation.AnnotationMetadata; +import io.micronaut.core.annotation.Internal; +import io.micronaut.core.annotation.Nullable; +import io.micronaut.http.HttpVersion; +import io.micronaut.http.client.annotation.Client; + +import java.util.Arrays; + +/** + * This class collects information about HTTP client protocol version settings, such as the + * {@link PlaintextMode} and the ALPN configuration. + * + * @author Jonas Konrad + * @since 4.0 + */ +public final class HttpVersionSelection { + /** + * ALPN protocol ID for HTTP/1.1 + */ + public static final String ALPN_HTTP_1 = "http/1.1"; + /** + * ALPN protocol ID for HTTP/2 + */ + public static final String ALPN_HTTP_2 = "h2"; + + private static final HttpVersionSelection LEGACY_1 = new HttpVersionSelection( + PlaintextMode.HTTP_1, + false, + new String[]{ALPN_HTTP_1}, + false + ); + + private static final HttpVersionSelection LEGACY_2 = new HttpVersionSelection( + PlaintextMode.HTTP_1, + true, + new String[]{ALPN_HTTP_1, ALPN_HTTP_2}, + true + ); + + private final PlaintextMode plaintextMode; + private final boolean alpn; + private final String[] alpnSupportedProtocols; + private final boolean http2CipherSuites; + + private HttpVersionSelection(PlaintextMode plaintextMode, boolean alpn, String[] alpnSupportedProtocols, boolean http2CipherSuites) { + this.plaintextMode = plaintextMode; + this.alpn = alpn; + this.alpnSupportedProtocols = alpnSupportedProtocols; + this.http2CipherSuites = http2CipherSuites; + } + + /** + * Get the {@link HttpVersionSelection} that matches Micronaut HTTP client 3.x behavior for the + * given version setting. + * + * @param httpVersion The HTTP version as configured for Micronaut HTTP client 3.x + * @return The version selection + */ + public static HttpVersionSelection forLegacyVersion(HttpVersion httpVersion) { + switch (httpVersion) { + case HTTP_1_0: + case HTTP_1_1: + return LEGACY_1; + case HTTP_2_0: + return LEGACY_2; + default: + throw new IllegalArgumentException("HTTP version " + httpVersion + " not supported here"); + } + } + + /** + * Construct a version selection from the given client configuration. + * + * @param clientConfiguration The client configuration + * @return The configured version selection + */ + public static HttpVersionSelection forClientConfiguration(HttpClientConfiguration clientConfiguration) { + @SuppressWarnings("deprecation") + HttpVersion legacyHttpVersion = clientConfiguration.getHttpVersion(); + if (legacyHttpVersion != null) { + return forLegacyVersion(legacyHttpVersion); + } else { + String[] alpnModes = clientConfiguration.getAlpnModes().toArray(new String[0]); + return new HttpVersionSelection( + clientConfiguration.getPlaintextMode(), + true, + alpnModes, + Arrays.asList(alpnModes).contains(ALPN_HTTP_2) + ); + } + } + + /** + * Infer the version selection for the given {@link Client} annotation, if any version settings + * are set. + * + * @param metadata The annotation metadata possibly containing a {@link Client} annotation + * @return The configured version selection, or {@code null} if the version is not explicitly + * set and should be inherited from the normal configuration instead. + */ + @Internal + @Nullable + public static HttpVersionSelection forClientAnnotation(AnnotationMetadata metadata) { + HttpVersion legacyHttpVersion = + metadata.enumValue(Client.class, "httpVersion", HttpVersion.class).orElse(null); + if (legacyHttpVersion != null) { + return forLegacyVersion(legacyHttpVersion); + } else { + String[] alpnModes = metadata.stringValues(Client.class, "alpnModes"); + PlaintextMode plaintextMode = metadata.enumValue(Client.class, "plaintextMode", PlaintextMode.class) + .orElse(null); + if (alpnModes.length == 0 && plaintextMode == null) { + // nothing set at all, default to client configuration + return null; + } + + // defaults + if (alpnModes.length == 0) { + alpnModes = new String[]{ALPN_HTTP_2, ALPN_HTTP_1}; + } + if (plaintextMode == null) { + plaintextMode = PlaintextMode.HTTP_1; + } + return new HttpVersionSelection( + plaintextMode, + true, + alpnModes, + Arrays.asList(alpnModes).contains(ALPN_HTTP_2) + ); + } + } + + /** + * @return Connection mode to use for plaintext connections + */ + @Internal + public PlaintextMode getPlaintextMode() { + return plaintextMode; + } + + /** + * @return Protocols that should be shown as supported via ALPN + */ + @Internal + public String[] getAlpnSupportedProtocols() { + return alpnSupportedProtocols; + } + + /** + * @return Whether ALPN should be used + */ + @Internal + public boolean isAlpn() { + return alpn; + } + + /** + * @return Whether TLS cipher suites should be constrained to those defined by the HTTP/2 spec + */ + @Internal + public boolean isHttp2CipherSuites() { + return http2CipherSuites; + } + + /** + * The connection mode to use for plaintext (non-TLS) connections. + */ + public enum PlaintextMode { + /** + * Normal HTTP/1.1 connection. + */ + HTTP_1, + /** + * HTTP/2 cleartext upgrade from HTTP/1.1. + */ + H2C, + } +} diff --git a/http-client-core/src/main/java/io/micronaut/http/client/ServiceHttpClientFactory.java b/http-client-core/src/main/java/io/micronaut/http/client/ServiceHttpClientFactory.java index f7250d46473..4b53a554f7d 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/ServiceHttpClientFactory.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/ServiceHttpClientFactory.java @@ -32,6 +32,7 @@ import io.micronaut.runtime.server.event.ServerStartupEvent; import io.micronaut.scheduling.TaskScheduler; import reactor.core.publisher.Flux; + import java.net.URI; import java.time.Duration; import java.util.Collection; @@ -98,7 +99,7 @@ ApplicationEventListener healthCheckStarter(@Parameter Servi Collection loadBalancedURIs = instanceList.getLoadBalancedURIs(); final HttpClient httpClient = clientFactory.get() .getClient( - configuration.getHttpVersion(), + HttpVersionSelection.forClientConfiguration(configuration), configuration.getServiceId(), configuration.getPath().orElse(null)); final Duration initialDelay = configuration.getHealthCheckInterval(); diff --git a/http-client-core/src/main/java/io/micronaut/http/client/annotation/Client.java b/http-client-core/src/main/java/io/micronaut/http/client/annotation/Client.java index 71794cdb4f4..5fa5b5c907a 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/annotation/Client.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/annotation/Client.java @@ -20,6 +20,7 @@ import io.micronaut.context.annotation.Type; import io.micronaut.http.HttpVersion; import io.micronaut.http.client.HttpClientConfiguration; +import io.micronaut.http.client.HttpVersionSelection; import io.micronaut.http.client.interceptor.HttpClientIntroductionAdvice; import io.micronaut.http.hateoas.JsonError; import io.micronaut.retry.annotation.Recoverable; @@ -80,6 +81,34 @@ * The HTTP version. * * @return The HTTP version of the client. + * @deprecated There are now separate settings for HTTP and HTTPS connections. To configure + * HTTP connections (e.g. for h2c), use {@link #plaintextMode}. To configure ALPN, set + * {@link #alpnModes}. */ + @Deprecated HttpVersion httpVersion() default HttpVersion.HTTP_1_1; + + /** + * The connection mode to use for plaintext (http as opposed to https) connections. + *
+ * Note: If {@link #httpVersion} is set, this setting is ignored! + * + * @return The plaintext connection mode. + * @since 4.0.0 + */ + HttpVersionSelection.PlaintextMode plaintextMode() default HttpVersionSelection.PlaintextMode.HTTP_1; + + /** + * The protocols to support for TLS ALPN. If HTTP 2 is included, this will also restrict the + * TLS cipher suites to those supported by the HTTP 2 standard. + *
+ * Note: If {@link #httpVersion} is set, this setting is ignored! + * + * @return The supported ALPN protocols. + * @since 4.0.0 + */ + String[] alpnModes() default { + HttpVersionSelection.ALPN_HTTP_2, + HttpVersionSelection.ALPN_HTTP_1 + }; } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 1588b98ee6b..1ce708f7010 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -19,8 +19,8 @@ import io.micronaut.core.annotation.Nullable; import io.micronaut.core.reflect.InstantiationUtils; import io.micronaut.core.util.StringUtils; -import io.micronaut.http.HttpVersion; import io.micronaut.http.client.HttpClientConfiguration; +import io.micronaut.http.client.HttpVersionSelection; import io.micronaut.http.client.exceptions.HttpClientException; import io.micronaut.http.client.netty.ssl.NettyClientSslBuilder; import io.micronaut.http.netty.channel.ChannelPipelineCustomizer; @@ -127,7 +127,7 @@ class ConnectionManager { private static final AttributeKey STREAM_KEY = AttributeKey.valueOf("micronaut.http2.stream"); final InvocationInstrumenter instrumenter; - final HttpVersion httpVersion; + final HttpVersionSelection httpVersion; private final Logger log; private final Map pools = new ConcurrentHashMap<>(); @@ -161,7 +161,7 @@ class ConnectionManager { @Nullable EventLoopGroup eventLoopGroup, @Nullable ThreadFactory threadFactory, HttpClientConfiguration configuration, - HttpVersion httpVersion, + @Nullable HttpVersionSelection httpVersion, InvocationInstrumenter instrumenter, ChannelFactory socketChannelFactory, NettyClientSslBuilder nettyClientSslBuilder, @@ -170,7 +170,7 @@ class ConnectionManager { String informationalServiceId) { if (httpVersion == null) { - httpVersion = configuration.getHttpVersion(); + httpVersion = HttpVersionSelection.forClientConfiguration(configuration); } this.log = log; @@ -182,7 +182,7 @@ class ConnectionManager { this.pipelineListeners = pipelineListeners; this.informationalServiceId = informationalServiceId; - this.sslContext = nettyClientSslBuilder.build(configuration.getSslConfiguration(), HttpVersion.HTTP_2_0).orElse(null); // TODO: alpn config + this.sslContext = nettyClientSslBuilder.build(configuration.getSslConfiguration(), httpVersion); if (eventLoopGroup != null) { group = eventLoopGroup; @@ -783,6 +783,7 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { } static abstract class PoolHandle { + final boolean http2; final Channel channel; /** @@ -790,7 +791,8 @@ static abstract class PoolHandle { */ abstract void taint(); - private PoolHandle(Channel channel) { + private PoolHandle(boolean http2, Channel channel) { + this.http2 = http2; this.channel = channel; } @@ -879,7 +881,7 @@ private void openNewConnection() { requestKey.getPort() ); } else { - switch (configuration.getPlaintextMode()) { + switch (httpVersion.getPlaintextMode()) { case HTTP_1: initializer = new ChannelInitializer() { @Override @@ -1055,7 +1057,7 @@ void satisfy0(Sinks.One sink) { returnPendingRequest(sink); return; } - Sinks.EmitResult emitResult = sink.tryEmitValue(new PoolHandle(channel) { + Sinks.EmitResult emitResult = sink.tryEmitValue(new PoolHandle(false, channel) { final ChannelHandlerContext lastContext = channel.pipeline().lastContext(); @Override @@ -1176,7 +1178,7 @@ void satisfy0(Sinks.One sink) { .addLast(new Http2StreamFrameToHttpObjectCodec(false)) .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECOMPRESSOR, new HttpContentDecompressor()); NettyClientCustomizer streamCustomizer = connectionCustomizer.specializeForChannel(streamChannel, NettyClientCustomizer.ChannelRole.HTTP2_STREAM); - PoolHandle ph = new PoolHandle(streamChannel) { + PoolHandle ph = new PoolHandle(true, streamChannel) { @Override void taint() { // todo diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index e82519ac474..c3489200217 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -47,6 +47,7 @@ import io.micronaut.http.client.DefaultHttpClientConfiguration; import io.micronaut.http.client.HttpClient; import io.micronaut.http.client.HttpClientConfiguration; +import io.micronaut.http.client.HttpVersionSelection; import io.micronaut.http.client.LoadBalancer; import io.micronaut.http.client.ProxyHttpClient; import io.micronaut.http.client.ProxyRequestOptions; @@ -269,7 +270,7 @@ public DefaultHttpClient(@Nullable LoadBalancer loadBalancer, List invocationInstrumenterFactories, HttpClientFilter... filters) { this(loadBalancer, - configuration.getHttpVersion(), + null, configuration, contextPath, new DefaultHttpClientFilterResolver(annotationMetadataResolver, Arrays.asList(filters)), @@ -307,7 +308,7 @@ public DefaultHttpClient(@Nullable LoadBalancer loadBalancer, * @param informationalServiceId Optional service ID that will be passed to exceptions created by this client */ public DefaultHttpClient(@Nullable LoadBalancer loadBalancer, - @Nullable io.micronaut.http.HttpVersion explicitHttpVersion, + @Nullable HttpVersionSelection explicitHttpVersion, @NonNull HttpClientConfiguration configuration, @Nullable String contextPath, @NonNull HttpClientFilterResolver filterResolver, @@ -1461,11 +1462,11 @@ private void sendRequestThroughChannel( HttpRequest nettyRequest = requestWriter.getNettyRequest(); prepareHttpHeaders( - requestURI, - finalRequest, - nettyRequest, - permitsBody, - poolHandle.canReturn() + poolHandle, + requestURI, + finalRequest, + nettyRequest, + permitsBody ); if (log.isDebugEnabled()) { @@ -1542,7 +1543,7 @@ private void streamRequestThroughChannel0( } } ); - prepareHttpHeaders(requestURI, finalRequest, requestWriter.getNettyRequest(), permitsBody, poolHandle.canReturn()); + prepareHttpHeaders(poolHandle, requestURI, finalRequest, requestWriter.getNettyRequest(), permitsBody); HttpRequest nettyRequest = requestWriter.getNettyRequest(); Promise> responsePromise = poolHandle.channel.eventLoop().newPromise(); @@ -1590,20 +1591,19 @@ private String getHostHeader(URI requestURI) { } private void prepareHttpHeaders( + ConnectionManager.PoolHandle poolHandle, URI requestURI, io.micronaut.http.HttpRequest request, - io.netty.handler.codec.http.HttpRequest nettyRequest, - boolean permitsBody, - boolean keepAlive) { + HttpRequest nettyRequest, + boolean permitsBody) { HttpHeaders headers = nettyRequest.headers(); if (!headers.contains(HttpHeaderNames.HOST)) { headers.set(HttpHeaderNames.HOST, getHostHeader(requestURI)); } - // HTTP/2 assumes keep-alive connections - if (connectionManager.httpVersion != io.micronaut.http.HttpVersion.HTTP_2_0) { - if (keepAlive) { + if (!poolHandle.http2) { + if (poolHandle.canReturn()) { headers.set(HttpHeaderNames.CONNECTION, HttpHeaderValues.KEEP_ALIVE); } else { headers.set(HttpHeaderNames.CONNECTION, HttpHeaderValues.CLOSE); @@ -1929,7 +1929,7 @@ private class NettyRequestWriter { * @param emitter The emitter */ protected void write(ConnectionManager.PoolHandle poolHandle, boolean isSecure, FluxSink emitter) { - if (connectionManager.httpVersion == io.micronaut.http.HttpVersion.HTTP_2_0) { + if (poolHandle.http2) { // todo: move to CM if (isSecure) { nettyRequest.headers().add(AbstractNettyHttpRequest.HTTP2_SCHEME, HttpScheme.HTTPS); diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java index 6e0698a2a6e..2f219ef9a31 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java @@ -27,20 +27,20 @@ import io.micronaut.core.annotation.Nullable; import io.micronaut.core.convert.ConversionService; import io.micronaut.core.util.StringUtils; -import io.micronaut.http.HttpVersion; import io.micronaut.http.MediaType; import io.micronaut.http.annotation.FilterMatcher; import io.micronaut.http.bind.DefaultRequestBinderRegistry; import io.micronaut.http.bind.RequestBinderRegistry; -import io.micronaut.http.client.HttpClientRegistry; -import io.micronaut.http.client.StreamingHttpClientRegistry; -import io.micronaut.http.client.ProxyHttpClient; -import io.micronaut.http.client.HttpClientConfiguration; import io.micronaut.http.client.HttpClient; -import io.micronaut.http.client.StreamingHttpClient; -import io.micronaut.http.client.ProxyHttpClientRegistry; +import io.micronaut.http.client.HttpClientConfiguration; +import io.micronaut.http.client.HttpClientRegistry; +import io.micronaut.http.client.HttpVersionSelection; import io.micronaut.http.client.LoadBalancer; import io.micronaut.http.client.LoadBalancerResolver; +import io.micronaut.http.client.ProxyHttpClient; +import io.micronaut.http.client.ProxyHttpClientRegistry; +import io.micronaut.http.client.StreamingHttpClient; +import io.micronaut.http.client.StreamingHttpClientRegistry; import io.micronaut.http.client.annotation.Client; import io.micronaut.http.client.exceptions.HttpClientException; import io.micronaut.http.client.filter.ClientFilterResolutionContext; @@ -58,8 +58,8 @@ import io.micronaut.http.netty.channel.EventLoopGroupRegistry; import io.micronaut.inject.InjectionPoint; import io.micronaut.inject.qualifiers.Qualifiers; -import io.micronaut.json.JsonMapper; import io.micronaut.json.JsonFeatures; +import io.micronaut.json.JsonMapper; import io.micronaut.json.codec.MapperMediaTypeCodec; import io.micronaut.scheduling.instrument.InvocationInstrumenterFactory; import io.micronaut.websocket.WebSocketClient; @@ -157,7 +157,7 @@ public DefaultNettyHttpClientRegistry( @NonNull @Override - public HttpClient getClient(HttpVersion httpVersion, @NonNull String clientId, @Nullable String path) { + public HttpClient getClient(HttpVersionSelection httpVersion, @NonNull String clientId, @Nullable String path) { final ClientKey key = new ClientKey( httpVersion, clientId, @@ -391,7 +391,7 @@ private DefaultHttpClient getClient(ClientKey key, BeanContext beanContext, Anno private DefaultHttpClient buildClient( LoadBalancer loadBalancer, - HttpVersion httpVersion, + HttpVersionSelection httpVersion, HttpClientConfiguration configuration, String clientId, String contextPath, @@ -476,8 +476,7 @@ private ChannelFactory resolveSocketChannelFactory(HttpClientConfiguration confi } private ClientKey getClientKey(AnnotationMetadata metadata) { - final HttpVersion httpVersion = - metadata.enumValue(Client.class, "httpVersion", HttpVersion.class).orElse(null); + HttpVersionSelection httpVersionSelection = HttpVersionSelection.forClientAnnotation(metadata); String clientId = metadata.stringValue(Client.class).orElse(null); String path = metadata.stringValue(Client.class, "path").orElse(null); List filterAnnotation = metadata @@ -486,7 +485,7 @@ private ClientKey getClientKey(AnnotationMetadata metadata) { metadata.classValue(Client.class, "configuration").orElse(null); JsonFeatures jsonFeatures = jsonMapper.detectFeatures(metadata).orElse(null); - return new ClientKey(httpVersion, clientId, filterAnnotation, path, configurationClass, jsonFeatures); + return new ClientKey(httpVersionSelection, clientId, filterAnnotation, path, configurationClass, jsonFeatures); } private static MediaTypeCodec createNewJsonCodec(BeanContext beanContext, JsonFeatures jsonFeatures) { @@ -502,7 +501,7 @@ private static MapperMediaTypeCodec getJsonCodec(BeanContext beanContext) { */ @Internal private static final class ClientKey { - final HttpVersion httpVersion; + final HttpVersionSelection httpVersion; final String clientId; final List filterAnnotations; final String path; @@ -510,7 +509,7 @@ private static final class ClientKey { final JsonFeatures jsonFeatures; ClientKey( - HttpVersion httpVersion, + HttpVersionSelection httpVersion, String clientId, List filterAnnotations, String path, diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java b/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java index 9c043c6c91f..58bdba5c012 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java @@ -19,6 +19,7 @@ import io.micronaut.core.annotation.Internal; import io.micronaut.core.io.ResourceResolver; import io.micronaut.http.HttpVersion; +import io.micronaut.http.client.HttpVersionSelection; import io.micronaut.http.ssl.AbstractClientSslConfiguration; import io.micronaut.http.ssl.ClientAuthentication; import io.micronaut.http.ssl.SslBuilder; @@ -26,7 +27,6 @@ import io.micronaut.http.ssl.SslConfigurationException; import io.netty.handler.codec.http2.Http2SecurityUtil; import io.netty.handler.ssl.ApplicationProtocolConfig; -import io.netty.handler.ssl.ApplicationProtocolNames; import io.netty.handler.ssl.ClientAuth; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslContextBuilder; @@ -74,17 +74,20 @@ public Optional build(SslConfiguration ssl, HttpVersion httpVersion) if (!ssl.isEnabled()) { return Optional.empty(); } - final boolean isHttp2 = httpVersion == HttpVersion.HTTP_2_0; + return Optional.of(build(ssl, HttpVersionSelection.forLegacyVersion(httpVersion))); + } + + public SslContext build(SslConfiguration ssl, HttpVersionSelection versionSelection) { SslContextBuilder sslBuilder = SslContextBuilder - .forClient() - .keyManager(getKeyManagerFactory(ssl)) - .trustManager(getTrustManagerFactory(ssl)); + .forClient() + .keyManager(getKeyManagerFactory(ssl)) + .trustManager(getTrustManagerFactory(ssl)); if (ssl.getProtocols().isPresent()) { sslBuilder.protocols(ssl.getProtocols().get()); } if (ssl.getCiphers().isPresent()) { sslBuilder = sslBuilder.ciphers(Arrays.asList(ssl.getCiphers().get())); - } else if (isHttp2) { + } else if (versionSelection.isHttp2CipherSuites()) { sslBuilder.ciphers(Http2SecurityUtil.CIPHERS, SupportedCipherSuiteFilter.INSTANCE); } if (ssl.getClientAuthentication().isPresent()) { @@ -95,20 +98,19 @@ public Optional build(SslConfiguration ssl, HttpVersion httpVersion) sslBuilder = sslBuilder.clientAuth(ClientAuth.OPTIONAL); } } - if (isHttp2) { + if (versionSelection.isAlpn()) { SslProvider provider = SslProvider.isAlpnSupported(SslProvider.OPENSSL) ? SslProvider.OPENSSL : SslProvider.JDK; sslBuilder.sslProvider(provider); sslBuilder.applicationProtocolConfig(new ApplicationProtocolConfig( - ApplicationProtocolConfig.Protocol.ALPN, - ApplicationProtocolConfig.SelectorFailureBehavior.NO_ADVERTISE, - ApplicationProtocolConfig.SelectedListenerFailureBehavior.ACCEPT, - ApplicationProtocolNames.HTTP_1_1, - ApplicationProtocolNames.HTTP_2 + ApplicationProtocolConfig.Protocol.ALPN, + ApplicationProtocolConfig.SelectorFailureBehavior.NO_ADVERTISE, + ApplicationProtocolConfig.SelectedListenerFailureBehavior.ACCEPT, + versionSelection.getAlpnSupportedProtocols() )); } try { - return Optional.of(sslBuilder.build()); + return sslBuilder.build(); } catch (SSLException ex) { throw new SslConfigurationException("An error occurred while setting up SSL", ex); } diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 677aa5a540a..33548e4d213 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -135,7 +135,9 @@ class ConnectionManagerSpec extends Specification { conn.setupHttp1() patch(client, conn.clientChannel) - conn.testExchange(client) + def request = conn.testExchangeRequest(client) + conn.fireClientActive() + conn.testExchangeResponse(request) cleanup: client.close() @@ -155,6 +157,7 @@ class ConnectionManagerSpec extends Specification { HttpRequest.GET('http://example.com/foo').header('accept-encoding', 'gzip'), String)).toFuture() future.exceptionally(t -> t.printStackTrace()) conn.advance() + conn.fireClientActive() assert conn.serverChannel.readInbound() instanceof io.netty.handler.codec.http.HttpRequest @@ -216,7 +219,7 @@ class ConnectionManagerSpec extends Specification { conn.setupHttp1Tls() patch(client, conn.clientChannel) - conn.testExchange(client) + conn.testExchangeResponse(conn.testExchangeRequest(client)) cleanup: client.close() @@ -250,7 +253,9 @@ class ConnectionManagerSpec extends Specification { conn.setupHttp1() patch(client, conn.clientChannel) - conn.testStreaming(client) + Queue responseData = conn.testStreamingRequest(client) + conn.fireClientActive() + conn.testStreamingResponse(responseData) cleanup: client.close() @@ -332,10 +337,14 @@ class ConnectionManagerSpec extends Specification { conn.setupHttp1() patch(client, conn.clientChannel) - conn.testExchange(client) - conn.testStreaming(client) - conn.testExchange(client) - conn.testStreaming(client) + def r1 = conn.testExchangeRequest(client) + conn.fireClientActive() + conn.testExchangeResponse(r1) + Queue responseData1 = conn.testStreamingRequest(client) + conn.testStreamingResponse(responseData1) + conn.testExchangeResponse(conn.testExchangeRequest(client)) + Queue responseData = conn.testStreamingRequest(client) + conn.testStreamingResponse(responseData) cleanup: client.close() @@ -353,8 +362,11 @@ class ConnectionManagerSpec extends Specification { patch(client, conn.clientChannel) when: - conn.testExchange(client) - conn.testStreaming(client) + def r1 = conn.testExchangeRequest(client) + conn.fireClientActive() + conn.testExchangeResponse(r1) + Queue responseData = conn.testStreamingRequest(client) + conn.testStreamingResponse(responseData) then: def outerChannel = tracker.initialPipelineBuilt.poll() @@ -461,7 +473,9 @@ class ConnectionManagerSpec extends Specification { patch(client, conn.clientChannel) // do one request - conn.testExchange(client) + def r1 = conn.testExchangeRequest(client) + conn.fireClientActive() + conn.testExchangeResponse(r1) conn.clientChannel.unfreezeTime() // connection is in reserve, should not time out TimeUnit.SECONDS.sleep(10) @@ -541,11 +555,14 @@ class ConnectionManagerSpec extends Specification { patch(client, conn1.clientChannel, conn2.clientChannel) def r1 = conn1.testExchangeRequest(client) + conn1.fireClientActive() conn1.clientChannel.advanceTimeBy(101, TimeUnit.SECONDS) conn1.testExchangeResponse(r1) // conn1 should expire now, conn2 will be the next connection - conn2.testExchange(client) + def r2 = conn2.testExchangeRequest(client) + conn2.fireClientActive() + conn2.testExchangeResponse(r2) cleanup: client.close() @@ -592,13 +609,17 @@ class ConnectionManagerSpec extends Specification { conn2.setupHttp1() patch(client, conn1.clientChannel, conn2.clientChannel) - conn1.testExchange(client) + def r1 = conn1.testExchangeRequest(client) + conn1.fireClientActive() + conn1.testExchangeResponse(r1) conn1.clientChannel.unfreezeTime() // todo: move to advanceTime once IdleStateHandler supports it TimeUnit.SECONDS.sleep(5) conn1.advance() // conn1 should expire now, conn2 will be the next connection - conn2.testExchange(client) + def r2 = conn2.testExchangeRequest(client) + conn2.fireClientActive() + conn2.testExchangeResponse(r2) cleanup: client.close() @@ -625,8 +646,7 @@ class ConnectionManagerSpec extends Specification { def uri = conn.scheme + "://example.com/foo" Mono.from(client.connect(Ws, uri)).subscribe() conn.advance() - conn.clientChannel.pipeline().fireChannelActive() - conn.advance() + conn.fireClientActive() io.netty.handler.codec.http.HttpRequest req = conn.serverChannel.readInbound() def handshaker = new WebSocketServerHandshakerFactory(uri, null, false).newHandshaker(req) handshaker.handshake(conn.serverChannel, req) @@ -674,6 +694,11 @@ class ConnectionManagerSpec extends Specification { void advance() { EmbeddedTestUtil.advance(serverChannel, clientChannel) } + + void fireClientActive() { + clientChannel.pipeline().fireChannelActive() + advance() + } } static class EmbeddedServerChannel extends EmbeddedChannel implements ServerChannel { @@ -709,10 +734,6 @@ class ConnectionManagerSpec extends Specification { serverChannel.writeOutbound(response) } - void testExchange(HttpClient client) { - testExchangeResponse(testExchangeRequest(client)) - } - CompletableFuture> testExchangeRequest(HttpClient client) { def future = Mono.from(client.exchange(scheme + '://example.com/foo')).toFuture() future.exceptionally(t -> t.printStackTrace()) @@ -736,13 +757,16 @@ class ConnectionManagerSpec extends Specification { assert future.get().status() == HttpStatus.OK } - void testStreaming(StreamingHttpClient client) { + private Queue testStreamingRequest(StreamingHttpClient client) { def responseData = new ArrayDeque() - def responseComplete = false Flux.from(client.dataStream(HttpRequest.GET(scheme + '://example.com/foo'))) .doOnError(t -> t.printStackTrace()) - .doOnComplete(() -> responseComplete = true) + .doOnComplete(() -> responseData.add("END")) .subscribe(b -> responseData.add(b.toString(StandardCharsets.UTF_8))) + responseData + } + + private void testStreamingResponse(Queue responseData) { advance() io.netty.handler.codec.http.HttpRequest request = serverChannel.readInbound() @@ -761,13 +785,13 @@ class ConnectionManagerSpec extends Specification { advance() assert responseData.poll() == 'foo' - assert !responseComplete + assert responseData.isEmpty() serverChannel.writeOutbound(new DefaultLastHttpContent(Unpooled.wrappedBuffer('bar'.bytes))) advance() assert responseData.poll() == 'bar' - assert responseComplete + assert responseData.poll() == 'END' } } From 10fcbca8d58ac7140e997f5f83e392037a1e5f18 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 26 Sep 2022 12:30:49 +0200 Subject: [PATCH 25/82] connection pool size settings --- .../http/client/HttpClientConfiguration.java | 50 +++++ .../http/client/netty/ConnectionManager.java | 86 +++++---- .../http/client/netty/PoolResizer.java | 171 ++++++++++++++++++ 3 files changed, 269 insertions(+), 38 deletions(-) create mode 100644 http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java index 5f9fb81ca51..82f6038c398 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java @@ -728,6 +728,12 @@ public static class ConnectionPoolConfiguration implements Toggleable { @SuppressWarnings("WeakerAccess") public static final int DEFAULT_MAXCONNECTIONS = -1; + private int maxPendingConnections = 4; + + private int maxConcurrentRequestsPerHttp2Connection = Integer.MAX_VALUE; + private int maxConcurrentHttp1Connections = Integer.MAX_VALUE; + private int maxConcurrentHttp2Connections = 1; + private int maxConnections = DEFAULT_MAXCONNECTIONS; private int maxPendingAcquires = Integer.MAX_VALUE; @@ -808,6 +814,50 @@ public Optional getAcquireTimeout() { public void setAcquireTimeout(@Nullable Duration acquireTimeout) { this.acquireTimeout = acquireTimeout; } + + /** + * The maximum number of pending (new) connections before they are assigned to a + * pool. + * + * @return The maximum number of pending connections + */ + public int getMaxPendingConnections() { + return maxPendingConnections; + } + + /** + * The maximum number of pending (new) connections before they are assigned to a + * pool. + * + * @param maxPendingConnections The maximum number of pending connections + */ + public void setMaxPendingConnections(int maxPendingConnections) { + this.maxPendingConnections = maxPendingConnections; + } + + public int getMaxConcurrentRequestsPerHttp2Connection() { + return maxConcurrentRequestsPerHttp2Connection; + } + + public void setMaxConcurrentRequestsPerHttp2Connection(int maxConcurrentRequestsPerHttp2Connection) { + this.maxConcurrentRequestsPerHttp2Connection = maxConcurrentRequestsPerHttp2Connection; + } + + public int getMaxConcurrentHttp1Connections() { + return maxConcurrentHttp1Connections; + } + + public void setMaxConcurrentHttp1Connections(int maxConcurrentHttp1Connections) { + this.maxConcurrentHttp1Connections = maxConcurrentHttp1Connections; + } + + public int getMaxConcurrentHttp2Connections() { + return maxConcurrentHttp2Connections; + } + + public void setMaxConcurrentHttp2Connections(int maxConcurrentHttp2Connections) { + this.maxConcurrentHttp2Connections = maxConcurrentHttp2Connections; + } } } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 1ce708f7010..96d54ff533a 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -653,6 +653,7 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { private class InitialConnectionErrorHandler extends ChannelInboundHandlerAdapter { private final Pool pool; + private Throwable cause; InitialConnectionErrorHandler(Pool pool) { this.pool = pool; @@ -661,12 +662,13 @@ private class InitialConnectionErrorHandler extends ChannelInboundHandlerAdapter @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { log.error("Failed to open connection", cause); + this.cause = cause; } @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { super.channelInactive(ctx); - pool.onNewConnectionFailure(); + pool.onNewConnectionFailure(cause); } } @@ -817,30 +819,15 @@ void notifyRequestPipelineBuilt() { } } - private static boolean incrementWithLimit(AtomicInteger variable, int limit) { - while (true) { - int old = variable.get(); - if (old >= limit) { - return false; - } - if (variable.compareAndSet(old, old + 1)) { - return true; - } - } - } - - private final class Pool { - private static final int MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION = 4; // TODO: config - private static final int MAX_PENDING_CONNECTIONS = 1; - + private final class Pool extends PoolResizer { private final DefaultHttpClient.RequestKey requestKey; private final Queue> pendingRequests = new ConcurrentLinkedQueue<>(); private final List http1Connections = new CopyOnWriteArrayList<>(); private final List http2Connections = new CopyOnWriteArrayList<>(); - private final AtomicInteger pendingConnectionCount = new AtomicInteger(0); Pool(DefaultHttpClient.RequestKey requestKey) { + super(log, configuration.getConnectionPoolConfiguration()); this.requestKey = requestKey; } @@ -848,7 +835,12 @@ Mono acquire() { Sinks.One sink = Sinks.one(); acquire(sink); // todo: if the subscriber cancels before the connection is acquired, what happens? - return sink.asMono(); + Optional acquireTimeout = configuration.getConnectionPoolConfiguration().getAcquireTimeout(); + if (acquireTimeout.isPresent()) { + return sink.asMono().timeout(acquireTimeout.get(), getEventLoopScheduler()); + } else { + return sink.asMono(); + } } private void acquire(Sinks.One sink) { @@ -863,14 +855,36 @@ private void acquire(Sinks.One sink) { } } // no connection open that has room + addPendingRequest(sink); + } + + @Override + void onNewConnectionFailure(@Nullable Throwable error) throws Exception { + log.error("Failed to connect to remote", error); + super.onNewConnectionFailure(error); + // to avoid an infinite loop, fail one pending request. + Sinks.One pending = pollPendingRequest(); + if (pending != null) { + pending.tryEmitError(new HttpClientException("Failed to establish connection", error)); + } + } + + private void addPendingRequest(Sinks.One sink) { pendingRequests.add(sink); - openNewConnection(); + onPendingRequestChange(1); } - private void openNewConnection() { - if (!incrementWithLimit(pendingConnectionCount, MAX_PENDING_CONNECTIONS)) { - return; + @Nullable + private Sinks.One pollPendingRequest() { + Sinks.One req = pendingRequests.poll(); + if (req != null) { + onPendingRequestChange(-1); } + return req; + } + + @Override + void openNewConnection() { // open a new connection ChannelInitializer initializer; if (requestKey.isSecure()) { @@ -911,17 +925,11 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { } addInstrumentedListener(doConnect(requestKey, initializer), future -> { if (!future.isSuccess()) { - log.error("Failed to connect to remote", future.cause()); - onNewConnectionFailure(); + onNewConnectionFailure(future.cause()); } }); } - void onNewConnectionFailure() { - pendingConnectionCount.decrementAndGet(); - // todo: retry connection - } - public void shutdown() { for (Http1ConnectionHolder http1Connection : http1Connections) { http1Connection.channel.close(); @@ -1027,13 +1035,13 @@ void init(boolean fireInitialPipelineBuilt) { } connectionCustomizer.onStreamPipelineBuilt(); - Sinks.One pendingRequest = pendingRequests.poll(); + Sinks.One pendingRequest = pollPendingRequest(); if (pendingRequest != null) { hasLiveRequest.set(true); satisfy0(pendingRequest); } http1Connections.add(this); - pendingConnectionCount.decrementAndGet(); + onNewConnectionEstablished1(); } @Override @@ -1100,7 +1108,7 @@ void notifyRequestPipelineBuilt() { private void returnPendingRequest(Sinks.One sink) { // failed, but the pending request may still work on another connection. - pendingRequests.add(sink); + addPendingRequest(sink); hasLiveRequest.set(false); } @@ -1116,6 +1124,7 @@ void windDownConnection() { void onInactive() { super.onInactive(); http1Connections.remove(this); + onConnectionInactive1(); } } @@ -1136,8 +1145,8 @@ void init() { connectionCustomizer.onStreamPipelineBuilt(); - for (int i = 0; i < MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION; i++) { - Sinks.One pendingRequest = pendingRequests.poll(); + for (int i = 0; i < configuration.getConnectionPoolConfiguration().getMaxConcurrentRequestsPerHttp2Connection(); i++) { + Sinks.One pendingRequest = pollPendingRequest(); if (pendingRequest == null) { break; } @@ -1145,12 +1154,12 @@ void init() { satisfy0(pendingRequest); } http2Connections.add(this); - pendingConnectionCount.decrementAndGet(); + onNewConnectionEstablished2(); } @Override boolean tryEarmarkForRequest() { - return !windDownConnection && incrementWithLimit(liveRequests, MAX_CONCURRENT_REQUESTS_PER_HTTP2_CONNECTION); + return !windDownConnection && incrementWithLimit(liveRequests, configuration.getConnectionPoolConfiguration().getMaxConcurrentRequestsPerHttp2Connection()); } @Override @@ -1216,7 +1225,7 @@ void notifyRequestPipelineBuilt() { private void returnPendingRequest(Sinks.One sink) { // failed, but the pending request may still work on another connection. - pendingRequests.add(sink); + addPendingRequest(sink); liveRequests.decrementAndGet(); } @@ -1232,6 +1241,7 @@ void windDownConnection() { void onInactive() { super.onInactive(); http2Connections.remove(Http2ConnectionHolder.this); + onConnectionInactive2(); } } } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java new file mode 100644 index 00000000000..89a90925fc7 --- /dev/null +++ b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java @@ -0,0 +1,171 @@ +/* + * Copyright 2017-2022 original authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.micronaut.http.client.netty; + +import io.micronaut.core.annotation.Internal; +import io.micronaut.core.annotation.Nullable; +import io.micronaut.http.client.HttpClientConfiguration; +import org.slf4j.Logger; + +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class handles the sizing of a connection pool to conform to the configuration in + * {@link io.micronaut.http.client.HttpClientConfiguration.ConnectionPoolConfiguration}. + */ +@Internal +abstract class PoolResizer { + private final Logger log; + private final HttpClientConfiguration.ConnectionPoolConfiguration connectionPoolConfiguration; + + private final AtomicReference state = new AtomicReference<>(WorkState.IDLE); + + private final AtomicInteger pendingRequests = new AtomicInteger(0); + private final AtomicInteger pendingConnectionCount = new AtomicInteger(0); + private final AtomicInteger http1ConnectionCount = new AtomicInteger(0); + private final AtomicInteger http2ConnectionCount = new AtomicInteger(0); + + PoolResizer(Logger log, HttpClientConfiguration.ConnectionPoolConfiguration connectionPoolConfiguration) { + this.log = log; + this.connectionPoolConfiguration = connectionPoolConfiguration; + } + + private void dirty() { + WorkState before = state.getAndUpdate(ws -> { + if (ws == WorkState.IDLE) { + return WorkState.ACTIVE_WITHOUT_PENDING_WORK; + } else { + return WorkState.ACTIVE_WITH_PENDING_WORK; + } + }); + if (before != WorkState.IDLE) { + // already in one of the active states, another thread will take care of our changes + return; + } + // we were in idle state, this thread will handle the changes. + while (true) { + doSomeWork(); + + WorkState endState = state.updateAndGet(ws -> { + if (ws == WorkState.ACTIVE_WITH_PENDING_WORK) { + return WorkState.ACTIVE_WITHOUT_PENDING_WORK; + } else { + return WorkState.IDLE; + } + }); + if (endState == WorkState.IDLE) { + // nothing else to do \o/ + break; + } + } + } + + private void doSomeWork() { + // snapshot our fields + int pendingRequests = this.pendingRequests.get(); + int pendingConnectionCount = this.pendingConnectionCount.get(); + int http1ConnectionCount = this.http1ConnectionCount.get(); + int http2ConnectionCount = this.http2ConnectionCount.get(); + + if (pendingRequests == 0) { + // if there are no pending requests, there is nothing to do. + return; + } + int connectionsToOpen = pendingRequests - pendingConnectionCount; + // make sure we won't exceed our config setting for pending connections + connectionsToOpen = Math.min(connectionsToOpen, connectionPoolConfiguration.getMaxPendingConnections() - pendingConnectionCount); + // limit the connection count to the protocol-specific settings, but only if that protocol was seen for this pool. + if (http1ConnectionCount > 0) { + connectionsToOpen = Math.min(connectionsToOpen, connectionPoolConfiguration.getMaxConcurrentHttp2Connections() - http1ConnectionCount); + } + if (http2ConnectionCount > 0) { + connectionsToOpen = Math.min(connectionsToOpen, connectionPoolConfiguration.getMaxConcurrentHttp2Connections() - http2ConnectionCount); + } + + if (connectionsToOpen > 0) { + this.pendingConnectionCount.addAndGet(connectionsToOpen); + for (int i = 0; i < connectionsToOpen; i++) { + try { + openNewConnection(); + } catch (Exception e) { + try { + onNewConnectionFailure(e); + } catch (Exception f) { + log.error("Internal error", f); + } + } + } + dirty(); + } + } + + abstract void openNewConnection() throws Exception; + + static boolean incrementWithLimit(AtomicInteger variable, int limit) { + while (true) { + int old = variable.get(); + if (old >= limit) { + return false; + } + if (variable.compareAndSet(old, old + 1)) { + return true; + } + } + } + + final void onPendingRequestChange(int delta) { + if (pendingRequests.addAndGet(delta) < 0) { + throw new IllegalStateException("Negative pending requests"); + } + dirty(); + } + + // can be overridden, so `throws Exception` ensures we handle any errors + void onNewConnectionFailure(@Nullable Throwable error) throws Exception { + // todo: circuit breaker? + pendingConnectionCount.decrementAndGet(); + dirty(); + } + + final void onNewConnectionEstablished1() { + http1ConnectionCount.incrementAndGet(); + pendingConnectionCount.decrementAndGet(); + dirty(); + } + + final void onNewConnectionEstablished2() { + http2ConnectionCount.incrementAndGet(); + pendingConnectionCount.decrementAndGet(); + dirty(); + } + + final void onConnectionInactive1() { + http1ConnectionCount.decrementAndGet(); + dirty(); + } + + final void onConnectionInactive2() { + http2ConnectionCount.decrementAndGet(); + dirty(); + } + + private enum WorkState { + IDLE, + ACTIVE_WITH_PENDING_WORK, + ACTIVE_WITHOUT_PENDING_WORK, + } +} From 6f190ac8b3215aeea0e47e7f95b6cd4f880d6371 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 26 Sep 2022 12:34:21 +0200 Subject: [PATCH 26/82] fix some todos --- .../http/client/netty/ConnectionManager.java | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 96d54ff533a..de08667f368 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -987,6 +987,19 @@ void windDownConnection() { windDownConnection = true; } + final void satisfySomePendingRequests() { + while (true) { + Sinks.One pending = pollPendingRequest(); + if (pending == null) { + break; + } + if (!satisfy(pending)) { + addPendingRequest(pending); + break; + } + } + } + boolean satisfy(Sinks.One sink) { if (!tryEarmarkForRequest()) { return false; @@ -1084,7 +1097,7 @@ void release() { } if (!windDownConnection) { hasLiveRequest.set(false); - // todo: claim a new request + satisfySomePendingRequests(); } else { channel.close(); } @@ -1102,7 +1115,7 @@ void notifyRequestPipelineBuilt() { }); if (emitResult.isFailure()) { hasLiveRequest.set(false); - // todo: claim a new request + satisfySomePendingRequests(); } } @@ -1190,7 +1203,7 @@ void satisfy0(Sinks.One sink) { PoolHandle ph = new PoolHandle(true, streamChannel) { @Override void taint() { - // todo + // do nothing, we don't reuse stream channels } @Override @@ -1198,7 +1211,7 @@ void release() { liveStreamChannels.remove(streamChannel); streamChannel.close(); liveRequests.decrementAndGet(); - // todo: claim a new request + satisfySomePendingRequests(); } @Override From e96d8f6809dd1d1ef7f9da10f8a898861d11763d Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 26 Sep 2022 12:59:53 +0200 Subject: [PATCH 27/82] handle cancelled pool acquisition --- .../client/netty/CancellableMonoSink.java | 118 ++++++++++++++++++ .../http/client/netty/ConnectionManager.java | 3 +- .../client/netty/ConnectionManagerSpec.groovy | 20 +++ 3 files changed, 139 insertions(+), 2 deletions(-) create mode 100644 http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java b/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java new file mode 100644 index 00000000000..d1a2b13e4a7 --- /dev/null +++ b/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java @@ -0,0 +1,118 @@ +package io.micronaut.http.client.netty; + +import io.micronaut.core.annotation.Internal; +import io.micronaut.core.annotation.NonNull; +import org.reactivestreams.Publisher; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; +import reactor.core.publisher.Mono; +import reactor.core.publisher.Sinks; + +/** + * Version of {@link Sinks#one()} where cancellation of the {@link Mono} will make future emit + * calls fail. + */ +@Internal +final class CancellableMonoSink implements Publisher, Sinks.One, Subscription { + private T value; + private Throwable failure; + private boolean complete = false; + private Subscriber subscriber = null; + private boolean subscriberWaiting = false; + + @Override + public synchronized void subscribe(Subscriber s) { + if (this.subscriber != null) { + s.onError(new IllegalStateException("Only one subscriber allowed")); + } + subscriber = s; + subscriber.onSubscribe(this); + } + + private void tryForward() { + if (subscriberWaiting && complete) { + if (failure == null) { + subscriber.onNext(value); + subscriber.onComplete(); + } else { + subscriber.onError(failure); + } + } + } + + @NonNull + @Override + public synchronized Sinks.EmitResult tryEmitValue(T value) { + if (complete) { + return Sinks.EmitResult.FAIL_OVERFLOW; + } else { + this.value = value; + complete = true; + tryForward(); + return Sinks.EmitResult.OK; + } + } + + @Override + public void emitValue(T value, @NonNull Sinks.EmitFailureHandler failureHandler) { + throw new UnsupportedOperationException(); + } + + @NonNull + @Override + public Sinks.EmitResult tryEmitEmpty() { + throw new UnsupportedOperationException(); + } + + @NonNull + @Override + public synchronized Sinks.EmitResult tryEmitError(@NonNull Throwable error) { + if (complete) { + return Sinks.EmitResult.FAIL_OVERFLOW; + } else { + this.failure = error; + complete = true; + tryForward(); + return Sinks.EmitResult.OK; + } + } + + @Override + public void emitEmpty(@NonNull Sinks.EmitFailureHandler failureHandler) { + throw new UnsupportedOperationException(); + } + + @Override + public void emitError(@NonNull Throwable error, @NonNull Sinks.EmitFailureHandler failureHandler) { + throw new UnsupportedOperationException(); + } + + @Override + public synchronized int currentSubscriberCount() { + return subscriber == null ? 0 : 1; + } + + @NonNull + @Override + public Mono asMono() { + return Mono.from(this); + } + + @Override + public Object scanUnsafe(@NonNull Attr key) { + return null; + } + + @Override + public synchronized void request(long n) { + if (n > 0 && !subscriberWaiting) { + subscriberWaiting = true; + tryForward(); + } + } + + @Override + public synchronized void cancel() { + complete = true; + } +} diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index de08667f368..5482433bf8b 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -832,9 +832,8 @@ private final class Pool extends PoolResizer { } Mono acquire() { - Sinks.One sink = Sinks.one(); + Sinks.One sink = new CancellableMonoSink<>(); acquire(sink); - // todo: if the subscriber cancels before the connection is acquired, what happens? Optional acquireTimeout = configuration.getConnectionPoolConfiguration().getAcquireTimeout(); if (acquireTimeout.isPresent()) { return sink.asMono().timeout(acquireTimeout.get(), getEventLoopScheduler()); diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 33548e4d213..f97d2509788 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -677,6 +677,26 @@ class ConnectionManagerSpec extends Specification { } } + def 'cancel pool acquisition'() { + def ctx = ApplicationContext.run() + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp1() + conn.setupHttp1() + patch(client, conn.clientChannel) + + def subscription = Mono.from(client.exchange(conn.scheme + '://example.com/foo')).subscribe() + conn.advance() + subscription.dispose() + conn.fireClientActive() + + conn.testExchangeResponse(conn.testExchangeRequest(client)) + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnectionBase { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel From 13a70f68f5fccff569d312309950e31ec7168a45 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 26 Sep 2022 17:53:46 +0200 Subject: [PATCH 28/82] fix multipart requests --- .../http/client/netty/ConnectionManager.java | 16 ++++++++ .../http/client/netty/DefaultHttpClient.java | 9 +++++ .../client/netty/ConnectionManagerSpec.groovy | 38 +++++++++++++++++++ 3 files changed, 63 insertions(+) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 5482433bf8b..0a13d67ec69 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -354,6 +354,22 @@ protected void finishAggregation(FullHttpMessage aggregated) throws Exception { } } }); + if (multipart) { + ph.channel.pipeline().addLast( + ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, + new HttpStreamsClientHandler() { + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof IdleStateEvent) { + // close the connection if it is idle for too long + ph.taint(); + ph.release(); + } + super.userEventTriggered(ctx, evt); + } + } + ); + } return ph; }); } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index c3489200217..e1095b8db5d 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -173,6 +173,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.Objects; import java.util.Optional; import java.util.concurrent.ThreadFactory; @@ -2273,6 +2274,14 @@ public Argument getErrorType(MediaType mediaType) { @Override public void handlerRemoved(ChannelHandlerContext ctx) { ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR); + try { + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP_CHUNK); + } catch (NoSuchElementException ignored) { + } + try { + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); + } catch (NoSuchElementException ignored) { + } poolHandle.release(); } diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index f97d2509788..cdec42ce8bf 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -7,9 +7,11 @@ import io.micronaut.http.HttpRequest import io.micronaut.http.HttpResponse import io.micronaut.http.HttpStatus import io.micronaut.http.HttpVersion +import io.micronaut.http.MediaType import io.micronaut.http.client.HttpClient import io.micronaut.http.client.StreamingHttpClient import io.micronaut.http.client.exceptions.ReadTimeoutException +import io.micronaut.http.client.multipart.MultipartBody import io.micronaut.http.netty.channel.ChannelPipelineCustomizer import io.micronaut.http.server.netty.ssl.CertificateProvidedSslBuilder import io.micronaut.http.ssl.SslConfiguration @@ -31,6 +33,7 @@ import io.netty.handler.codec.http.DefaultFullHttpResponse import io.netty.handler.codec.http.DefaultHttpContent import io.netty.handler.codec.http.DefaultHttpResponse import io.netty.handler.codec.http.DefaultLastHttpContent +import io.netty.handler.codec.http.FullHttpRequest import io.netty.handler.codec.http.HttpContentCompressor import io.netty.handler.codec.http.HttpMethod import io.netty.handler.codec.http.HttpObjectAggregator @@ -697,6 +700,41 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'multipart request'() { + def ctx = ApplicationContext.run() + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp1() + conn.setupHttp1() + patch(client, conn.clientChannel) + conn.serverChannel.pipeline().addLast(new HttpObjectAggregator(1024)) + + def future = Mono.from(client.exchange(HttpRequest.POST(conn.scheme + '://example.com/foo', MultipartBody.builder() + .addPart('foo', 'fn', MediaType.TEXT_PLAIN_TYPE, 'bar'.bytes) + .build()) + .contentType(MediaType.MULTIPART_FORM_DATA), String)).toFuture() + future.exceptionally(t -> t.printStackTrace()) + conn.advance() + conn.fireClientActive() + + FullHttpRequest request = conn.serverChannel.readInbound() + assert request.uri() == '/foo' + assert request.method() == HttpMethod.POST + assert request.headers().get('host') == 'example.com' + assert request.headers().get("connection") == "keep-alive" + assert request.content().isReadable(100) // cba to check the exact content + + def response = new DefaultFullHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer('foo'.bytes)) + response.headers().add("Content-Length", 3) + conn.serverChannel.writeOutbound(response) + conn.advance() + assert future.get().body() == 'foo' + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnectionBase { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel From 045b73155a520c9a36c0e0dd5a7b78f6dc44a328 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 26 Sep 2022 19:43:13 +0200 Subject: [PATCH 29/82] improve channel emulation --- .../client/netty/CancellableMonoSink.java | 9 ++- .../http/client/netty/ConnectionManager.java | 8 +-- .../NettyWebSocketClientHandler.java | 9 ++- .../client/netty/ConnectionManagerSpec.groovy | 9 ++- .../http/client/netty/EmbeddedTestUtil.groovy | 63 +++++++++++-------- 5 files changed, 61 insertions(+), 37 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java b/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java index d1a2b13e4a7..ae9765bc7b2 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java @@ -14,6 +14,8 @@ */ @Internal final class CancellableMonoSink implements Publisher, Sinks.One, Subscription { + private static final Object EMPTY = new Object(); + private T value; private Throwable failure; private boolean complete = false; @@ -32,7 +34,9 @@ public synchronized void subscribe(Subscriber s) { private void tryForward() { if (subscriberWaiting && complete) { if (failure == null) { - subscriber.onNext(value); + if (value != EMPTY) { + subscriber.onNext(value); + } subscriber.onComplete(); } else { subscriber.onError(failure); @@ -58,10 +62,11 @@ public void emitValue(T value, @NonNull Sinks.EmitFailureHandler failureHandler) throw new UnsupportedOperationException(); } + @SuppressWarnings("unchecked") @NonNull @Override public Sinks.EmitResult tryEmitEmpty() { - throw new UnsupportedOperationException(); + return tryEmitValue((T) EMPTY); } @NonNull diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 0a13d67ec69..21c979791db 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -490,7 +490,7 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc * @return A mono that will complete when the handshakes complete */ Mono connectForWebsocket(DefaultHttpClient.RequestKey requestKey, ChannelHandler handler) { - Sinks.Empty initial = Sinks.empty(); + Sinks.Empty initial = new CancellableMonoSink<>(); ChannelFuture connectFuture = doConnect(requestKey, new ChannelInitializer() { @Override @@ -791,6 +791,8 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { upgradeRequest.headers().set(HttpHeaderNames.HOST, pool.requestKey.getHost() + ':' + pool.requestKey.getPort()); ctx.writeAndFlush(upgradeRequest); ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP2_UPGRADE_REQUEST); + // read the upgrade response + ctx.read(); super.channelActive(ctx); } @@ -930,9 +932,7 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { }; break; case H2C: - initializer = new Http2UpgradeInitializer( - this - ); + initializer = new Http2UpgradeInitializer(this); break; default: throw new AssertionError("Unknown plaintext mode"); diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/websocket/NettyWebSocketClientHandler.java b/http-client/src/main/java/io/micronaut/http/client/netty/websocket/NettyWebSocketClientHandler.java index 5e825b38400..0bf36fc9700 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/websocket/NettyWebSocketClientHandler.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/websocket/NettyWebSocketClientHandler.java @@ -146,7 +146,14 @@ public void handlerAdded(final ChannelHandlerContext ctx) { @Override public void channelActive(final ChannelHandlerContext ctx) { - handshaker.handshake(ctx.channel()); + handshaker.handshake(ctx.channel()).addListener(future -> { + if (future.isSuccess()) { + ctx.channel().config().setAutoRead(true); + ctx.read(); + } else { + handshakeFuture.tryFailure(future.cause()); + } + }); } @Override diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index cdec42ce8bf..b08750ac6c8 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -64,8 +64,10 @@ import io.netty.handler.ssl.SslContextBuilder import io.netty.handler.ssl.util.SelfSignedCertificate import io.netty.util.AsciiString import jakarta.inject.Singleton +import org.spockframework.runtime.model.parallel.ExecutionMode import reactor.core.publisher.Flux import reactor.core.publisher.Mono +import spock.lang.Execution import spock.lang.Specification import spock.lang.Unroll @@ -76,6 +78,7 @@ import java.util.concurrent.Future import java.util.concurrent.TimeUnit import java.util.zip.GZIPOutputStream +@Execution(ExecutionMode.CONCURRENT) class ConnectionManagerSpec extends Specification { private static void patch(DefaultHttpClient httpClient, EmbeddedChannel... channels) { httpClient.connectionManager = new ConnectionManager(httpClient.connectionManager) { @@ -746,6 +749,7 @@ class ConnectionManagerSpec extends Specification { clientChannel = new EmbeddedChannel(new DummyChannelId('client')) clientChannel.freezeTime() + clientChannel.config().setAutoRead(false) EmbeddedTestUtil.connect(serverChannel, clientChannel) } @@ -907,15 +911,14 @@ class ConnectionManagerSpec extends Specification { } void exchangeSettings() { - advance() + fireClientActive() assert serverChannel.readInbound() instanceof Http2SettingsFrame assert serverChannel.readInbound() instanceof Http2SettingsAckFrame } void exchangeH2c() { - clientChannel.pipeline().fireChannelActive() - advance() + fireClientActive() Http2HeadersFrame upgradeRequest = serverChannel.readInbound() assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy index bd46f086b74..38441839f08 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/EmbeddedTestUtil.groovy @@ -1,6 +1,7 @@ package io.micronaut.http.client.netty import io.netty.buffer.ByteBuf +import io.netty.buffer.CompositeByteBuf import io.netty.channel.ChannelHandlerContext import io.netty.channel.ChannelOutboundHandlerAdapter import io.netty.channel.ChannelPromise @@ -28,38 +29,24 @@ class EmbeddedTestUtil { } private static class ConnectionDirection { - private static final Object FLUSH = new Object() - final EmbeddedChannel source final EmbeddedChannel dest - final Queue queue = new ArrayDeque<>() + CompositeByteBuf sourceQueue + final List sourceQueueFutures = new ArrayList<>(); + final Queue destQueue = new ArrayDeque<>() boolean readPending - boolean newData = false ConnectionDirection(EmbeddedChannel source, EmbeddedChannel dest) { this.source = source this.dest = dest } - private void forwardLater(Object msg) { - if (readPending || dest.config().isAutoRead()) { - dest.eventLoop().execute(() -> forwardNow(msg)) - readPending = false - } else { - queue.add(msg) - } - } - - private void forwardNow(Object msg) { - if (msg == FLUSH) { - if (dest.isOpen() && newData) { - dest.pipeline().fireChannelReadComplete() - newData = false - } - } else { - newData = true - dest.writeOneInbound(msg) + private void forwardNow(ByteBuf msg) { + if (!dest.isOpen()) { + return } + dest.writeOneInbound(msg) + dest.pipeline().fireChannelReadComplete() } void register() { @@ -75,22 +62,44 @@ class EmbeddedTestUtil { promise.setSuccess() return } - forwardLater(msg) - promise.setSuccess() + + if (sourceQueue == null) { + sourceQueue = ((ByteBuf) msg).alloc().compositeBuffer() + } + sourceQueue.addComponent(true, (ByteBuf) msg) + if (!promise.isVoid()) { + sourceQueueFutures.add(promise) + } } @Override void flush(ChannelHandlerContext ctx_) throws Exception { - forwardLater(FLUSH) + if (sourceQueue != null) { + ByteBuf packet = sourceQueue + sourceQueue = null + + for (ChannelPromise promise : sourceQueueFutures) { + promise.trySuccess() + } + sourceQueueFutures.clear() + + if (readPending || dest.config().isAutoRead()) { + dest.eventLoop().execute(() -> forwardNow(packet)) + readPending = false + } else { + destQueue.add(packet) + } + } } }) dest.pipeline().addFirst(new ChannelOutboundHandlerAdapter() { @Override void read(ChannelHandlerContext ctx) throws Exception { - if (queue.isEmpty()) { + if (destQueue.isEmpty()) { readPending = true } else { - ctx.fireChannelRead(queue.poll()) + ByteBuf msg = destQueue.poll() + ctx.channel().eventLoop().execute(() -> forwardNow(msg)) } } }) From 69891544bf351ef0dd3c14a2eee66f5ce54aac2e Mon Sep 17 00:00:00 2001 From: yawkat Date: Tue, 27 Sep 2022 11:57:15 +0200 Subject: [PATCH 30/82] fix publisher requests --- .../http/client/netty/ConnectionManager.java | 26 +++++++------- .../http/client/netty/DefaultHttpClient.java | 5 +-- .../client/netty/ConnectionManagerSpec.groovy | 34 +++++++++++++++++++ .../stream/HttpStreamsClientHandler.java | 1 + 4 files changed, 48 insertions(+), 18 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 21c979791db..feae1a94575 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -354,22 +354,20 @@ protected void finishAggregation(FullHttpMessage aggregated) throws Exception { } } }); - if (multipart) { - ph.channel.pipeline().addLast( - ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, - new HttpStreamsClientHandler() { - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - if (evt instanceof IdleStateEvent) { - // close the connection if it is idle for too long - ph.taint(); - ph.release(); - } - super.userEventTriggered(ctx, evt); + ph.channel.pipeline().addLast( + ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, + new HttpStreamsClientHandler() { + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof IdleStateEvent) { + // close the connection if it is idle for too long + ph.taint(); + ph.release(); } + super.userEventTriggered(ctx, evt); } - ); - } + } + ); return ph; }); } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index e1095b8db5d..7db0c91b49a 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -2278,10 +2278,7 @@ public void handlerRemoved(ChannelHandlerContext ctx) { ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP_CHUNK); } catch (NoSuchElementException ignored) { } - try { - ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); - } catch (NoSuchElementException ignored) { - } + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); poolHandle.release(); } diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index b08750ac6c8..00d24dac912 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -738,6 +738,40 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'publisher request'() { + def ctx = ApplicationContext.run() + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp1() + conn.serverChannel.pipeline().addLast(new LoggingHandler(LogLevel.INFO)) // TODO + conn.setupHttp1() + patch(client, conn.clientChannel) + conn.serverChannel.pipeline().addLast(new HttpObjectAggregator(1024)) + + def future = Mono.from(client.exchange(HttpRequest.POST(conn.scheme + '://example.com/foo', Flux.fromIterable([1,2,3,4,5])) + .contentType(MediaType.APPLICATION_JSON_TYPE), String)).toFuture() + future.exceptionally(t -> t.printStackTrace()) + conn.advance() + conn.fireClientActive() + + FullHttpRequest request = conn.serverChannel.readInbound() + assert request.uri() == '/foo' + assert request.method() == HttpMethod.POST + assert request.headers().get('host') == 'example.com' + assert request.headers().get("connection") == "keep-alive" + assert request.content().toString(StandardCharsets.UTF_8) == '[1,2,3,4,5]' + + def response = new DefaultFullHttpResponse(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer('foo'.bytes)) + response.headers().add("Content-Length", 3) + conn.serverChannel.writeOutbound(response) + conn.advance() + assert future.get().body() == 'foo' + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnectionBase { final EmbeddedChannel serverChannel final EmbeddedChannel clientChannel diff --git a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java index fecd63000fc..46346a7341b 100644 --- a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java +++ b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java @@ -189,6 +189,7 @@ protected int getStreamId(Object msg) { @Override public void write(final ChannelHandlerContext ctx, Object msg, final ChannelPromise promise) throws Exception { + // todo if (ctx.channel().attr(AttributeKey.valueOf(ChannelPipelineCustomizer.HANDLER_HTTP_CHUNK)).get() == Boolean.TRUE) { ctx.write(msg, promise); } else { From 39973e6bcc741996fac2cfc257557a062e922f18 Mon Sep 17 00:00:00 2001 From: yawkat Date: Tue, 27 Sep 2022 14:32:35 +0200 Subject: [PATCH 31/82] add missing read call in server --- .../http/netty/stream/HttpStreamsServerHandler.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java index f819aad2bc3..7447e176796 100644 --- a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java +++ b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java @@ -23,7 +23,16 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; import io.netty.channel.ChannelPromise; -import io.netty.handler.codec.http.*; +import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpContent; +import io.netty.handler.codec.http.HttpHeaderNames; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpUtil; +import io.netty.handler.codec.http.HttpVersion; import io.netty.handler.codec.http.websocketx.WebSocketFrame; import io.netty.handler.codec.http.websocketx.WebSocketServerHandshaker; import io.netty.handler.codec.http.websocketx.WebSocketVersion; @@ -211,6 +220,8 @@ protected void consumedInMessage(ChannelHandlerContext ctx) { webSocketResponse = null; webSocketResponseChannelPromise = null; } + // read next request + ctx.read(); } private void handleWebSocketResponse(ChannelHandlerContext ctx, HttpResponse message, ChannelPromise promise) { From f43b0b1b27a896a8efefeb822241b58f856fd9d6 Mon Sep 17 00:00:00 2001 From: yawkat Date: Tue, 27 Sep 2022 15:11:21 +0200 Subject: [PATCH 32/82] improve error messages --- .../http/client/netty/ConnectionManager.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index feae1a94575..13ff07a1bb8 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -875,13 +875,22 @@ private void acquire(Sinks.One sink) { @Override void onNewConnectionFailure(@Nullable Throwable error) throws Exception { - log.error("Failed to connect to remote", error); super.onNewConnectionFailure(error); // to avoid an infinite loop, fail one pending request. Sinks.One pending = pollPendingRequest(); if (pending != null) { - pending.tryEmitError(new HttpClientException("Failed to establish connection", error)); + HttpClientException wrapped; + if (error == null) { + wrapped = new HttpClientException("Unknown connect error"); + } else { + wrapped = new HttpClientException("Connect Error: " + error.getMessage(), error); + } + if (pending.tryEmitError(customizeException(wrapped)) == Sinks.EmitResult.OK) { + // no need to log + return; + } } + log.error("Failed to connect to remote", error); } private void addPendingRequest(Sinks.One sink) { From 0f43b5dd8e129df5368806ce3ca5d59d5b109df0 Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 11:28:35 +0200 Subject: [PATCH 33/82] fix http1 pool sizing --- .../main/java/io/micronaut/http/client/netty/PoolResizer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java index 89a90925fc7..74ba6faeefe 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java @@ -90,7 +90,7 @@ private void doSomeWork() { connectionsToOpen = Math.min(connectionsToOpen, connectionPoolConfiguration.getMaxPendingConnections() - pendingConnectionCount); // limit the connection count to the protocol-specific settings, but only if that protocol was seen for this pool. if (http1ConnectionCount > 0) { - connectionsToOpen = Math.min(connectionsToOpen, connectionPoolConfiguration.getMaxConcurrentHttp2Connections() - http1ConnectionCount); + connectionsToOpen = Math.min(connectionsToOpen, connectionPoolConfiguration.getMaxConcurrentHttp1Connections() - http1ConnectionCount); } if (http2ConnectionCount > 0) { connectionsToOpen = Math.min(connectionsToOpen, connectionPoolConfiguration.getMaxConcurrentHttp2Connections() - http2ConnectionCount); From 88c0f7ab1ea076dfeea073d97fcbc6b256929ce6 Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 12:11:16 +0200 Subject: [PATCH 34/82] move request dispatch logic into PoolResizer to avoid concurrency issues --- .../http/client/netty/ConnectionManager.java | 95 ++------------ .../http/client/netty/PoolResizer.java | 122 ++++++++++++++---- 2 files changed, 112 insertions(+), 105 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 13ff07a1bb8..286e99eb781 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -102,15 +102,11 @@ import java.time.Duration; import java.util.Collection; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.OptionalInt; -import java.util.Queue; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -838,10 +834,6 @@ void notifyRequestPipelineBuilt() { private final class Pool extends PoolResizer { private final DefaultHttpClient.RequestKey requestKey; - private final Queue> pendingRequests = new ConcurrentLinkedQueue<>(); - private final List http1Connections = new CopyOnWriteArrayList<>(); - private final List http2Connections = new CopyOnWriteArrayList<>(); - Pool(DefaultHttpClient.RequestKey requestKey) { super(log, configuration.getConnectionPoolConfiguration()); this.requestKey = requestKey; @@ -849,7 +841,7 @@ private final class Pool extends PoolResizer { Mono acquire() { Sinks.One sink = new CancellableMonoSink<>(); - acquire(sink); + addPendingRequest(sink); Optional acquireTimeout = configuration.getConnectionPoolConfiguration().getAcquireTimeout(); if (acquireTimeout.isPresent()) { return sink.asMono().timeout(acquireTimeout.get(), getEventLoopScheduler()); @@ -858,21 +850,6 @@ Mono acquire() { } } - private void acquire(Sinks.One sink) { - for (Http2ConnectionHolder http2Connection : http2Connections) { - if (http2Connection.satisfy(sink)) { - return; - } - } - for (Http1ConnectionHolder http1Connection : http1Connections) { - if (http1Connection.satisfy(sink)) { - return; - } - } - // no connection open that has room - addPendingRequest(sink); - } - @Override void onNewConnectionFailure(@Nullable Throwable error) throws Exception { super.onNewConnectionFailure(error); @@ -893,20 +870,6 @@ void onNewConnectionFailure(@Nullable Throwable error) throws Exception { log.error("Failed to connect to remote", error); } - private void addPendingRequest(Sinks.One sink) { - pendingRequests.add(sink); - onPendingRequestChange(1); - } - - @Nullable - private Sinks.One pollPendingRequest() { - Sinks.One req = pendingRequests.poll(); - if (req != null) { - onPendingRequestChange(-1); - } - return req; - } - @Override void openNewConnection() { // open a new connection @@ -953,15 +916,10 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { } public void shutdown() { - for (Http1ConnectionHolder http1Connection : http1Connections) { - http1Connection.channel.close(); - } - for (Http2ConnectionHolder http2Connection : http2Connections) { - http2Connection.channel.close(); - } + forEachConnection(c -> ((ConnectionHolder) c).channel.close()); } - abstract class ConnectionHolder { + abstract class ConnectionHolder extends ResizerConnection { final Channel channel; final NettyClientCustomizer connectionCustomizer; ScheduledFuture ttlFuture; @@ -1009,20 +967,8 @@ void windDownConnection() { windDownConnection = true; } - final void satisfySomePendingRequests() { - while (true) { - Sinks.One pending = pollPendingRequest(); - if (pending == null) { - break; - } - if (!satisfy(pending)) { - addPendingRequest(pending); - break; - } - } - } - - boolean satisfy(Sinks.One sink) { + @Override + public boolean dispatch(Sinks.One sink) { if (!tryEarmarkForRequest()) { return false; } @@ -1070,13 +1016,7 @@ void init(boolean fireInitialPipelineBuilt) { } connectionCustomizer.onStreamPipelineBuilt(); - Sinks.One pendingRequest = pollPendingRequest(); - if (pendingRequest != null) { - hasLiveRequest.set(true); - satisfy0(pendingRequest); - } - http1Connections.add(this); - onNewConnectionEstablished1(); + onNewConnectionEstablished1(this); } @Override @@ -1119,7 +1059,7 @@ void release() { } if (!windDownConnection) { hasLiveRequest.set(false); - satisfySomePendingRequests(); + markConnectionAvailable(); } else { channel.close(); } @@ -1137,7 +1077,7 @@ void notifyRequestPipelineBuilt() { }); if (emitResult.isFailure()) { hasLiveRequest.set(false); - satisfySomePendingRequests(); + markConnectionAvailable(); } } @@ -1158,8 +1098,7 @@ void windDownConnection() { @Override void onInactive() { super.onInactive(); - http1Connections.remove(this); - onConnectionInactive1(); + onConnectionInactive1(this); } } @@ -1180,16 +1119,7 @@ void init() { connectionCustomizer.onStreamPipelineBuilt(); - for (int i = 0; i < configuration.getConnectionPoolConfiguration().getMaxConcurrentRequestsPerHttp2Connection(); i++) { - Sinks.One pendingRequest = pollPendingRequest(); - if (pendingRequest == null) { - break; - } - liveRequests.incrementAndGet(); - satisfy0(pendingRequest); - } - http2Connections.add(this); - onNewConnectionEstablished2(); + onNewConnectionEstablished2(this); } @Override @@ -1233,7 +1163,7 @@ void release() { liveStreamChannels.remove(streamChannel); streamChannel.close(); liveRequests.decrementAndGet(); - satisfySomePendingRequests(); + markConnectionAvailable(); } @Override @@ -1275,8 +1205,7 @@ void windDownConnection() { @Override void onInactive() { super.onInactive(); - http2Connections.remove(Http2ConnectionHolder.this); - onConnectionInactive2(); + onConnectionInactive2(this); } } } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java index 74ba6faeefe..16a5840aa76 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java @@ -19,13 +19,26 @@ import io.micronaut.core.annotation.Nullable; import io.micronaut.http.client.HttpClientConfiguration; import org.slf4j.Logger; +import reactor.core.publisher.Sinks; +import java.util.Deque; +import java.util.List; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; /** * This class handles the sizing of a connection pool to conform to the configuration in * {@link io.micronaut.http.client.HttpClientConfiguration.ConnectionPoolConfiguration}. + * + * @implNote This class consists of various mutator methods (e.g. {@link #addPendingRequest}) that + * may be called concurrently and in a reentrant fashion (e.g. inside {@link #openNewConnection()}). + * These mutator methods update their respective fields and then mark this class as + * {@link #dirty()}. The state management logic ensures that {@link #doSomeWork()} is called in a + * serialized fashion (no concurrency or reentrancy) at least once after each {@link #dirty()} + * call. */ @Internal abstract class PoolResizer { @@ -34,10 +47,11 @@ abstract class PoolResizer { private final AtomicReference state = new AtomicReference<>(WorkState.IDLE); - private final AtomicInteger pendingRequests = new AtomicInteger(0); private final AtomicInteger pendingConnectionCount = new AtomicInteger(0); - private final AtomicInteger http1ConnectionCount = new AtomicInteger(0); - private final AtomicInteger http2ConnectionCount = new AtomicInteger(0); + + private final Deque> pendingRequests = new ConcurrentLinkedDeque<>(); + private final List http1Connections = new CopyOnWriteArrayList<>(); + private final List http2Connections = new CopyOnWriteArrayList<>(); PoolResizer(Logger log, HttpClientConfiguration.ConnectionPoolConfiguration connectionPoolConfiguration) { this.log = log; @@ -58,7 +72,14 @@ private void dirty() { } // we were in idle state, this thread will handle the changes. while (true) { - doSomeWork(); + try { + doSomeWork(); + } catch (Throwable t) { + // this is probably an irrecoverable failure, we need to bail immediately, but + // avoid locking up the state. Another thread might be able to continue work. + state.set(WorkState.IDLE); + throw t; + } WorkState endState = state.updateAndGet(ws -> { if (ws == WorkState.ACTIVE_WITH_PENDING_WORK) { @@ -75,11 +96,37 @@ private void dirty() { } private void doSomeWork() { + while (true) { + Sinks.One toDispatch = pendingRequests.pollFirst(); + if (toDispatch == null) { + break; + } + boolean dispatched = false; + for (ResizerConnection c : http2Connections) { + if (c.dispatch(toDispatch)) { + dispatched = true; + break; + } + } + if (!dispatched) { + for (ResizerConnection c : http1Connections) { + if (c.dispatch(toDispatch)) { + dispatched = true; + break; + } + } + } + if (!dispatched) { + pendingRequests.addFirst(toDispatch); + break; + } + } + // snapshot our fields - int pendingRequests = this.pendingRequests.get(); + int pendingRequests = this.pendingRequests.size(); int pendingConnectionCount = this.pendingConnectionCount.get(); - int http1ConnectionCount = this.http1ConnectionCount.get(); - int http2ConnectionCount = this.http2ConnectionCount.get(); + int http1ConnectionCount = this.http1Connections.size(); + int http2ConnectionCount = this.http2Connections.size(); if (pendingRequests == 0) { // if there are no pending requests, there is nothing to do. @@ -127,13 +174,6 @@ static boolean incrementWithLimit(AtomicInteger variable, int limit) { } } - final void onPendingRequestChange(int delta) { - if (pendingRequests.addAndGet(delta) < 0) { - throw new IllegalStateException("Negative pending requests"); - } - dirty(); - } - // can be overridden, so `throws Exception` ensures we handle any errors void onNewConnectionFailure(@Nullable Throwable error) throws Exception { // todo: circuit breaker? @@ -141,31 +181,69 @@ void onNewConnectionFailure(@Nullable Throwable error) throws Exception { dirty(); } - final void onNewConnectionEstablished1() { - http1ConnectionCount.incrementAndGet(); + final void onNewConnectionEstablished1(ResizerConnection connection) { + http1Connections.add(connection); pendingConnectionCount.decrementAndGet(); dirty(); } - final void onNewConnectionEstablished2() { - http2ConnectionCount.incrementAndGet(); + final void onNewConnectionEstablished2(ResizerConnection connection) { + http2Connections.add(connection); pendingConnectionCount.decrementAndGet(); dirty(); } - final void onConnectionInactive1() { - http1ConnectionCount.decrementAndGet(); + final void onConnectionInactive1(ResizerConnection connection) { + http1Connections.remove(connection); dirty(); } - final void onConnectionInactive2() { - http2ConnectionCount.decrementAndGet(); + final void onConnectionInactive2(ResizerConnection connection) { + http2Connections.remove(connection); dirty(); } + final void addPendingRequest(Sinks.One sink) { + pendingRequests.addLast(sink); + dirty(); + } + + @Nullable + final Sinks.One pollPendingRequest() { + Sinks.One req = pendingRequests.pollFirst(); + if (req != null) { + dirty(); + } + return req; + } + + final void markConnectionAvailable() { + dirty(); + } + + final void forEachConnection(Consumer c) { + for (ResizerConnection http1Connection : http1Connections) { + c.accept(http1Connection); + } + for (ResizerConnection http2Connection : http2Connections) { + c.accept(http2Connection); + } + } + private enum WorkState { IDLE, ACTIVE_WITH_PENDING_WORK, ACTIVE_WITHOUT_PENDING_WORK, } + + abstract static class ResizerConnection { + /** + * Attempt to dispatch a stream on this connection. + * + * @param sink The pending request that wants to acquire this connection + * @return {@code true} if the acquisition may succeed (if it fails later, the pending + * request must be readded), or {@code false} if it fails immediately + */ + abstract boolean dispatch(Sinks.One sink); + } } From 0419a3efbb6ceaeb3e384806d551796ac632abf7 Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 12:16:57 +0200 Subject: [PATCH 35/82] improve error handling for dispatch --- .../http/client/netty/PoolResizer.java | 22 ++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java index 16a5840aa76..501c7ea05dd 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java @@ -103,14 +103,14 @@ private void doSomeWork() { } boolean dispatched = false; for (ResizerConnection c : http2Connections) { - if (c.dispatch(toDispatch)) { + if (dispatchSafe(c, toDispatch)) { dispatched = true; break; } } if (!dispatched) { for (ResizerConnection c : http1Connections) { - if (c.dispatch(toDispatch)) { + if (dispatchSafe(c, toDispatch)) { dispatched = true; break; } @@ -160,6 +160,22 @@ private void doSomeWork() { } } + private boolean dispatchSafe(ResizerConnection connection, Sinks.One toDispatch) { + try { + return connection.dispatch(toDispatch); + } catch (Exception e) { + try { + if (toDispatch.tryEmitError(e) != Sinks.EmitResult.OK) { + // this is probably fine, log it anyway + log.debug("Failure during connection dispatch operation, but dispatch request was already complete.", e); + } + } catch (Exception f) { + log.error("Internal error", f); + } + return true; + } + } + abstract void openNewConnection() throws Exception; static boolean incrementWithLimit(AtomicInteger variable, int limit) { @@ -244,6 +260,6 @@ abstract static class ResizerConnection { * @return {@code true} if the acquisition may succeed (if it fails later, the pending * request must be readded), or {@code false} if it fails immediately */ - abstract boolean dispatch(Sinks.One sink); + abstract boolean dispatch(Sinks.One sink) throws Exception; } } From e177515c53581cf7fb7bfb4e5687449797a2deef Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 12:34:28 +0200 Subject: [PATCH 36/82] improve handshake error handling --- .../http/client/netty/ConnectionManager.java | 54 +++++++++---------- .../netty/InitialConnectionErrorHandler.java | 32 +++++++++++ .../channel/ChannelPipelineCustomizer.java | 1 + 3 files changed, 60 insertions(+), 27 deletions(-) create mode 100644 http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 286e99eb781..636723aaa13 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -46,6 +46,7 @@ import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.handler.codec.DecoderException; import io.netty.handler.codec.LineBasedFrameDecoder; import io.netty.handler.codec.http.DefaultFullHttpRequest; import io.netty.handler.codec.http.DefaultHttpContent; @@ -96,6 +97,7 @@ import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; +import javax.net.ssl.SSLException; import java.net.InetSocketAddress; import java.net.Proxy; import java.net.SocketAddress; @@ -631,7 +633,6 @@ protected void initChannel(Channel ch) { configureProxy(ch.pipeline(), true, host, port); - InitialConnectionErrorHandler initialErrorHandler = new InitialConnectionErrorHandler(pool); SslHandler sslHandler = sslContext.newHandler(ch.alloc(), host, port); sslHandler.setHandshakeTimeoutMillis(configuration.getSslConfiguration().getHandshakeTimeout().toMillis()); ch.pipeline() @@ -644,44 +645,33 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { ctx.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, makeFrameCodec()); initHttp2(pool, ctx.channel(), channelCustomizer); - ctx.pipeline().remove(initialErrorHandler); + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR); } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { initHttp1(ctx.channel()); pool.new Http1ConnectionHolder(ch, channelCustomizer).init(false); - ctx.pipeline().remove(initialErrorHandler); + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR); } else { ctx.close(); throw customizeException(new HttpClientException("Unknown Protocol: " + protocol)); } } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + // let the HANDLER_INITIAL_ERROR handle the failure + if (cause instanceof DecoderException && cause.getCause() instanceof SSLException) { + // unwrap DecoderException + cause = cause.getCause(); + } + ctx.fireExceptionCaught(cause); + } }) - .addLast(initialErrorHandler); + .addLast(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR, pool.initialErrorHandler); channelCustomizer.onInitialPipelineBuilt(); } } - private class InitialConnectionErrorHandler extends ChannelInboundHandlerAdapter { - private final Pool pool; - private Throwable cause; - - InitialConnectionErrorHandler(Pool pool) { - this.pool = pool; - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - log.error("Failed to open connection", cause); - this.cause = cause; - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - super.channelInactive(ctx); - pool.onNewConnectionFailure(cause); - } - } - private void initHttp1(Channel ch) { addLogHandler(ch); @@ -718,11 +708,12 @@ protected void initChannel(Http2StreamChannel ch) throws Exception { } }); ch.pipeline().addLast(multiplexHandler); - ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new InitialConnectionErrorHandler(pool) { + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new ChannelInboundHandlerAdapter() { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { if (msg instanceof Http2SettingsFrame) { - ctx.pipeline().remove(this); + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS); + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR); pool.new Http2ConnectionHolder(ch, connectionCustomizer).init(); return; } else { @@ -732,6 +723,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception super.channelRead(ctx, msg); } }); + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR, pool.initialErrorHandler); // stream frames should be handled by the multiplexer ch.pipeline().addLast(new ChannelInboundHandlerAdapter() { @Override @@ -834,6 +826,13 @@ void notifyRequestPipelineBuilt() { private final class Pool extends PoolResizer { private final DefaultHttpClient.RequestKey requestKey; + final InitialConnectionErrorHandler initialErrorHandler = new InitialConnectionErrorHandler() { + @Override + protected void onNewConnectionFailure(@Nullable Throwable cause) throws Exception { + Pool.this.onNewConnectionFailure(cause); + } + }; + Pool(DefaultHttpClient.RequestKey requestKey) { super(log, configuration.getConnectionPoolConfiguration()); this.requestKey = requestKey; @@ -858,6 +857,7 @@ void onNewConnectionFailure(@Nullable Throwable error) throws Exception { if (pending != null) { HttpClientException wrapped; if (error == null) { + // no failure observed, but channel closed wrapped = new HttpClientException("Unknown connect error"); } else { wrapped = new HttpClientException("Connect Error: " + error.getMessage(), error); diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java b/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java new file mode 100644 index 00000000000..0e07983f605 --- /dev/null +++ b/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java @@ -0,0 +1,32 @@ +package io.micronaut.http.client.netty; + +import io.micronaut.core.annotation.Internal; +import io.micronaut.core.annotation.Nullable; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.util.AttributeKey; + +/** + * Handler for connection failures that happen during the handshake phases of a connection. + */ +@Internal +@ChannelHandler.Sharable +abstract class InitialConnectionErrorHandler extends ChannelInboundHandlerAdapter { + private static final AttributeKey FAILURE_KEY = + AttributeKey.valueOf(InitialConnectionErrorHandler.class, "FAILURE_KEY"); + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + ctx.channel().attr(FAILURE_KEY).set(cause); + ctx.close(); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + super.channelInactive(ctx); + onNewConnectionFailure(ctx.channel().attr(FAILURE_KEY).get()); + } + + protected abstract void onNewConnectionFailure(@Nullable Throwable cause) throws Exception; +} diff --git a/http-netty/src/main/java/io/micronaut/http/netty/channel/ChannelPipelineCustomizer.java b/http-netty/src/main/java/io/micronaut/http/netty/channel/ChannelPipelineCustomizer.java index 9883e5779dc..f2e28c9e17a 100644 --- a/http-netty/src/main/java/io/micronaut/http/netty/channel/ChannelPipelineCustomizer.java +++ b/http-netty/src/main/java/io/micronaut/http/netty/channel/ChannelPipelineCustomizer.java @@ -56,6 +56,7 @@ public interface ChannelPipelineCustomizer { String HANDLER_WEBSOCKET_UPGRADE = "websocket-upgrade-handler"; String HANDLER_MICRONAUT_INBOUND = "micronaut-inbound-handler"; String HANDLER_ACCESS_LOGGER = "http-access-logger"; + String HANDLER_INITIAL_ERROR = "initial-error"; /** * @return Is this customizer the client. From dc507361edfad5d629ba0ab0bfb02336937cbc5c Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 12:42:12 +0200 Subject: [PATCH 37/82] fix ssl handshake timeout error --- .../http/client/netty/ConnectionManager.java | 12 ++++++++++++ .../client/netty/InitialConnectionErrorHandler.java | 7 ++++++- 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 636723aaa13..2d27d655ec4 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -80,6 +80,7 @@ import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; +import io.netty.handler.ssl.SslHandshakeCompletionEvent; import io.netty.handler.timeout.IdleStateEvent; import io.netty.handler.timeout.IdleStateHandler; import io.netty.handler.timeout.ReadTimeoutException; @@ -656,6 +657,17 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { } } + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof SslHandshakeCompletionEvent) { + SslHandshakeCompletionEvent event = (SslHandshakeCompletionEvent) evt; + if (!event.isSuccess()) { + InitialConnectionErrorHandler.setFailureCause(ctx.channel(), event.cause()); + } + } + super.userEventTriggered(ctx, evt); + } + @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { // let the HANDLER_INITIAL_ERROR handle the failure diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java b/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java index 0e07983f605..0c333c343e7 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java @@ -2,6 +2,7 @@ import io.micronaut.core.annotation.Internal; import io.micronaut.core.annotation.Nullable; +import io.netty.channel.Channel; import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; @@ -18,10 +19,14 @@ abstract class InitialConnectionErrorHandler extends ChannelInboundHandlerAdapte @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - ctx.channel().attr(FAILURE_KEY).set(cause); + setFailureCause(ctx.channel(), cause); ctx.close(); } + static void setFailureCause(Channel channel, Throwable cause) { + channel.attr(FAILURE_KEY).set(cause); + } + @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { super.channelInactive(ctx); From ea74a0bc5b6a509a22229bfccf404a8b642cf73b Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 12:44:02 +0200 Subject: [PATCH 38/82] fix duplicate handler --- .../java/io/micronaut/http/client/netty/ConnectionManager.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 2d27d655ec4..ad616144363 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -646,7 +646,6 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { ctx.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, makeFrameCodec()); initHttp2(pool, ctx.channel(), channelCustomizer); - ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR); } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { initHttp1(ctx.channel()); pool.new Http1ConnectionHolder(ch, channelCustomizer).init(false); @@ -735,7 +734,6 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception super.channelRead(ctx, msg); } }); - ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR, pool.initialErrorHandler); // stream frames should be handled by the multiplexer ch.pipeline().addLast(new ChannelInboundHandlerAdapter() { @Override @@ -795,6 +793,7 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { super.channelActive(ctx); } }); + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR, pool.initialErrorHandler); connectionCustomizer.onInitialPipelineBuilt(); } From 4a21439605adc1ef625655aa56e43b080d087586 Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 13:02:12 +0200 Subject: [PATCH 39/82] fix http version selection --- .../java/io/micronaut/http/client/HttpVersionSelection.java | 2 +- .../io/micronaut/http/client/http2/Http2RequestSpec.groovy | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java index 21f0f5d9b6c..922e85322d9 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java @@ -48,7 +48,7 @@ public final class HttpVersionSelection { ); private static final HttpVersionSelection LEGACY_2 = new HttpVersionSelection( - PlaintextMode.HTTP_1, + PlaintextMode.H2C, true, new String[]{ALPN_HTTP_1, ALPN_HTTP_2}, true diff --git a/test-suite/src/test/groovy/io/micronaut/http/client/http2/Http2RequestSpec.groovy b/test-suite/src/test/groovy/io/micronaut/http/client/http2/Http2RequestSpec.groovy index 88407a6ecd4..561bb44d1e8 100644 --- a/test-suite/src/test/groovy/io/micronaut/http/client/http2/Http2RequestSpec.groovy +++ b/test-suite/src/test/groovy/io/micronaut/http/client/http2/Http2RequestSpec.groovy @@ -152,6 +152,7 @@ class Http2RequestSpec extends Specification { "micronaut.server.http-version" : "2.0", 'micronaut.server.ssl.buildSelfSigned': true, 'micronaut.server.ssl.port': -1, + "micronaut.http.client.http-version" : "1.1", "micronaut.http.client.log-level" : "TRACE", "micronaut.server.netty.log-level" : "TRACE", 'micronaut.http.client.ssl.insecure-trust-all-certificates': true @@ -198,6 +199,7 @@ class Http2RequestSpec extends Specification { "micronaut.server.http-version" : "2.0", 'micronaut.server.ssl.buildSelfSigned': true, 'micronaut.server.ssl.port': -1, + "micronaut.http.client.http-version" : "1.1", "micronaut.http.client.log-level" : "TRACE", "micronaut.server.netty.log-level" : "TRACE" ]) From 40693298c6b24f692e925b7564bb77abaedf024b Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 13:05:50 +0200 Subject: [PATCH 40/82] remove obsolete Http2Stream handling for sse --- .../http/client/netty/ConnectionManager.java | 38 +------------------ 1 file changed, 2 insertions(+), 36 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index ad616144363..8d249aad454 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -26,8 +26,6 @@ import io.micronaut.http.netty.channel.ChannelPipelineCustomizer; import io.micronaut.http.netty.channel.ChannelPipelineListener; import io.micronaut.http.netty.channel.NettyThreadFactory; -import io.micronaut.http.netty.stream.DefaultHttp2Content; -import io.micronaut.http.netty.stream.Http2Content; import io.micronaut.http.netty.stream.HttpStreamsClientHandler; import io.micronaut.scheduling.instrument.Instrumentation; import io.micronaut.scheduling.instrument.InvocationInstrumenter; @@ -53,7 +51,6 @@ import io.netty.handler.codec.http.FullHttpMessage; import io.netty.handler.codec.http.HttpClientCodec; import io.netty.handler.codec.http.HttpClientUpgradeHandler; -import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.HttpContentDecompressor; import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpMethod; @@ -69,7 +66,6 @@ import io.netty.handler.codec.http2.Http2FrameLogger; import io.netty.handler.codec.http2.Http2MultiplexHandler; import io.netty.handler.codec.http2.Http2SettingsFrame; -import io.netty.handler.codec.http2.Http2Stream; import io.netty.handler.codec.http2.Http2StreamChannel; import io.netty.handler.codec.http2.Http2StreamChannelBootstrap; import io.netty.handler.codec.http2.Http2StreamFrameToHttpObjectCodec; @@ -86,7 +82,6 @@ import io.netty.handler.timeout.ReadTimeoutException; import io.netty.handler.timeout.ReadTimeoutHandler; import io.netty.resolver.NoopAddressResolverGroup; -import io.netty.util.Attribute; import io.netty.util.AttributeKey; import io.netty.util.ReferenceCountUtil; import io.netty.util.concurrent.Future; @@ -123,7 +118,6 @@ class ConnectionManager { private static final AttributeKey CHANNEL_CUSTOMIZER_KEY = AttributeKey.valueOf("micronaut.http.customizer"); - private static final AttributeKey STREAM_KEY = AttributeKey.valueOf("micronaut.http2.stream"); final InvocationInstrumenter instrumenter; final HttpVersionSelection httpVersion; @@ -414,29 +408,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } }); if (sse) { - ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_EVENT_STREAM, new LineBasedFrameDecoder(configuration.getMaxContentLength(), true, true) { - - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof HttpContent) { - if (msg instanceof LastHttpContent) { - super.channelRead(ctx, msg); - } else { - Attribute streamKey = ctx.channel().attr(STREAM_KEY); - if (msg instanceof Http2Content) { - streamKey.set(((Http2Content) msg).stream()); - } - try { - super.channelRead(ctx, ((HttpContent) msg).content()); - } finally { - streamKey.set(null); - } - } - } else { - super.channelRead(ctx, msg); - } - } - }); + ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_EVENT_STREAM, new LineBasedFrameDecoder(configuration.getMaxContentLength(), true, true)); ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_CONTENT, new SimpleChannelInboundHandlerInstrumented(instrumenter, false) { @Override @@ -447,13 +419,7 @@ public boolean acceptInboundMessage(Object msg) { @Override protected void channelReadInstrumented(ChannelHandlerContext ctx, ByteBuf msg) { try { - Attribute streamKey = ctx.channel().attr(STREAM_KEY); - Http2Stream http2Stream = streamKey.get(); - if (http2Stream != null) { - ctx.fireChannelRead(new DefaultHttp2Content(msg.copy(), http2Stream)); - } else { - ctx.fireChannelRead(new DefaultHttpContent(msg.copy())); - } + ctx.fireChannelRead(new DefaultHttpContent(msg.copy())); } finally { msg.release(); } From 84a90070123c5c01cdc74d03da9a76f8241a1118 Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 13:10:24 +0200 Subject: [PATCH 41/82] fix merge --- .../http/netty/stream/HttpStreamsClientHandler.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java index 46346a7341b..4367e1615bb 100644 --- a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java +++ b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java @@ -181,12 +181,6 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } } - @Override - protected int getStreamId(Object msg) { - // TODO - return -1; - } - @Override public void write(final ChannelHandlerContext ctx, Object msg, final ChannelPromise promise) throws Exception { // todo From a106f2023daf4b4957063b71c247c4f5260e87f7 Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 13:49:31 +0200 Subject: [PATCH 42/82] fix sse decoding --- .../http/client/netty/ConnectionManager.java | 24 +------ .../netty/HttpLineBasedFrameDecoder.java | 65 +++++++++++++++++++ 2 files changed, 67 insertions(+), 22 deletions(-) create mode 100644 http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 1683f04d672..e66a1ab4313 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -31,7 +31,6 @@ import io.micronaut.scheduling.instrument.InvocationInstrumenter; import io.micronaut.websocket.exceptions.WebSocketSessionException; import io.netty.bootstrap.Bootstrap; -import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.Channel; import io.netty.channel.ChannelFactory; @@ -45,9 +44,7 @@ import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.handler.codec.DecoderException; -import io.netty.handler.codec.LineBasedFrameDecoder; import io.netty.handler.codec.http.DefaultFullHttpRequest; -import io.netty.handler.codec.http.DefaultHttpContent; import io.netty.handler.codec.http.FullHttpMessage; import io.netty.handler.codec.http.HttpClientCodec; import io.netty.handler.codec.http.HttpClientUpgradeHandler; @@ -396,8 +393,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception ignoreOneLast = false; } else { if (sse) { - ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_EVENT_STREAM); - ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_CONTENT); + ctx.pipeline().remove(HttpLineBasedFrameDecoder.NAME); } ctx.pipeline() .remove(this) @@ -408,23 +404,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } }); if (sse) { - ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_EVENT_STREAM, new LineBasedFrameDecoder(configuration.getMaxContentLength(), true, true)); - ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_CONTENT, new SimpleChannelInboundHandlerInstrumented(instrumenter, false) { - - @Override - public boolean acceptInboundMessage(Object msg) { - return msg instanceof ByteBuf; - } - - @Override - protected void channelReadInstrumented(ChannelHandlerContext ctx, ByteBuf msg) { - try { - ctx.fireChannelRead(new DefaultHttpContent(msg.copy())); - } finally { - msg.release(); - } - } - }); + ph.channel.pipeline().addLast(HttpLineBasedFrameDecoder.NAME, new HttpLineBasedFrameDecoder(configuration.getMaxContentLength(), true, true)); } ph.channel.pipeline().addLast( ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java b/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java new file mode 100644 index 00000000000..af7ba6d206a --- /dev/null +++ b/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java @@ -0,0 +1,65 @@ +package io.micronaut.http.client.netty; + +import io.micronaut.core.annotation.Internal; +import io.micronaut.core.annotation.NonNull; +import io.micronaut.http.netty.channel.ChannelPipelineCustomizer; +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.handler.codec.LineBasedFrameDecoder; +import io.netty.handler.codec.http.DefaultHttpContent; +import io.netty.handler.codec.http.HttpContent; +import io.netty.handler.codec.http.LastHttpContent; + +/** + * Variant of {@link LineBasedFrameDecoder} that accepts + * {@link io.netty.handler.codec.http.HttpContent} data. Note: this loses {@link LastHttpContent}. + */ +@Internal +final class HttpLineBasedFrameDecoder extends LineBasedFrameDecoder { + static final String NAME = ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_EVENT_STREAM; + + public HttpLineBasedFrameDecoder(int maxLength, boolean stripDelimiter, boolean failFast) { + super(maxLength, stripDelimiter, failFast); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof HttpContent) { + super.channelRead(ctx, ((HttpContent) msg).content()); + } else { + ctx.fireChannelRead(msg); + } + } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) { + ctx.pipeline().addAfter(NAME, Wrap.NAME, Wrap.INSTANCE); + } + + @Override + protected void handlerRemoved0(ChannelHandlerContext ctx) { + ctx.pipeline().remove(Wrap.NAME); + } + + @Sharable + private static class Wrap extends ChannelInboundHandlerAdapter { + static final ChannelHandler INSTANCE = new Wrap(); + static final String NAME = ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_CONTENT; + + @Override + public void channelRead(@NonNull ChannelHandlerContext ctx, @NonNull Object msg) throws Exception { + if (msg instanceof ByteBuf) { + ByteBuf buffer = (ByteBuf) msg; + // todo: this is necessary because downstream handlers sometimes do the + // `if (refcnt > 0) release` pattern. We should eventually fix that. + ByteBuf copy = buffer.copy(); + buffer.release(); + ctx.fireChannelRead(new DefaultHttpContent(copy)); + } else { + ctx.fireChannelRead(msg); + } + } + } +} From d18a4df61f037b8a6000fb8e9fb681e6bb760f5e Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 13:52:32 +0200 Subject: [PATCH 43/82] comments on PoolResizer --- .../io/micronaut/http/client/netty/PoolResizer.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java index 501c7ea05dd..0995b4070e2 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java @@ -247,8 +247,19 @@ final void forEachConnection(Consumer c) { } private enum WorkState { + /** + * There are no pending changes, and nobody is currently executing {@link #doSomeWork()}. + */ IDLE, + /** + * Someone is currently executing {@link #doSomeWork()}, but there were further changes + * after {@link #doSomeWork()} was called, so it needs to be called again. + */ ACTIVE_WITH_PENDING_WORK, + /** + * Someone is currently executing {@link #doSomeWork()}, and there were no other changes + * since then. + */ ACTIVE_WITHOUT_PENDING_WORK, } From 94441bf873fe8604ccda77eca94709442eed4fea Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 13:56:00 +0200 Subject: [PATCH 44/82] move HttpLineBasedFrameDecoder removal logic --- .../http/client/netty/ConnectionManager.java | 3 --- .../netty/HttpLineBasedFrameDecoder.java | 23 ++++++++++++++++++- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index e66a1ab4313..deab5c94eca 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -392,9 +392,6 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception if (ignoreOneLast) { ignoreOneLast = false; } else { - if (sse) { - ctx.pipeline().remove(HttpLineBasedFrameDecoder.NAME); - } ctx.pipeline() .remove(this) .remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java b/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java index af7ba6d206a..08678346ad5 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java @@ -10,27 +10,48 @@ import io.netty.handler.codec.LineBasedFrameDecoder; import io.netty.handler.codec.http.DefaultHttpContent; import io.netty.handler.codec.http.HttpContent; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.LastHttpContent; /** * Variant of {@link LineBasedFrameDecoder} that accepts - * {@link io.netty.handler.codec.http.HttpContent} data. Note: this loses {@link LastHttpContent}. + * {@link io.netty.handler.codec.http.HttpContent} data. Note: this handler removes itself when the + * response has been consumed. */ @Internal final class HttpLineBasedFrameDecoder extends LineBasedFrameDecoder { static final String NAME = ChannelPipelineCustomizer.HANDLER_MICRONAUT_SSE_EVENT_STREAM; + private boolean ignoreOneLast = false; + public HttpLineBasedFrameDecoder(int maxLength, boolean stripDelimiter, boolean failFast) { super(maxLength, stripDelimiter, failFast); } @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof HttpResponse && + ((HttpResponse) msg).status().equals(HttpResponseStatus.CONTINUE)) { + ignoreOneLast = true; + } + if (msg instanceof HttpContent) { super.channelRead(ctx, ((HttpContent) msg).content()); } else { ctx.fireChannelRead(msg); } + + if (msg instanceof LastHttpContent) { + if (ignoreOneLast) { + ignoreOneLast = false; + } else { + // first, remove the handler so that LineBasedFrameDecoder flushes any further + // data. Then forward the LastHttpContent. + ctx.pipeline().remove(NAME); + ctx.fireChannelRead(LastHttpContent.EMPTY_LAST_CONTENT); + } + } } @Override From 0431175333f135765bfe69944e9d9eec5083af63 Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 17:58:10 +0200 Subject: [PATCH 45/82] implement pool disable option --- .../http/client/HttpClientConfiguration.java | 2 +- .../http/client/netty/ConnectionManager.java | 51 +++-- .../http/client/SslRefreshSpec.groovy | 3 +- .../client/netty/ConnectionManagerSpec.groovy | 212 +++++++++++------- 4 files changed, 176 insertions(+), 92 deletions(-) diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java index 82f6038c398..1a906d9648e 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java @@ -720,7 +720,7 @@ public static class ConnectionPoolConfiguration implements Toggleable { * The default enable value. */ @SuppressWarnings("WeakerAccess") - public static final boolean DEFAULT_ENABLED = false; + public static final boolean DEFAULT_ENABLED = true; /** * The default max connections value. diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index deab5c94eca..b3fba7cdc7c 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -909,10 +909,23 @@ protected void readTimedOut(ChannelHandlerContext ctx) throws Exception { configuration.getConnectTtl().ifPresent(ttl -> ttlFuture = channel.eventLoop().schedule(this::windDownConnection, ttl.toNanos(), TimeUnit.NANOSECONDS)); channel.pipeline().addBefore(before, "connection-cleaner", new ChannelInboundHandlerAdapter() { + boolean inactiveCalled = false; + @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { super.channelInactive(ctx); - onInactive(); + if (!inactiveCalled) { + inactiveCalled = true; + onInactive(); + } + } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + if (!inactiveCalled) { + inactiveCalled = true; + onInactive(); + } } }); } @@ -921,6 +934,18 @@ void windDownConnection() { windDownConnection = true; } + final void emitPoolHandle(Sinks.One sink, PoolHandle ph) { + Sinks.EmitResult emitResult = sink.tryEmitValue(ph); + if (emitResult.isFailure()) { + ph.release(); + } else { + if (!configuration.getConnectionPoolConfiguration().isEnabled()) { + // if pooling is off, release the connection after this. + windDownConnection(); + } + } + } + @Override public boolean dispatch(Sinks.One sink) { if (!tryEarmarkForRequest()) { @@ -994,7 +1019,7 @@ void satisfy0(Sinks.One sink) { returnPendingRequest(sink); return; } - Sinks.EmitResult emitResult = sink.tryEmitValue(new PoolHandle(false, channel) { + PoolHandle ph = new PoolHandle(false, channel) { final ChannelHandlerContext lastContext = channel.pipeline().lastContext(); @Override @@ -1028,11 +1053,8 @@ boolean canReturn() { void notifyRequestPipelineBuilt() { connectionCustomizer.onRequestPipelineBuilt(); } - }); - if (emitResult.isFailure()) { - hasLiveRequest.set(false); - markConnectionAvailable(); - } + }; + emitPoolHandle(sink, ph); } private void returnPendingRequest(Sinks.One sink) { @@ -1095,7 +1117,7 @@ void fireReadTimeout(ChannelHandlerContext ctx) { @Override void satisfy0(Sinks.One sink) { - if (!channel.isActive()) { + if (!channel.isActive() || windDownConnection) { returnPendingRequest(sink); return; } @@ -1116,8 +1138,12 @@ void taint() { void release() { liveStreamChannels.remove(streamChannel); streamChannel.close(); - liveRequests.decrementAndGet(); - markConnectionAvailable(); + int newCount = liveRequests.decrementAndGet(); + if (windDownConnection && newCount <= 0) { + Http2ConnectionHolder.this.channel.close(); + } else { + markConnectionAvailable(); + } } @Override @@ -1131,10 +1157,7 @@ void notifyRequestPipelineBuilt() { } }; liveStreamChannels.add(streamChannel); - Sinks.EmitResult emitResult = sink.tryEmitValue(ph); - if (emitResult.isFailure()) { - ph.release(); - } + emitPoolHandle(sink, ph); } else { log.debug("Failed to open http2 stream", future.cause()); returnPendingRequest(sink); diff --git a/http-client/src/test/groovy/io/micronaut/http/client/SslRefreshSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/SslRefreshSpec.groovy index 40a68f9eb8b..28335c1c778 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/SslRefreshSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/SslRefreshSpec.groovy @@ -43,7 +43,8 @@ class SslRefreshSpec extends Specification { 'micronaut.http.client.ssl.client-authentication': 'NEED', 'micronaut.http.client.ssl.key-store.path': 'classpath:certs/client1.p12', 'micronaut.http.client.ssl.key-store.password': 'secret', - 'micronaut.http.client.ssl.insecure-trust-all-certificates': true + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + 'micronaut.http.client.pool.enabled': false, ] @Shared @AutoCleanup EmbeddedServer embeddedServer = ApplicationContext .builder() diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 00d24dac912..3d8a7a32857 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -27,6 +27,7 @@ import io.netty.channel.ChannelHandlerContext import io.netty.channel.ChannelId import io.netty.channel.ChannelInboundHandlerAdapter import io.netty.channel.ChannelInitializer +import io.netty.channel.ChannelPromise import io.netty.channel.ServerChannel import io.netty.channel.embedded.EmbeddedChannel import io.netty.handler.codec.http.DefaultFullHttpResponse @@ -63,6 +64,7 @@ import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler import io.netty.handler.ssl.SslContextBuilder import io.netty.handler.ssl.util.SelfSignedCertificate import io.netty.util.AsciiString +import io.netty.util.concurrent.GenericFutureListener import jakarta.inject.Singleton import org.spockframework.runtime.model.parallel.ExecutionMode import reactor.core.publisher.Flux @@ -80,17 +82,23 @@ import java.util.zip.GZIPOutputStream @Execution(ExecutionMode.CONCURRENT) class ConnectionManagerSpec extends Specification { - private static void patch(DefaultHttpClient httpClient, EmbeddedChannel... channels) { + private static void patch(DefaultHttpClient httpClient, EmbeddedTestConnectionBase... connections) { httpClient.connectionManager = new ConnectionManager(httpClient.connectionManager) { int i = 0 @Override protected ChannelFuture doConnect(DefaultHttpClient.RequestKey requestKey, ChannelInitializer channelInitializer) { - def channel = channels[i++] - channel.pipeline().addLast(channelInitializer) - def promise = channel.newPromise() - promise.setSuccess() - return promise + try { + def connection = connections[i++] + connection.clientChannel = new EmbeddedChannel(new DummyChannelId('client' + i), connection.clientInitializer, channelInitializer) + def promise = connection.clientChannel.newPromise() + promise.setSuccess() + return promise + } catch (Throwable t) { + // print it immediately to make sure it's not swallowed + t.printStackTrace() + throw t + } } } } @@ -103,7 +111,7 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp2() conn.setupHttp2Tls() - patch(client, conn.clientChannel) + patch(client, conn) def future = conn.testExchangeRequest(client) conn.exchangeSettings() @@ -122,7 +130,7 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp2() conn.setupHttp2Tls() - patch(client, conn.clientChannel) + patch(client, conn) def r1 = conn.testStreamingRequest(client) conn.exchangeSettings() @@ -139,11 +147,9 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() - patch(client, conn.clientChannel) + patch(client, conn) - def request = conn.testExchangeRequest(client) - conn.fireClientActive() - conn.testExchangeResponse(request) + conn.testExchangeResponse(conn.testExchangeRequest(client)) cleanup: client.close() @@ -157,13 +163,12 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() conn.serverChannel.pipeline().addLast(new HttpContentCompressor()) - patch(client, conn.clientChannel) + patch(client, conn) def future = Mono.from(client.exchange( HttpRequest.GET('http://example.com/foo').header('accept-encoding', 'gzip'), String)).toFuture() future.exceptionally(t -> t.printStackTrace()) conn.advance() - conn.fireClientActive() assert conn.serverChannel.readInbound() instanceof io.netty.handler.codec.http.HttpRequest @@ -188,7 +193,7 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp2() conn.setupHttp2Tls() - patch(client, conn.clientChannel) + patch(client, conn) def future = Mono.from(client.exchange('https://example.com/foo', String)).toFuture() future.exceptionally(t -> t.printStackTrace()) @@ -223,7 +228,7 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1Tls() - patch(client, conn.clientChannel) + patch(client, conn) conn.testExchangeResponse(conn.testExchangeRequest(client)) @@ -240,7 +245,7 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp2() conn.setupH2c() - patch(client, conn.clientChannel) + patch(client, conn) def future = conn.testExchangeRequest(client) conn.exchangeH2c() @@ -257,11 +262,9 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() - patch(client, conn.clientChannel) + patch(client, conn) - Queue responseData = conn.testStreamingRequest(client) - conn.fireClientActive() - conn.testStreamingResponse(responseData) + conn.testStreamingResponse(conn.testStreamingRequest(client)) cleanup: client.close() @@ -279,7 +282,7 @@ class ConnectionManagerSpec extends Specification { conn1.setupHttp2Tls() def conn2 = new EmbeddedTestConnectionHttp2() conn2.setupHttp2Tls() - patch(client, conn1.clientChannel, conn2.clientChannel) + patch(client, conn1, conn2) when: // start two requests. this will open two connections @@ -341,11 +344,10 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() - patch(client, conn.clientChannel) + patch(client, conn) + + conn.testExchangeResponse(conn.testExchangeRequest(client)) - def r1 = conn.testExchangeRequest(client) - conn.fireClientActive() - conn.testExchangeResponse(r1) Queue responseData1 = conn.testStreamingRequest(client) conn.testStreamingResponse(responseData1) conn.testExchangeResponse(conn.testExchangeRequest(client)) @@ -365,12 +367,11 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() - patch(client, conn.clientChannel) + patch(client, conn) when: - def r1 = conn.testExchangeRequest(client) - conn.fireClientActive() - conn.testExchangeResponse(r1) + conn.testExchangeResponse(conn.testExchangeRequest(client)) + Queue responseData = conn.testStreamingRequest(client) conn.testStreamingResponse(responseData) @@ -420,7 +421,7 @@ class ConnectionManagerSpec extends Specification { } else { conn.setupH2c() } - patch(client, conn.clientChannel) + patch(client, conn) when: def r1 = conn.testExchangeRequest(client) @@ -476,12 +477,10 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() - patch(client, conn.clientChannel) + patch(client, conn) // do one request - def r1 = conn.testExchangeRequest(client) - conn.fireClientActive() - conn.testExchangeResponse(r1) + conn.testExchangeResponse(conn.testExchangeRequest(client)) conn.clientChannel.unfreezeTime() // connection is in reserve, should not time out TimeUnit.SECONDS.sleep(10) @@ -516,7 +515,7 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp2() conn.setupHttp2Tls() - patch(client, conn.clientChannel) + patch(client, conn) // one request opens the connection def r1 = conn.testExchangeRequest(client) @@ -558,17 +557,14 @@ class ConnectionManagerSpec extends Specification { conn1.setupHttp1() def conn2 = new EmbeddedTestConnectionHttp1() conn2.setupHttp1() - patch(client, conn1.clientChannel, conn2.clientChannel) + patch(client, conn1, conn2) def r1 = conn1.testExchangeRequest(client) - conn1.fireClientActive() conn1.clientChannel.advanceTimeBy(101, TimeUnit.SECONDS) conn1.testExchangeResponse(r1) // conn1 should expire now, conn2 will be the next connection - def r2 = conn2.testExchangeRequest(client) - conn2.fireClientActive() - conn2.testExchangeResponse(r2) + conn2.testExchangeResponse(conn2.testExchangeRequest(client)) cleanup: client.close() @@ -586,7 +582,7 @@ class ConnectionManagerSpec extends Specification { conn1.setupHttp2Tls() def conn2 = new EmbeddedTestConnectionHttp2() conn2.setupHttp2Tls() - patch(client, conn1.clientChannel, conn2.clientChannel) + patch(client, conn1, conn2) def r1 = conn1.testExchangeRequest(client) conn1.exchangeSettings() @@ -613,19 +609,15 @@ class ConnectionManagerSpec extends Specification { conn1.setupHttp1() def conn2 = new EmbeddedTestConnectionHttp1() conn2.setupHttp1() - patch(client, conn1.clientChannel, conn2.clientChannel) + patch(client, conn1, conn2) - def r1 = conn1.testExchangeRequest(client) - conn1.fireClientActive() - conn1.testExchangeResponse(r1) + conn1.testExchangeResponse(conn1.testExchangeRequest(client)) conn1.clientChannel.unfreezeTime() // todo: move to advanceTime once IdleStateHandler supports it TimeUnit.SECONDS.sleep(5) conn1.advance() // conn1 should expire now, conn2 will be the next connection - def r2 = conn2.testExchangeRequest(client) - conn2.fireClientActive() - conn2.testExchangeResponse(r2) + conn2.testExchangeResponse(conn2.testExchangeRequest(client)) cleanup: client.close() @@ -647,12 +639,11 @@ class ConnectionManagerSpec extends Specification { conn.setupHttp1() } conn.serverChannel.pipeline().addLast(new HttpObjectAggregator(1024)) - patch(client, conn.clientChannel) + patch(client, conn) def uri = conn.scheme + "://example.com/foo" Mono.from(client.connect(Ws, uri)).subscribe() conn.advance() - conn.fireClientActive() io.netty.handler.codec.http.HttpRequest req = conn.serverChannel.readInbound() def handshaker = new WebSocketServerHandshakerFactory(uri, null, false).newHandshaker(req) handshaker.handshake(conn.serverChannel, req) @@ -689,12 +680,37 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() - patch(client, conn.clientChannel) + + ChannelPromise delayPromise + def normalInit = conn.clientInitializer + // hack: delay the channelActive call until we complete delayPromise + conn.clientInitializer = new ChannelInitializer() { + @Override + protected void initChannel(EmbeddedChannel ch) throws Exception { + ch.pipeline().addLast(new ChannelInboundHandlerAdapter() { + @Override + void channelActive(ChannelHandlerContext chtx) throws Exception { + delayPromise = chtx.newPromise() + delayPromise.addListener(new GenericFutureListener>() { + @Override + void operationComplete(io.netty.util.concurrent.Future future) throws Exception { + chtx.fireChannelActive() + } + }) + } + }) + ch.pipeline().addLast(normalInit) + } + } + + patch(client, conn) def subscription = Mono.from(client.exchange(conn.scheme + '://example.com/foo')).subscribe() conn.advance() subscription.dispose() - conn.fireClientActive() + // this completes the handshake + delayPromise.setSuccess() + conn.advance() conn.testExchangeResponse(conn.testExchangeRequest(client)) @@ -709,16 +725,15 @@ class ConnectionManagerSpec extends Specification { def conn = new EmbeddedTestConnectionHttp1() conn.setupHttp1() - patch(client, conn.clientChannel) + patch(client, conn) conn.serverChannel.pipeline().addLast(new HttpObjectAggregator(1024)) def future = Mono.from(client.exchange(HttpRequest.POST(conn.scheme + '://example.com/foo', MultipartBody.builder() - .addPart('foo', 'fn', MediaType.TEXT_PLAIN_TYPE, 'bar'.bytes) - .build()) + .addPart('foo', 'fn', MediaType.TEXT_PLAIN_TYPE, 'bar'.bytes) + .build()) .contentType(MediaType.MULTIPART_FORM_DATA), String)).toFuture() future.exceptionally(t -> t.printStackTrace()) conn.advance() - conn.fireClientActive() FullHttpRequest request = conn.serverChannel.readInbound() assert request.uri() == '/foo' @@ -743,16 +758,14 @@ class ConnectionManagerSpec extends Specification { def client = ctx.getBean(DefaultHttpClient) def conn = new EmbeddedTestConnectionHttp1() - conn.serverChannel.pipeline().addLast(new LoggingHandler(LogLevel.INFO)) // TODO conn.setupHttp1() - patch(client, conn.clientChannel) + patch(client, conn) conn.serverChannel.pipeline().addLast(new HttpObjectAggregator(1024)) def future = Mono.from(client.exchange(HttpRequest.POST(conn.scheme + '://example.com/foo', Flux.fromIterable([1,2,3,4,5])) .contentType(MediaType.APPLICATION_JSON_TYPE), String)).toFuture() future.exceptionally(t -> t.printStackTrace()) conn.advance() - conn.fireClientActive() FullHttpRequest request = conn.serverChannel.readInbound() assert request.uri() == '/foo' @@ -772,29 +785,76 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'connection pool disabled http1'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.pool.enabled': false, + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn1 = new EmbeddedTestConnectionHttp1() + conn1.setupHttp1() + def conn2 = new EmbeddedTestConnectionHttp1() + conn2.setupHttp1() + patch(client, conn1, conn2) + + def r1 = conn1.testExchangeRequest(client) + conn1.testExchangeResponse(r1, "close") + + def r2 = conn2.testExchangeRequest(client) + conn2.testExchangeResponse(r2, "close") + + cleanup: + client.close() + ctx.close() + } + + def 'connection pool disabled http2'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.pool.enabled': false, + 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn1 = new EmbeddedTestConnectionHttp2() + conn1.setupHttp2Tls() + def conn2 = new EmbeddedTestConnectionHttp2() + conn2.setupHttp2Tls() + patch(client, conn1, conn2) + + def r1 = conn1.testExchangeRequest(client) + conn1.exchangeSettings() + conn1.testExchangeResponse(r1) + + def r2 = conn2.testExchangeRequest(client) + conn2.exchangeSettings() + conn2.testExchangeResponse(r2) + + cleanup: + client.close() + ctx.close() + } + static class EmbeddedTestConnectionBase { final EmbeddedChannel serverChannel - final EmbeddedChannel clientChannel + EmbeddedChannel clientChannel + ChannelInitializer clientInitializer = new ChannelInitializer() { + @Override + protected void initChannel(EmbeddedChannel ch) throws Exception { + ch.freezeTime() + ch.config().setAutoRead(false) + EmbeddedTestUtil.connect(serverChannel, ch) + } + } EmbeddedTestConnectionBase() { serverChannel = new EmbeddedServerChannel(new DummyChannelId('server')) serverChannel.freezeTime() serverChannel.config().setAutoRead(true) - - clientChannel = new EmbeddedChannel(new DummyChannelId('client')) - clientChannel.freezeTime() - clientChannel.config().setAutoRead(false) - EmbeddedTestUtil.connect(serverChannel, clientChannel) } - void advance() { + final void advance() { EmbeddedTestUtil.advance(serverChannel, clientChannel) } - - void fireClientActive() { - clientChannel.pipeline().fireChannelActive() - advance() - } } static class EmbeddedServerChannel extends EmbeddedChannel implements ServerChannel { @@ -837,12 +897,12 @@ class ConnectionManagerSpec extends Specification { return future } - void testExchangeResponse(CompletableFuture> future) { + void testExchangeResponse(CompletableFuture> future, String connectionHeader = "keep-alive") { io.netty.handler.codec.http.HttpRequest request = serverChannel.readInbound() assert request.uri() == '/foo' assert request.method() == HttpMethod.GET assert request.headers().get('host') == 'example.com' - assert request.headers().get("connection") == "keep-alive" + assert request.headers().get("connection") == connectionHeader def tail = serverChannel.readInbound() assert tail == null || tail instanceof LastHttpContent @@ -945,14 +1005,14 @@ class ConnectionManagerSpec extends Specification { } void exchangeSettings() { - fireClientActive() + advance() assert serverChannel.readInbound() instanceof Http2SettingsFrame assert serverChannel.readInbound() instanceof Http2SettingsAckFrame } void exchangeH2c() { - fireClientActive() + advance() Http2HeadersFrame upgradeRequest = serverChannel.readInbound() assert upgradeRequest.headers().get(Http2Headers.PseudoHeaderName.METHOD.value()) == 'GET' From 4df4d3478e070b965b78447c1e7175e3e9fa1c5b Mon Sep 17 00:00:00 2001 From: yawkat Date: Wed, 28 Sep 2022 18:02:56 +0200 Subject: [PATCH 46/82] fix SslRefreshSpec --- .../test/groovy/io/micronaut/http/client/SslRefreshSpec.groovy | 2 ++ 1 file changed, 2 insertions(+) diff --git a/http-client/src/test/groovy/io/micronaut/http/client/SslRefreshSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/SslRefreshSpec.groovy index 28335c1c778..90770f722d8 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/SslRefreshSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/SslRefreshSpec.groovy @@ -45,6 +45,8 @@ class SslRefreshSpec extends Specification { 'micronaut.http.client.ssl.key-store.password': 'secret', 'micronaut.http.client.ssl.insecure-trust-all-certificates': true, 'micronaut.http.client.pool.enabled': false, + // need to force http1 because our ciphers are not supported by http2 + 'micronaut.http.client.http-version': '1.1', ] @Shared @AutoCleanup EmbeddedServer embeddedServer = ApplicationContext .builder() From ce54cedb383f2e62d498e177afc8feca08da3cf3 Mon Sep 17 00:00:00 2001 From: yawkat Date: Thu, 29 Sep 2022 12:22:40 +0200 Subject: [PATCH 47/82] make test not use ChannelPipelineCustomizer --- .../DefaultNettyHttpClientRegistrySpec.groovy | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistrySpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistrySpec.groovy index 5abda077f12..2aa503905ad 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistrySpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistrySpec.groovy @@ -1,17 +1,13 @@ package io.micronaut.http.client.netty import io.micronaut.context.ApplicationContext -import io.micronaut.context.BeanContext import io.micronaut.context.annotation.Requires import io.micronaut.context.event.BeanCreatedEvent import io.micronaut.context.event.BeanCreatedEventListener import io.micronaut.http.annotation.Get import io.micronaut.http.client.HttpClient import io.micronaut.http.client.annotation.Client -import io.micronaut.http.netty.channel.ChannelPipelineCustomizer -import io.micronaut.runtime.server.EmbeddedServer -import io.micronaut.test.extensions.spock.annotation.MicronautTest -import io.netty.util.Attribute +import io.netty.channel.Channel import io.netty.util.AttributeKey import jakarta.inject.Inject import jakarta.inject.Singleton @@ -88,22 +84,27 @@ class DefaultNettyHttpClientRegistrySpec extends Specification { @Requires(property = 'spec.name', value = 'DefaultNettyHttpClientRegistrySpec') @Singleton - static class MyCustomizer implements BeanCreatedEventListener { + static class MyCustomizer implements BeanCreatedEventListener { static final AttributeKey CUSTOMIZED = AttributeKey.valueOf('micronaut.test.customized') def connected = 0 def duplicate = false @Override - ChannelPipelineCustomizer onCreated(BeanCreatedEvent event) { - event.bean.doOnConnect { - if (it.channel().hasAttr(CUSTOMIZED)) { - duplicate = true + NettyClientCustomizer.Registry onCreated(BeanCreatedEvent event) { + event.bean.register(new NettyClientCustomizer() { + @Override + NettyClientCustomizer specializeForChannel(Channel channel, NettyClientCustomizer.ChannelRole role) { + if (role == NettyClientCustomizer.ChannelRole.CONNECTION) { + if (channel.hasAttr(CUSTOMIZED)) { + duplicate = true + } + channel.attr(CUSTOMIZED).set(true) + connected++ + } + return this } - it.channel().attr(CUSTOMIZED).set(true) - connected++ - return it - } + }) return event.bean } } From e76c2a9d230013544d77c1a213fd4fece12b576f Mon Sep 17 00:00:00 2001 From: yawkat Date: Thu, 29 Sep 2022 12:28:09 +0200 Subject: [PATCH 48/82] handle sslConfiguration.enabled properly --- .../micronaut/http/client/netty/ConnectionManager.java | 3 ++- .../http/client/netty/ssl/NettyClientSslBuilder.java | 10 ++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index b3fba7cdc7c..15a6a1d4206 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -437,7 +437,8 @@ Mono connectForWebsocket(DefaultHttpClient.RequestKey requestKey, ChannelHand protected void initChannel(Channel ch) throws Exception { addLogHandler(ch); - if (requestKey.isSecure()) { + SslContext sslContext = buildSslContext(requestKey); + if (sslContext != null) { SslHandler sslHandler = sslContext.newHandler(ch.alloc(), requestKey.getHost(), requestKey.getPort()); sslHandler.setHandshakeTimeoutMillis(configuration.getSslConfiguration().getHandshakeTimeout().toMillis()); ch.pipeline().addLast(sslHandler); diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java b/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java index 58bdba5c012..79bca822002 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java @@ -17,6 +17,7 @@ import io.micronaut.context.annotation.BootstrapContextCompatible; import io.micronaut.core.annotation.Internal; +import io.micronaut.core.annotation.Nullable; import io.micronaut.core.io.ResourceResolver; import io.micronaut.http.HttpVersion; import io.micronaut.http.client.HttpVersionSelection; @@ -71,13 +72,14 @@ public Optional build(SslConfiguration ssl) { @Override public Optional build(SslConfiguration ssl, HttpVersion httpVersion) { - if (!ssl.isEnabled()) { - return Optional.empty(); - } - return Optional.of(build(ssl, HttpVersionSelection.forLegacyVersion(httpVersion))); + return Optional.ofNullable(build(ssl, HttpVersionSelection.forLegacyVersion(httpVersion))); } + @Nullable public SslContext build(SslConfiguration ssl, HttpVersionSelection versionSelection) { + if (!ssl.isEnabled()) { + return null; + } SslContextBuilder sslBuilder = SslContextBuilder .forClient() .keyManager(getKeyManagerFactory(ssl)) From 0b61862e7cb74aeb47100bfb20d92846f7a05b95 Mon Sep 17 00:00:00 2001 From: yawkat Date: Thu, 29 Sep 2022 12:42:57 +0200 Subject: [PATCH 49/82] fix client tests that rely on pool internals --- .../http/client/netty/ConnectionManager.java | 32 +++++++++++++++++++ .../http/client/ConnectionTTLSpec.groovy | 27 ++++++---------- .../http/client/IdleTimeoutSpec.groovy | 27 ++++++---------- .../http/client/ReadTimeoutSpec.groovy | 3 +- 4 files changed, 53 insertions(+), 36 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 15a6a1d4206..b1920b4a002 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -95,8 +95,10 @@ import java.net.Proxy; import java.net.SocketAddress; import java.time.Duration; +import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.OptionalInt; @@ -234,6 +236,36 @@ private static NioEventLoopGroup createEventLoopGroup(HttpClientConfiguration co return group; } + /** + * For testing + */ + List getChannels() { + List channels = new ArrayList<>(); + for (Pool pool : pools.values()) { + pool.forEachConnection(c -> channels.add(((Pool.ConnectionHolder) c).channel)); + } + return channels; + } + + /** + * For testing + */ + int liveRequestCount() { + AtomicInteger count = new AtomicInteger(); + for (Pool pool : pools.values()) { + pool.forEachConnection(c -> { + if (c instanceof Pool.Http1ConnectionHolder) { + if (((Pool.Http1ConnectionHolder) c).hasLiveRequests()) { + count.incrementAndGet(); + } + } else { + count.addAndGet(((Pool.Http2ConnectionHolder) c).liveRequests.get()); + } + }); + } + return count.get(); + } + /** * @see DefaultHttpClient#start() */ diff --git a/http-client/src/test/groovy/io/micronaut/http/client/ConnectionTTLSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/ConnectionTTLSpec.groovy index e2660359945..dc03f800445 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/ConnectionTTLSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/ConnectionTTLSpec.groovy @@ -8,15 +8,12 @@ import io.micronaut.http.annotation.Controller import io.micronaut.http.annotation.Get import io.micronaut.runtime.server.EmbeddedServer import io.netty.channel.Channel -import io.netty.channel.pool.AbstractChannelPoolMap import spock.lang.AutoCleanup import spock.lang.Retry import spock.lang.Shared import spock.lang.Specification import spock.util.concurrent.PollingConditions -import java.lang.reflect.Field - @Retry class ConnectionTTLSpec extends Specification { @@ -37,7 +34,7 @@ class ConnectionTTLSpec extends Specification { when:"make first request" httpClient.toBlocking().retrieve(HttpRequest.GET('/connectTTL/'),String) - Channel ch = getQueuedChannels(httpClient).first + Channel ch = getQueuedChannels(httpClient).get(0) then:"ensure that connection is open as connect-ttl is not reached" getQueuedChannels(httpClient).size() == 1 @@ -67,11 +64,11 @@ class ConnectionTTLSpec extends Specification { when:"make first request" httpClient.toBlocking().retrieve(HttpRequest.GET('/connectTTL/'),String) - Deque deque = getQueuedChannels(httpClient) + List deque = getQueuedChannels(httpClient) then:"ensure that connection is open as connect-ttl is not reached" new PollingConditions().eventually { - deque.first.isOpen() + deque.get(0).isOpen() } when:"make another request after some time" @@ -80,7 +77,7 @@ class ConnectionTTLSpec extends Specification { then:"ensure channel is still open" new PollingConditions().eventually { - deque.first.isOpen() + deque.get(0).isOpen() } cleanup: @@ -99,11 +96,11 @@ class ConnectionTTLSpec extends Specification { when:"make first request" httpClient.toBlocking().retrieve(HttpRequest.GET('/connectTTL/'),String) - Deque deque = getQueuedChannels(httpClient) + Collection deque = getQueuedChannels(httpClient) then:"ensure that connection is open as connect-ttl is not reached" new PollingConditions().eventually { - deque.first.isOpen() + deque.get(0).isOpen() } when:"make another request" @@ -111,7 +108,7 @@ class ConnectionTTLSpec extends Specification { then:"ensure channel is still open" new PollingConditions().eventually { - deque.first.isOpen() + deque.get(0).isOpen() } cleanup: @@ -130,7 +127,7 @@ class ConnectionTTLSpec extends Specification { when:"make first request" httpClient.toBlocking().retrieve(HttpRequest.GET('/connectTTL/'),String) - Channel ch = getQueuedChannels(httpClient).first + Channel ch = getQueuedChannels(httpClient).get(0) then:"ensure that connection is open as connect-ttl is not reached" getQueuedChannels(httpClient).size() == 1 @@ -149,12 +146,8 @@ class ConnectionTTLSpec extends Specification { clientContext.close() } - Deque getQueuedChannels(HttpClient client) { - AbstractChannelPoolMap poolMap = client.connectionManager.poolMap - Field mapField = AbstractChannelPoolMap.getDeclaredField("map") - mapField.setAccessible(true) - Map innerMap = mapField.get(poolMap) - return innerMap.values().first().deque + List getQueuedChannels(HttpClient client) { + return client.connectionManager.channels } @Requires(property = 'spec.name', value = 'ConnectionTTLSpec') diff --git a/http-client/src/test/groovy/io/micronaut/http/client/IdleTimeoutSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/IdleTimeoutSpec.groovy index a65805a92bf..9f75a05e552 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/IdleTimeoutSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/IdleTimeoutSpec.groovy @@ -8,15 +8,12 @@ import io.micronaut.http.annotation.Controller import io.micronaut.http.annotation.Get import io.micronaut.runtime.server.EmbeddedServer import io.netty.channel.Channel -import io.netty.channel.pool.AbstractChannelPoolMap import spock.lang.AutoCleanup import spock.lang.Retry import spock.lang.Shared import spock.lang.Specification import spock.util.concurrent.PollingConditions -import java.lang.reflect.Field - @Retry class IdleTimeoutSpec extends Specification { @@ -35,8 +32,8 @@ class IdleTimeoutSpec extends Specification { when: "make first request" httpClient.toBlocking().retrieve(HttpRequest.GET('/idleTimeout/'), String) - Deque deque = getQueuedChannels(httpClient) - Channel ch1 = deque.first + List deque = getQueuedChannels(httpClient) + Channel ch1 = deque.get(0) then: "ensure that connection is open as connection-pool-idle-timeout is not reached" deque.size() == 1 @@ -54,7 +51,7 @@ class IdleTimeoutSpec extends Specification { } when: - Channel ch2 = deque.first + Channel ch2 = deque.get(0) then: "ensure channel 2 is open and channel 2 != channel 1" deque.size() == 1 @@ -79,13 +76,13 @@ class IdleTimeoutSpec extends Specification { when: "make first request" httpClient.toBlocking().retrieve(HttpRequest.GET('/idleTimeout/'), String) - Deque deque = getQueuedChannels(httpClient) - Channel ch1 = deque.first + List deque = getQueuedChannels(httpClient) + Channel ch1 = deque.get(0) then: "ensure that connection is open as connection-pool-idle-timeout is not reached" deque.size() == 1 new PollingConditions().eventually { - deque.first.isOpen() + deque.get(0).isOpen() } when: "make another request" @@ -97,13 +94,13 @@ class IdleTimeoutSpec extends Specification { } when: - Channel ch2 = deque.first + Channel ch2 = deque.get(0) then: "ensure channel is still open" ch1 == ch2 deque.size() == 1 new PollingConditions().eventually { - deque.first.isOpen() + deque.get(0).isOpen() } cleanup: @@ -111,12 +108,8 @@ class IdleTimeoutSpec extends Specification { clientContext.close() } - Deque getQueuedChannels(HttpClient client) { - AbstractChannelPoolMap poolMap = client.connectionManager.poolMap - Field mapField = AbstractChannelPoolMap.getDeclaredField("map") - mapField.setAccessible(true) - Map innerMap = mapField.get(poolMap) - return innerMap.values().first().deque + List getQueuedChannels(HttpClient client) { + return client.connectionManager.channels } @Requires(property = 'spec.name', value = 'IdleTimeoutSpec') diff --git a/http-client/src/test/groovy/io/micronaut/http/client/ReadTimeoutSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/ReadTimeoutSpec.groovy index a04d8e137c0..9debe5c3b98 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/ReadTimeoutSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/ReadTimeoutSpec.groovy @@ -298,11 +298,10 @@ class ReadTimeoutSpec extends Specification { .filter { it.clientId == "http://localhost:${embeddedServer.getPort()}" } .findFirst() .get() - def pool = getPool(clients.get(clientKey)) then:"Connections are not leaked" conditions.eventually { - pool.acquiredChannelCount() == 0 + clients.get(clientKey).connectionManager.liveRequestCount() == 0 } cleanup: From 40b0fad459fc7284f64b82c184020d550e53a306 Mon Sep 17 00:00:00 2001 From: yawkat Date: Thu, 29 Sep 2022 13:00:30 +0200 Subject: [PATCH 50/82] fix HttpResponseSpec --- .../netty/binding/HttpResponseSpec.groovy | 56 +++++++++---------- 1 file changed, 27 insertions(+), 29 deletions(-) diff --git a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy index df3fe92a1cc..80637e326fe 100644 --- a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy +++ b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy @@ -23,19 +23,16 @@ import io.micronaut.http.HttpResponse import io.micronaut.http.HttpStatus import io.micronaut.http.annotation.Controller import io.micronaut.http.annotation.Get -import io.micronaut.http.client.HttpClient import io.micronaut.http.client.DefaultHttpClientConfiguration +import io.micronaut.http.client.HttpClient import io.micronaut.http.client.exceptions.HttpClientResponseException import io.micronaut.http.server.netty.AbstractMicronautSpec -import io.micronaut.runtime.Micronaut import io.micronaut.runtime.server.EmbeddedServer import reactor.core.publisher.Flux +import spock.lang.Ignore import spock.lang.Shared import spock.lang.Unroll -import java.time.Duration -import java.time.temporal.ChronoUnit - /** * @author Graeme Rocher * @since 1.0 @@ -71,19 +68,19 @@ class HttpResponseSpec extends AbstractMicronautSpec { where: action | status | body | headers - "ok" | HttpStatus.OK | null | [connection: 'close'] - "ok-with-body" | HttpStatus.OK | "some text" | ['content-length': '9', 'content-type': 'text/plain'] + [connection: 'close'] - "error-with-body" | HttpStatus.INTERNAL_SERVER_ERROR | "some text" | ['content-length': '9', 'content-type': 'text/plain'] + [connection: 'close'] - "ok-with-body-object" | HttpStatus.OK | '{"name":"blah","age":10}' | defaultHeaders + ['content-length': '24', 'content-type': 'application/json'] + [connection: 'close'] - "status" | HttpStatus.MOVED_PERMANENTLY | null | [connection: 'close'] - "created-body" | HttpStatus.CREATED | '{"name":"blah","age":10}' | defaultHeaders + ['content-length': '24', 'content-type': 'application/json'] + [connection: 'close'] - "created-uri" | HttpStatus.CREATED | null | [connection: 'close', 'location': 'http://test.com'] - "created-body-uri" | HttpStatus.CREATED | '{"name":"blah","age":10}' | defaultHeaders + ['content-length': '24', 'content-type': 'application/json'] + [connection: 'close', 'location': 'http://test.com'] - "accepted" | HttpStatus.ACCEPTED | null | [connection: 'close'] - "accepted-uri" | HttpStatus.ACCEPTED | null | [connection: 'close', 'location': 'http://example.com'] - "disallow" | HttpStatus.METHOD_NOT_ALLOWED | null | [connection: "close", 'allow': 'DELETE'] - "optional-response/false" | HttpStatus.OK | null | [connection: 'close'] - "optional-response/true" | HttpStatus.NOT_FOUND | null | ['content-type': 'application/json', 'content-length': '162', connection: 'close'] + "ok" | HttpStatus.OK | null | [connection: 'keep-alive'] + "ok-with-body" | HttpStatus.OK | "some text" | ['content-length': '9', 'content-type': 'text/plain'] + [connection: 'keep-alive'] + "error-with-body" | HttpStatus.INTERNAL_SERVER_ERROR | "some text" | ['content-length': '9', 'content-type': 'text/plain'] + [connection: 'keep-alive'] + "ok-with-body-object" | HttpStatus.OK | '{"name":"blah","age":10}' | defaultHeaders + ['content-length': '24', 'content-type': 'application/json'] + [connection: 'keep-alive'] + "status" | HttpStatus.MOVED_PERMANENTLY | null | [connection: 'keep-alive'] + "created-body" | HttpStatus.CREATED | '{"name":"blah","age":10}' | defaultHeaders + ['content-length': '24', 'content-type': 'application/json'] + [connection: 'keep-alive'] + "created-uri" | HttpStatus.CREATED | null | [connection: 'keep-alive', 'location': 'http://test.com'] + "created-body-uri" | HttpStatus.CREATED | '{"name":"blah","age":10}' | defaultHeaders + ['content-length': '24', 'content-type': 'application/json'] + [connection: 'keep-alive', 'location': 'http://test.com'] + "accepted" | HttpStatus.ACCEPTED | null | [connection: 'keep-alive'] + "accepted-uri" | HttpStatus.ACCEPTED | null | [connection: 'keep-alive', 'location': 'http://example.com'] + "disallow" | HttpStatus.METHOD_NOT_ALLOWED | null | [connection: "keep-alive", 'allow': 'DELETE'] + "optional-response/false" | HttpStatus.OK | null | [connection: 'keep-alive'] + "optional-response/true" | HttpStatus.NOT_FOUND | null | ['content-type': 'application/json', 'content-length': '162', connection: 'keep-alive'] } @@ -104,7 +101,7 @@ class HttpResponseSpec extends AbstractMicronautSpec { } def responseBody = response.body.orElse(null) - def defaultHeaders = [connection: 'close'] + def defaultHeaders = [connection: 'keep-alive'] then: response.code() == status.code @@ -113,15 +110,15 @@ class HttpResponseSpec extends AbstractMicronautSpec { where: action | status | body | headers - "ok" | HttpStatus.OK | null | [connection: 'close'] - "ok-with-body" | HttpStatus.OK | "some text" | ['content-length': '9', 'content-type': 'text/plain'] + [connection: 'close'] - "error-with-body" | HttpStatus.INTERNAL_SERVER_ERROR | "some text" | ['content-length': '9', 'content-type': 'text/plain'] + [connection: 'close'] - "ok-with-body-object" | HttpStatus.OK | '{"name":"blah","age":10}' | defaultHeaders + ['content-length': '24', 'content-type': 'application/json'] + [connection: 'close'] - "status" | HttpStatus.MOVED_PERMANENTLY | null | [connection: 'close'] - "created-body" | HttpStatus.CREATED | '{"name":"blah","age":10}' | defaultHeaders + ['content-length': '24', 'content-type': 'application/json'] + [connection: 'close'] - "created-uri" | HttpStatus.CREATED | null | [connection: 'close', 'location': 'http://test.com'] - "accepted" | HttpStatus.ACCEPTED | null | [connection: 'close'] - "accepted-uri" | HttpStatus.ACCEPTED | null | [connection: 'close', 'location': 'http://example.com'] + "ok" | HttpStatus.OK | null | [connection: 'keep-alive'] + "ok-with-body" | HttpStatus.OK | "some text" | ['content-length': '9', 'content-type': 'text/plain'] + [connection: 'keep-alive'] + "error-with-body" | HttpStatus.INTERNAL_SERVER_ERROR | "some text" | ['content-length': '9', 'content-type': 'text/plain'] + [connection: 'keep-alive'] + "ok-with-body-object" | HttpStatus.OK | '{"name":"blah","age":10}' | defaultHeaders + ['content-length': '24', 'content-type': 'application/json'] + [connection: 'keep-alive'] + "status" | HttpStatus.MOVED_PERMANENTLY | null | [connection: 'keep-alive'] + "created-body" | HttpStatus.CREATED | '{"name":"blah","age":10}' | defaultHeaders + ['content-length': '24', 'content-type': 'application/json'] + [connection: 'keep-alive'] + "created-uri" | HttpStatus.CREATED | null | [connection: 'keep-alive', 'location': 'http://test.com'] + "accepted" | HttpStatus.ACCEPTED | null | [connection: 'keep-alive'] + "accepted-uri" | HttpStatus.ACCEPTED | null | [connection: 'keep-alive', 'location': 'http://example.com'] } void "test content encoding"() { @@ -232,6 +229,7 @@ class HttpResponseSpec extends AbstractMicronautSpec { server.close() } + @Ignore // why wouldn't keep-alive be set? void "test keep alive connection header is not set by default for > 499 response"() { when: EmbeddedServer server = applicationContext.run(EmbeddedServer, [(SPEC_NAME_PROPERTY):getClass().simpleName]) @@ -256,7 +254,7 @@ class HttpResponseSpec extends AbstractMicronautSpec { void "test connection header is defaulted to keep-alive when configured to true for > 499 response"() { when: DefaultHttpClientConfiguration config = new DefaultHttpClientConfiguration() - // The client will explicitly request "Connection: close" unless using a connection pool, so set it up + // The client will explicitly request "Connection: keep-alive" unless using a connection pool, so set it up config.connectionPoolConfiguration.enabled = true EmbeddedServer server = applicationContext.run(EmbeddedServer, [ (SPEC_NAME_PROPERTY):getClass().simpleName, From 5f7d2bd4a3908b16be6fd80c8a643eabe886dc16 Mon Sep 17 00:00:00 2001 From: yawkat Date: Thu, 29 Sep 2022 15:04:15 +0200 Subject: [PATCH 51/82] fix test --- .../io/micronaut/http/client/IdleTimeoutSpec.groovy | 11 +++++------ .../http/netty/stream/HttpStreamsServerHandler.java | 2 -- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/http-client/src/test/groovy/io/micronaut/http/client/IdleTimeoutSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/IdleTimeoutSpec.groovy index 9f75a05e552..c5edd768f3f 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/IdleTimeoutSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/IdleTimeoutSpec.groovy @@ -32,11 +32,10 @@ class IdleTimeoutSpec extends Specification { when: "make first request" httpClient.toBlocking().retrieve(HttpRequest.GET('/idleTimeout/'), String) - List deque = getQueuedChannels(httpClient) - Channel ch1 = deque.get(0) + Channel ch1 = getQueuedChannels(httpClient).get(0) then: "ensure that connection is open as connection-pool-idle-timeout is not reached" - deque.size() == 1 + getQueuedChannels(httpClient).size() == 1 ch1.isOpen() new PollingConditions(timeout: 2).eventually { !ch1.isOpen() @@ -47,14 +46,14 @@ class IdleTimeoutSpec extends Specification { then: new PollingConditions().eventually { - assert deque.size() > 0 + assert getQueuedChannels(httpClient).size() > 0 } when: - Channel ch2 = deque.get(0) + Channel ch2 = getQueuedChannels(httpClient).get(0) then: "ensure channel 2 is open and channel 2 != channel 1" - deque.size() == 1 + getQueuedChannels(httpClient).size() == 1 ch1 != ch2 ch2.isOpen() new PollingConditions(timeout: 2).eventually { diff --git a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java index 7447e176796..93810a91e48 100644 --- a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java +++ b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java @@ -220,8 +220,6 @@ protected void consumedInMessage(ChannelHandlerContext ctx) { webSocketResponse = null; webSocketResponseChannelPromise = null; } - // read next request - ctx.read(); } private void handleWebSocketResponse(ChannelHandlerContext ctx, HttpResponse message, ChannelPromise promise) { From 033d1358f16a0344411eacdc0f3cabd960b38896 Mon Sep 17 00:00:00 2001 From: yawkat Date: Thu, 29 Sep 2022 19:28:31 +0200 Subject: [PATCH 52/82] fix test --- .../http/netty/stream/HttpStreamsServerHandler.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java index 93810a91e48..a2d4cf67584 100644 --- a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java +++ b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsServerHandler.java @@ -220,6 +220,15 @@ protected void consumedInMessage(ChannelHandlerContext ctx) { webSocketResponse = null; webSocketResponseChannelPromise = null; } + if (inFlight == 0) { + // normally, after writing the response, the routing handler triggers a read() for the + // next request. However, if at this point the request is not fully read yet (e.g. + // still missing a LastHttpContent), then that read() call will simply read the + // remaining content, and the HandlerPublisher also won't trigger more read()s since + // it's complete. To prevent the connection from being stuck in that case, we trigger a + // read here. + ctx.read(); + } } private void handleWebSocketResponse(ChannelHandlerContext ctx, HttpResponse message, ChannelPromise promise) { From f7f1c08d1d32fcd2647e5730e84b0e13761161a6 Mon Sep 17 00:00:00 2001 From: yawkat Date: Thu, 29 Sep 2022 19:36:12 +0200 Subject: [PATCH 53/82] fix BinaryWebSocketSpec --- .../http/client/netty/ConnectionManager.java | 1 + .../websocket/BinaryWebSocketSpec.groovy | 67 ++++++++++++++++--- 2 files changed, 59 insertions(+), 9 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index b1920b4a002..9b3252f56ee 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -492,6 +492,7 @@ protected void initChannel(Channel ch) throws Exception { try { ch.pipeline().addLast(WebSocketClientCompressionHandler.INSTANCE); ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_WEBSOCKET_CLIENT, handler); + clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION).onInitialPipelineBuilt(); if (initial.tryEmitEmpty().isSuccess()) { return; } diff --git a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/websocket/BinaryWebSocketSpec.groovy b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/websocket/BinaryWebSocketSpec.groovy index a4bd3a39ca2..4fc32dac55c 100644 --- a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/websocket/BinaryWebSocketSpec.groovy +++ b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/websocket/BinaryWebSocketSpec.groovy @@ -19,10 +19,12 @@ import io.micronaut.context.ApplicationContext import io.micronaut.context.annotation.Requires import io.micronaut.context.event.BeanCreatedEvent import io.micronaut.context.event.BeanCreatedEventListener -import io.micronaut.http.netty.channel.ChannelPipelineCustomizer +import io.micronaut.http.client.netty.NettyClientCustomizer +import io.micronaut.http.server.netty.NettyServerCustomizer import io.micronaut.runtime.server.EmbeddedServer import io.micronaut.websocket.WebSocketClient import io.netty.buffer.Unpooled +import io.netty.channel.Channel import io.netty.channel.ChannelHandlerContext import io.netty.channel.ChannelOutboundHandlerAdapter import io.netty.channel.ChannelPipeline @@ -218,7 +220,8 @@ class BinaryWebSocketSpec extends Specification { 'spec.name' : 'test per-message compression', 'micronaut.server.port': -1 ]) - def compressionDetectionCustomizer = ctx.getBean(CompressionDetectionCustomizer) + def cdcServer = ctx.getBean(CompressionDetectionCustomizerServer) + def cdcClient = ctx.getBean(CompressionDetectionCustomizerClient) EmbeddedServer embeddedServer = ctx.getBean(EmbeddedServer) embeddedServer.start() PollingConditions conditions = new PollingConditions(timeout: 15, delay: 0.5) @@ -237,11 +240,12 @@ class BinaryWebSocketSpec extends Specification { fred.replies.size() == 1 } - compressionDetectionCustomizer.getPipelines().size() == 4 + cdcServer.getPipelines().size() == 2 + cdcClient.getPipelines().size() == 2 when: "A message is sent" List interceptors = new ArrayList<>() - for (ChannelPipeline pipeline : compressionDetectionCustomizer.getPipelines()) { + for (ChannelPipeline pipeline : cdcServer.getPipelines() + cdcClient.getPipelines()) { def interceptor = new MessageInterceptor() if (pipeline.get('ws-encoder') != null) { pipeline.addAfter('ws-encoder', 'MessageInterceptor', interceptor) @@ -268,17 +272,62 @@ class BinaryWebSocketSpec extends Specification { @Singleton @Requires(property = 'spec.name', value = 'test per-message compression') - static class CompressionDetectionCustomizer implements BeanCreatedEventListener { + static class CompressionDetectionCustomizerServer implements BeanCreatedEventListener { List pipelines = Collections.synchronizedList(new ArrayList<>()) @Override - ChannelPipelineCustomizer onCreated(BeanCreatedEvent event) { - event.getBean().doOnConnect { - pipelines.add(it) - return it + NettyServerCustomizer.Registry onCreated(BeanCreatedEvent event) { + event.getBean().register(new Customizer(null)) + return event.bean + } + + class Customizer implements NettyServerCustomizer { + final Channel channel + + Customizer(Channel channel) { + this.channel = channel + } + + @Override + NettyServerCustomizer specializeForChannel(Channel channel, ChannelRole role) { + return new Customizer(channel) } + + @Override + void onInitialPipelineBuilt() { + pipelines.add(channel.pipeline()) + } + } + } + + @Singleton + @Requires(property = 'spec.name', value = 'test per-message compression') + static class CompressionDetectionCustomizerClient implements BeanCreatedEventListener { + List pipelines = Collections.synchronizedList(new ArrayList<>()) + + @Override + NettyClientCustomizer.Registry onCreated(BeanCreatedEvent event) { + event.getBean().register(new Customizer(null)) return event.bean } + + class Customizer implements NettyClientCustomizer { + final Channel channel + + Customizer(Channel channel) { + this.channel = channel + } + + @Override + NettyClientCustomizer specializeForChannel(Channel channel, ChannelRole role) { + return new Customizer(channel) + } + + @Override + void onInitialPipelineBuilt() { + pipelines.add(channel.pipeline()) + } + } } static class MessageInterceptor extends ChannelOutboundHandlerAdapter { From dce9aa77206e2bcc8094cacf6f7bbee83252f431 Mon Sep 17 00:00:00 2001 From: yawkat Date: Thu, 29 Sep 2022 19:58:32 +0200 Subject: [PATCH 54/82] fix LogbookNettyClientCustomizer --- .../docs/netty/LogbookNettyClientCustomizer.groovy | 10 ++++++---- .../docs/netty/LogbookNettyClientCustomizer.kt | 6 ++++-- .../docs/netty/LogbookNettyClientCustomizer.java | 6 +++--- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/test-suite-groovy/src/test/groovy/io/micronaut/docs/netty/LogbookNettyClientCustomizer.groovy b/test-suite-groovy/src/test/groovy/io/micronaut/docs/netty/LogbookNettyClientCustomizer.groovy index de84e9777b7..a4d6b4b1aa5 100644 --- a/test-suite-groovy/src/test/groovy/io/micronaut/docs/netty/LogbookNettyClientCustomizer.groovy +++ b/test-suite-groovy/src/test/groovy/io/micronaut/docs/netty/LogbookNettyClientCustomizer.groovy @@ -1,12 +1,13 @@ package io.micronaut.docs.netty -import io.micronaut.context.annotation.Requires; +import io.micronaut.context.annotation.Requires +import io.micronaut.context.event.BeanCreatedEvent; // tag::imports[] -import io.micronaut.context.event.BeanCreatedEvent import io.micronaut.context.event.BeanCreatedEventListener import io.micronaut.http.client.netty.NettyClientCustomizer +import io.micronaut.http.netty.channel.ChannelPipelineCustomizer import io.netty.channel.Channel import jakarta.inject.Singleton import org.zalando.logbook.Logbook @@ -47,8 +48,9 @@ class LogbookNettyClientCustomizer } @Override - void onStreamPipelineBuilt() { - channel.pipeline().addLast( // <5> + void onRequestPipelineBuilt() { + channel.pipeline().addBefore( // <5> + ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, "logbook", new LogbookClientHandler(logbook) ) diff --git a/test-suite-kotlin/src/test/kotlin/io/micronaut/docs/netty/LogbookNettyClientCustomizer.kt b/test-suite-kotlin/src/test/kotlin/io/micronaut/docs/netty/LogbookNettyClientCustomizer.kt index 5b980a2fcb8..7fdeb32a624 100644 --- a/test-suite-kotlin/src/test/kotlin/io/micronaut/docs/netty/LogbookNettyClientCustomizer.kt +++ b/test-suite-kotlin/src/test/kotlin/io/micronaut/docs/netty/LogbookNettyClientCustomizer.kt @@ -6,6 +6,7 @@ import io.micronaut.context.event.BeanCreatedEvent import io.micronaut.context.event.BeanCreatedEventListener import io.micronaut.http.client.netty.NettyClientCustomizer import io.micronaut.http.client.netty.NettyClientCustomizer.ChannelRole +import io.micronaut.http.netty.channel.ChannelPipelineCustomizer import io.netty.channel.Channel import jakarta.inject.Singleton import org.zalando.logbook.Logbook @@ -29,8 +30,9 @@ class LogbookNettyClientCustomizer(private val logbook: Logbook) : override fun specializeForChannel(channel: Channel, role: ChannelRole) = Customizer(channel) // <4> - override fun onStreamPipelineBuilt() { - channel!!.pipeline().addLast( // <5> + override fun onRequestPipelineBuilt() { + channel!!.pipeline().addBefore( // <5> + ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, "logbook", LogbookClientHandler(logbook) ) diff --git a/test-suite/src/test/java/io/micronaut/docs/netty/LogbookNettyClientCustomizer.java b/test-suite/src/test/java/io/micronaut/docs/netty/LogbookNettyClientCustomizer.java index a2d0069f968..6f37b73fc11 100644 --- a/test-suite/src/test/java/io/micronaut/docs/netty/LogbookNettyClientCustomizer.java +++ b/test-suite/src/test/java/io/micronaut/docs/netty/LogbookNettyClientCustomizer.java @@ -8,7 +8,6 @@ import io.micronaut.http.client.netty.NettyClientCustomizer; import io.micronaut.http.netty.channel.ChannelPipelineCustomizer; import io.netty.channel.Channel; -import io.netty.channel.ChannelPipeline; import jakarta.inject.Singleton; import org.zalando.logbook.Logbook; import org.zalando.logbook.netty.LogbookClientHandler; @@ -47,8 +46,9 @@ public NettyClientCustomizer specializeForChannel(Channel channel, ChannelRole r } @Override - public void onStreamPipelineBuilt() { - channel.pipeline().addLast( // <5> + public void onRequestPipelineBuilt() { + channel.pipeline().addBefore( // <5> + ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, "logbook", new LogbookClientHandler(logbook) ); From aa55b2dbc2c848be9a96c99d639b5df62e8978af Mon Sep 17 00:00:00 2001 From: yawkat Date: Thu, 29 Sep 2022 20:09:45 +0200 Subject: [PATCH 55/82] fix Http2AccessLoggerSpec --- .../io/micronaut/http2/Http2AccessLoggerSpec.groovy | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/test-suite/src/test/groovy/io/micronaut/http2/Http2AccessLoggerSpec.groovy b/test-suite/src/test/groovy/io/micronaut/http2/Http2AccessLoggerSpec.groovy index fcfc136201f..8839249b5d0 100644 --- a/test-suite/src/test/groovy/io/micronaut/http2/Http2AccessLoggerSpec.groovy +++ b/test-suite/src/test/groovy/io/micronaut/http2/Http2AccessLoggerSpec.groovy @@ -1,22 +1,21 @@ package io.micronaut.http2 +import ch.qos.logback.classic.Logger import ch.qos.logback.classic.spi.ILoggingEvent import ch.qos.logback.core.AppenderBase -import ch.qos.logback.classic.Logger -import io.micronaut.http.client.HttpClient -import io.micronaut.http.client.StreamingHttpClient -import org.reactivestreams.Publisher -import org.slf4j.LoggerFactory - import io.micronaut.context.ApplicationContext import io.micronaut.core.type.Argument import io.micronaut.docs.server.json.Person import io.micronaut.http.HttpRequest import io.micronaut.http.MediaType import io.micronaut.http.annotation.Get +import io.micronaut.http.client.HttpClient +import io.micronaut.http.client.StreamingHttpClient import io.micronaut.http.client.annotation.Client import io.micronaut.http.sse.Event import io.micronaut.runtime.server.EmbeddedServer +import org.reactivestreams.Publisher +import org.slf4j.LoggerFactory import reactor.core.publisher.Flux import spock.lang.AutoCleanup import spock.lang.Shared @@ -144,6 +143,7 @@ class Http2AccessLoggerSpec extends Specification { 'micronaut.server.ssl.buildSelfSigned': true, 'micronaut.server.ssl.port': -1, "micronaut.http.client.log-level" : "TRACE", + "micronaut.http.client.http-version" : "1.1", "micronaut.server.netty.log-level" : "TRACE", 'micronaut.server.netty.access-logger.enabled': true, 'micronaut.http.client.ssl.insecure-trust-all-certificates': true From 1dbc9beba64083acd251f357727f409dcf71de5c Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 11:06:10 +0200 Subject: [PATCH 56/82] remove obsolete idle state handling --- .../http/client/netty/ConnectionManager.java | 31 ++----------------- 1 file changed, 2 insertions(+), 29 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 9b3252f56ee..87f90465c08 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -74,7 +74,6 @@ import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; import io.netty.handler.ssl.SslHandshakeCompletionEvent; -import io.netty.handler.timeout.IdleStateEvent; import io.netty.handler.timeout.IdleStateHandler; import io.netty.handler.timeout.ReadTimeoutException; import io.netty.handler.timeout.ReadTimeoutHandler; @@ -376,20 +375,7 @@ protected void finishAggregation(FullHttpMessage aggregated) throws Exception { } } }); - ph.channel.pipeline().addLast( - ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, - new HttpStreamsClientHandler() { - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - if (evt instanceof IdleStateEvent) { - // close the connection if it is idle for too long - ph.taint(); - ph.release(); - } - super.userEventTriggered(ctx, evt); - } - } - ); + ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, new HttpStreamsClientHandler()); return ph; }); } @@ -435,20 +421,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception if (sse) { ph.channel.pipeline().addLast(HttpLineBasedFrameDecoder.NAME, new HttpLineBasedFrameDecoder(configuration.getMaxContentLength(), true, true)); } - ph.channel.pipeline().addLast( - ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, - new HttpStreamsClientHandler() { - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - if (evt instanceof IdleStateEvent) { - // close the connection if it is idle for too long - ph.taint(); - ph.release(); - } - super.userEventTriggered(ctx, evt); - } - } - ); + ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, new HttpStreamsClientHandler()); return ph; }); } From 130b492cf97cc124fded2baa58534a5a30ff1d74 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 11:12:54 +0200 Subject: [PATCH 57/82] remove todo --- .../micronaut/http/netty/stream/HttpStreamsClientHandler.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java index 4367e1615bb..3ff8be28cda 100644 --- a/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java +++ b/http-netty/src/main/java/io/micronaut/http/netty/stream/HttpStreamsClientHandler.java @@ -88,7 +88,6 @@ protected boolean hasBody(HttpResponse response) { return true; } - if (HttpUtil.isContentLengthSet(response)) { return HttpUtil.getContentLength(response) > 0; } @@ -183,8 +182,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception @Override public void write(final ChannelHandlerContext ctx, Object msg, final ChannelPromise promise) throws Exception { - // todo - if (ctx.channel().attr(AttributeKey.valueOf(ChannelPipelineCustomizer.HANDLER_HTTP_CHUNK)).get() == Boolean.TRUE) { + if (Boolean.TRUE.equals(ctx.channel().attr(AttributeKey.valueOf(ChannelPipelineCustomizer.HANDLER_HTTP_CHUNK)).get())) { ctx.write(msg, promise); } else { super.write(ctx, msg, promise); From 8e4ac0df979277c785e66b60516fe261eb7da7b8 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 11:23:25 +0200 Subject: [PATCH 58/82] move exchange/stream-specific handlers to DefaultHttpClient --- .../http/client/netty/ConnectionManager.java | 75 +------------------ .../http/client/netty/DefaultHttpClient.java | 54 ++++++++++++- 2 files changed, 55 insertions(+), 74 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 87f90465c08..ebfff9a6530 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -26,7 +26,6 @@ import io.micronaut.http.netty.channel.ChannelPipelineCustomizer; import io.micronaut.http.netty.channel.ChannelPipelineListener; import io.micronaut.http.netty.channel.NettyThreadFactory; -import io.micronaut.http.netty.stream.HttpStreamsClientHandler; import io.micronaut.scheduling.instrument.Instrumentation; import io.micronaut.scheduling.instrument.InvocationInstrumenter; import io.micronaut.websocket.exceptions.WebSocketSessionException; @@ -45,17 +44,12 @@ import io.netty.channel.nio.NioEventLoopGroup; import io.netty.handler.codec.DecoderException; import io.netty.handler.codec.http.DefaultFullHttpRequest; -import io.netty.handler.codec.http.FullHttpMessage; import io.netty.handler.codec.http.HttpClientCodec; import io.netty.handler.codec.http.HttpClientUpgradeHandler; import io.netty.handler.codec.http.HttpContentDecompressor; import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpObjectAggregator; -import io.netty.handler.codec.http.HttpResponse; -import io.netty.handler.codec.http.HttpResponseStatus; -import io.netty.handler.codec.http.HttpUtil; -import io.netty.handler.codec.http.LastHttpContent; import io.netty.handler.codec.http.websocketx.extensions.compression.WebSocketClientCompressionHandler; import io.netty.handler.codec.http2.Http2ClientUpgradeCodec; import io.netty.handler.codec.http2.Http2FrameCodec; @@ -354,76 +348,13 @@ private SslContext buildSslContext(DefaultHttpClient.RequestKey requestKey) { } /** - * Get a connection for exchange-like (non-streaming) http client methods. + * Get a connection for non-websocket http client methods. * * @param requestKey The remote to connect to - * @param multipart Whether the request should be multipart - * @param acceptEvents Whether the response may be an event stream * @return A mono that will complete once the channel is ready for transmission */ - Mono connectForExchange(DefaultHttpClient.RequestKey requestKey, boolean multipart, boolean acceptEvents) { - Pool pool = pools.computeIfAbsent(requestKey, Pool::new); - return pool.acquire().map(ph -> { - // TODO: this sucks - ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR, new HttpObjectAggregator(configuration.getMaxContentLength()) { - @Override - protected void finishAggregation(FullHttpMessage aggregated) throws Exception { - if (!HttpUtil.isContentLengthSet(aggregated)) { - if (aggregated.content().readableBytes() > 0) { - super.finishAggregation(aggregated); - } - } - } - }); - ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, new HttpStreamsClientHandler()); - return ph; - }); - } - - /** - * Get a connection for streaming http client methods. - * - * @param requestKey The remote to connect to - * @param isProxy Whether the request is for a {@link io.micronaut.http.client.ProxyHttpClient} call - * @param acceptEvents Whether the response may be an event stream - * @return A mono that will complete once the channel is ready for transmission - */ - Mono connectForStream(DefaultHttpClient.RequestKey requestKey, boolean isProxy, boolean acceptEvents) { - Pool pool = pools.computeIfAbsent(requestKey, Pool::new); - return pool.acquire() - .map(ph -> { - // TODO: this sucks - boolean sse = !isProxy && acceptEvents; - ph.channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { - boolean ignoreOneLast = false; - - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof HttpResponse && - ((HttpResponse) msg).status().equals(HttpResponseStatus.CONTINUE)) { - ignoreOneLast = true; - } - - super.channelRead(ctx, msg); - - if (msg instanceof LastHttpContent) { - if (ignoreOneLast) { - ignoreOneLast = false; - } else { - ctx.pipeline() - .remove(this) - .remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); - ph.release(); - } - } - } - }); - if (sse) { - ph.channel.pipeline().addLast(HttpLineBasedFrameDecoder.NAME, new HttpLineBasedFrameDecoder(configuration.getMaxContentLength(), true, true)); - } - ph.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, new HttpStreamsClientHandler()); - return ph; - }); + Mono connect(DefaultHttpClient.RequestKey requestKey) { + return pools.computeIfAbsent(requestKey, Pool::new).acquire(); } /** diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index 7db0c91b49a..152152ea41d 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -83,6 +83,7 @@ import io.micronaut.http.netty.channel.ChannelPipelineCustomizer; import io.micronaut.http.netty.channel.ChannelPipelineListener; import io.micronaut.http.netty.stream.DefaultStreamedHttpResponse; +import io.micronaut.http.netty.stream.HttpStreamsClientHandler; import io.micronaut.http.netty.stream.JsonSubscriber; import io.micronaut.http.netty.stream.StreamedHttpRequest; import io.micronaut.http.netty.stream.StreamedHttpResponse; @@ -113,6 +114,7 @@ import io.netty.channel.ChannelFactory; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; import io.netty.channel.MultithreadEventLoopGroup; @@ -123,16 +125,19 @@ import io.netty.handler.codec.http.DefaultHttpHeaders; import io.netty.handler.codec.http.DefaultLastHttpContent; import io.netty.handler.codec.http.EmptyHttpHeaders; +import io.netty.handler.codec.http.FullHttpMessage; import io.netty.handler.codec.http.FullHttpRequest; import io.netty.handler.codec.http.FullHttpResponse; import io.netty.handler.codec.http.HttpContent; import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpHeaderValues; import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpObjectAggregator; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpScheme; import io.netty.handler.codec.http.HttpUtil; +import io.netty.handler.codec.http.LastHttpContent; import io.netty.handler.codec.http.multipart.DefaultHttpDataFactory; import io.netty.handler.codec.http.multipart.FileUpload; import io.netty.handler.codec.http.multipart.HttpDataFactory; @@ -1014,8 +1019,39 @@ private Flux> connectAndStream( } catch (Exception e) { return Flux.error(e); } - return connectionManager.connectForStream(requestKey, isProxy, isAcceptEvents(request)).flatMapMany(poolHandle -> { + return connectionManager.connect(requestKey).flatMapMany(poolHandle -> { request.setAttribute(NettyClientHttpRequest.CHANNEL, poolHandle.channel); + + boolean sse = !isProxy && isAcceptEvents(request); + poolHandle.channel.pipeline().addLast(new ChannelInboundHandlerAdapter() { + boolean ignoreOneLast = false; + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof io.netty.handler.codec.http.HttpResponse && + ((io.netty.handler.codec.http.HttpResponse) msg).status().equals(HttpResponseStatus.CONTINUE)) { + ignoreOneLast = true; + } + + super.channelRead(ctx, msg); + + if (msg instanceof LastHttpContent) { + if (ignoreOneLast) { + ignoreOneLast = false; + } else { + ctx.pipeline() + .remove(this) + .remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); + poolHandle.release(); + } + } + } + }); + if (sse) { + poolHandle.channel.pipeline().addLast(HttpLineBasedFrameDecoder.NAME, new HttpLineBasedFrameDecoder(configuration.getMaxContentLength(), true, true)); + } + poolHandle.channel.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, new HttpStreamsClientHandler()); + return this.streamRequestThroughChannel( parentRequest, requestWrapper.get(), @@ -1044,9 +1080,23 @@ private Publisher> exchang return Flux.error(e); } - Mono handlePublisher = connectionManager.connectForExchange(requestKey, MediaType.MULTIPART_FORM_DATA_TYPE.equals(request.getContentType().orElse(null)), isAcceptEvents(request)); + Mono handlePublisher = connectionManager.connect(requestKey); Flux> responsePublisher = handlePublisher.flatMapMany(poolHandle -> { + poolHandle.channel.pipeline() + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_AGGREGATOR, new HttpObjectAggregator(configuration.getMaxContentLength()) { + @Override + protected void finishAggregation(FullHttpMessage aggregated) throws Exception { + // only set content-length if there's any content + if (!HttpUtil.isContentLengthSet(aggregated)) { + if (aggregated.content().readableBytes() > 0) { + super.finishAggregation(aggregated); + } + } + } + }) + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM, new HttpStreamsClientHandler()); + return Flux.create(emitter -> { try { sendRequestThroughChannel( From 122d65c83b20962ae008279028f32fb86da7b2b4 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 12:01:19 +0200 Subject: [PATCH 59/82] remove some raw types --- .../http/client/netty/DefaultHttpClient.java | 57 +++++++++++-------- 1 file changed, 33 insertions(+), 24 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index 152152ea41d..b48e7c8ea78 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -150,6 +150,7 @@ import io.netty.util.CharsetUtil; import io.netty.util.ReferenceCountUtil; import io.netty.util.concurrent.DefaultThreadFactory; +import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; import org.reactivestreams.Processor; import org.reactivestreams.Publisher; @@ -852,7 +853,7 @@ private Publisher connectWebSocket(URI uri, MutableHttpRequest request } private Flux>> exchangeStreamImpl(io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest request, Argument errorType, URI requestURI) { - Flux> streamResponsePublisher = Flux.from(buildStreamExchange(parentRequest, request, requestURI, errorType)); + Flux> streamResponsePublisher = Flux.from(buildStreamExchange(parentRequest, request, requestURI, errorType)); return streamResponsePublisher.switchMap(response -> { StreamedHttpResponse streamedHttpResponse = NettyHttpResponseBuilder.toStreamResponse(response); Flux httpContentReactiveSequence = Flux.from(streamedHttpResponse); @@ -879,7 +880,7 @@ private Flux>> exchangeStreamImpl(io.micronaut.ht } private Flux jsonStreamImpl(io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest request, Argument type, Argument errorType, URI requestURI) { - Flux> streamResponsePublisher = + Flux> streamResponsePublisher = Flux.from(buildStreamExchange(parentRequest, request, requestURI, errorType)); return streamResponsePublisher.switchMap(response -> { if (!(response instanceof NettyStreamedHttpResponse)) { @@ -917,7 +918,7 @@ private Flux jsonStreamImpl(io.micronaut.http.HttpRequest parentReq } private Flux> dataStreamImpl(io.micronaut.http.HttpRequest request, Argument errorType, io.micronaut.http.HttpRequest parentRequest, URI requestURI) { - Flux> streamResponsePublisher = Flux.from(buildStreamExchange(parentRequest, request, requestURI, errorType)); + Flux> streamResponsePublisher = Flux.from(buildStreamExchange(parentRequest, request, requestURI, errorType)); Function> contentMapper = message -> { ByteBuf byteBuf = message.content(); return byteBufferFactory.wrap(byteBuf); @@ -951,14 +952,14 @@ private static void terminateRequestChannel(io.micronaut.http.HttpRequest req * Implementation of {@link #jsonStream}, {@link #dataStream}, {@link #exchangeStream}. */ @SuppressWarnings("MagicNumber") - private Publisher> buildStreamExchange( + private Publisher> buildStreamExchange( @Nullable io.micronaut.http.HttpRequest parentRequest, @NonNull io.micronaut.http.HttpRequest request, @NonNull URI requestURI, @Nullable Argument errorType) { AtomicReference> requestWrapper = new AtomicReference<>(request); - Flux> streamResponsePublisher = connectAndStream(parentRequest, request, requestURI, requestWrapper, false, true); + Flux> streamResponsePublisher = connectAndStream(parentRequest, request, requestURI, requestWrapper, false, true); streamResponsePublisher = readBodyOnError(errorType, streamResponsePublisher); @@ -989,7 +990,7 @@ public Publisher> proxy(@NonNull io.micronaut.http.HttpRe } AtomicReference> requestWrapper = new AtomicReference<>(httpRequest); - Flux> proxyResponsePublisher = connectAndStream(request, request, requestURI, requestWrapper, true, false); + Flux> proxyResponsePublisher = connectAndStream(request, request, requestURI, requestWrapper, true, false); // apply filters //noinspection unchecked proxyResponsePublisher = Flux.from( @@ -1005,7 +1006,7 @@ public Publisher> proxy(@NonNull io.micronaut.http.HttpRe }); } - private Flux> connectAndStream( + private Flux> connectAndStream( io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest request, URI requestURI, @@ -1214,7 +1215,7 @@ protected Object getLoadBalancerDiscriminator() { return null; } - private > Publisher applyFilterToResponsePublisher( + private > Publisher applyFilterToResponsePublisher( io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest request, URI requestURI, @@ -1406,7 +1407,7 @@ protected NettyRequestWriter buildNettyRequest( return new NettyRequestWriter(nettyRequest, postRequestEncoder); } - private Flux> readBodyOnError(@Nullable Argument errorType, @NonNull Flux> publisher) { + private Flux> readBodyOnError(@Nullable Argument errorType, @NonNull Flux> publisher) { if (errorType != null && errorType != HttpClient.DEFAULT_ERROR_TYPE) { return publisher.onErrorResume(clientException -> { if (clientException instanceof HttpClientResponseException) { @@ -1542,13 +1543,13 @@ private void sendRequestThroughChannel( requestWriter.write(poolHandle, secure, emitter); } - private Flux> streamRequestThroughChannel( + private Flux> streamRequestThroughChannel( io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest request, ConnectionManager.PoolHandle poolHandle, boolean failOnError, boolean secure) { - return Flux.>create(sink -> { + return Flux.>create(sink -> { try { streamRequestThroughChannel0(parentRequest, request, sink, poolHandle, secure); } catch (HttpPostRequestEncoder.ErrorDataEncoderException e) { @@ -1572,7 +1573,7 @@ private > Flux handleStreamHttpError( private void streamRequestThroughChannel0( io.micronaut.http.HttpRequest parentRequest, final io.micronaut.http.HttpRequest finalRequest, - FluxSink emitter, + FluxSink> emitter, ConnectionManager.PoolHandle poolHandle, boolean secure) throws HttpPostRequestEncoder.ErrorDataEncoderException { if (!(finalRequest instanceof MutableHttpRequest)) { @@ -1597,7 +1598,7 @@ private void streamRequestThroughChannel0( prepareHttpHeaders(poolHandle, requestURI, finalRequest, requestWriter.getNettyRequest(), permitsBody); HttpRequest nettyRequest = requestWriter.getNettyRequest(); - Promise> responsePromise = poolHandle.channel.eventLoop().newPromise(); + Promise> responsePromise = poolHandle.channel.eventLoop().newPromise(); ChannelPipeline pipeline = poolHandle.channel.pipeline(); pipeline.addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_FULL, new StreamFullHttpResponseHandler(responsePromise, parentRequest, finalRequest)); pipeline.addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_STREAM, new StreamStreamHttpResponseHandler(responsePromise, parentRequest, finalRequest)); @@ -1612,7 +1613,7 @@ private void streamRequestThroughChannel0( } requestWriter.write(poolHandle, secure, emitter); - responsePromise.addListener(future -> { + responsePromise.addListener((Future> future) -> { if (future.isSuccess()) { emitter.next(future.getNow()); emitter.complete(); @@ -2039,11 +2040,11 @@ private static class CurrentEvent { } private abstract class BaseHttpResponseHandler extends SimpleChannelInboundHandlerInstrumented { - private final Promise responsePromise; + private final Promise responsePromise; private final io.micronaut.http.HttpRequest parentRequest; private final io.micronaut.http.HttpRequest finalRequest; - public BaseHttpResponseHandler(Promise responsePromise, io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest finalRequest) { + public BaseHttpResponseHandler(Promise responsePromise, io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest finalRequest) { super(connectionManager.instrumenter); this.responsePromise = responsePromise; this.parentRequest = parentRequest; @@ -2340,8 +2341,12 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { } } - private class StreamFullHttpResponseHandler extends BaseHttpResponseHandler> { - public StreamFullHttpResponseHandler(Promise> responsePromise, io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest finalRequest) { + private class StreamFullHttpResponseHandler extends BaseHttpResponseHandler> { + public StreamFullHttpResponseHandler( + Promise> responsePromise, + io.micronaut.http.HttpRequest parentRequest, + io.micronaut.http.HttpRequest finalRequest) { + super(responsePromise, parentRequest, finalRequest); } @@ -2357,7 +2362,7 @@ protected void removeHandler(ChannelHandlerContext ctx) { } @Override - protected void buildResponse(Promise> promise, FullHttpResponse msg, HttpStatus httpStatus) { + protected void buildResponse(Promise> promise, FullHttpResponse msg, HttpStatus httpStatus) { Publisher bodyPublisher; if (msg.content() instanceof EmptyByteBuf) { bodyPublisher = Publishers.empty(); @@ -2374,13 +2379,17 @@ protected void buildResponse(Promise> promise, FullHttpR } @Override - protected Function>> makeRedirectHandler(io.micronaut.http.HttpRequest parentRequest, MutableHttpRequest redirectRequest) { + protected Function>> makeRedirectHandler(io.micronaut.http.HttpRequest parentRequest, MutableHttpRequest redirectRequest) { return uri -> buildStreamExchange(parentRequest, redirectRequest, uri, null); } } - private class StreamStreamHttpResponseHandler extends BaseHttpResponseHandler> { - public StreamStreamHttpResponseHandler(Promise> responsePromise, io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest finalRequest) { + private class StreamStreamHttpResponseHandler extends BaseHttpResponseHandler> { + public StreamStreamHttpResponseHandler( + Promise> responsePromise, + io.micronaut.http.HttpRequest parentRequest, + io.micronaut.http.HttpRequest finalRequest) { + super(responsePromise, parentRequest, finalRequest); } @@ -2396,12 +2405,12 @@ protected void removeHandler(ChannelHandlerContext ctx) { } @Override - protected void buildResponse(Promise> promise, StreamedHttpResponse msg, HttpStatus httpStatus) { + protected void buildResponse(Promise> promise, StreamedHttpResponse msg, HttpStatus httpStatus) { promise.trySuccess(new NettyStreamedHttpResponse<>(msg, httpStatus)); } @Override - protected Function>> makeRedirectHandler(io.micronaut.http.HttpRequest parentRequest, MutableHttpRequest redirectRequest) { + protected Function>> makeRedirectHandler(io.micronaut.http.HttpRequest parentRequest, MutableHttpRequest redirectRequest) { return uri -> buildStreamExchange(parentRequest, redirectRequest, uri, null); } } From 1626acb696315ce1eab06ecf60a2b1da2d3c363c Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 12:12:13 +0200 Subject: [PATCH 60/82] remove todo --- .../java/io/micronaut/http/client/netty/DefaultHttpClient.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index b48e7c8ea78..1a76028cb19 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -968,8 +968,7 @@ private Publisher> buildStreamExchange( applyFilterToResponsePublisher(parentRequest, request, requestURI, requestWrapper, streamResponsePublisher) ); - //return streamResponsePublisher.subscribeOn(connectionManager.getEventLoopScheduler()); - return streamResponsePublisher; // todo + return streamResponsePublisher; } @Override From 9d9460bbadfb579fd4f431c20fe6adf877cfcc7d Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 12:37:26 +0200 Subject: [PATCH 61/82] fix compile error --- .../io/micronaut/http/client/netty/DefaultHttpClient.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index 1a76028cb19..7d411b4be34 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -1589,8 +1589,8 @@ private void streamRequestThroughChannel0( permitsBody, null, throwable -> { - if (!((FluxSink>) emitter).isCancelled()) { - ((FluxSink>) emitter).error(throwable); + if (!emitter.isCancelled()) { + emitter.error(throwable); } } ); From e0b4ba7ae92a61b0ae73dc388a45f0662036d615 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 12:37:40 +0200 Subject: [PATCH 62/82] prevent double release --- .../http/client/netty/ConnectionManager.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index ebfff9a6530..eda282d3a4b 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -959,6 +959,7 @@ void satisfy0(Sinks.One sink) { } PoolHandle ph = new PoolHandle(false, channel) { final ChannelHandlerContext lastContext = channel.pipeline().lastContext(); + boolean released = false; @Override void taint() { @@ -967,6 +968,11 @@ void taint() { @Override void release() { + if (released) { + throw new IllegalStateException("Already released"); + } + released = true; + if (!windDownConnection) { ChannelHandlerContext newLast = channel.pipeline().lastContext(); if (lastContext != newLast) { @@ -1067,6 +1073,8 @@ void satisfy0(Sinks.One sink) { .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECOMPRESSOR, new HttpContentDecompressor()); NettyClientCustomizer streamCustomizer = connectionCustomizer.specializeForChannel(streamChannel, NettyClientCustomizer.ChannelRole.HTTP2_STREAM); PoolHandle ph = new PoolHandle(true, streamChannel) { + boolean released = false; + @Override void taint() { // do nothing, we don't reuse stream channels @@ -1074,6 +1082,11 @@ void taint() { @Override void release() { + if (released) { + throw new IllegalStateException("Already released"); + } + released = true; + liveStreamChannels.remove(streamChannel); streamChannel.close(); int newCount = liveRequests.decrementAndGet(); From 45be5c048f852aa2c2c9958bb76f42016b780279 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 12:38:56 +0200 Subject: [PATCH 63/82] remove terminateRequestChannel --- .../http/client/netty/DefaultHttpClient.java | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index 7d411b4be34..759e34f5ef9 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -874,8 +874,6 @@ private Flux>> exchangeStreamImpl(io.micronaut.ht thisResponse.setBody(byteBuffer); return (HttpResponse>) new HttpResponseWrapper<>(thisResponse); }); - }).doOnTerminate(() -> { - terminateRequestChannel(request); }); } @@ -912,8 +910,6 @@ private Flux jsonStreamImpl(io.micronaut.http.HttpRequest parentReq }, streamArray); return Flux.from(jsonProcessor) .map(jsonNode -> mediaTypeCodec.decode(type, jsonNode)); - }).doOnTerminate(() -> { - terminateRequestChannel(request); }); } @@ -932,22 +928,9 @@ private Flux> dataStreamImpl(io.micronaut.http.HttpRequest return httpContentReactiveSequence .filter(message -> !(message.content() instanceof EmptyByteBuf)) .map(contentMapper); - }) - .doOnTerminate(() -> { - terminateRequestChannel(request); }); } - private static void terminateRequestChannel(io.micronaut.http.HttpRequest req) { - final Object o = req.getAttribute(NettyClientHttpRequest.CHANNEL).orElse(null); - if (o instanceof Channel) { - final Channel c = (Channel) o; - if (c.isOpen()) { - //c.close(); TODO - } - } - } - /** * Implementation of {@link #jsonStream}, {@link #dataStream}, {@link #exchangeStream}. */ From 4337ca4feef43291b85282f3853c4ea014f9404c Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 12:48:56 +0200 Subject: [PATCH 64/82] implement pool handle leak detection --- .../http/client/netty/ConnectionManager.java | 42 +++++++++++-------- 1 file changed, 24 insertions(+), 18 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index eda282d3a4b..8746a401e53 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -19,6 +19,7 @@ import io.micronaut.core.annotation.Nullable; import io.micronaut.core.reflect.InstantiationUtils; import io.micronaut.core.util.StringUtils; +import io.micronaut.core.util.SupplierUtil; import io.micronaut.http.client.HttpClientConfiguration; import io.micronaut.http.client.HttpVersionSelection; import io.micronaut.http.client.exceptions.HttpClientException; @@ -74,6 +75,9 @@ import io.netty.resolver.NoopAddressResolverGroup; import io.netty.util.AttributeKey; import io.netty.util.ReferenceCountUtil; +import io.netty.util.ResourceLeakDetector; +import io.netty.util.ResourceLeakDetectorFactory; +import io.netty.util.ResourceLeakTracker; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; import io.netty.util.concurrent.ScheduledFuture; @@ -101,6 +105,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; /** * Connection manager for {@link DefaultHttpClient}. This class manages the lifecycle of netty @@ -680,10 +685,16 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { } } - abstract class PoolHandle { + static abstract class PoolHandle { + private static final Supplier> LEAK_DETECTOR = SupplierUtil.memoized(() -> + ResourceLeakDetectorFactory.instance().newResourceLeakDetector(PoolHandle.class)); + + private final ResourceLeakTracker tracker = LEAK_DETECTOR.get().track(this); final boolean http2; final Channel channel; + boolean released = false; + /** * Prevent this connection from being reused. */ @@ -697,7 +708,15 @@ private PoolHandle(boolean http2, Channel channel) { /** * Close this connection or release it back to the pool. */ - abstract void release(); + void release() { + if (released) { + throw new IllegalStateException("Already released"); + } + released = true; + if (tracker != null) { + tracker.close(this); + } + } /** * Whether this connection may be returned to a connection pool (i.e. should be kept @@ -710,9 +729,7 @@ private PoolHandle(boolean http2, Channel channel) { /** * Notify any {@link NettyClientCustomizer} that the request pipeline has been built. */ - void notifyRequestPipelineBuilt() { - channel.attr(CHANNEL_CUSTOMIZER_KEY).get().onRequestPipelineBuilt(); - } + abstract void notifyRequestPipelineBuilt(); } private final class Pool extends PoolResizer { @@ -959,7 +976,6 @@ void satisfy0(Sinks.One sink) { } PoolHandle ph = new PoolHandle(false, channel) { final ChannelHandlerContext lastContext = channel.pipeline().lastContext(); - boolean released = false; @Override void taint() { @@ -968,11 +984,7 @@ void taint() { @Override void release() { - if (released) { - throw new IllegalStateException("Already released"); - } - released = true; - + super.release(); if (!windDownConnection) { ChannelHandlerContext newLast = channel.pipeline().lastContext(); if (lastContext != newLast) { @@ -1073,8 +1085,6 @@ void satisfy0(Sinks.One sink) { .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECOMPRESSOR, new HttpContentDecompressor()); NettyClientCustomizer streamCustomizer = connectionCustomizer.specializeForChannel(streamChannel, NettyClientCustomizer.ChannelRole.HTTP2_STREAM); PoolHandle ph = new PoolHandle(true, streamChannel) { - boolean released = false; - @Override void taint() { // do nothing, we don't reuse stream channels @@ -1082,11 +1092,7 @@ void taint() { @Override void release() { - if (released) { - throw new IllegalStateException("Already released"); - } - released = true; - + super.release(); liveStreamChannels.remove(streamChannel); streamChannel.close(); int newCount = liveRequests.decrementAndGet(); From aeea7026eb99cdc78f9db615d7d57489c1849bb9 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 12:53:25 +0200 Subject: [PATCH 65/82] fix some todos --- .../java/io/micronaut/http/client/netty/ConnectionManager.java | 3 +-- .../main/java/io/micronaut/http/client/netty/PoolResizer.java | 3 ++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 8746a401e53..2022b792c99 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -593,7 +593,6 @@ private void initHttp2(Pool pool, Channel ch, NettyClientCustomizer connectionCu Http2MultiplexHandler multiplexHandler = new Http2MultiplexHandler(new ChannelInitializer() { @Override protected void initChannel(Http2StreamChannel ch) throws Exception { - // todo: fail connection? log.warn("Server opened HTTP2 stream {}, closing immediately", ch.stream().id()); ch.close(); } @@ -629,7 +628,7 @@ public void handlerAdded(ChannelHandlerContext ctx) throws Exception { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - // todo: log + log.warn("Unexpected message on HTTP2 connection channel: {}", msg); ReferenceCountUtil.release(msg); ctx.read(); } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java index 0995b4070e2..0c097ade210 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java @@ -192,7 +192,8 @@ static boolean incrementWithLimit(AtomicInteger variable, int limit) { // can be overridden, so `throws Exception` ensures we handle any errors void onNewConnectionFailure(@Nullable Throwable error) throws Exception { - // todo: circuit breaker? + // todo: implement a circuit breaker here? right now, we just fail one connection in the + // subclass implementation, but maybe we should do more. pendingConnectionCount.decrementAndGet(); dirty(); } From b13ee300dd54368ccac503fb91918ce82724d624 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 12:53:33 +0200 Subject: [PATCH 66/82] spotless --- .../http/client/netty/CancellableMonoSink.java | 15 +++++++++++++++ .../client/netty/HttpLineBasedFrameDecoder.java | 15 +++++++++++++++ .../netty/InitialConnectionErrorHandler.java | 15 +++++++++++++++ 3 files changed, 45 insertions(+) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java b/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java index ae9765bc7b2..492e4a97c84 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017-2022 original authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package io.micronaut.http.client.netty; import io.micronaut.core.annotation.Internal; diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java b/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java index 08678346ad5..7b9680e59d1 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017-2022 original authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package io.micronaut.http.client.netty; import io.micronaut.core.annotation.Internal; diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java b/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java index 0c333c343e7..5c6aea4d4f3 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/InitialConnectionErrorHandler.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017-2022 original authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package io.micronaut.http.client.netty; import io.micronaut.core.annotation.Internal; From 4ed49489334fdd23ab8652bbfabd7c82c22fbefb Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 13:08:04 +0200 Subject: [PATCH 67/82] checkstyle --- .../http/client/HttpClientConfiguration.java | 32 ++++ .../http/client/HttpVersionSelection.java | 4 +- .../client/netty/CancellableMonoSink.java | 2 + .../http/client/netty/ConnectionManager.java | 171 +++++++++--------- .../http/client/netty/PoolResizer.java | 4 +- .../netty/ssl/NettyClientSslBuilder.java | 2 +- 6 files changed, 124 insertions(+), 91 deletions(-) diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java index 1a906d9648e..fd4d85ac306 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java @@ -835,26 +835,58 @@ public void setMaxPendingConnections(int maxPendingConnections) { this.maxPendingConnections = maxPendingConnections; } + /** + * The maximum number of requests (streams) that can run concurrently on one HTTP2 + * connection. + * + * @return The maximum concurrent request count + */ public int getMaxConcurrentRequestsPerHttp2Connection() { return maxConcurrentRequestsPerHttp2Connection; } + /** + * The maximum number of requests (streams) that can run concurrently on one HTTP2 + * connection. + * + * @param maxConcurrentRequestsPerHttp2Connection The maximum concurrent request count + */ public void setMaxConcurrentRequestsPerHttp2Connection(int maxConcurrentRequestsPerHttp2Connection) { this.maxConcurrentRequestsPerHttp2Connection = maxConcurrentRequestsPerHttp2Connection; } + /** + * The maximum number of concurrent HTTP1 connections in the pool. + * + * @return The maximum concurrent connection count + */ public int getMaxConcurrentHttp1Connections() { return maxConcurrentHttp1Connections; } + /** + * The maximum number of concurrent HTTP1 connections in the pool. + * + * @param maxConcurrentHttp1Connections The maximum concurrent connection count + */ public void setMaxConcurrentHttp1Connections(int maxConcurrentHttp1Connections) { this.maxConcurrentHttp1Connections = maxConcurrentHttp1Connections; } + /** + * The maximum number of concurrent HTTP2 connections in the pool. + * + * @return The maximum concurrent connection count + */ public int getMaxConcurrentHttp2Connections() { return maxConcurrentHttp2Connections; } + /** + * The maximum number of concurrent HTTP2 connections in the pool. + * + * @param maxConcurrentHttp2Connections The maximum concurrent connection count + */ public void setMaxConcurrentHttp2Connections(int maxConcurrentHttp2Connections) { this.maxConcurrentHttp2Connections = maxConcurrentHttp2Connections; } diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java index 922e85322d9..b54e67b2d98 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java @@ -32,11 +32,11 @@ */ public final class HttpVersionSelection { /** - * ALPN protocol ID for HTTP/1.1 + * ALPN protocol ID for HTTP/1.1. */ public static final String ALPN_HTTP_1 = "http/1.1"; /** - * ALPN protocol ID for HTTP/2 + * ALPN protocol ID for HTTP/2. */ public static final String ALPN_HTTP_2 = "h2"; diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java b/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java index 492e4a97c84..c56a84accf5 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/CancellableMonoSink.java @@ -26,6 +26,8 @@ /** * Version of {@link Sinks#one()} where cancellation of the {@link Mono} will make future emit * calls fail. + * + * @param Element type */ @Internal final class CancellableMonoSink implements Publisher, Sinks.One, Subscription { diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 2022b792c99..3a7af220030 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -73,7 +73,6 @@ import io.netty.handler.timeout.ReadTimeoutException; import io.netty.handler.timeout.ReadTimeoutHandler; import io.netty.resolver.NoopAddressResolverGroup; -import io.netty.util.AttributeKey; import io.netty.util.ReferenceCountUtil; import io.netty.util.ResourceLeakDetector; import io.netty.util.ResourceLeakDetectorFactory; @@ -113,12 +112,9 @@ */ @Internal class ConnectionManager { - private final AttributeKey CHANNEL_CUSTOMIZER_KEY = - AttributeKey.valueOf("micronaut.http.customizer"); - final InvocationInstrumenter instrumenter; - final HttpVersionSelection httpVersion; + private final HttpVersionSelection httpVersion; private final Logger log; private final Map pools = new ConcurrentHashMap<>(); private EventLoopGroup group; @@ -235,7 +231,9 @@ private static NioEventLoopGroup createEventLoopGroup(HttpClientConfiguration co } /** - * For testing + * For testing. + * + * @return Connected channels in all pools */ List getChannels() { List channels = new ArrayList<>(); @@ -246,7 +244,9 @@ List getChannels() { } /** - * For testing + * For testing. + * + * @return Number of running requests */ int liveRequestCount() { AtomicInteger count = new AtomicInteger(); @@ -492,13 +492,78 @@ private Http2FrameCodec makeFrameCodec() { return builder.build(); } + private void initHttp1(Channel ch) { + addLogHandler(ch); + + ch.pipeline() + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()) + .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); + } + + private void addLogHandler(Channel ch) { + configuration.getLogLevel().ifPresent(logLevel -> { + try { + final io.netty.handler.logging.LogLevel nettyLevel = + io.netty.handler.logging.LogLevel.valueOf(logLevel.name()); + ch.pipeline().addLast(new LoggingHandler(DefaultHttpClient.class, nettyLevel)); + } catch (IllegalArgumentException e) { + throw customizeException(new HttpClientException("Unsupported log level: " + logLevel)); + } + }); + } + + private void initHttp2(Pool pool, Channel ch, NettyClientCustomizer connectionCustomizer) { + Http2MultiplexHandler multiplexHandler = new Http2MultiplexHandler(new ChannelInitializer() { + @Override + protected void initChannel(Http2StreamChannel ch) throws Exception { + log.warn("Server opened HTTP2 stream {}, closing immediately", ch.stream().id()); + ch.close(); + } + }, new ChannelInitializer() { + @Override + protected void initChannel(Http2StreamChannel ch) throws Exception { + // discard any response data for the upgrade request + ch.close(); + } + }); + ch.pipeline().addLast(multiplexHandler); + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new ChannelInboundHandlerAdapter() { + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof Http2SettingsFrame) { + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS); + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR); + pool.new Http2ConnectionHolder(ch, connectionCustomizer).init(); + return; + } else { + log.warn("Premature frame: {}", msg.getClass()); + } + + super.channelRead(ctx, msg); + } + }); + // stream frames should be handled by the multiplexer + ch.pipeline().addLast(new ChannelInboundHandlerAdapter() { + @Override + public void handlerAdded(ChannelHandlerContext ctx) throws Exception { + ctx.read(); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + log.warn("Unexpected message on HTTP2 connection channel: {}", msg); + ReferenceCountUtil.release(msg); + ctx.read(); + } + }); + } + private class AdaptiveAlpnChannelInitializer extends ChannelInitializer { private final Pool pool; - final SslContext sslContext; - final String host; - final int port; - private NettyClientCustomizer channelCustomizer; + private final SslContext sslContext; + private final String host; + private final int port; AdaptiveAlpnChannelInitializer(Pool pool, SslContext sslContext, @@ -515,8 +580,7 @@ private class AdaptiveAlpnChannelInitializer extends ChannelInitializer */ @Override protected void initChannel(Channel ch) { - channelCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); - ch.attr(CHANNEL_CUSTOMIZER_KEY).set(channelCustomizer); + NettyClientCustomizer channelCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); configureProxy(ch.pipeline(), true, host, port); @@ -569,72 +633,6 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E } } - private void initHttp1(Channel ch) { - addLogHandler(ch); - - ch.pipeline() - .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_CLIENT_CODEC, new HttpClientCodec()) - .addLast(ChannelPipelineCustomizer.HANDLER_HTTP_DECODER, new HttpContentDecompressor()); - } - - private void addLogHandler(Channel ch) { - configuration.getLogLevel().ifPresent(logLevel -> { - try { - final io.netty.handler.logging.LogLevel nettyLevel = - io.netty.handler.logging.LogLevel.valueOf(logLevel.name()); - ch.pipeline().addLast(new LoggingHandler(DefaultHttpClient.class, nettyLevel)); - } catch (IllegalArgumentException e) { - throw customizeException(new HttpClientException("Unsupported log level: " + logLevel)); - } - }); - } - - private void initHttp2(Pool pool, Channel ch, NettyClientCustomizer connectionCustomizer) { - Http2MultiplexHandler multiplexHandler = new Http2MultiplexHandler(new ChannelInitializer() { - @Override - protected void initChannel(Http2StreamChannel ch) throws Exception { - log.warn("Server opened HTTP2 stream {}, closing immediately", ch.stream().id()); - ch.close(); - } - }, new ChannelInitializer() { - @Override - protected void initChannel(Http2StreamChannel ch) throws Exception { - // discard any response data for the upgrade request - ch.close(); - } - }); - ch.pipeline().addLast(multiplexHandler); - ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new ChannelInboundHandlerAdapter() { - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof Http2SettingsFrame) { - ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS); - ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR); - pool.new Http2ConnectionHolder(ch, connectionCustomizer).init(); - return; - } else { - log.warn("Premature frame: {}", msg.getClass()); - } - - super.channelRead(ctx, msg); - } - }); - // stream frames should be handled by the multiplexer - ch.pipeline().addLast(new ChannelInboundHandlerAdapter() { - @Override - public void handlerAdded(ChannelHandlerContext ctx) throws Exception { - ctx.read(); - } - - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - log.warn("Unexpected message on HTTP2 connection channel: {}", msg); - ReferenceCountUtil.release(msg); - ctx.read(); - } - }); - } - private class Http2UpgradeInitializer extends ChannelInitializer { private final Pool pool; @@ -684,26 +682,27 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { } } - static abstract class PoolHandle { + abstract static class PoolHandle { private static final Supplier> LEAK_DETECTOR = SupplierUtil.memoized(() -> ResourceLeakDetectorFactory.instance().newResourceLeakDetector(PoolHandle.class)); - private final ResourceLeakTracker tracker = LEAK_DETECTOR.get().track(this); final boolean http2; final Channel channel; boolean released = false; - /** - * Prevent this connection from being reused. - */ - abstract void taint(); + private final ResourceLeakTracker tracker = LEAK_DETECTOR.get().track(this); private PoolHandle(boolean http2, Channel channel) { this.http2 = http2; this.channel = channel; } + /** + * Prevent this connection from being reused. + */ + abstract void taint(); + /** * Close this connection or release it back to the pool. */ @@ -734,7 +733,7 @@ void release() { private final class Pool extends PoolResizer { private final DefaultHttpClient.RequestKey requestKey; - final InitialConnectionErrorHandler initialErrorHandler = new InitialConnectionErrorHandler() { + private final InitialConnectionErrorHandler initialErrorHandler = new InitialConnectionErrorHandler() { @Override protected void onNewConnectionFailure(@Nullable Throwable cause) throws Exception { Pool.this.onNewConnectionFailure(cause); diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java index 0c097ade210..bc2cb71f31a 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java @@ -32,8 +32,8 @@ /** * This class handles the sizing of a connection pool to conform to the configuration in * {@link io.micronaut.http.client.HttpClientConfiguration.ConnectionPoolConfiguration}. - * - * @implNote This class consists of various mutator methods (e.g. {@link #addPendingRequest}) that + *

+ * This class consists of various mutator methods (e.g. {@link #addPendingRequest}) that * may be called concurrently and in a reentrant fashion (e.g. inside {@link #openNewConnection()}). * These mutator methods update their respective fields and then mark this class as * {@link #dirty()}. The state management logic ensures that {@link #doSomeWork()} is called in a diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java b/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java index 79bca822002..e8e3864415f 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ssl/NettyClientSslBuilder.java @@ -54,7 +54,7 @@ @Singleton @Internal @BootstrapContextCompatible -public class NettyClientSslBuilder extends SslBuilder { +public final class NettyClientSslBuilder extends SslBuilder { private static final Logger LOG = LoggerFactory.getLogger(NettyClientSslBuilder.class); /** From e780c186165c0ab445b7b7fb1d8fa4aeb84309e3 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 13:58:03 +0200 Subject: [PATCH 68/82] docs --- .../http/client/netty/ConnectionManager.java | 185 +++++++++++++----- .../http/client/netty/DefaultHttpClient.java | 5 - .../netty/DefaultNettyHttpClientRegistry.java | 1 - 3 files changed, 136 insertions(+), 55 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 3a7af220030..12a25e7137b 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -16,6 +16,7 @@ package io.micronaut.http.client.netty; import io.micronaut.core.annotation.Internal; +import io.micronaut.core.annotation.NonNull; import io.micronaut.core.annotation.Nullable; import io.micronaut.core.reflect.InstantiationUtils; import io.micronaut.core.util.StringUtils; @@ -25,7 +26,6 @@ import io.micronaut.http.client.exceptions.HttpClientException; import io.micronaut.http.client.netty.ssl.NettyClientSslBuilder; import io.micronaut.http.netty.channel.ChannelPipelineCustomizer; -import io.micronaut.http.netty.channel.ChannelPipelineListener; import io.micronaut.http.netty.channel.NettyThreadFactory; import io.micronaut.scheduling.instrument.Instrumentation; import io.micronaut.scheduling.instrument.InvocationInstrumenter; @@ -83,7 +83,6 @@ import org.slf4j.Logger; import reactor.core.publisher.Mono; import reactor.core.publisher.Sinks; -import reactor.core.scheduler.Scheduler; import reactor.core.scheduler.Schedulers; import javax.net.ssl.SSLException; @@ -92,7 +91,6 @@ import java.net.SocketAddress; import java.time.Duration; import java.util.ArrayList; -import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -124,9 +122,13 @@ class ConnectionManager { private final HttpClientConfiguration configuration; private final SslContext sslContext; private final NettyClientCustomizer clientCustomizer; - private final Collection pipelineListeners; private final String informationalServiceId; + /** + * Copy constructor used by the test suite to patch this manager + * + * @param from Original connection manager + */ ConnectionManager(ConnectionManager from) { this.instrumenter = from.instrumenter; this.httpVersion = from.httpVersion; @@ -138,7 +140,6 @@ class ConnectionManager { this.configuration = from.configuration; this.sslContext = from.sslContext; this.clientCustomizer = from.clientCustomizer; - this.pipelineListeners = from.pipelineListeners; this.informationalServiceId = from.informationalServiceId; } @@ -152,7 +153,6 @@ class ConnectionManager { ChannelFactory socketChannelFactory, NettyClientSslBuilder nettyClientSslBuilder, NettyClientCustomizer clientCustomizer, - Collection pipelineListeners, String informationalServiceId) { if (httpVersion == null) { @@ -165,7 +165,6 @@ class ConnectionManager { this.configuration = configuration; this.instrumenter = instrumenter; this.clientCustomizer = clientCustomizer; - this.pipelineListeners = pipelineListeners; this.informationalServiceId = informationalServiceId; this.sslContext = nettyClientSslBuilder.build(configuration.getSslConfiguration(), httpVersion); @@ -235,6 +234,7 @@ private static NioEventLoopGroup createEventLoopGroup(HttpClientConfiguration co * * @return Connected channels in all pools */ + @SuppressWarnings("unused") List getChannels() { List channels = new ArrayList<>(); for (Pool pool : pools.values()) { @@ -248,6 +248,7 @@ List getChannels() { * * @return Number of running requests */ + @SuppressWarnings("unused") int liveRequestCount() { AtomicInteger count = new AtomicInteger(); for (Pool pool : pools.values()) { @@ -308,29 +309,23 @@ public boolean isRunning() { } /** - * Get a reactive scheduler that runs on the event loop group of this connection manager. + * Use the bootstrap to connect to the given host. Also does some proxy setup. This method is + * protected: The test suite overrides it to return embedded channels instead. * - * @return A scheduler that runs on the event loop + * @param requestKey The host to connect to + * @param channelInitializer The initializer to use + * @return Future that terminates when the TCP connection is established. */ - public Scheduler getEventLoopScheduler() { - return Schedulers.fromExecutor(group); - } - - // for testing protected ChannelFuture doConnect(DefaultHttpClient.RequestKey requestKey, ChannelInitializer channelInitializer) { String host = requestKey.getHost(); int port = requestKey.getPort(); Bootstrap localBootstrap = bootstrap.clone(); - initBootstrapForProxy(localBootstrap, requestKey.isSecure(), host, port); - localBootstrap.handler(channelInitializer); - return localBootstrap.connect(host, port); - } - - private void initBootstrapForProxy(Bootstrap localBootstrap, boolean sslCtx, String host, int port) { - Proxy proxy = configuration.resolveProxy(sslCtx, host, port); + Proxy proxy = configuration.resolveProxy(requestKey.isSecure(), host, port); if (proxy.type() != Proxy.Type.DIRECT) { localBootstrap.resolver(NoopAddressResolverGroup.INSTANCE); } + localBootstrap.handler(channelInitializer); + return localBootstrap.connect(host, port); } /** @@ -338,6 +333,7 @@ private void initBootstrapForProxy(Bootstrap localBootstrap, boolean sslCtx, Str * * @return The {@link SslContext} instance */ + @Nullable private SslContext buildSslContext(DefaultHttpClient.RequestKey requestKey) { final SslContext sslCtx; if (requestKey.isSecure()) { @@ -375,7 +371,7 @@ Mono connectForWebsocket(DefaultHttpClient.RequestKey requestKey, ChannelHand ChannelFuture connectFuture = doConnect(requestKey, new ChannelInitializer() { @Override - protected void initChannel(Channel ch) throws Exception { + protected void initChannel(@NonNull Channel ch) { addLogHandler(ch); SslContext sslContext = buildSslContext(requestKey); @@ -463,10 +459,11 @@ private void configureProxy(ChannelPipeline pipeline, boolean secure, String hos } } - > Future addInstrumentedListener( + > void addInstrumentedListener( Future channelFuture, GenericFutureListener listener) { - return channelFuture.addListener(f -> { + channelFuture.addListener(f -> { try (Instrumentation ignored = instrumenter.newInstrumentation()) { + //noinspection unchecked listener.operationComplete((C) f); } }); @@ -481,9 +478,8 @@ private Http2FrameCodec makeFrameCodec() { Http2FrameCodecBuilder builder = Http2FrameCodecBuilder.forClient(); configuration.getLogLevel().ifPresent(logLevel -> { try { - final io.netty.handler.logging.LogLevel nettyLevel = io.netty.handler.logging.LogLevel.valueOf( - logLevel.name() - ); + final io.netty.handler.logging.LogLevel nettyLevel = + io.netty.handler.logging.LogLevel.valueOf(logLevel.name()); builder.frameLogger(new Http2FrameLogger(nettyLevel, DefaultHttpClient.class)); } catch (IllegalArgumentException e) { throw customizeException(new HttpClientException("Unsupported log level: " + logLevel)); @@ -492,6 +488,11 @@ private Http2FrameCodec makeFrameCodec() { return builder.build(); } + /** + * Initializer for HTTP1.1, called either in plaintext mode, or after ALPN in TLS. + * + * @param ch The plaintext channel + */ private void initHttp1(Channel ch) { addLogHandler(ch); @@ -512,16 +513,25 @@ private void addLogHandler(Channel ch) { }); } + /** + * Initializer for HTTP2 multiplexing, called either in h2c mode, or after ALPN in TLS. The + * channel should already contain a {@link #makeFrameCodec() frame codec} that does the HTTP2 + * parsing, this method adds the handlers that do multiplexing, error handling, etc. + * + * @param pool The pool to add the connection to once the handshake is done + * @param ch The plaintext channel + * @param connectionCustomizer Customizer for the connection + */ private void initHttp2(Pool pool, Channel ch, NettyClientCustomizer connectionCustomizer) { Http2MultiplexHandler multiplexHandler = new Http2MultiplexHandler(new ChannelInitializer() { @Override - protected void initChannel(Http2StreamChannel ch) throws Exception { + protected void initChannel(@NonNull Http2StreamChannel ch) throws Exception { log.warn("Server opened HTTP2 stream {}, closing immediately", ch.stream().id()); ch.close(); } }, new ChannelInitializer() { @Override - protected void initChannel(Http2StreamChannel ch) throws Exception { + protected void initChannel(@NonNull Http2StreamChannel ch) throws Exception { // discard any response data for the upgrade request ch.close(); } @@ -529,7 +539,7 @@ protected void initChannel(Http2StreamChannel ch) throws Exception { ch.pipeline().addLast(multiplexHandler); ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS, new ChannelInboundHandlerAdapter() { @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + public void channelRead(@NonNull ChannelHandlerContext ctx, @NonNull Object msg) throws Exception { if (msg instanceof Http2SettingsFrame) { ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP2_SETTINGS); ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_INITIAL_ERROR); @@ -550,7 +560,7 @@ public void handlerAdded(ChannelHandlerContext ctx) throws Exception { } @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + public void channelRead(@NonNull ChannelHandlerContext ctx, @NonNull Object msg) throws Exception { log.warn("Unexpected message on HTTP2 connection channel: {}", msg); ReferenceCountUtil.release(msg); ctx.read(); @@ -558,6 +568,10 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception }); } + /** + * Initializer for TLS channels. After ALPN we will proceed either with + * {@link #initHttp1(Channel)} or {@link #initHttp2(Pool, Channel, NettyClientCustomizer)}. + */ private class AdaptiveAlpnChannelInitializer extends ChannelInitializer { private final Pool pool; @@ -579,7 +593,7 @@ private class AdaptiveAlpnChannelInitializer extends ChannelInitializer * @param ch The channel */ @Override - protected void initChannel(Channel ch) { + protected void initChannel(@NonNull Channel ch) { NettyClientCustomizer channelCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); configureProxy(ch.pipeline(), true, host, port); @@ -590,6 +604,7 @@ protected void initChannel(Channel ch) { .addLast(ChannelPipelineCustomizer.HANDLER_SSL, sslHandler) .addLast( ChannelPipelineCustomizer.HANDLER_HTTP2_PROTOCOL_NEGOTIATOR, + // if the server doesn't do ALPN, fall back to HTTP 1 new ApplicationProtocolNegotiationHandler(ApplicationProtocolNames.HTTP_1_1) { @Override protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { @@ -633,6 +648,10 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E } } + /** + * Initializer for H2C connections. Will proceed with + * {@link #initHttp2(Pool, Channel, NettyClientCustomizer)} when the upgrade is done. + */ private class Http2UpgradeInitializer extends ChannelInitializer { private final Pool pool; @@ -641,7 +660,7 @@ private class Http2UpgradeInitializer extends ChannelInitializer { } @Override - protected void initChannel(Channel ch) throws Exception { + protected void initChannel(@NonNull Channel ch) throws Exception { NettyClientCustomizer connectionCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); Http2FrameCodec frameCodec = makeFrameCodec(); @@ -650,7 +669,7 @@ protected void initChannel(Channel ch) throws Exception { Http2ClientUpgradeCodec upgradeCodec = new Http2ClientUpgradeCodec(frameCodec, new ChannelInitializer() { @Override - protected void initChannel(Channel ch) throws Exception { + protected void initChannel(@NonNull Channel ch) throws Exception { ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_CONNECTION, frameCodec); initHttp2(pool, ch, connectionCustomizer); } @@ -662,7 +681,7 @@ protected void initChannel(Channel ch) throws Exception { ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_HTTP2_UPGRADE_REQUEST, new ChannelInboundHandlerAdapter() { @Override - public void channelActive(ChannelHandlerContext ctx) throws Exception { + public void channelActive(@NonNull ChannelHandlerContext ctx) throws Exception { DefaultFullHttpRequest upgradeRequest = new DefaultFullHttpRequest(io.netty.handler.codec.http.HttpVersion.HTTP_1_1, HttpMethod.GET, "/", Unpooled.EMPTY_BUFFER); @@ -682,6 +701,11 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { } } + /** + * Handle for a pooled connection. One pool handle generally corresponds to one request, and + * once the request and response are done, the handle is {@link #release() released} and a new + * request can claim the same connection. + */ abstract static class PoolHandle { private static final Supplier> LEAK_DETECTOR = SupplierUtil.memoized(() -> ResourceLeakDetectorFactory.instance().newResourceLeakDetector(PoolHandle.class)); @@ -699,7 +723,8 @@ private PoolHandle(boolean http2, Channel channel) { } /** - * Prevent this connection from being reused. + * Prevent this connection from being reused, e.g. because garbage was written because of + * an error. */ abstract void taint(); @@ -730,9 +755,21 @@ void release() { abstract void notifyRequestPipelineBuilt(); } + /** + * This class represents one pool, and matches to exactly one + * {@link io.micronaut.http.client.netty.DefaultHttpClient.RequestKey} (i.e. host, port and + * protocol are the same for one pool). + *

+ * The superclass {@link PoolResizer} handles pool size management, this class just implements + * the HTTP parts. + */ private final class Pool extends PoolResizer { private final DefaultHttpClient.RequestKey requestKey; + /** + * {@link ChannelHandler} that is added to a connection to report failures during + * handshakes. It's removed once the connection is established and processes requests. + */ private final InitialConnectionErrorHandler initialErrorHandler = new InitialConnectionErrorHandler() { @Override protected void onNewConnectionFailure(@Nullable Throwable cause) throws Exception { @@ -749,8 +786,9 @@ Mono acquire() { Sinks.One sink = new CancellableMonoSink<>(); addPendingRequest(sink); Optional acquireTimeout = configuration.getConnectionPoolConfiguration().getAcquireTimeout(); + //noinspection OptionalIsPresent if (acquireTimeout.isPresent()) { - return sink.asMono().timeout(acquireTimeout.get(), getEventLoopScheduler()); + return sink.asMono().timeout(acquireTimeout.get(), Schedulers.fromExecutor(group)); } else { return sink.asMono(); } @@ -793,12 +831,12 @@ void openNewConnection() { case HTTP_1: initializer = new ChannelInitializer() { @Override - protected void initChannel(Channel ch) throws Exception { + protected void initChannel(@NonNull Channel ch) throws Exception { configureProxy(ch.pipeline(), false, requestKey.getHost(), requestKey.getPort()); initHttp1(ch); ch.pipeline().addLast("activity-listener", new ChannelInboundHandlerAdapter() { @Override - public void channelActive(ChannelHandlerContext ctx) throws Exception { + public void channelActive(@NonNull ChannelHandlerContext ctx) throws Exception { super.channelActive(ctx); ctx.pipeline().remove(this); NettyClientCustomizer channelCustomizer = clientCustomizer.specializeForChannel(ch, NettyClientCustomizer.ChannelRole.CONNECTION); @@ -826,9 +864,17 @@ public void shutdown() { forEachConnection(c -> ((ConnectionHolder) c).channel.close()); } + /** + * Base class for one HTTP1/HTTP2 connection. + */ abstract class ConnectionHolder extends ResizerConnection { final Channel channel; final NettyClientCustomizer connectionCustomizer; + /** + * Future for the scheduled task that runs when the configured time-to-live for the + * connection passes. + */ + @Nullable ScheduledFuture ttlFuture; volatile boolean windDownConnection = false; @@ -837,12 +883,19 @@ abstract class ConnectionHolder extends ResizerConnection { this.connectionCustomizer = connectionCustomizer; } + /** + * Add connection-level timeout-related handlers to the channel + * (read timeout, TTL, ...). + * + * @param before Reference handler name, the timeout handlers will be placed before + * this handler. + */ final void addTimeoutHandlers(String before) { // read timeout handles timeouts *during* a request configuration.getReadTimeout() .ifPresent(dur -> channel.pipeline().addBefore(before, ChannelPipelineCustomizer.HANDLER_READ_TIMEOUT, new ReadTimeoutHandler(dur.toNanos(), TimeUnit.NANOSECONDS) { @Override - protected void readTimedOut(ChannelHandlerContext ctx) throws Exception { + protected void readTimedOut(ChannelHandlerContext ctx) { if (hasLiveRequests()) { fireReadTimeout(ctx); ctx.close(); @@ -853,7 +906,7 @@ protected void readTimedOut(ChannelHandlerContext ctx) throws Exception { configuration.getConnectionPoolIdleTimeout() .ifPresent(dur -> channel.pipeline().addBefore(before, ChannelPipelineCustomizer.HANDLER_IDLE_STATE, new ReadTimeoutHandler(dur.toNanos(), TimeUnit.NANOSECONDS) { @Override - protected void readTimedOut(ChannelHandlerContext ctx) throws Exception { + protected void readTimedOut(ChannelHandlerContext ctx) { if (!hasLiveRequests()) { ctx.close(); } @@ -865,7 +918,7 @@ protected void readTimedOut(ChannelHandlerContext ctx) throws Exception { boolean inactiveCalled = false; @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { + public void channelInactive(@NonNull ChannelHandlerContext ctx) throws Exception { super.channelInactive(ctx); if (!inactiveCalled) { inactiveCalled = true; @@ -874,7 +927,7 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } @Override - public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + public void handlerRemoved(ChannelHandlerContext ctx) { if (!inactiveCalled) { inactiveCalled = true; onInactive(); @@ -883,10 +936,20 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { }); } + /** + * Stop accepting new requests on this connection, but finish up the running requests + * if possible. + */ void windDownConnection() { windDownConnection = true; } + /** + * Send the finished pool handle to the given requester, if possible. + * + * @param sink The request for a pool handle + * @param ph The pool handle + */ final void emitPoolHandle(Sinks.One sink, PoolHandle ph) { Sinks.EmitResult emitResult = sink.tryEmitValue(ph); if (emitResult.isFailure()) { @@ -906,21 +969,45 @@ public boolean dispatch(Sinks.One sink) { } if (channel.eventLoop().inEventLoop()) { - satisfy0(sink); + dispatch0(sink); } else { - channel.eventLoop().execute(() -> satisfy0(sink)); + channel.eventLoop().execute(() -> dispatch0(sink)); } return true; } - abstract void satisfy0(Sinks.One sink); - + /** + * Called on event loop only. Dispatch a stream/connection to the given pool + * handle request. + * + * @param sink The request for a pool handle + */ + abstract void dispatch0(Sinks.One sink); + + /** + * Try to add a new request to this connection. This is called outside the event loop, + * and if this succeeds, we will proceed with a {@link #dispatch0} call on the + * event loop. + * + * @return {@code true} if the request may be added to this connection + */ abstract boolean tryEarmarkForRequest(); + /** + * @return {@code true} iff there are any requests running on this connection. + */ abstract boolean hasLiveRequests(); + /** + * Send a read timeout exception to all requests on this connection. + * + * @param ctx The connection-level channel handler context to use. + */ abstract void fireReadTimeout(ChannelHandlerContext ctx); + /** + * Called when the connection becomes inactive, i.e. on disconnect. + */ void onInactive() { if (ttlFuture != null) { ttlFuture.cancel(false); @@ -967,7 +1054,7 @@ void fireReadTimeout(ChannelHandlerContext ctx) { } @Override - void satisfy0(Sinks.One sink) { + void dispatch0(Sinks.One sink) { if (!channel.isActive()) { returnPendingRequest(sink); return; @@ -1070,7 +1157,7 @@ void fireReadTimeout(ChannelHandlerContext ctx) { } @Override - void satisfy0(Sinks.One sink) { + void dispatch0(Sinks.One sink) { if (!channel.isActive() || windDownConnection) { returnPendingRequest(sink); return; diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index 759e34f5ef9..cfcfba8ae9a 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -81,7 +81,6 @@ import io.micronaut.http.netty.NettyHttpRequestBuilder; import io.micronaut.http.netty.NettyHttpResponseBuilder; import io.micronaut.http.netty.channel.ChannelPipelineCustomizer; -import io.micronaut.http.netty.channel.ChannelPipelineListener; import io.micronaut.http.netty.stream.DefaultStreamedHttpResponse; import io.micronaut.http.netty.stream.HttpStreamsClientHandler; import io.micronaut.http.netty.stream.JsonSubscriber; @@ -289,7 +288,6 @@ public DefaultHttpClient(@Nullable LoadBalancer loadBalancer, new DefaultRequestBinderRegistry(ConversionService.SHARED), null, NioSocketChannel::new, - Collections.emptySet(), CompositeNettyClientCustomizer.EMPTY, invocationInstrumenterFactories, null); } @@ -309,7 +307,6 @@ public DefaultHttpClient(@Nullable LoadBalancer loadBalancer, * @param requestBinderRegistry The request binder registry * @param eventLoopGroup The event loop group to use * @param socketChannelFactory The socket channel factory - * @param pipelineListeners The listeners to call for pipeline customization * @param clientCustomizer The pipeline customizer * @param invocationInstrumenterFactories The invocation instrumeter factories to instrument netty handlers execution with * @param informationalServiceId Optional service ID that will be passed to exceptions created by this client @@ -327,7 +324,6 @@ public DefaultHttpClient(@Nullable LoadBalancer loadBalancer, @NonNull RequestBinderRegistry requestBinderRegistry, @Nullable EventLoopGroup eventLoopGroup, @NonNull ChannelFactory socketChannelFactory, - Collection pipelineListeners, NettyClientCustomizer clientCustomizer, List invocationInstrumenterFactories, @Nullable String informationalServiceId @@ -378,7 +374,6 @@ public DefaultHttpClient(@Nullable LoadBalancer loadBalancer, socketChannelFactory, nettyClientSslBuilder, clientCustomizer, - pipelineListeners, informationalServiceId); } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java index 2f219ef9a31..23193bc14b8 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java @@ -418,7 +418,6 @@ private DefaultHttpClient buildClient( ), eventLoopGroup, resolveSocketChannelFactory(configuration, beanContext), - pipelineListeners, clientCustomizer, invocationInstrumenterFactories, clientId From 966cd48a280e16e31dfa3f3a837ae7402f4dae01 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 15:00:11 +0200 Subject: [PATCH 69/82] sonar --- .../io/micronaut/http/client/netty/ConnectionManager.java | 2 +- .../io/micronaut/http/client/netty/DefaultHttpClient.java | 7 +++---- .../java/io/micronaut/http/client/netty/PoolResizer.java | 6 +++--- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 12a25e7137b..384b6d50d5e 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -125,7 +125,7 @@ class ConnectionManager { private final String informationalServiceId; /** - * Copy constructor used by the test suite to patch this manager + * Copy constructor used by the test suite to patch this manager. * * @param from Original connection manager */ diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index cfcfba8ae9a..3c6d05924b9 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -1066,10 +1066,9 @@ private Publisher> exchang @Override protected void finishAggregation(FullHttpMessage aggregated) throws Exception { // only set content-length if there's any content - if (!HttpUtil.isContentLengthSet(aggregated)) { - if (aggregated.content().readableBytes() > 0) { - super.finishAggregation(aggregated); - } + if (!HttpUtil.isContentLengthSet(aggregated) && + aggregated.content().readableBytes() > 0) { + super.finishAggregation(aggregated); } } }) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java index bc2cb71f31a..7a1fda0cbc4 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java @@ -123,16 +123,16 @@ private void doSomeWork() { } // snapshot our fields - int pendingRequests = this.pendingRequests.size(); + int pendingRequestCount = this.pendingRequests.size(); int pendingConnectionCount = this.pendingConnectionCount.get(); int http1ConnectionCount = this.http1Connections.size(); int http2ConnectionCount = this.http2Connections.size(); - if (pendingRequests == 0) { + if (pendingRequestCount == 0) { // if there are no pending requests, there is nothing to do. return; } - int connectionsToOpen = pendingRequests - pendingConnectionCount; + int connectionsToOpen = pendingRequestCount - pendingConnectionCount; // make sure we won't exceed our config setting for pending connections connectionsToOpen = Math.min(connectionsToOpen, connectionPoolConfiguration.getMaxPendingConnections() - pendingConnectionCount); // limit the connection count to the protocol-specific settings, but only if that protocol was seen for this pool. From 826586bffe4fc2b37a6885f1f17974b00194f6eb Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 16:07:33 +0200 Subject: [PATCH 70/82] fix keepAliveOnServerError and enable it by default --- .../io/micronaut/http/server/netty/RoutingInBoundHandler.java | 2 +- .../netty/configuration/NettyHttpServerConfiguration.java | 2 +- .../micronaut/http/server/netty/binding/HttpResponseSpec.groovy | 2 -- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/http-server-netty/src/main/java/io/micronaut/http/server/netty/RoutingInBoundHandler.java b/http-server-netty/src/main/java/io/micronaut/http/server/netty/RoutingInBoundHandler.java index 44b6da7d8ff..a1e8b8151b3 100644 --- a/http-server-netty/src/main/java/io/micronaut/http/server/netty/RoutingInBoundHandler.java +++ b/http-server-netty/src/main/java/io/micronaut/http/server/netty/RoutingInBoundHandler.java @@ -1251,7 +1251,7 @@ private void writeFinalNettyResponse(MutableHttpResponse message, HttpRequest if (!isHttp2) { if (!nettyHeaders.contains(HttpHeaderNames.CONNECTION)) { boolean expectKeepAlive = nettyResponse.protocolVersion().isKeepAliveDefault() || request.getHeaders().isKeepAlive(); - if (!decodeError && (expectKeepAlive || httpStatus.getCode() < 500 || serverConfiguration.isKeepAliveOnServerError())) { + if (!decodeError && expectKeepAlive && (httpStatus.getCode() < 500 || serverConfiguration.isKeepAliveOnServerError())) { nettyHeaders.set(HttpHeaderNames.CONNECTION, HttpHeaderValues.KEEP_ALIVE); } else { nettyHeaders.set(HttpHeaderNames.CONNECTION, HttpHeaderValues.CLOSE); diff --git a/http-server-netty/src/main/java/io/micronaut/http/server/netty/configuration/NettyHttpServerConfiguration.java b/http-server-netty/src/main/java/io/micronaut/http/server/netty/configuration/NettyHttpServerConfiguration.java index 4c05c0dcae3..d615bb1dc8a 100644 --- a/http-server-netty/src/main/java/io/micronaut/http/server/netty/configuration/NettyHttpServerConfiguration.java +++ b/http-server-netty/src/main/java/io/micronaut/http/server/netty/configuration/NettyHttpServerConfiguration.java @@ -111,7 +111,7 @@ public class NettyHttpServerConfiguration extends HttpServerConfiguration { * The default configuration for boolean flag indicating whether to add connection header `keep-alive` to responses with HttpStatus > 499. */ @SuppressWarnings("WeakerAccess") - public static final boolean DEFAULT_KEEP_ALIVE_ON_SERVER_ERROR = false; + public static final boolean DEFAULT_KEEP_ALIVE_ON_SERVER_ERROR = true; private static final Logger LOG = LoggerFactory.getLogger(NettyHttpServerConfiguration.class); diff --git a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy index 80637e326fe..bf5c81546a6 100644 --- a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy +++ b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy @@ -29,7 +29,6 @@ import io.micronaut.http.client.exceptions.HttpClientResponseException import io.micronaut.http.server.netty.AbstractMicronautSpec import io.micronaut.runtime.server.EmbeddedServer import reactor.core.publisher.Flux -import spock.lang.Ignore import spock.lang.Shared import spock.lang.Unroll @@ -229,7 +228,6 @@ class HttpResponseSpec extends AbstractMicronautSpec { server.close() } - @Ignore // why wouldn't keep-alive be set? void "test keep alive connection header is not set by default for > 499 response"() { when: EmbeddedServer server = applicationContext.run(EmbeddedServer, [(SPEC_NAME_PROPERTY):getClass().simpleName]) From 4de283e11d2822e4c85440c86b07b9191a43f1fa Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 16:22:51 +0200 Subject: [PATCH 71/82] fix test --- .../server/netty/binding/HttpResponseSpec.groovy | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy index bf5c81546a6..94a2f06eb79 100644 --- a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy +++ b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy @@ -228,9 +228,12 @@ class HttpResponseSpec extends AbstractMicronautSpec { server.close() } - void "test keep alive connection header is not set by default for > 499 response"() { + void "test keep alive connection header is not set for 500 response"() { when: - EmbeddedServer server = applicationContext.run(EmbeddedServer, [(SPEC_NAME_PROPERTY):getClass().simpleName]) + EmbeddedServer server = applicationContext.run(EmbeddedServer, [ + (SPEC_NAME_PROPERTY):getClass().simpleName, + 'micronaut.server.netty.keepAliveOnServerError': false + ]) ApplicationContext ctx = server.getApplicationContext() HttpClient client = applicationContext.createBean(HttpClient, embeddedServer.getURL()) @@ -249,14 +252,13 @@ class HttpResponseSpec extends AbstractMicronautSpec { server.close() } - void "test connection header is defaulted to keep-alive when configured to true for > 499 response"() { + void "test connection header is defaulted to keep-alive by default for > 499 response"() { when: DefaultHttpClientConfiguration config = new DefaultHttpClientConfiguration() // The client will explicitly request "Connection: keep-alive" unless using a connection pool, so set it up config.connectionPoolConfiguration.enabled = true EmbeddedServer server = applicationContext.run(EmbeddedServer, [ - (SPEC_NAME_PROPERTY):getClass().simpleName, - 'micronaut.server.netty.keepAliveOnServerError':true + (SPEC_NAME_PROPERTY):getClass().simpleName ]) def ctx = server.getApplicationContext() HttpClient client = applicationContext.createBean(HttpClient, embeddedServer.getURL(), config) From 9ac7fb5988e51c75355360689ccdc98e3ff43e2e Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 16:29:35 +0200 Subject: [PATCH 72/82] fix test --- .../http/server/netty/binding/HttpResponseSpec.groovy | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy index 94a2f06eb79..760bcaf9352 100644 --- a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy +++ b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/HttpResponseSpec.groovy @@ -23,7 +23,6 @@ import io.micronaut.http.HttpResponse import io.micronaut.http.HttpStatus import io.micronaut.http.annotation.Controller import io.micronaut.http.annotation.Get -import io.micronaut.http.client.DefaultHttpClientConfiguration import io.micronaut.http.client.HttpClient import io.micronaut.http.client.exceptions.HttpClientResponseException import io.micronaut.http.server.netty.AbstractMicronautSpec @@ -231,11 +230,11 @@ class HttpResponseSpec extends AbstractMicronautSpec { void "test keep alive connection header is not set for 500 response"() { when: EmbeddedServer server = applicationContext.run(EmbeddedServer, [ + 'micronaut.server.netty.keepAliveOnServerError': false, (SPEC_NAME_PROPERTY):getClass().simpleName, - 'micronaut.server.netty.keepAliveOnServerError': false ]) ApplicationContext ctx = server.getApplicationContext() - HttpClient client = applicationContext.createBean(HttpClient, embeddedServer.getURL()) + HttpClient client = applicationContext.createBean(HttpClient, server.getURL()) Flux.from(client.exchange( HttpRequest.GET('/test-header/fail') @@ -254,14 +253,11 @@ class HttpResponseSpec extends AbstractMicronautSpec { void "test connection header is defaulted to keep-alive by default for > 499 response"() { when: - DefaultHttpClientConfiguration config = new DefaultHttpClientConfiguration() - // The client will explicitly request "Connection: keep-alive" unless using a connection pool, so set it up - config.connectionPoolConfiguration.enabled = true EmbeddedServer server = applicationContext.run(EmbeddedServer, [ (SPEC_NAME_PROPERTY):getClass().simpleName ]) def ctx = server.getApplicationContext() - HttpClient client = applicationContext.createBean(HttpClient, embeddedServer.getURL(), config) + HttpClient client = applicationContext.createBean(HttpClient, server.getURL()) Flux.from(client.exchange( HttpRequest.GET('/test-header/fail') From 8fb7b242ca32f32f4789f348b7409b00fa45a3c7 Mon Sep 17 00:00:00 2001 From: yawkat Date: Fri, 30 Sep 2022 16:34:51 +0200 Subject: [PATCH 73/82] fix *another* test --- .../configuration/NettyHttpServerConfigurationSpec.groovy | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/configuration/NettyHttpServerConfigurationSpec.groovy b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/configuration/NettyHttpServerConfigurationSpec.groovy index 06a32e21179..8eec5b1c913 100644 --- a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/configuration/NettyHttpServerConfigurationSpec.groovy +++ b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/configuration/NettyHttpServerConfigurationSpec.groovy @@ -472,17 +472,17 @@ class NettyHttpServerConfigurationSpec extends Specification { NettyHttpServerConfiguration config = beanContext.getBean(NettyHttpServerConfiguration) then: - !config.keepAliveOnServerError + config.keepAliveOnServerError cleanup: beanContext.close() } - void "test keepAlive configuration set to true"() { + void "test keepAlive configuration set to false"() { given: ApplicationContext beanContext = new DefaultApplicationContext("test") beanContext.environment.addPropertySource(PropertySource.of("test", - ['micronaut.server.netty.keepAliveOnServerError': true] + ['micronaut.server.netty.keepAliveOnServerError': false] )) beanContext.start() @@ -490,7 +490,7 @@ class NettyHttpServerConfigurationSpec extends Specification { NettyHttpServerConfiguration config = beanContext.getBean(NettyHttpServerConfiguration) then: - config.keepAliveOnServerError + !config.keepAliveOnServerError cleanup: beanContext.close() From 74bccc7ddfcca7d567a481bd783563fd935e54f1 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 10 Oct 2022 10:56:05 +0200 Subject: [PATCH 74/82] implement max-pending-acquires, remove unused config options --- .../http/client/HttpClientConfiguration.java | 26 ------------------- .../http/client/netty/PoolResizer.java | 5 ++++ .../DefaultHttpClientConfigurationSpec.groovy | 1 - .../netty/binding/NettyHttpServerSpec.groovy | 8 +++--- 4 files changed, 10 insertions(+), 30 deletions(-) diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java index fd4d85ac306..5bb20483ff3 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java @@ -722,20 +722,12 @@ public static class ConnectionPoolConfiguration implements Toggleable { @SuppressWarnings("WeakerAccess") public static final boolean DEFAULT_ENABLED = true; - /** - * The default max connections value. - */ - @SuppressWarnings("WeakerAccess") - public static final int DEFAULT_MAXCONNECTIONS = -1; - private int maxPendingConnections = 4; private int maxConcurrentRequestsPerHttp2Connection = Integer.MAX_VALUE; private int maxConcurrentHttp1Connections = Integer.MAX_VALUE; private int maxConcurrentHttp2Connections = 1; - private int maxConnections = DEFAULT_MAXCONNECTIONS; - private int maxPendingAcquires = Integer.MAX_VALUE; private Duration acquireTimeout; @@ -761,24 +753,6 @@ public void setEnabled(boolean enabled) { this.enabled = enabled; } - /** - * The maximum number of connections. Defaults to ({@value io.micronaut.http.client.HttpClientConfiguration.ConnectionPoolConfiguration#DEFAULT_MAXCONNECTIONS}); no maximum. - * - * @return The max connections - */ - public int getMaxConnections() { - return maxConnections; - } - - /** - * Sets the maximum number of connections. Defaults to no maximum. - * - * @param maxConnections The count - */ - public void setMaxConnections(int maxConnections) { - this.maxConnections = maxConnections; - } - /** * Maximum number of futures awaiting connection acquisition. Defaults to no maximum. * diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java index 7a1fda0cbc4..2a0a28e65c4 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/PoolResizer.java @@ -18,6 +18,7 @@ import io.micronaut.core.annotation.Internal; import io.micronaut.core.annotation.Nullable; import io.micronaut.http.client.HttpClientConfiguration; +import io.micronaut.http.client.exceptions.HttpClientException; import org.slf4j.Logger; import reactor.core.publisher.Sinks; @@ -221,6 +222,10 @@ final void onConnectionInactive2(ResizerConnection connection) { } final void addPendingRequest(Sinks.One sink) { + if (pendingRequests.size() >= connectionPoolConfiguration.getMaxPendingAcquires()) { + sink.tryEmitError(new HttpClientException("Cannot acquire connection, exceeded max pending acquires configuration")); + return; + } pendingRequests.addLast(sink); dirty(); } diff --git a/http-client/src/test/groovy/io/micronaut/http/client/config/DefaultHttpClientConfigurationSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/config/DefaultHttpClientConfigurationSpec.groovy index ec839c1b0ae..6a2ed45e48b 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/config/DefaultHttpClientConfigurationSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/config/DefaultHttpClientConfigurationSpec.groovy @@ -75,7 +75,6 @@ class DefaultHttpClientConfigurationSpec extends Specification { where: key | property | value | expected 'enabled' | 'enabled' | 'false' | false - 'max-connections' | 'maxConnections' | '10' | 10 } void "test overriding logger for the client"() { diff --git a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/NettyHttpServerSpec.groovy b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/NettyHttpServerSpec.groovy index fc59c5aaad7..6ac5f9fd84d 100644 --- a/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/NettyHttpServerSpec.groovy +++ b/http-server-netty/src/test/groovy/io/micronaut/http/server/netty/binding/NettyHttpServerSpec.groovy @@ -20,7 +20,11 @@ import io.micronaut.context.env.Environment import io.micronaut.context.env.PropertySource import io.micronaut.context.event.StartupEvent import io.micronaut.core.io.socket.SocketUtils -import io.micronaut.http.* +import io.micronaut.http.HttpHeaders +import io.micronaut.http.HttpMethod +import io.micronaut.http.HttpRequest +import io.micronaut.http.HttpResponse +import io.micronaut.http.HttpStatus import io.micronaut.http.annotation.Controller import io.micronaut.http.annotation.Get import io.micronaut.http.annotation.Put @@ -32,7 +36,6 @@ import io.micronaut.runtime.Micronaut import io.micronaut.runtime.event.annotation.EventListener import io.micronaut.runtime.server.EmbeddedServer import jakarta.inject.Singleton -import reactor.core.publisher.Flux import spock.lang.Retry import spock.lang.Specification import spock.lang.Stepwise @@ -179,7 +182,6 @@ class NettyHttpServerSpec extends Specification { DefaultHttpClientConfiguration config = new DefaultHttpClientConfiguration() // The client will explicitly request "Connection: close" unless using a connection pool, so set it up config.connectionPoolConfiguration.enabled = true - config.connectionPoolConfiguration.maxConnections = 2; config.connectionPoolConfiguration.acquireTimeout = Duration.of(3, ChronoUnit.SECONDS); ApplicationContext applicationContext = Micronaut.run() From 02209b16a27d52cad4da22a68e4f91e6c12308e2 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 10 Oct 2022 10:56:11 +0200 Subject: [PATCH 75/82] docs --- .../clientConfiguration.adoc | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/src/main/docs/guide/httpClient/lowLevelHttpClient/clientConfiguration.adoc b/src/main/docs/guide/httpClient/lowLevelHttpClient/clientConfiguration.adoc index 0b65eb51d34..350737bdaeb 100644 --- a/src/main/docs/guide/httpClient/lowLevelHttpClient/clientConfiguration.adoc +++ b/src/main/docs/guide/httpClient/lowLevelHttpClient/clientConfiguration.adoc @@ -75,9 +75,17 @@ Alternatively, if you don't use service discovery you can use the `configuration ReactorHttpClient httpClient; ---- -=== Using HTTP Client Connection Pooling +=== Connection Pooling and HTTP 2 -A client that handles a significant number of requests will benefit from enabling HTTP client connection pooling. The following configuration enables pooling for the `foo` client: +Connections using normal HTTP (without TLS/SSL) use HTTP 1.1. This can be configured using the `plaintext-mode` configuration option. + +Secure connections (i.e. HTTP**S**, with TLS/SSL) use a feature called "Application Layer Protocol Negotiation" (ALPN) that is part of TLS to select the HTTP version. If the server supports HTTP 2, the Micronaut HTTP Client will use that capability by default, but if it doesn't, HTTP 1.1 is still supported. This is configured using the `alpn-modes` option, which is a list of supported ALPN protocol IDs (`"h2"` and `"http/1.1"`). + +NOTE: The HTTP 2 standard forbids the use of certain less secure TLS cipher suites for HTTP 2 connections. When the HTTP client supports HTTP 2 (which is the default), it will not support those cipher suites. Removing `"h2"` from `alpn-modes` will enable support for all cipher suites. + +Each HTTP 1.1 connection can only support one request at a time, but can be reused for subsequent requests using the `keep-alive` mechanism. HTTP 2 connections can support any number of concurrent requests. + +To remove the overhead of opening a new connection for each request, the Micronaut HTTP Client will reuse HTTP connections wherever possible. They are managed in a _connection pool_. HTTP 1.1 connections are kept around using keep-alive and are used for new requests, and for HTTP 2, a single connection is used for all requests. .Manually configuring HTTP services [source,yaml] @@ -90,15 +98,15 @@ micronaut: - http://foo1 - http://foo2 pool: - enabled: true # <1> - max-connections: 50 # <2> + max-concurrent-http1-connections: 50 # <1> ---- -<1> Enables the pool -<2> Sets the maximum number of connections in the pool +<1> Limit maximum concurrent HTTP 1.1 connections See the API for link:{api}/io/micronaut/http/client/HttpClientConfiguration.ConnectionPoolConfiguration.html[ConnectionPoolConfiguration] for details on available pool configuration options. +By setting the `pool.enabled` property to `false`, you can disable connection reuse. The pool is still used and other configuration options (e.g. concurrent HTTP 1 connections) still apply, but one connection will only serve one request. + === Configuring Event Loop Groups By default, Micronaut shares a common Netty `EventLoopGroup` for worker threads and all HTTP client threads. From 14d679e48c3041baa47e5a3863bccfa17bed1f61 Mon Sep 17 00:00:00 2001 From: yawkat Date: Tue, 11 Oct 2022 10:21:56 +0200 Subject: [PATCH 76/82] test for max-pending-acquires --- .../client/netty/ConnectionManagerSpec.groovy | 51 +++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 3d8a7a32857..1236367f175 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -719,6 +719,57 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'max pending acquires'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.pool.max-pending-acquires': 5, + 'micronaut.http.client.pool.max-pending-connections': 1, + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn = new EmbeddedTestConnectionHttp1() + conn.setupHttp1() + + ChannelPromise delayPromise + def normalInit = conn.clientInitializer + // hack: delay the channelActive call until we complete delayPromise + conn.clientInitializer = new ChannelInitializer() { + @Override + protected void initChannel(EmbeddedChannel ch) throws Exception { + ch.pipeline().addLast(new ChannelInboundHandlerAdapter() { + @Override + void channelActive(ChannelHandlerContext chtx) throws Exception { + delayPromise = chtx.newPromise() + delayPromise.addListener(new GenericFutureListener>() { + @Override + void operationComplete(io.netty.util.concurrent.Future future) throws Exception { + chtx.fireChannelActive() + } + }) + } + }) + ch.pipeline().addLast(normalInit) + } + } + + patch(client, conn) + + List> futures = new ArrayList<>() + for (int i = 0; i < 6; i++) { + futures.add(Mono.from(client.exchange(conn.scheme + '://example.com/foo')).toFuture()) + } + conn.advance() + + for (int i = 0; i < 5; i++) { + assert !futures.get(i).isDone() + } + assert futures.get(5).isDone() + assert futures.get(5).completedExceptionally + + cleanup: + client.close() + ctx.close() + } + def 'multipart request'() { def ctx = ApplicationContext.run() def client = ctx.getBean(DefaultHttpClient) From ee060473baac552a58c35bc470e559cfe1a03d7b Mon Sep 17 00:00:00 2001 From: yawkat Date: Tue, 11 Oct 2022 10:30:00 +0200 Subject: [PATCH 77/82] test for max-concurrent-http1-connections --- .../client/netty/ConnectionManagerSpec.groovy | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy index 1236367f175..3aba2aab49f 100644 --- a/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy +++ b/http-client/src/test/groovy/io/micronaut/http/client/netty/ConnectionManagerSpec.groovy @@ -770,6 +770,38 @@ class ConnectionManagerSpec extends Specification { ctx.close() } + def 'max http1 connections'() { + def ctx = ApplicationContext.run([ + 'micronaut.http.client.pool.max-pending-connections': 1, + 'micronaut.http.client.pool.max-concurrent-http1-connections': 2, + ]) + def client = ctx.getBean(DefaultHttpClient) + + def conn1 = new EmbeddedTestConnectionHttp1() + conn1.setupHttp1() + def conn2 = new EmbeddedTestConnectionHttp1() + conn2.setupHttp1() + + patch(client, conn1, conn2) + + // we open four requests, the first two of which will open connections. + List>> futures = [ + conn1.testExchangeRequest(client), + conn2.testExchangeRequest(client), + conn1.testExchangeRequest(client), + conn1.testExchangeRequest(client), + ] + + conn1.testExchangeResponse(futures.get(0)) + conn1.testExchangeResponse(futures.get(2)) + conn1.testExchangeResponse(futures.get(3)) + conn2.testExchangeResponse(futures.get(1)) + + cleanup: + client.close() + ctx.close() + } + def 'multipart request'() { def ctx = ApplicationContext.run() def client = ctx.getBean(DefaultHttpClient) From f40b38c0a4a7e16d13a30f0679b4dd18daa1f733 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 17 Oct 2022 10:58:53 +0200 Subject: [PATCH 78/82] address feedback --- .../http/client/HttpClientConfiguration.java | 14 ++++++++++++-- .../micronaut/http/client/HttpClientRegistry.java | 3 ++- .../http/client/HttpVersionSelection.java | 6 ++++-- .../micronaut/http/client/annotation/Client.java | 3 +++ .../http/client/netty/ConnectionManager.java | 9 ++++++--- .../http/client/netty/DefaultHttpClient.java | 8 ++++++-- .../netty/DefaultNettyHttpClientRegistry.java | 2 +- .../client/netty/HttpLineBasedFrameDecoder.java | 4 +++- .../http/client/netty/NettyClientCustomizer.java | 2 ++ .../netty/channel/ChannelPipelineCustomizer.java | 5 +++++ .../lowLevelHttpClient/clientConfiguration.adoc | 14 +++++++------- 11 files changed, 51 insertions(+), 19 deletions(-) diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java index 5bb20483ff3..29a284cc561 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientConfiguration.java @@ -665,6 +665,7 @@ public Proxy resolveProxy(boolean isSsl, String host, int port) { * @return The plaintext connection mode. * @since 4.0.0 */ + @NonNull public HttpVersionSelection.PlaintextMode getPlaintextMode() { return plaintextMode; } @@ -677,7 +678,7 @@ public HttpVersionSelection.PlaintextMode getPlaintextMode() { * @param plaintextMode The plaintext connection mode. * @since 4.0.0 */ - public void setPlaintextMode(HttpVersionSelection.PlaintextMode plaintextMode) { + public void setPlaintextMode(@NonNull HttpVersionSelection.PlaintextMode plaintextMode) { this.plaintextMode = Objects.requireNonNull(plaintextMode, "plaintextMode"); } @@ -690,6 +691,7 @@ public void setPlaintextMode(HttpVersionSelection.PlaintextMode plaintextMode) { * @return The supported ALPN protocols. * @since 4.0.0 */ + @NonNull public List getAlpnModes() { return alpnModes; } @@ -703,7 +705,7 @@ public List getAlpnModes() { * @param alpnModes The supported ALPN protocols. * @since 4.0.0 */ - public void setAlpnModes(List alpnModes) { + public void setAlpnModes(@NonNull List alpnModes) { this.alpnModes = Objects.requireNonNull(alpnModes, "alpnModes"); } @@ -794,6 +796,7 @@ public void setAcquireTimeout(@Nullable Duration acquireTimeout) { * pool. * * @return The maximum number of pending connections + * @since 4.0.0 */ public int getMaxPendingConnections() { return maxPendingConnections; @@ -804,6 +807,7 @@ public int getMaxPendingConnections() { * pool. * * @param maxPendingConnections The maximum number of pending connections + * @since 4.0.0 */ public void setMaxPendingConnections(int maxPendingConnections) { this.maxPendingConnections = maxPendingConnections; @@ -814,6 +818,7 @@ public void setMaxPendingConnections(int maxPendingConnections) { * connection. * * @return The maximum concurrent request count + * @since 4.0.0 */ public int getMaxConcurrentRequestsPerHttp2Connection() { return maxConcurrentRequestsPerHttp2Connection; @@ -824,6 +829,7 @@ public int getMaxConcurrentRequestsPerHttp2Connection() { * connection. * * @param maxConcurrentRequestsPerHttp2Connection The maximum concurrent request count + * @since 4.0.0 */ public void setMaxConcurrentRequestsPerHttp2Connection(int maxConcurrentRequestsPerHttp2Connection) { this.maxConcurrentRequestsPerHttp2Connection = maxConcurrentRequestsPerHttp2Connection; @@ -833,6 +839,7 @@ public void setMaxConcurrentRequestsPerHttp2Connection(int maxConcurrentRequests * The maximum number of concurrent HTTP1 connections in the pool. * * @return The maximum concurrent connection count + * @since 4.0.0 */ public int getMaxConcurrentHttp1Connections() { return maxConcurrentHttp1Connections; @@ -842,6 +849,7 @@ public int getMaxConcurrentHttp1Connections() { * The maximum number of concurrent HTTP1 connections in the pool. * * @param maxConcurrentHttp1Connections The maximum concurrent connection count + * @since 4.0.0 */ public void setMaxConcurrentHttp1Connections(int maxConcurrentHttp1Connections) { this.maxConcurrentHttp1Connections = maxConcurrentHttp1Connections; @@ -851,6 +859,7 @@ public void setMaxConcurrentHttp1Connections(int maxConcurrentHttp1Connections) * The maximum number of concurrent HTTP2 connections in the pool. * * @return The maximum concurrent connection count + * @since 4.0.0 */ public int getMaxConcurrentHttp2Connections() { return maxConcurrentHttp2Connections; @@ -860,6 +869,7 @@ public int getMaxConcurrentHttp2Connections() { * The maximum number of concurrent HTTP2 connections in the pool. * * @param maxConcurrentHttp2Connections The maximum concurrent connection count + * @since 4.0.0 */ public void setMaxConcurrentHttp2Connections(int maxConcurrentHttp2Connections) { this.maxConcurrentHttp2Connections = maxConcurrentHttp2Connections; diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientRegistry.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientRegistry.java index cb84cb8a36c..068a1c315de 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpClientRegistry.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpClientRegistry.java @@ -62,9 +62,10 @@ default T getClient(HttpVersion httpVersion, @NonNull String clientId, @Nullable * @param clientId The client ID * @param path The path (Optional) * @return The client + * @since 4.0.0 */ @NonNull - T getClient(HttpVersionSelection httpVersion, @NonNull String clientId, @Nullable String path); + T getClient(@NonNull HttpVersionSelection httpVersion, @NonNull String clientId, @Nullable String path); /** * Resolves a {@link HttpClient} for the given injection point. diff --git a/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java b/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java index b54e67b2d98..4d8d8afce7a 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/HttpVersionSelection.java @@ -17,6 +17,7 @@ import io.micronaut.core.annotation.AnnotationMetadata; import io.micronaut.core.annotation.Internal; +import io.micronaut.core.annotation.NonNull; import io.micronaut.core.annotation.Nullable; import io.micronaut.http.HttpVersion; import io.micronaut.http.client.annotation.Client; @@ -59,7 +60,7 @@ public final class HttpVersionSelection { private final String[] alpnSupportedProtocols; private final boolean http2CipherSuites; - private HttpVersionSelection(PlaintextMode plaintextMode, boolean alpn, String[] alpnSupportedProtocols, boolean http2CipherSuites) { + private HttpVersionSelection(@NonNull PlaintextMode plaintextMode, boolean alpn, @NonNull String[] alpnSupportedProtocols, boolean http2CipherSuites) { this.plaintextMode = plaintextMode; this.alpn = alpn; this.alpnSupportedProtocols = alpnSupportedProtocols; @@ -73,7 +74,8 @@ private HttpVersionSelection(PlaintextMode plaintextMode, boolean alpn, String[] * @param httpVersion The HTTP version as configured for Micronaut HTTP client 3.x * @return The version selection */ - public static HttpVersionSelection forLegacyVersion(HttpVersion httpVersion) { + @NonNull + public static HttpVersionSelection forLegacyVersion(@NonNull HttpVersion httpVersion) { switch (httpVersion) { case HTTP_1_0: case HTTP_1_1: diff --git a/http-client-core/src/main/java/io/micronaut/http/client/annotation/Client.java b/http-client-core/src/main/java/io/micronaut/http/client/annotation/Client.java index 5fa5b5c907a..5eb3cb824ac 100644 --- a/http-client-core/src/main/java/io/micronaut/http/client/annotation/Client.java +++ b/http-client-core/src/main/java/io/micronaut/http/client/annotation/Client.java @@ -18,6 +18,7 @@ import io.micronaut.aop.Introduction; import io.micronaut.context.annotation.AliasFor; import io.micronaut.context.annotation.Type; +import io.micronaut.core.annotation.NonNull; import io.micronaut.http.HttpVersion; import io.micronaut.http.client.HttpClientConfiguration; import io.micronaut.http.client.HttpVersionSelection; @@ -96,6 +97,7 @@ * @return The plaintext connection mode. * @since 4.0.0 */ + @NonNull HttpVersionSelection.PlaintextMode plaintextMode() default HttpVersionSelection.PlaintextMode.HTTP_1; /** @@ -107,6 +109,7 @@ * @return The supported ALPN protocols. * @since 4.0.0 */ + @NonNull String[] alpnModes() default { HttpVersionSelection.ALPN_HTTP_2, HttpVersionSelection.ALPN_HTTP_1 diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 384b6d50d5e..5e5c0ae0774 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -233,7 +233,9 @@ private static NioEventLoopGroup createEventLoopGroup(HttpClientConfiguration co * For testing. * * @return Connected channels in all pools + * @since 4.0.0 */ + @NonNull @SuppressWarnings("unused") List getChannels() { List channels = new ArrayList<>(); @@ -247,6 +249,7 @@ List getChannels() { * For testing. * * @return Number of running requests + * @since 4.0.0 */ @SuppressWarnings("unused") int liveRequestCount() { @@ -572,7 +575,7 @@ public void channelRead(@NonNull ChannelHandlerContext ctx, @NonNull Object msg) * Initializer for TLS channels. After ALPN we will proceed either with * {@link #initHttp1(Channel)} or {@link #initHttp2(Pool, Channel, NettyClientCustomizer)}. */ - private class AdaptiveAlpnChannelInitializer extends ChannelInitializer { + private final class AdaptiveAlpnChannelInitializer extends ChannelInitializer { private final Pool pool; private final SslContext sslContext; @@ -652,7 +655,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E * Initializer for H2C connections. Will proceed with * {@link #initHttp2(Pool, Channel, NettyClientCustomizer)} when the upgrade is done. */ - private class Http2UpgradeInitializer extends ChannelInitializer { + private final class Http2UpgradeInitializer extends ChannelInitializer { private final Pool pool; Http2UpgradeInitializer(Pool pool) { @@ -834,7 +837,7 @@ void openNewConnection() { protected void initChannel(@NonNull Channel ch) throws Exception { configureProxy(ch.pipeline(), false, requestKey.getHost(), requestKey.getPort()); initHttp1(ch); - ch.pipeline().addLast("activity-listener", new ChannelInboundHandlerAdapter() { + ch.pipeline().addLast(ChannelPipelineCustomizer.HANDLER_ACTIVITY_LISTENER, new ChannelInboundHandlerAdapter() { @Override public void channelActive(@NonNull ChannelHandlerContext ctx) throws Exception { super.channelActive(ctx); diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index b9d62badea5..9353ebc02ea 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -1017,10 +1017,14 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception ctx.pipeline() .remove(this) .remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); - poolHandle.release(); } } } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + poolHandle.release(); + } }); if (sse) { poolHandle.channel.pipeline().addLast(HttpLineBasedFrameDecoder.NAME, new HttpLineBasedFrameDecoder(configuration.getMaxContentLength(), true, true)); @@ -1955,7 +1959,7 @@ private class NettyRequestWriter { */ protected void write(ConnectionManager.PoolHandle poolHandle, boolean isSecure, FluxSink emitter) { if (poolHandle.http2) { - // todo: move to CM + // todo: move to ConnectionManager, DefaultHttpClient shouldn't care about the scheme if (isSecure) { nettyRequest.headers().add(AbstractNettyHttpRequest.HTTP2_SCHEME, HttpScheme.HTTPS); } else { diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java index 23193bc14b8..98fccbd1a81 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java @@ -157,7 +157,7 @@ public DefaultNettyHttpClientRegistry( @NonNull @Override - public HttpClient getClient(HttpVersionSelection httpVersion, @NonNull String clientId, @Nullable String path) { + public HttpClient getClient(@NonNull HttpVersionSelection httpVersion, @NonNull String clientId, @Nullable String path) { final ClientKey key = new ClientKey( httpVersion, clientId, diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java b/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java index 7b9680e59d1..eacc0125f52 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/HttpLineBasedFrameDecoder.java @@ -33,6 +33,8 @@ * Variant of {@link LineBasedFrameDecoder} that accepts * {@link io.netty.handler.codec.http.HttpContent} data. Note: this handler removes itself when the * response has been consumed. + * + * @since 4.0.0 */ @Internal final class HttpLineBasedFrameDecoder extends LineBasedFrameDecoder { @@ -40,7 +42,7 @@ final class HttpLineBasedFrameDecoder extends LineBasedFrameDecoder { private boolean ignoreOneLast = false; - public HttpLineBasedFrameDecoder(int maxLength, boolean stripDelimiter, boolean failFast) { + HttpLineBasedFrameDecoder(int maxLength, boolean stripDelimiter, boolean failFast) { super(maxLength, stripDelimiter, failFast); } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/NettyClientCustomizer.java b/http-client/src/main/java/io/micronaut/http/client/netty/NettyClientCustomizer.java index 8963996aa66..a95e029f37f 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/NettyClientCustomizer.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/NettyClientCustomizer.java @@ -81,6 +81,8 @@ enum ChannelRole { CONNECTION, /** * The channel is a HTTP2 stream channel. + * + * @since 4.0.0 */ HTTP2_STREAM, } diff --git a/http-netty/src/main/java/io/micronaut/http/netty/channel/ChannelPipelineCustomizer.java b/http-netty/src/main/java/io/micronaut/http/netty/channel/ChannelPipelineCustomizer.java index f2e28c9e17a..746774dc1c3 100644 --- a/http-netty/src/main/java/io/micronaut/http/netty/channel/ChannelPipelineCustomizer.java +++ b/http-netty/src/main/java/io/micronaut/http/netty/channel/ChannelPipelineCustomizer.java @@ -57,6 +57,11 @@ public interface ChannelPipelineCustomizer { String HANDLER_MICRONAUT_INBOUND = "micronaut-inbound-handler"; String HANDLER_ACCESS_LOGGER = "http-access-logger"; String HANDLER_INITIAL_ERROR = "initial-error"; + /** + * Handler that listens for channelActive to trigger, which will finish up the connection + * setup. + */ + String HANDLER_ACTIVITY_LISTENER = "activity-listener"; /** * @return Is this customizer the client. diff --git a/src/main/docs/guide/httpClient/lowLevelHttpClient/clientConfiguration.adoc b/src/main/docs/guide/httpClient/lowLevelHttpClient/clientConfiguration.adoc index 350737bdaeb..aedaeffbe25 100644 --- a/src/main/docs/guide/httpClient/lowLevelHttpClient/clientConfiguration.adoc +++ b/src/main/docs/guide/httpClient/lowLevelHttpClient/clientConfiguration.adoc @@ -75,17 +75,17 @@ Alternatively, if you don't use service discovery you can use the `configuration ReactorHttpClient httpClient; ---- -=== Connection Pooling and HTTP 2 +=== Connection Pooling and HTTP/2 -Connections using normal HTTP (without TLS/SSL) use HTTP 1.1. This can be configured using the `plaintext-mode` configuration option. +Connections using normal HTTP (without TLS/SSL) use HTTP/1.1. This can be configured using the `plaintext-mode` configuration option. -Secure connections (i.e. HTTP**S**, with TLS/SSL) use a feature called "Application Layer Protocol Negotiation" (ALPN) that is part of TLS to select the HTTP version. If the server supports HTTP 2, the Micronaut HTTP Client will use that capability by default, but if it doesn't, HTTP 1.1 is still supported. This is configured using the `alpn-modes` option, which is a list of supported ALPN protocol IDs (`"h2"` and `"http/1.1"`). +Secure connections (i.e. HTTP**S**, with TLS/SSL) use a feature called "Application Layer Protocol Negotiation" (ALPN) that is part of TLS to select the HTTP version. If the server supports HTTP/2, the Micronaut HTTP Client will use that capability by default, but if it doesn't, HTTP/1.1 is still supported. This is configured using the `alpn-modes` option, which is a list of supported ALPN protocol IDs (`"h2"` and `"http/1.1"`). -NOTE: The HTTP 2 standard forbids the use of certain less secure TLS cipher suites for HTTP 2 connections. When the HTTP client supports HTTP 2 (which is the default), it will not support those cipher suites. Removing `"h2"` from `alpn-modes` will enable support for all cipher suites. +NOTE: The HTTP/2 standard forbids the use of certain less secure TLS cipher suites for HTTP/2 connections. When the HTTP client supports HTTP/2 (which is the default), it will not support those cipher suites. Removing `"h2"` from `alpn-modes` will enable support for all cipher suites. -Each HTTP 1.1 connection can only support one request at a time, but can be reused for subsequent requests using the `keep-alive` mechanism. HTTP 2 connections can support any number of concurrent requests. +Each HTTP/1.1 connection can only support one request at a time, but can be reused for subsequent requests using the `keep-alive` mechanism. HTTP/2 connections can support any number of concurrent requests. -To remove the overhead of opening a new connection for each request, the Micronaut HTTP Client will reuse HTTP connections wherever possible. They are managed in a _connection pool_. HTTP 1.1 connections are kept around using keep-alive and are used for new requests, and for HTTP 2, a single connection is used for all requests. +To remove the overhead of opening a new connection for each request, the Micronaut HTTP Client will reuse HTTP connections wherever possible. They are managed in a _connection pool_. HTTP/1.1 connections are kept around using keep-alive and are used for new requests, and for HTTP/2, a single connection is used for all requests. .Manually configuring HTTP services [source,yaml] @@ -101,7 +101,7 @@ micronaut: max-concurrent-http1-connections: 50 # <1> ---- -<1> Limit maximum concurrent HTTP 1.1 connections +<1> Limit maximum concurrent HTTP/1.1 connections See the API for link:{api}/io/micronaut/http/client/HttpClientConfiguration.ConnectionPoolConfiguration.html[ConnectionPoolConfiguration] for details on available pool configuration options. From af905192950201d26b9f5f891db0f0543bc96e14 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 17 Oct 2022 11:19:15 +0200 Subject: [PATCH 79/82] fix remove order --- .../io/micronaut/http/client/netty/DefaultHttpClient.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index 9353ebc02ea..8e1d1078f75 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -1014,9 +1014,8 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception if (ignoreOneLast) { ignoreOneLast = false; } else { - ctx.pipeline() - .remove(this) - .remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); + ctx.pipeline().remove(ChannelPipelineCustomizer.HANDLER_HTTP_STREAM); + ctx.pipeline().remove(this); } } } From e93e189715d5809947a31ff6b3ec48f321231f3c Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 17 Oct 2022 11:56:22 +0200 Subject: [PATCH 80/82] handle non-mutable http requests properly --- .../http/client/netty/DefaultHttpClient.java | 79 ++++++------ .../netty/DefaultNettyHttpClientRegistry.java | 6 +- .../netty/MutableHttpRequestWrapper.java | 116 ++++++++++++++++++ 3 files changed, 162 insertions(+), 39 deletions(-) create mode 100644 http-client/src/main/java/io/micronaut/http/client/netty/MutableHttpRequestWrapper.java diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java index 8e1d1078f75..c1cdb016b40 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultHttpClient.java @@ -252,6 +252,7 @@ public class DefaultHttpClient implements private final RequestBinderRegistry requestBinderRegistry; private final List invocationInstrumenterFactories; private final String informationalServiceId; + private final ConversionService conversionService; /** * Construct a client for the given arguments. @@ -289,7 +290,9 @@ public DefaultHttpClient(@Nullable LoadBalancer loadBalancer, null, NioSocketChannel::new, CompositeNettyClientCustomizer.EMPTY, - invocationInstrumenterFactories, null); + invocationInstrumenterFactories, + null, + ConversionService.SHARED); } /** @@ -326,7 +329,8 @@ public DefaultHttpClient(@Nullable LoadBalancer loadBalancer, @NonNull ChannelFactory socketChannelFactory, NettyClientCustomizer clientCustomizer, List invocationInstrumenterFactories, - @Nullable String informationalServiceId + @Nullable String informationalServiceId, + @NonNull ConversionService conversionService ) { ArgumentUtils.requireNonNull("nettyClientSslBuilder", nettyClientSslBuilder); ArgumentUtils.requireNonNull("codecRegistry", codecRegistry); @@ -363,6 +367,7 @@ public DefaultHttpClient(@Nullable LoadBalancer loadBalancer, this.webSocketRegistry = webSocketBeanRegistry != null ? webSocketBeanRegistry : WebSocketBeanRegistry.EMPTY; this.requestBinderRegistry = requestBinderRegistry; this.informationalServiceId = informationalServiceId; + this.conversionService = conversionService; this.connectionManager = new ConnectionManager( log, @@ -526,6 +531,11 @@ public O retrieve(io.micronaut.http.HttpRequest request, Argument MutableHttpRequest toMutableRequest(io.micronaut.http.HttpRequest request) { + return MutableHttpRequestWrapper.wrapIfNecessary(conversionService, request); + } + @SuppressWarnings("SubscriberImplementation") @Override public Publisher>> eventStream(@NonNull io.micronaut.http.HttpRequest request) { @@ -694,7 +704,7 @@ public Publisher> dataStream(@NonNull io.micronaut.http.HttpRe public Publisher> dataStream(@NonNull io.micronaut.http.HttpRequest request, @NonNull Argument errorType) { final io.micronaut.http.HttpRequest parentRequest = ServerRequestContext.currentRequest().orElse(null); return new MicronautFlux<>(Flux.from(resolveRequestURI(request)) - .flatMap(requestURI -> dataStreamImpl(request, errorType, parentRequest, requestURI))) + .flatMap(requestURI -> dataStreamImpl(toMutableRequest(request), errorType, parentRequest, requestURI))) .doAfterNext(buffer -> { Object o = buffer.asNativeBuffer(); if (o instanceof ByteBuf) { @@ -715,7 +725,7 @@ public Publisher>> exchangeStre public Publisher>> exchangeStream(@NonNull io.micronaut.http.HttpRequest request, @NonNull Argument errorType) { io.micronaut.http.HttpRequest parentRequest = ServerRequestContext.currentRequest().orElse(null); return new MicronautFlux<>(Flux.from(resolveRequestURI(request)) - .flatMap(uri -> exchangeStreamImpl(parentRequest, request, errorType, uri))) + .flatMap(uri -> exchangeStreamImpl(parentRequest, toMutableRequest(request), errorType, uri))) .doAfterNext(byteBufferHttpResponse -> { ByteBuffer buffer = byteBufferHttpResponse.body(); if (buffer instanceof ReferenceCounted) { @@ -733,7 +743,7 @@ public Publisher jsonStream(@NonNull io.micronaut.http.HttpRequest public Publisher jsonStream(@NonNull io.micronaut.http.HttpRequest request, @NonNull Argument type, @NonNull Argument errorType) { final io.micronaut.http.HttpRequest parentRequest = ServerRequestContext.currentRequest().orElse(null); return Flux.from(resolveRequestURI(request)) - .flatMap(requestURI -> jsonStreamImpl(parentRequest, request, type, errorType, requestURI)); + .flatMap(requestURI -> jsonStreamImpl(parentRequest, toMutableRequest(request), type, errorType, requestURI)); } @SuppressWarnings("unchecked") @@ -752,7 +762,7 @@ public Publisher> exchange(@NonNull final io.micronaut.http.HttpRequest parentRequest = ServerRequestContext.currentRequest().orElse(null); Publisher uriPublisher = resolveRequestURI(request); return Flux.from(uriPublisher) - .switchMap(uri -> exchangeImpl(uri, parentRequest, request, bodyType, errorType)); + .switchMap(uri -> exchangeImpl(uri, parentRequest, toMutableRequest(request), bodyType, errorType)); } @Override @@ -847,7 +857,7 @@ private Publisher connectWebSocket(URI uri, MutableHttpRequest request .then(handler.getHandshakeCompletedMono()); } - private Flux>> exchangeStreamImpl(io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest request, Argument errorType, URI requestURI) { + private Flux>> exchangeStreamImpl(io.micronaut.http.HttpRequest parentRequest, MutableHttpRequest request, Argument errorType, URI requestURI) { Flux> streamResponsePublisher = Flux.from(buildStreamExchange(parentRequest, request, requestURI, errorType)); return streamResponsePublisher.switchMap(response -> { StreamedHttpResponse streamedHttpResponse = NettyHttpResponseBuilder.toStreamResponse(response); @@ -869,7 +879,7 @@ private Flux>> exchangeStreamImpl(io.micronaut.ht }); } - private Flux jsonStreamImpl(io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest request, Argument type, Argument errorType, URI requestURI) { + private Flux jsonStreamImpl(io.micronaut.http.HttpRequest parentRequest, MutableHttpRequest request, Argument type, Argument errorType, URI requestURI) { Flux> streamResponsePublisher = Flux.from(buildStreamExchange(parentRequest, request, requestURI, errorType)); return streamResponsePublisher.switchMap(response -> { @@ -905,7 +915,7 @@ private Flux jsonStreamImpl(io.micronaut.http.HttpRequest parentReq }); } - private Flux> dataStreamImpl(io.micronaut.http.HttpRequest request, Argument errorType, io.micronaut.http.HttpRequest parentRequest, URI requestURI) { + private Flux> dataStreamImpl(MutableHttpRequest request, Argument errorType, io.micronaut.http.HttpRequest parentRequest, URI requestURI) { Flux> streamResponsePublisher = Flux.from(buildStreamExchange(parentRequest, request, requestURI, errorType)); Function> contentMapper = message -> { ByteBuf byteBuf = message.content(); @@ -929,11 +939,11 @@ private Flux> dataStreamImpl(io.micronaut.http.HttpRequest @SuppressWarnings("MagicNumber") private Publisher> buildStreamExchange( @Nullable io.micronaut.http.HttpRequest parentRequest, - @NonNull io.micronaut.http.HttpRequest request, + @NonNull MutableHttpRequest request, @NonNull URI requestURI, @Nullable Argument errorType) { - AtomicReference> requestWrapper = new AtomicReference<>(request); + AtomicReference> requestWrapper = new AtomicReference<>(request); Flux> streamResponsePublisher = connectAndStream(parentRequest, request, requestURI, requestWrapper, false, true); streamResponsePublisher = readBodyOnError(errorType, streamResponsePublisher); @@ -956,14 +966,12 @@ public Publisher> proxy(@NonNull io.micronaut.http.HttpRe Objects.requireNonNull(options, "options"); return Flux.from(resolveRequestURI(request)) .flatMap(requestURI -> { - io.micronaut.http.MutableHttpRequest httpRequest = request instanceof MutableHttpRequest - ? (io.micronaut.http.MutableHttpRequest) request - : request.mutate(); + io.micronaut.http.MutableHttpRequest httpRequest = toMutableRequest(request); if (!options.isRetainHostHeader()) { httpRequest.headers(headers -> headers.remove(HttpHeaderNames.HOST)); } - AtomicReference> requestWrapper = new AtomicReference<>(httpRequest); + AtomicReference> requestWrapper = new AtomicReference<>(httpRequest); Flux> proxyResponsePublisher = connectAndStream(request, request, requestURI, requestWrapper, true, false); // apply filters //noinspection unchecked @@ -984,7 +992,7 @@ private Flux> connectAndStream( io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest request, URI requestURI, - AtomicReference> requestWrapper, + AtomicReference> requestWrapper, boolean isProxy, boolean failOnError ) { @@ -1046,10 +1054,10 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { private Publisher> exchangeImpl( URI requestURI, io.micronaut.http.HttpRequest parentRequest, - io.micronaut.http.HttpRequest request, + MutableHttpRequest request, @NonNull Argument bodyType, @NonNull Argument errorType) { - AtomicReference> requestWrapper = new AtomicReference<>(request); + AtomicReference> requestWrapper = new AtomicReference<>(request); RequestKey requestKey; try { @@ -1195,7 +1203,7 @@ private > Publisher applyFilte io.micronaut.http.HttpRequest parentRequest, io.micronaut.http.HttpRequest request, URI requestURI, - AtomicReference> requestWrapper, + AtomicReference> requestWrapper, Publisher responsePublisher) { if (request instanceof MutableHttpRequest) { @@ -1277,7 +1285,7 @@ protected NettyRequestWriter buildNettyRequest( if (Publishers.isConvertibleToPublisher(bodyValue)) { boolean isSingle = Publishers.isSingle(bodyValue.getClass()); - Publisher publisher = ConversionService.SHARED.convert(bodyValue, Publisher.class).orElseThrow(() -> + Publisher publisher = conversionService.convert(bodyValue, Publisher.class).orElseThrow(() -> new IllegalArgumentException("Unconvertible reactive type: " + bodyValue) ); @@ -1359,7 +1367,7 @@ protected NettyRequestWriter buildNettyRequest( .orElse(null); } if (bodyContent == null) { - bodyContent = ConversionService.SHARED.convert(bodyValue, ByteBuf.class).orElseThrow(() -> + bodyContent = conversionService.convert(bodyValue, ByteBuf.class).orElseThrow(() -> customizeException(new HttpClientException("Body [" + bodyValue + "] cannot be encoded to content type [" + requestContentType + "]. No possible codecs or converters found.")) ); } @@ -1521,7 +1529,7 @@ private void sendRequestThroughChannel( private Flux> streamRequestThroughChannel( io.micronaut.http.HttpRequest parentRequest, - io.micronaut.http.HttpRequest request, + MutableHttpRequest request, ConnectionManager.PoolHandle poolHandle, boolean failOnError, boolean secure) { @@ -1548,19 +1556,16 @@ private > Flux handleStreamHttpError( private void streamRequestThroughChannel0( io.micronaut.http.HttpRequest parentRequest, - final io.micronaut.http.HttpRequest finalRequest, + MutableHttpRequest request, FluxSink> emitter, ConnectionManager.PoolHandle poolHandle, boolean secure) throws HttpPostRequestEncoder.ErrorDataEncoderException { - if (!(finalRequest instanceof MutableHttpRequest)) { - throw new IllegalArgumentException("A MutableHttpRequest is required"); - } - URI requestURI = finalRequest.getUri(); - boolean permitsBody = io.micronaut.http.HttpMethod.permitsRequestBody(finalRequest.getMethod()); + URI requestURI = request.getUri(); + boolean permitsBody = io.micronaut.http.HttpMethod.permitsRequestBody(request.getMethod()); NettyRequestWriter requestWriter = buildNettyRequest( - (MutableHttpRequest) finalRequest, + request, requestURI, - finalRequest + request .getContentType() .orElse(MediaType.APPLICATION_JSON_TYPE), permitsBody, @@ -1571,21 +1576,21 @@ private void streamRequestThroughChannel0( } } ); - prepareHttpHeaders(poolHandle, requestURI, finalRequest, requestWriter.getNettyRequest(), permitsBody); + prepareHttpHeaders(poolHandle, requestURI, request, requestWriter.getNettyRequest(), permitsBody); HttpRequest nettyRequest = requestWriter.getNettyRequest(); Promise> responsePromise = poolHandle.channel.eventLoop().newPromise(); ChannelPipeline pipeline = poolHandle.channel.pipeline(); - pipeline.addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_FULL, new StreamFullHttpResponseHandler(responsePromise, parentRequest, finalRequest)); - pipeline.addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_STREAM, new StreamStreamHttpResponseHandler(responsePromise, parentRequest, finalRequest)); + pipeline.addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_FULL, new StreamFullHttpResponseHandler(responsePromise, parentRequest, request)); + pipeline.addLast(ChannelPipelineCustomizer.HANDLER_MICRONAUT_HTTP_RESPONSE_STREAM, new StreamStreamHttpResponseHandler(responsePromise, parentRequest, request)); poolHandle.notifyRequestPipelineBuilt(); if (log.isDebugEnabled()) { - debugRequest(finalRequest.getUri(), nettyRequest); + debugRequest(request.getUri(), nettyRequest); } if (log.isTraceEnabled()) { - traceRequest(finalRequest, nettyRequest); + traceRequest(request, nettyRequest); } requestWriter.write(poolHandle, secure, emitter); @@ -1659,7 +1664,7 @@ private void prepareHttpHeaders( } } - private ClientFilterChain buildChain(AtomicReference> requestWrapper, List filters) { + private ClientFilterChain buildChain(AtomicReference> requestWrapper, List filters) { AtomicInteger integer = new AtomicInteger(); int len = filters.size(); return new ClientFilterChain() { @@ -1706,7 +1711,7 @@ private HttpPostRequestEncoder buildFormDataRequest(MutableHttpRequest clientHtt } private void addBodyAttribute(HttpPostRequestEncoder postRequestEncoder, String key, Object value) throws HttpPostRequestEncoder.ErrorDataEncoderException { - Optional converted = ConversionService.SHARED.convert(value, String.class); + Optional converted = conversionService.convert(value, String.class); if (converted.isPresent()) { postRequestEncoder.addBodyAttribute(key, converted.get()); } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java index 98fccbd1a81..07acfebeee2 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/DefaultNettyHttpClientRegistry.java @@ -399,6 +399,7 @@ private DefaultHttpClient buildClient( AnnotationMetadata annotationMetadata) { EventLoopGroup eventLoopGroup = resolveEventLoopGroup(configuration, beanContext); + ConversionService conversionService = beanContext.getBean(ConversionService.class); return new DefaultHttpClient( loadBalancer, httpVersion, @@ -414,13 +415,14 @@ private DefaultHttpClient buildClient( codecRegistry, WebSocketBeanRegistry.forClient(beanContext), beanContext.findBean(RequestBinderRegistry.class).orElseGet(() -> - new DefaultRequestBinderRegistry(ConversionService.SHARED) + new DefaultRequestBinderRegistry(conversionService) ), eventLoopGroup, resolveSocketChannelFactory(configuration, beanContext), clientCustomizer, invocationInstrumenterFactories, - clientId + clientId, + conversionService ); } diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/MutableHttpRequestWrapper.java b/http-client/src/main/java/io/micronaut/http/client/netty/MutableHttpRequestWrapper.java new file mode 100644 index 00000000000..3943971b075 --- /dev/null +++ b/http-client/src/main/java/io/micronaut/http/client/netty/MutableHttpRequestWrapper.java @@ -0,0 +1,116 @@ +package io.micronaut.http.client.netty; + +import io.micronaut.core.annotation.Internal; +import io.micronaut.core.annotation.NonNull; +import io.micronaut.core.annotation.Nullable; +import io.micronaut.core.convert.ConversionContext; +import io.micronaut.core.convert.ConversionService; +import io.micronaut.core.type.Argument; +import io.micronaut.http.HttpRequest; +import io.micronaut.http.HttpRequestWrapper; +import io.micronaut.http.MutableHttpHeaders; +import io.micronaut.http.MutableHttpParameters; +import io.micronaut.http.MutableHttpRequest; +import io.micronaut.http.cookie.Cookie; + +import java.net.URI; +import java.util.Optional; + +/** + * Wrapper around an immutable {@link HttpRequest} that allows mutation. + * + * @param Body type + * @since 4.0.0 + */ +@Internal +final class MutableHttpRequestWrapper extends HttpRequestWrapper implements MutableHttpRequest { + private final ConversionService conversionService; + + @Nullable + private B body; + @Nullable + private URI uri; + + MutableHttpRequestWrapper(ConversionService conversionService, HttpRequest delegate) { + super(delegate); + this.conversionService = conversionService; + } + + static MutableHttpRequest wrapIfNecessary(ConversionService conversionService, HttpRequest request) { + if (request instanceof MutableHttpRequest) { + return (MutableHttpRequest) request; + } else { + return new MutableHttpRequestWrapper<>(conversionService, request); + } + } + + @NonNull + @Override + public Optional getBody() { + if (body == null) { + return getDelegate().getBody(); + } else { + return Optional.of(body); + } + } + + @NonNull + @Override + public Optional getBody(@NonNull Class type) { + if (body == null) { + return getDelegate().getBody(type); + } else { + return conversionService.convert(body, ConversionContext.of(type)); + } + } + + @NonNull + @Override + public Optional getBody(@NonNull Argument type) { + if (body == null) { + return getDelegate().getBody(type); + } else { + return conversionService.convert(body, ConversionContext.of(type)); + } + } + + @Override + public MutableHttpRequest cookie(Cookie cookie) { + throw new UnsupportedOperationException(); + } + + @Override + public MutableHttpRequest uri(URI uri) { + this.uri = uri; + return this; + } + + @Override + @NonNull + public URI getUri() { + if (uri == null) { + return getDelegate().getUri(); + } else { + return uri; + } + } + + @NonNull + @Override + public MutableHttpParameters getParameters() { + return (MutableHttpParameters) super.getParameters(); + } + + @NonNull + @Override + public MutableHttpHeaders getHeaders() { + return (MutableHttpHeaders) super.getHeaders(); + } + + @SuppressWarnings("unchecked") + @Override + public MutableHttpRequest body(T body) { + this.body = (B) body; + return (MutableHttpRequest) this; + } +} From 68bc6545a8d0b5f41cfb2e8f904a50f8cf0ba322 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 17 Oct 2022 11:58:12 +0200 Subject: [PATCH 81/82] fix merge --- .../java/io/micronaut/http/client/netty/ConnectionManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java index 5b7f0d9fefb..7221852440c 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/ConnectionManager.java @@ -137,6 +137,7 @@ class ConnectionManager { this.group = from.group; this.shutdownGroup = from.shutdownGroup; this.threadFactory = from.threadFactory; + this.socketChannelFactory = from.socketChannelFactory; this.bootstrap = from.bootstrap; this.configuration = from.configuration; this.sslContext = from.sslContext; From 93bd6bb8c2201ca289a5656b76a280990f631037 Mon Sep 17 00:00:00 2001 From: yawkat Date: Mon, 17 Oct 2022 12:21:48 +0200 Subject: [PATCH 82/82] spotless --- .../client/netty/MutableHttpRequestWrapper.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/http-client/src/main/java/io/micronaut/http/client/netty/MutableHttpRequestWrapper.java b/http-client/src/main/java/io/micronaut/http/client/netty/MutableHttpRequestWrapper.java index 3943971b075..8e1343be39f 100644 --- a/http-client/src/main/java/io/micronaut/http/client/netty/MutableHttpRequestWrapper.java +++ b/http-client/src/main/java/io/micronaut/http/client/netty/MutableHttpRequestWrapper.java @@ -1,3 +1,18 @@ +/* + * Copyright 2017-2022 original authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package io.micronaut.http.client.netty; import io.micronaut.core.annotation.Internal;