From 168b1461d39771358d6b8dc0c517617e80c048b8 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sun, 2 Feb 2025 12:42:32 +0900 Subject: [PATCH 01/27] Support configuring the Ping frame interval in the HTTP/2 protocol Signed-off-by: raccoonback --- .../java/reactor/netty/NettyPipeline.java | 1 + .../reactor/netty/http/Http2SettingsSpec.java | 39 ++++++++++++++++++- .../netty/http/client/HttpClientConfig.java | 21 +++++++--- 3 files changed, 54 insertions(+), 7 deletions(-) diff --git a/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java b/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java index 380c66835d..9c4eb8b8ab 100644 --- a/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java +++ b/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java @@ -112,6 +112,7 @@ public interface NettyPipeline { String OnChannelReadIdle = LEFT + "onChannelReadIdle"; String OnChannelWriteIdle = LEFT + "onChannelWriteIdle"; String ProxyHandler = LEFT + "proxyHandler"; + String H2LivenessHandler = LEFT + "h2LivenessHandler"; /** * Use to register a special handler which ensures that any {@link io.netty.channel.VoidChannelPromise} * will be converted to "unvoided" promises. diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java b/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java index f09ab288da..1b44e348a1 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java @@ -19,6 +19,7 @@ import io.netty.handler.codec.http2.Http2Settings; import reactor.util.annotation.Nullable; +import java.time.Duration; import java.util.Objects; /** @@ -102,6 +103,21 @@ public interface Builder { * @return {@code this} */ //Builder pushEnabled(boolean pushEnabled); + + /** + * Sets the interval for checking ping frames. + * If a ping ACK frame is not received within the configured interval, the connection will be closed. + * + *

Be cautious when setting a very short interval, as it may cause the connection to be closed, + * even if the keep-alive setting is enabled.

+ * + *

If no interval is specified, no ping frame checking will be performed by default.

+ * + * @param pingInterval the duration between sending ping frames. If not specified, ping frame checking is disabled. + * @return {@code this} + * @since 1.2.3 + */ + Builder pingInterval(Duration pingInterval); } /** @@ -196,6 +212,16 @@ public Boolean pushEnabled() { return pushEnabled; } + /** + * Returns the configured {@code pingInterval} value or null. + * + * @return the configured {@code pingInterval} value or null + */ + @Nullable + public Duration pingInterval() { + return pingInterval; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -212,7 +238,8 @@ public boolean equals(Object o) { Objects.equals(maxFrameSize, that.maxFrameSize) && maxHeaderListSize.equals(that.maxHeaderListSize) && Objects.equals(maxStreams, that.maxStreams) && - Objects.equals(pushEnabled, that.pushEnabled); + Objects.equals(pushEnabled, that.pushEnabled) && + Objects.equals(pingInterval, that.pingInterval); } @Override @@ -226,6 +253,7 @@ public int hashCode() { result = 31 * result + (maxHeaderListSize == null ? 0 : Long.hashCode(maxHeaderListSize)); result = 31 * result + (maxStreams == null ? 0 : Long.hashCode(maxStreams)); result = 31 * result + (pushEnabled == null ? 0 : Boolean.hashCode(pushEnabled)); + result = 31 * result + (pingInterval == null ? 0 : Objects.hashCode(pingInterval)); return result; } @@ -237,6 +265,7 @@ public int hashCode() { final Long maxHeaderListSize; final Long maxStreams; final Boolean pushEnabled; + final Duration pingInterval; Http2SettingsSpec(Build build) { Http2Settings settings = build.http2Settings; @@ -254,11 +283,13 @@ public int hashCode() { maxHeaderListSize = settings.maxHeaderListSize(); maxStreams = build.maxStreams; pushEnabled = settings.pushEnabled(); + pingInterval = build.pingInterval; } static final class Build implements Builder { Boolean connectProtocolEnabled; Long maxStreams; + Duration pingInterval; final Http2Settings http2Settings = Http2Settings.defaultSettings(); @Override @@ -311,6 +342,12 @@ public Builder maxStreams(long maxStreams) { return this; } + @Override + public Builder pingInterval(Duration pingInterval) { + this.pingInterval = pingInterval; + return this; + } + /* @Override public Builder pushEnabled(boolean pushEnabled) { diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java index 5730675907..eebca60e3d 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java @@ -679,7 +679,7 @@ else if (metricsRecorder instanceof ContextAwareHttpClientMetricsRecorder) { } static void configureHttp2Pipeline(ChannelPipeline p, HttpResponseDecoderSpec decoder, - Http2Settings http2Settings, ConnectionObserver observer) { + Http2Settings http2Settings, Duration pingInterval, ConnectionObserver observer) { Http2FrameCodecBuilder http2FrameCodecBuilder = Http2FrameCodecBuilder.forClient() .validateHeaders(decoder.validateHeaders()) @@ -690,8 +690,11 @@ static void configureHttp2Pipeline(ChannelPipeline p, HttpResponseDecoderSpec de "reactor.netty.http.client.h2")); } + Http2FrameCodec codec = http2FrameCodecBuilder.build(); + p.addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2Flush, new FlushConsolidationHandler(1024, true)) - .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpCodec, http2FrameCodecBuilder.build()) + .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpCodec, codec) + .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2LivenessHandler, new Http2ConnectionLivenessHandler(codec.encoder(), pingInterval)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2MultiplexHandler, new Http2MultiplexHandler(H2InboundStreamHandler.INSTANCE)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpTrafficHandler, new HttpTrafficHandler(observer)); } @@ -717,6 +720,7 @@ static void configureHttp11OrH2CleartextPipeline( boolean acceptGzip, HttpResponseDecoderSpec decoder, Http2Settings http2Settings, + Duration pingInterval, @Nullable ChannelMetricsRecorder metricsRecorder, ConnectionObserver observer, ChannelOperations.OnSetup opsFactory, @@ -753,6 +757,7 @@ static void configureHttp11OrH2CleartextPipeline( new ReactorNettyHttpClientUpgradeHandler(httpClientCodec, upgradeCodec, decoder.h2cMaxContentLength()); p.addBefore(NettyPipeline.ReactiveBridge, null, httpClientCodec) + .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2LivenessHandler, new Http2ConnectionLivenessHandler(http2FrameCodec.encoder(), pingInterval)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2CUpgradeHandler, upgradeHandler) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpTrafficHandler, new HttpTrafficHandler(observer)); @@ -997,6 +1002,7 @@ static final class H2OrHttp11Codec extends ChannelInboundHandlerAdapter { final boolean acceptGzip; final HttpResponseDecoderSpec decoder; final Http2Settings http2Settings; + final Duration pingInterval; final ChannelMetricsRecorder metricsRecorder; final ConnectionObserver observer; final SocketAddress proxyAddress; @@ -1007,6 +1013,7 @@ static final class H2OrHttp11Codec extends ChannelInboundHandlerAdapter { this.acceptGzip = initializer.acceptGzip; this.decoder = initializer.decoder; this.http2Settings = initializer.http2Settings; + this.pingInterval = initializer.pingInterval; this.metricsRecorder = initializer.metricsRecorder; this.observer = observer; this.proxyAddress = initializer.proxyAddress; @@ -1025,7 +1032,7 @@ public void channelActive(ChannelHandlerContext ctx) { log.debug(format(ctx.channel(), "Negotiated application-level protocol [" + protocol + "]")); } if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { - configureHttp2Pipeline(ctx.channel().pipeline(), decoder, http2Settings, observer); + configureHttp2Pipeline(ctx.channel().pipeline(), decoder, http2Settings, pingInterval, observer); } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { configureHttp11Pipeline(ctx.channel().pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); @@ -1049,6 +1056,7 @@ static final class HttpClientChannelInitializer implements ChannelPipelineConfig final boolean acceptGzip; final HttpResponseDecoderSpec decoder; final Http2Settings http2Settings; + final Duration pingInterval; final ChannelMetricsRecorder metricsRecorder; final ChannelOperations.OnSetup opsFactory; final int protocols; @@ -1060,6 +1068,7 @@ static final class HttpClientChannelInitializer implements ChannelPipelineConfig this.acceptGzip = config.acceptGzip; this.decoder = config.decoder; this.http2Settings = config.http2Settings(); + this.pingInterval = config.http2Settings != null ? config.http2Settings.pingInterval() : null; this.metricsRecorder = config.metricsRecorderInternal(); this.opsFactory = config.channelOperationsProvider(); this.protocols = config._protocols; @@ -1084,7 +1093,7 @@ else if ((protocols & h11) == h11) { configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h2) == h2) { - configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, observer); + configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, pingInterval, observer); } else if ((protocols & h3) == h3) { configureHttp3Pipeline(channel.pipeline(), metricsRecorder != null, proxyAddress != null); @@ -1092,13 +1101,13 @@ else if ((protocols & h3) == h3) { } else { if ((protocols & h11orH2C) == h11orH2C) { - configureHttp11OrH2CleartextPipeline(channel.pipeline(), acceptGzip, decoder, http2Settings, metricsRecorder, observer, opsFactory, proxyAddress, remoteAddress, uriTagValue); + configureHttp11OrH2CleartextPipeline(channel.pipeline(), acceptGzip, decoder, http2Settings, pingInterval, metricsRecorder, observer, opsFactory, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h11) == h11) { configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h2c) == h2c) { - configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, observer); + configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, pingInterval, observer); } } } From 7680dcfd908625dfcce748f6951e2378c94f6ae2 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sun, 2 Feb 2025 12:44:37 +0900 Subject: [PATCH 02/27] Add a handler to check connection status via Ping frame health checks in HTTP/2 Signed-off-by: raccoonback --- .../Http2ConnectionLivenessHandler.java | 245 +++++++++++++++++ .../Http2ConnectionLivenessHandlerTest.java | 247 ++++++++++++++++++ 2 files changed, 492 insertions(+) create mode 100644 reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java create mode 100644 reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java new file mode 100644 index 0000000000..e3dd9a5eb7 --- /dev/null +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java @@ -0,0 +1,245 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.http.client; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPromise; +import io.netty.handler.codec.http2.DefaultHttp2PingFrame; +import io.netty.handler.codec.http2.Http2ChannelDuplexHandler; +import io.netty.handler.codec.http2.Http2ConnectionEncoder; +import reactor.util.Logger; +import reactor.util.Loggers; +import reactor.util.annotation.Nullable; + +import java.time.Duration; +import java.util.Objects; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ThreadLocalRandom; + +import static java.util.concurrent.TimeUnit.NANOSECONDS; + +/** + * Handler that supports connection health checks using HTTP/2 Ping Frames. + * + *

This Handler sends a ping frame at the specified interval when no frame is being read or written, + * ensuring the connection health is monitored. If a ping ACK frame is not received within the configured interval, + * the connection will be closed.

+ * + *

Ping frame checking will not be performed while a read or write operation is in progress.

+ * + *

Be cautious when setting a very short interval, as it may cause the connection to be closed, + * even if the keep-alive setting is enabled.

+ * + *

If no interval is specified, no ping frame checking will be performed.

+ * + * @author raccoonback + * @since 1.2.3 + */ +public class Http2ConnectionLivenessHandler extends Http2ChannelDuplexHandler { + + private static final Logger log = Loggers.getLogger(Http2ConnectionLivenessHandler.class); + + private ScheduledFuture pingScheduler; + private final ChannelFutureListener pingWriteListener = new PingWriteListener(); + private final Http2ConnectionEncoder encoder; + private final long pingIntervalNanos; + private long lastSentPingData; + private long lastReceivedPingTime; + private long lastIoTime; + private boolean isPingAckPending; + + public Http2ConnectionLivenessHandler(Http2ConnectionEncoder encoder, @Nullable Duration pingInterval) { + Objects.requireNonNull(encoder, "encoder"); + this.encoder = encoder; + + if (pingInterval != null) { + this.pingIntervalNanos = pingInterval.toNanos(); + } + else { + this.pingIntervalNanos = 0L; + } + } + + @Override + public void channelActive(ChannelHandlerContext ctx) throws Exception { + if (isPingIntervalConfigured()) { + isPingAckPending = false; + pingScheduler = ctx.executor() + .schedule( + new PingChecker(ctx), + pingIntervalNanos, + NANOSECONDS + ); + } + + ctx.fireChannelActive(); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (msg instanceof DefaultHttp2PingFrame) { + DefaultHttp2PingFrame frame = (DefaultHttp2PingFrame) msg; + if (frame.ack() && frame.content() == lastSentPingData) { + lastReceivedPingTime = System.nanoTime(); + } + } + else { + lastIoTime = System.nanoTime(); + } + + ctx.fireChannelRead(msg); + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { + lastIoTime = System.nanoTime(); + + ctx.write(msg, promise); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + cancel(); + ctx.fireChannelInactive(); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + cancel(); + ctx.fireExceptionCaught(cause); + } + + private boolean isPingIntervalConfigured() { + return pingIntervalNanos > 0; + } + + private void cancel() { + if (pingScheduler != null) { + pingScheduler.cancel(false); + } + } + + private class PingChecker implements Runnable { + + private final ChannelHandlerContext ctx; + + PingChecker(ChannelHandlerContext ctx) { + this.ctx = ctx; + } + + @Override + public void run() { + Channel channel = ctx.channel(); + if (channel == null || !channel.isOpen()) { + + return; + } + + if (lastIoTime == 0 || isIoInProgress()) { + if (log.isDebugEnabled()) { + log.debug("{} channel is currently reading or writing data.", channel); + } + + isPingAckPending = false; + pingScheduler = invokeNextSchedule(); + return; + } + + if (!isPingAckPending) { + if (log.isDebugEnabled()) { + log.debug("Attempting to send a ping frame to {} channel.", channel); + } + + writePing(ctx); + pingScheduler = invokeNextSchedule(); + return; + } + + if (isOutOfTimeRange()) { + if (log.isInfoEnabled()) { + log.info("Closing {} channel due to delayed ping frame response (timeout: {} ns).", channel, pingIntervalNanos); + } + + close(channel); + return; + } + + isPingAckPending = false; + pingScheduler = invokeNextSchedule(); + } + + private void writePing(ChannelHandlerContext ctx) { + lastSentPingData = ThreadLocalRandom.current().nextLong(); + + encoder.frameWriter() + .writePing(ctx, false, lastSentPingData, ctx.newPromise()) + .addListener(pingWriteListener); + ctx.flush(); + } + + private boolean isIoInProgress() { + return pingIntervalNanos > (System.nanoTime() - lastIoTime); + } + + private boolean isOutOfTimeRange() { + return pingIntervalNanos < (System.nanoTime() - lastReceivedPingTime); + } + + private ScheduledFuture invokeNextSchedule() { + return ctx.executor() + .schedule( + new PingChecker(ctx), + pingIntervalNanos, + NANOSECONDS + ); + } + + private void close(Channel channel) { + channel.close() + .addListener(future -> { + if (future.isSuccess()) { + if (log.isDebugEnabled()) { + log.debug("{} channel closed an channel", channel); + } + } + else if (log.isDebugEnabled()) { + log.debug("{} channel failed to close an channel", channel, future.cause()); + } + }); + } + } + + private class PingWriteListener implements ChannelFutureListener { + + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if (future.isSuccess()) { + if (log.isDebugEnabled()) { + log.debug("Wrote PING frame to {} channel.", future.channel()); + } + + isPingAckPending = true; + } + else if (log.isDebugEnabled()) { + log.debug("Failed to wrote PING frame to {} channel.", future.channel()); + } + } + } +} diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java new file mode 100644 index 0000000000..6bdb97dab9 --- /dev/null +++ b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java @@ -0,0 +1,247 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.http.client; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.codec.http2.DefaultHttp2PingFrame; +import io.netty.handler.codec.http2.Http2FrameCodec; +import io.netty.handler.codec.http2.Http2FrameCodecBuilder; +import io.netty.handler.codec.http2.Http2PingFrame; +import io.netty.handler.ssl.util.InsecureTrustManagerFactory; +import io.netty.handler.ssl.util.SelfSignedCertificate; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import reactor.core.publisher.Mono; +import reactor.netty.BaseHttpTest; +import reactor.netty.DisposableServer; +import reactor.netty.NettyPipeline; +import reactor.netty.http.Http2SslContextSpec; + +import java.security.cert.CertificateException; +import java.time.Duration; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.List; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static reactor.netty.http.HttpProtocol.H2; + +/** + * This test class verifies {@link Http2ConnectionLivenessHandler}. + * + * @author raccoonback + * @since 1.2.3 + */ +class Http2ConnectionLivenessHandlerTest extends BaseHttpTest { + + static SelfSignedCertificate ssc; + + @BeforeAll + static void createSelfSignedCertificate() throws CertificateException { + ssc = new SelfSignedCertificate(); + } + + @Test + void successReceiveResponse() { + DisposableServer disposableServer = createServer() + .protocol(H2) + .secure(spec -> spec.sslContext( + Http2SslContextSpec.forServer(ssc.certificate(), ssc.privateKey()) + )) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + String result = createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext( + Http2SslContextSpec.forClient() + .configure(builder -> builder.trustManager(InsecureTrustManagerFactory.INSTANCE)) + )) + .get() + .uri("/") + .responseSingle((resp, bytes) -> bytes.asString()) + .block(); + + assertThat(result).isEqualTo("Test"); + } + + @Test + void noPingCheckWhenNotConfigured() { + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + + DisposableServer disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext( + Http2SslContextSpec.forServer(ssc.certificate(), ssc.privateKey()) + )) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .keepAlive(true) + .secure(spec -> spec.sslContext( + Http2SslContextSpec.forClient() + .configure(builder -> builder.trustManager(InsecureTrustManagerFactory.INSTANCE)) + )) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofMillis(100)) + .block(); + + assertThat(handler.getReceivedPingTimes()).isEmpty(); + assertThat(channel.parent().isOpen()).isTrue(); + } + + @Test + void closePingFrameIfDelayed() { + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame) -> Mono.delay(Duration.ofMillis(150)) + .doOnNext(unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true))) + .subscribe() + ); + + DisposableServer disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext( + Http2SslContextSpec.forServer(ssc.certificate(), ssc.privateKey()) + )) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .keepAlive(true) + .secure(spec -> spec.sslContext( + Http2SslContextSpec.forClient() + .configure(builder -> builder.trustManager(InsecureTrustManagerFactory.INSTANCE)) + )) + .http2Settings(builder -> { + builder.pingInterval(Duration.ofMillis(100)); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofMillis(600)) + .block(); + + assertThat(handler.getReceivedPingTimes()).hasSize(1); + assertThat(channel.parent().isOpen()).isFalse(); + } + + @Test + void ackPingFrameWithinInterval() { + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + + DisposableServer disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext( + Http2SslContextSpec.forServer(ssc.certificate(), ssc.privateKey()) + )) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .keepAlive(true) + .secure(spec -> spec.sslContext( + Http2SslContextSpec.forClient() + .configure(builder -> builder.trustManager(InsecureTrustManagerFactory.INSTANCE)) + )) + .http2Settings(builder -> { + builder.pingInterval(Duration.ofMillis(100)); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofMillis(1000)) + .block(); + + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + assertThat(channel.parent().isOpen()).isTrue(); + } + + private static class Http2PingFrameHandler extends SimpleChannelInboundHandler { + + private List receivedPingTimes = new ArrayList<>(); + + private final BiConsumer consumer; + + private Http2PingFrameHandler() { + this.consumer = (ctx, frame) -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)); + } + + private Http2PingFrameHandler(BiConsumer consumer) { + this.consumer = consumer; + } + + @Override + protected void channelRead0(ChannelHandlerContext ctx, Http2PingFrame frame) throws InterruptedException { + receivedPingTimes.add(LocalDateTime.now()); + consumer.accept(ctx, frame); + } + + public List getReceivedPingTimes() { + return receivedPingTimes.stream() + .sorted() + .collect(Collectors.toList()); + } + } +} From f5aae94a8db1feee5af206e8d2030ee99dd18b5d Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sun, 2 Feb 2025 12:45:50 +0900 Subject: [PATCH 03/27] Update doc related with HTTP/2 Signed-off-by: raccoonback --- docs/modules/ROOT/pages/http-client.adoc | 1 + .../documentation/http/client/http2/H2Application.java | 7 +++++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/modules/ROOT/pages/http-client.adoc b/docs/modules/ROOT/pages/http-client.adoc index 9eb4125778..a7335bc70b 100644 --- a/docs/modules/ROOT/pages/http-client.adoc +++ b/docs/modules/ROOT/pages/http-client.adoc @@ -318,6 +318,7 @@ include::{examples-dir}/http2/H2Application.java[lines=18..42] ---- <1> Configures the client to support only `HTTP/2` <2> Configures `SSL` +<3> You can configure the interval for checking `Ping` frames The following listing presents a simple `H2C` example: diff --git a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java index 5030e8a2b5..150c29bde4 100644 --- a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java +++ b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2021 VMware, Inc. or its affiliates, All Rights Reserved. + * Copyright (c) 2020-2025 VMware, Inc. or its affiliates, All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,13 +21,16 @@ import reactor.netty.http.client.HttpClient; import reactor.util.function.Tuple2; +import java.time.Duration; + public class H2Application { public static void main(String[] args) { HttpClient client = HttpClient.create() .protocol(HttpProtocol.H2) //<1> - .secure(); //<2> + .secure() //<2> + .http2Settings(builder -> builder.pingInterval(Duration.ofMillis(100))); // <3> Tuple2 response = client.get() From 9cd9f61c05b9060d2ac62fd1f050ca8e74efa16f Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sun, 2 Feb 2025 13:13:55 +0900 Subject: [PATCH 04/27] Ensure backward compatibility while adding support for configuring the ping interval - ref. https://github.com/reactor/reactor-netty/actions/runs/13095356929/job/36537031690?pr=3612 Signed-off-by: raccoonback --- .../src/main/java/reactor/netty/http/Http2SettingsSpec.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java b/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java index 1b44e348a1..3e725705a7 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java @@ -117,7 +117,9 @@ public interface Builder { * @return {@code this} * @since 1.2.3 */ - Builder pingInterval(Duration pingInterval); + default Builder pingInterval(Duration pingInterval) { + return this; + } } /** From f245d91d416d7bdaad8aae8ebc83b047b73894e0 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sun, 2 Feb 2025 16:48:11 +0900 Subject: [PATCH 05/27] Fix unnecessary warnings in test code Signed-off-by: raccoonback --- .../Http2ConnectionLivenessHandlerTest.java | 160 ++++++++++++++---- 1 file changed, 123 insertions(+), 37 deletions(-) diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java index 6bdb97dab9..5572274326 100644 --- a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java +++ b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java @@ -16,12 +16,15 @@ package reactor.netty.http.client; import io.netty.channel.Channel; +import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.SimpleChannelInboundHandler; import io.netty.handler.codec.http2.DefaultHttp2PingFrame; import io.netty.handler.codec.http2.Http2FrameCodec; import io.netty.handler.codec.http2.Http2FrameCodecBuilder; import io.netty.handler.codec.http2.Http2PingFrame; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslContextBuilder; import io.netty.handler.ssl.util.InsecureTrustManagerFactory; import io.netty.handler.ssl.util.SelfSignedCertificate; import org.junit.jupiter.api.BeforeAll; @@ -30,11 +33,13 @@ import reactor.netty.BaseHttpTest; import reactor.netty.DisposableServer; import reactor.netty.NettyPipeline; -import reactor.netty.http.Http2SslContextSpec; +import reactor.netty.resources.ConnectionProvider; +import javax.net.ssl.SSLException; import java.security.cert.CertificateException; import java.time.Duration; import java.time.LocalDateTime; +import java.time.ZoneId; import java.util.ArrayList; import java.util.List; import java.util.function.BiConsumer; @@ -52,28 +57,30 @@ class Http2ConnectionLivenessHandlerTest extends BaseHttpTest { static SelfSignedCertificate ssc; + static SslContext sslServer; + static SslContext sslClient; @BeforeAll - static void createSelfSignedCertificate() throws CertificateException { + static void createSelfSignedCertificate() throws CertificateException, SSLException { ssc = new SelfSignedCertificate(); + sslServer = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey()) + .build(); + sslClient = SslContextBuilder.forClient() + .trustManager(InsecureTrustManagerFactory.INSTANCE) + .build(); } @Test void successReceiveResponse() { DisposableServer disposableServer = createServer() .protocol(H2) - .secure(spec -> spec.sslContext( - Http2SslContextSpec.forServer(ssc.certificate(), ssc.privateKey()) - )) + .secure(spec -> spec.sslContext(sslServer)) .handle((req, resp) -> resp.sendString(Mono.just("Test"))) .bindNow(); String result = createClient(disposableServer::address) .protocol(H2) - .secure(spec -> spec.sslContext( - Http2SslContextSpec.forClient() - .configure(builder -> builder.trustManager(InsecureTrustManagerFactory.INSTANCE)) - )) + .secure(spec -> spec.sslContext(sslClient)) .get() .uri("/") .responseSingle((resp, bytes) -> bytes.asString()) @@ -89,9 +96,7 @@ void noPingCheckWhenNotConfigured() { DisposableServer disposableServer = createServer() .protocol(H2) .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext( - Http2SslContextSpec.forServer(ssc.certificate(), ssc.privateKey()) - )) + .secure(spec -> spec.sslContext(sslServer)) .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() .autoAckPingFrame(false) @@ -107,10 +112,7 @@ void noPingCheckWhenNotConfigured() { Channel channel = createClient(disposableServer::address) .protocol(H2) .keepAlive(true) - .secure(spec -> spec.sslContext( - Http2SslContextSpec.forClient() - .configure(builder -> builder.trustManager(InsecureTrustManagerFactory.INSTANCE)) - )) + .secure(spec -> spec.sslContext(sslClient)) .get() .uri("/") .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) @@ -125,19 +127,20 @@ void noPingCheckWhenNotConfigured() { } @Test - void closePingFrameIfDelayed() { + void closeConnectionIfPingFrameDelayed() { Http2PingFrameHandler handler = new Http2PingFrameHandler( (ctx, frame) -> Mono.delay(Duration.ofMillis(150)) - .doOnNext(unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true))) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) .subscribe() ); DisposableServer disposableServer = createServer() .protocol(H2) .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext( - Http2SslContextSpec.forServer(ssc.certificate(), ssc.privateKey()) - )) + .secure(spec -> spec.sslContext(sslServer)) .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() .autoAckPingFrame(false) @@ -153,10 +156,55 @@ void closePingFrameIfDelayed() { Channel channel = createClient(disposableServer::address) .protocol(H2) .keepAlive(true) - .secure(spec -> spec.sslContext( - Http2SslContextSpec.forClient() - .configure(builder -> builder.trustManager(InsecureTrustManagerFactory.INSTANCE)) - )) + .secure(spec -> spec.sslContext(sslClient)) + .http2Settings(builder -> { + builder.pingInterval(Duration.ofMillis(100)); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofMillis(600)) + .block(); + + assertThat(handler.getReceivedPingTimes()).hasSize(1); + assertThat(channel.parent().isOpen()).isFalse(); + } + + @Test + void closeConnectionInPoolIfPingFrameDelayed() { + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame) -> Mono.delay(Duration.ofMillis(150)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe() + ); + + DisposableServer disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + ConnectionProvider pool = ConnectionProvider.create("closeConnectionInPoolIfPingFrameDelayed", 1); + Channel channel = createClient(pool, disposableServer::address) + .protocol(H2) + .keepAlive(true) + .secure(spec -> spec.sslContext(sslClient)) .http2Settings(builder -> { builder.pingInterval(Duration.ofMillis(100)); }) @@ -180,9 +228,7 @@ void ackPingFrameWithinInterval() { DisposableServer disposableServer = createServer() .protocol(H2) .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext( - Http2SslContextSpec.forServer(ssc.certificate(), ssc.privateKey()) - )) + .secure(spec -> spec.sslContext(sslServer)) .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() .autoAckPingFrame(false) @@ -198,10 +244,48 @@ void ackPingFrameWithinInterval() { Channel channel = createClient(disposableServer::address) .protocol(H2) .keepAlive(true) - .secure(spec -> spec.sslContext( - Http2SslContextSpec.forClient() - .configure(builder -> builder.trustManager(InsecureTrustManagerFactory.INSTANCE)) - )) + .secure(spec -> spec.sslContext(sslClient)) + .http2Settings(builder -> { + builder.pingInterval(Duration.ofMillis(100)); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(1)) + .block(); + + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + assertThat(channel.parent().isOpen()).isTrue(); + } + + @Test + void connectionRetentionInPoolOnPingFrameAck() { + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + + DisposableServer disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + ConnectionProvider pool = ConnectionProvider.create("connectionRetentionInPoolOnPingFrameAck", 1); + Channel channel = createClient(pool, disposableServer::address) + .protocol(H2) + .keepAlive(true) + .secure(spec -> spec.sslContext(sslClient)) .http2Settings(builder -> { builder.pingInterval(Duration.ofMillis(100)); }) @@ -211,21 +295,23 @@ void ackPingFrameWithinInterval() { .single() .block(); - Mono.delay(Duration.ofMillis(1000)) + Mono.delay(Duration.ofSeconds(1)) .block(); assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); assertThat(channel.parent().isOpen()).isTrue(); } - private static class Http2PingFrameHandler extends SimpleChannelInboundHandler { + private static final class Http2PingFrameHandler extends SimpleChannelInboundHandler { - private List receivedPingTimes = new ArrayList<>(); + private final List receivedPingTimes = new ArrayList<>(); private final BiConsumer consumer; private Http2PingFrameHandler() { - this.consumer = (ctx, frame) -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)); + this.consumer = (ctx, frame) -> + ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE); } private Http2PingFrameHandler(BiConsumer consumer) { @@ -234,7 +320,7 @@ private Http2PingFrameHandler(BiConsumer @Override protected void channelRead0(ChannelHandlerContext ctx, Http2PingFrame frame) throws InterruptedException { - receivedPingTimes.add(LocalDateTime.now()); + receivedPingTimes.add(LocalDateTime.now(ZoneId.systemDefault())); consumer.accept(ctx, frame); } From d7c0342bb45d7b3ac9a05a50b4de8d3a37886744 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sun, 2 Feb 2025 17:33:26 +0900 Subject: [PATCH 06/27] Http2ChannelDuplexHandler -> ChannelDuplexHandler Signed-off-by: raccoonback --- .../netty/http/client/Http2ConnectionLivenessHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java index e3dd9a5eb7..5e198bb040 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java @@ -16,12 +16,12 @@ package reactor.netty.http.client; import io.netty.channel.Channel; +import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPromise; import io.netty.handler.codec.http2.DefaultHttp2PingFrame; -import io.netty.handler.codec.http2.Http2ChannelDuplexHandler; import io.netty.handler.codec.http2.Http2ConnectionEncoder; import reactor.util.Logger; import reactor.util.Loggers; @@ -51,7 +51,7 @@ * @author raccoonback * @since 1.2.3 */ -public class Http2ConnectionLivenessHandler extends Http2ChannelDuplexHandler { +public class Http2ConnectionLivenessHandler extends ChannelDuplexHandler { private static final Logger log = Loggers.getLogger(Http2ConnectionLivenessHandler.class); From ba1025b93f7ee1190e8acde650efcd4c7a7ac09b Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sun, 2 Feb 2025 18:52:09 +0900 Subject: [PATCH 07/27] Reduce the scope of access control Signed-off-by: raccoonback --- .../netty/http/client/Http2ConnectionLivenessHandler.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java index 5e198bb040..c166a74773 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java @@ -28,7 +28,6 @@ import reactor.util.annotation.Nullable; import java.time.Duration; -import java.util.Objects; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ThreadLocalRandom; @@ -51,7 +50,7 @@ * @author raccoonback * @since 1.2.3 */ -public class Http2ConnectionLivenessHandler extends ChannelDuplexHandler { +final class Http2ConnectionLivenessHandler extends ChannelDuplexHandler { private static final Logger log = Loggers.getLogger(Http2ConnectionLivenessHandler.class); @@ -65,7 +64,6 @@ public class Http2ConnectionLivenessHandler extends ChannelDuplexHandler { private boolean isPingAckPending; public Http2ConnectionLivenessHandler(Http2ConnectionEncoder encoder, @Nullable Duration pingInterval) { - Objects.requireNonNull(encoder, "encoder"); this.encoder = encoder; if (pingInterval != null) { From e16cb7de9fcd4afd131bde7022f41180dbd3cb8d Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sun, 2 Feb 2025 19:33:56 +0900 Subject: [PATCH 08/27] Fix broken test code Signed-off-by: raccoonback --- .../Http2ConnectionLivenessHandler.java | 1 - .../reactor-netty-http/reflect-config.json | 7 +++++ .../Http2ConnectionLivenessHandlerTest.java | 31 +++++++++---------- ...dConnectionProviderDefaultMetricsTest.java | 4 ++- 4 files changed, 25 insertions(+), 18 deletions(-) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java index c166a74773..d3f751edbc 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java @@ -146,7 +146,6 @@ private class PingChecker implements Runnable { public void run() { Channel channel = ctx.channel(); if (channel == null || !channel.isOpen()) { - return; } diff --git a/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json b/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json index 65c4c74cdd..81f188e8ca 100644 --- a/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json +++ b/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json @@ -341,5 +341,12 @@ }, "name": "reactor.netty.http.server.logging.error.BaseErrorLogHandler", "queryAllPublicMethods": true + }, + { + "condition": { + "typeReachable": "reactor.netty.http.client.Http2ConnectionLivenessHandler" + }, + "name": "reactor.netty.http.client.Http2ConnectionLivenessHandler", + "queryAllPublicMethods": true } ] \ No newline at end of file diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java index 5572274326..19b313d9b5 100644 --- a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java +++ b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java @@ -31,7 +31,6 @@ import org.junit.jupiter.api.Test; import reactor.core.publisher.Mono; import reactor.netty.BaseHttpTest; -import reactor.netty.DisposableServer; import reactor.netty.NettyPipeline; import reactor.netty.resources.ConnectionProvider; @@ -56,13 +55,12 @@ */ class Http2ConnectionLivenessHandlerTest extends BaseHttpTest { - static SelfSignedCertificate ssc; static SslContext sslServer; static SslContext sslClient; @BeforeAll static void createSelfSignedCertificate() throws CertificateException, SSLException { - ssc = new SelfSignedCertificate(); + SelfSignedCertificate ssc = new SelfSignedCertificate(); sslServer = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey()) .build(); sslClient = SslContextBuilder.forClient() @@ -72,7 +70,7 @@ static void createSelfSignedCertificate() throws CertificateException, SSLExcept @Test void successReceiveResponse() { - DisposableServer disposableServer = createServer() + disposableServer = createServer() .protocol(H2) .secure(spec -> spec.sslContext(sslServer)) .handle((req, resp) -> resp.sendString(Mono.just("Test"))) @@ -93,7 +91,7 @@ void successReceiveResponse() { void noPingCheckWhenNotConfigured() { Http2PingFrameHandler handler = new Http2PingFrameHandler(); - DisposableServer disposableServer = createServer() + disposableServer = createServer() .protocol(H2) .maxKeepAliveRequests(1) .secure(spec -> spec.sslContext(sslServer)) @@ -119,7 +117,7 @@ void noPingCheckWhenNotConfigured() { .single() .block(); - Mono.delay(Duration.ofMillis(100)) + Mono.delay(Duration.ofSeconds(1)) .block(); assertThat(handler.getReceivedPingTimes()).isEmpty(); @@ -137,7 +135,7 @@ void closeConnectionIfPingFrameDelayed() { .subscribe() ); - DisposableServer disposableServer = createServer() + disposableServer = createServer() .protocol(H2) .maxKeepAliveRequests(1) .secure(spec -> spec.sslContext(sslServer)) @@ -166,7 +164,7 @@ void closeConnectionIfPingFrameDelayed() { .single() .block(); - Mono.delay(Duration.ofMillis(600)) + Mono.delay(Duration.ofSeconds(2)) .block(); assertThat(handler.getReceivedPingTimes()).hasSize(1); @@ -184,7 +182,7 @@ void closeConnectionInPoolIfPingFrameDelayed() { .subscribe() ); - DisposableServer disposableServer = createServer() + disposableServer = createServer() .protocol(H2) .maxKeepAliveRequests(1) .secure(spec -> spec.sslContext(sslServer)) @@ -214,7 +212,7 @@ void closeConnectionInPoolIfPingFrameDelayed() { .single() .block(); - Mono.delay(Duration.ofMillis(600)) + Mono.delay(Duration.ofSeconds(2)) .block(); assertThat(handler.getReceivedPingTimes()).hasSize(1); @@ -225,7 +223,7 @@ void closeConnectionInPoolIfPingFrameDelayed() { void ackPingFrameWithinInterval() { Http2PingFrameHandler handler = new Http2PingFrameHandler(); - DisposableServer disposableServer = createServer() + disposableServer = createServer() .protocol(H2) .maxKeepAliveRequests(1) .secure(spec -> spec.sslContext(sslServer)) @@ -246,7 +244,7 @@ void ackPingFrameWithinInterval() { .keepAlive(true) .secure(spec -> spec.sslContext(sslClient)) .http2Settings(builder -> { - builder.pingInterval(Duration.ofMillis(100)); + builder.pingInterval(Duration.ofSeconds(1)); }) .get() .uri("/") @@ -254,7 +252,7 @@ void ackPingFrameWithinInterval() { .single() .block(); - Mono.delay(Duration.ofSeconds(1)) + Mono.delay(Duration.ofSeconds(10)) .block(); assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); @@ -265,7 +263,7 @@ void ackPingFrameWithinInterval() { void connectionRetentionInPoolOnPingFrameAck() { Http2PingFrameHandler handler = new Http2PingFrameHandler(); - DisposableServer disposableServer = createServer() + disposableServer = createServer() .protocol(H2) .maxKeepAliveRequests(1) .secure(spec -> spec.sslContext(sslServer)) @@ -287,7 +285,7 @@ void connectionRetentionInPoolOnPingFrameAck() { .keepAlive(true) .secure(spec -> spec.sslContext(sslClient)) .http2Settings(builder -> { - builder.pingInterval(Duration.ofMillis(100)); + builder.pingInterval(Duration.ofSeconds(1)); }) .get() .uri("/") @@ -295,7 +293,7 @@ void connectionRetentionInPoolOnPingFrameAck() { .single() .block(); - Mono.delay(Duration.ofSeconds(1)) + Mono.delay(Duration.ofSeconds(10)) .block(); assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); @@ -322,6 +320,7 @@ private Http2PingFrameHandler(BiConsumer protected void channelRead0(ChannelHandlerContext ctx, Http2PingFrame frame) throws InterruptedException { receivedPingTimes.add(LocalDateTime.now(ZoneId.systemDefault())); consumer.accept(ctx, frame); + ctx.fireChannelRead(frame); } public List getReceivedPingTimes() { diff --git a/reactor-netty-http/src/test/java/reactor/netty/resources/PooledConnectionProviderDefaultMetricsTest.java b/reactor-netty-http/src/test/java/reactor/netty/resources/PooledConnectionProviderDefaultMetricsTest.java index b63dff62a7..74ebb039f6 100644 --- a/reactor-netty-http/src/test/java/reactor/netty/resources/PooledConnectionProviderDefaultMetricsTest.java +++ b/reactor-netty-http/src/test/java/reactor/netty/resources/PooledConnectionProviderDefaultMetricsTest.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024 VMware, Inc. or its affiliates, All Rights Reserved. + * Copyright (c) 2019-2025 VMware, Inc. or its affiliates, All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -388,6 +388,8 @@ private void testIssue3060(ConnectionProvider provider) throws Exception { /* https://github.com/reactor/reactor-netty/issues/3519 */ @Test public void testConnectionProviderDisableAllBuiltInMetrics() throws Exception { + REGISTRY.clear(); + disposableServer = createServer() .handle((req, res) -> res.sendString(Mono.just("testConnectionProviderDisableAllBuiltInMetrics"))) From 8987335102137a845d7f8eb662855f5cf25c2e4e Mon Sep 17 00:00:00 2001 From: raccoonback Date: Tue, 4 Feb 2025 23:36:08 +0900 Subject: [PATCH 09/27] Change comparison type for PING frame Signed-off-by: raccoonback --- .../netty/http/client/Http2ConnectionLivenessHandler.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java index d3f751edbc..9e3ca37509 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java @@ -21,8 +21,8 @@ import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPromise; -import io.netty.handler.codec.http2.DefaultHttp2PingFrame; import io.netty.handler.codec.http2.Http2ConnectionEncoder; +import io.netty.handler.codec.http2.Http2PingFrame; import reactor.util.Logger; import reactor.util.Loggers; import reactor.util.annotation.Nullable; @@ -91,8 +91,8 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg instanceof DefaultHttp2PingFrame) { - DefaultHttp2PingFrame frame = (DefaultHttp2PingFrame) msg; + if (msg instanceof Http2PingFrame) { + Http2PingFrame frame = (Http2PingFrame) msg; if (frame.ack() && frame.content() == lastSentPingData) { lastReceivedPingTime = System.nanoTime(); } From f9807c370ff40c78fd1998f69d835012c6e91305 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Wed, 5 Feb 2025 09:07:15 +0900 Subject: [PATCH 10/27] Fix broken test Signed-off-by: raccoonback --- .../Http2ConnectionLivenessHandlerTest.java | 23 +++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java index 19b313d9b5..c7fb3a85b3 100644 --- a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java +++ b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java @@ -33,6 +33,8 @@ import reactor.netty.BaseHttpTest; import reactor.netty.NettyPipeline; import reactor.netty.resources.ConnectionProvider; +import reactor.util.Logger; +import reactor.util.Loggers; import javax.net.ssl.SSLException; import java.security.cert.CertificateException; @@ -55,6 +57,8 @@ */ class Http2ConnectionLivenessHandlerTest extends BaseHttpTest { + private static final Logger log = Loggers.getLogger(Http2ConnectionLivenessHandlerTest.class); + static SslContext sslServer; static SslContext sslClient; @@ -199,6 +203,7 @@ void closeConnectionInPoolIfPingFrameDelayed() { .bindNow(); ConnectionProvider pool = ConnectionProvider.create("closeConnectionInPoolIfPingFrameDelayed", 1); + Channel channel = createClient(pool, disposableServer::address) .protocol(H2) .keepAlive(true) @@ -217,6 +222,8 @@ void closeConnectionInPoolIfPingFrameDelayed() { assertThat(handler.getReceivedPingTimes()).hasSize(1); assertThat(channel.parent().isOpen()).isFalse(); + + pool.dispose(); } @Test @@ -255,8 +262,8 @@ void ackPingFrameWithinInterval() { Mono.delay(Duration.ofSeconds(10)) .block(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); } @Test @@ -279,7 +286,11 @@ void connectionRetentionInPoolOnPingFrameAck() { .handle((req, resp) -> resp.sendString(Mono.just("Test"))) .bindNow(); - ConnectionProvider pool = ConnectionProvider.create("connectionRetentionInPoolOnPingFrameAck", 1); + ConnectionProvider pool = ConnectionProvider.builder("connectionRetentionInPoolOnPingFrameAck") + .maxConnections(10) + .maxIdleTime(Duration.ofSeconds(10)) + .maxLifeTime(Duration.ofSeconds(10)) + .build(); Channel channel = createClient(pool, disposableServer::address) .protocol(H2) .keepAlive(true) @@ -296,8 +307,10 @@ void connectionRetentionInPoolOnPingFrameAck() { Mono.delay(Duration.ofSeconds(10)) .block(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + + pool.dispose(); } private static final class Http2PingFrameHandler extends SimpleChannelInboundHandler { @@ -309,7 +322,9 @@ private static final class Http2PingFrameHandler extends SimpleChannelInboundHan private Http2PingFrameHandler() { this.consumer = (ctx, frame) -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE); + .addListener((listener) -> { + log.info("Wrote ping ack. data: {}, result: {}", frame.content(), listener.isSuccess()); + }); } private Http2PingFrameHandler(BiConsumer consumer) { From b9dd4bf527f00c4a87aff5c30a0a5caae9d82821 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Thu, 6 Feb 2025 22:54:11 +0900 Subject: [PATCH 11/27] Add configuration for HTTP/2 PING scheduler interval and retry threshold - Added a method to configure the execution interval of the scheduler that sends HTTP/2 PING frames and periodically checks for ACK responses - Introduced a retry threshold setting to limit the number of PING transmission attempts before considering the connection as unresponsive - Default values: - Scheduler interval must be explicitly set - Retry threshold defaults to 0 (no retries, only one PING attempt) Signed-off-by: raccoonback --- docs/modules/ROOT/pages/http-client.adoc | 5 +- .../java/reactor/netty/NettyPipeline.java | 2 +- .../http/client/http2/H2Application.java | 6 +- .../reactor/netty/http/Http2SettingsSpec.java | 138 +++++++++++++--- .../Http2ConnectionLivenessHandler.java | 79 +++++++--- .../netty/http/client/HttpClientConfig.java | 37 +++-- .../Http2ConnectionLivenessHandlerTest.java | 147 +++++++++++++----- 7 files changed, 320 insertions(+), 94 deletions(-) diff --git a/docs/modules/ROOT/pages/http-client.adoc b/docs/modules/ROOT/pages/http-client.adoc index a7335bc70b..b9815117ae 100644 --- a/docs/modules/ROOT/pages/http-client.adoc +++ b/docs/modules/ROOT/pages/http-client.adoc @@ -318,7 +318,10 @@ include::{examples-dir}/http2/H2Application.java[lines=18..42] ---- <1> Configures the client to support only `HTTP/2` <2> Configures `SSL` -<3> You can configure the interval for checking `Ping` frames +<3> Sets the interval for sending `HTTP/2` `PING` frames and receiving `ACK` responses +<4> Sets the execution interval for the scheduler that sends `HTTP/2` `PING frames and periodically checks for `ACK` responses +<5> Sets the threshold for retrying `HTTP/2` `PING` frame transmissions. + The following listing presents a simple `H2C` example: diff --git a/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java b/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java index 9c4eb8b8ab..f05181c2a1 100644 --- a/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java +++ b/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java @@ -112,7 +112,7 @@ public interface NettyPipeline { String OnChannelReadIdle = LEFT + "onChannelReadIdle"; String OnChannelWriteIdle = LEFT + "onChannelWriteIdle"; String ProxyHandler = LEFT + "proxyHandler"; - String H2LivenessHandler = LEFT + "h2LivenessHandler"; + String H2LivenessHandler = LEFT + "h2LivenessHandler"; /** * Use to register a special handler which ensures that any {@link io.netty.channel.VoidChannelPromise} * will be converted to "unvoided" promises. diff --git a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java index 150c29bde4..10aca39285 100644 --- a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java +++ b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java @@ -30,7 +30,11 @@ public static void main(String[] args) { HttpClient.create() .protocol(HttpProtocol.H2) //<1> .secure() //<2> - .http2Settings(builder -> builder.pingInterval(Duration.ofMillis(100))); // <3> + .http2Settings( + builder -> builder.pingAckTimeout(Duration.ofMillis(600)) // <3> + .pingScheduleInterval(Duration.ofMillis(300)) // <4> + .pingAckDropThreshold(2) // <5> + ); Tuple2 response = client.get() diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java b/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java index 3e725705a7..21d0bbc06b 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java @@ -105,19 +105,75 @@ public interface Builder { //Builder pushEnabled(boolean pushEnabled); /** - * Sets the interval for checking ping frames. - * If a ping ACK frame is not received within the configured interval, the connection will be closed. + * Sets the interval for sending HTTP/2 PING frames and receiving ACK responses. * - *

Be cautious when setting a very short interval, as it may cause the connection to be closed, - * even if the keep-alive setting is enabled.

+ *

+ * This method configures the time interval at which PING frames are sent to the peer. + * The interval should be chosen carefully to balance between detecting connection issues + * and minimizing unnecessary network traffic. + *

* - *

If no interval is specified, no ping frame checking will be performed by default.

+ *

+ * If the interval is set too short, it may cause excessive network overhead. + * If set too long, connection failures may not be detected in a timely manner. + *

* - * @param pingInterval the duration between sending ping frames. If not specified, ping frame checking is disabled. - * @return {@code this} - * @since 1.2.3 + * @param pingAckTimeout the interval in between consecutive PING frames + * and ACK responses. Must be a positive value. + */ + default Builder pingAckTimeout(Duration pingAckTimeout) { + return this; + } + + /** + * Sets the execution interval for the scheduler that sends HTTP/2 PING frames + * and periodically checks for ACK responses. + * + *

+ * This method configures the time interval at which the scheduler runs + * to send PING frames and verify if ACK responses are received within + * the expected timeframe. + * Proper tuning of this interval helps in detecting connection issues + * while avoiding unnecessary network overhead. + *

+ * + *

+ * If the interval is too short, it may increase network and CPU usage. + * Conversely, setting it too long may delay the detection of connection failures. + *

+ * + * @param pingScheduleInterval the interval in at which the scheduler executes. + * Must be a positive value. */ - default Builder pingInterval(Duration pingInterval) { + default Builder pingScheduleInterval(Duration pingScheduleInterval) { + return this; + } + + /** + * Sets the threshold for retrying HTTP/2 PING frame transmissions. + * + *

+ * This method defines the maximum number of attempts to send a PING frame + * before considering the connection as unresponsive. + * If the threshold is exceeded without receiving an ACK response, + * the connection may be closed or marked as unhealthy. + *

+ * + *

+ * A lower threshold can detect connection failures more quickly but may lead + * to premature disconnections. Conversely, a higher threshold allows more retries + * but may delay failure detection. + *

+ * + *

+ * If this value is not specified, it defaults to 0, meaning only one attempt to send a PING frame is made without retries. + *

+ * + * @param pingAckDropThreshold the maximum number of PING transmission attempts. + * Must be a positive integer. + * The default value is 0, meaning no retries will occur and only one PING frame will be sent. + */ + default Builder pingAckDropThreshold(Integer pingAckDropThreshold) { return this; } } @@ -215,13 +271,33 @@ public Boolean pushEnabled() { } /** - * Returns the configured {@code pingInterval} value or null. + * Returns the configured {@code pingAckTimeout} value or null. + * + * @return the configured {@code pingAckTimeout} value or null + */ + @Nullable + public Duration pingAckTimeout() { + return pingAckTimeout; + } + + /** + * Returns the configured {@code pingScheduleInterval} value or null. * - * @return the configured {@code pingInterval} value or null + * @return the configured {@code pingScheduleInterval} value or null */ @Nullable - public Duration pingInterval() { - return pingInterval; + public Duration pingScheduleInterval() { + return pingScheduleInterval; + } + + /** + * Returns the configured {@code pingAckDropThreshold} value or null. + * + * @return the configured {@code pingAckDropThreshold} value or null + */ + @Nullable + public Integer pingAckDropThreshold() { + return pingAckDropThreshold; } @Override @@ -241,7 +317,9 @@ public boolean equals(Object o) { maxHeaderListSize.equals(that.maxHeaderListSize) && Objects.equals(maxStreams, that.maxStreams) && Objects.equals(pushEnabled, that.pushEnabled) && - Objects.equals(pingInterval, that.pingInterval); + Objects.equals(pingAckTimeout, that.pingAckTimeout) && + Objects.equals(pingScheduleInterval, that.pingScheduleInterval) && + Objects.equals(pingAckDropThreshold, that.pingAckDropThreshold); } @Override @@ -255,7 +333,9 @@ public int hashCode() { result = 31 * result + (maxHeaderListSize == null ? 0 : Long.hashCode(maxHeaderListSize)); result = 31 * result + (maxStreams == null ? 0 : Long.hashCode(maxStreams)); result = 31 * result + (pushEnabled == null ? 0 : Boolean.hashCode(pushEnabled)); - result = 31 * result + (pingInterval == null ? 0 : Objects.hashCode(pingInterval)); + result = 31 * result + (pingAckTimeout == null ? 0 : Objects.hashCode(pingAckTimeout)); + result = 31 * result + (pingScheduleInterval == null ? 0 : Objects.hashCode(pingScheduleInterval)); + result = 31 * result + (pingAckDropThreshold == null ? 0 : Integer.hashCode(pingAckDropThreshold)); return result; } @@ -267,7 +347,9 @@ public int hashCode() { final Long maxHeaderListSize; final Long maxStreams; final Boolean pushEnabled; - final Duration pingInterval; + final Duration pingAckTimeout; + final Duration pingScheduleInterval; + final Integer pingAckDropThreshold; Http2SettingsSpec(Build build) { Http2Settings settings = build.http2Settings; @@ -285,13 +367,17 @@ public int hashCode() { maxHeaderListSize = settings.maxHeaderListSize(); maxStreams = build.maxStreams; pushEnabled = settings.pushEnabled(); - pingInterval = build.pingInterval; + pingAckTimeout = build.pingAckTimeout; + pingScheduleInterval = build.pingScheduleInterval; + pingAckDropThreshold = build.pingAckDropThreshold; } static final class Build implements Builder { Boolean connectProtocolEnabled; Long maxStreams; - Duration pingInterval; + Duration pingAckTimeout; + Duration pingScheduleInterval; + Integer pingAckDropThreshold; final Http2Settings http2Settings = Http2Settings.defaultSettings(); @Override @@ -345,8 +431,20 @@ public Builder maxStreams(long maxStreams) { } @Override - public Builder pingInterval(Duration pingInterval) { - this.pingInterval = pingInterval; + public Builder pingAckTimeout(Duration pingAckTimeout) { + this.pingAckTimeout = pingAckTimeout; + return this; + } + + @Override + public Builder pingScheduleInterval(Duration pingScheduleInterval) { + this.pingScheduleInterval = pingScheduleInterval; + return this; + } + + @Override + public Builder pingAckDropThreshold(Integer pingAckDropThreshold) { + this.pingAckDropThreshold = pingAckDropThreshold; return this; } diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java index 9e3ca37509..fc5aa42437 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java @@ -28,6 +28,7 @@ import reactor.util.annotation.Nullable; import java.time.Duration; +import java.util.Objects; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ThreadLocalRandom; @@ -55,22 +56,45 @@ final class Http2ConnectionLivenessHandler extends ChannelDuplexHandler { private static final Logger log = Loggers.getLogger(Http2ConnectionLivenessHandler.class); private ScheduledFuture pingScheduler; + private final ChannelFutureListener pingWriteListener = new PingWriteListener(); private final Http2ConnectionEncoder encoder; - private final long pingIntervalNanos; + private final long pingAckTimeoutNanos; + private final long pingScheduleIntervalNanos; + private final int pingAckDropThreshold; + + private int pingAckDropCount; private long lastSentPingData; private long lastReceivedPingTime; + private long lastSendingPingTime; private long lastIoTime; private boolean isPingAckPending; - public Http2ConnectionLivenessHandler(Http2ConnectionEncoder encoder, @Nullable Duration pingInterval) { + public Http2ConnectionLivenessHandler(Http2ConnectionEncoder encoder, @Nullable Duration pingAckTimeout, + @Nullable Duration pintScheduleInterval, @Nullable Integer pingAckDropThreshold) { + Objects.requireNonNull(encoder, "encoder"); + this.encoder = encoder; - if (pingInterval != null) { - this.pingIntervalNanos = pingInterval.toNanos(); + if (pingAckTimeout != null) { + this.pingAckTimeoutNanos = pingAckTimeout.toNanos(); + } + else { + this.pingAckTimeoutNanos = 0L; + } + + if (pintScheduleInterval != null) { + this.pingScheduleIntervalNanos = pintScheduleInterval.toNanos(); } else { - this.pingIntervalNanos = 0L; + this.pingScheduleIntervalNanos = 0L; + } + + if (pingAckDropThreshold != null) { + this.pingAckDropThreshold = pingAckDropThreshold; + } + else { + this.pingAckDropThreshold = 0; } } @@ -78,10 +102,11 @@ public Http2ConnectionLivenessHandler(Http2ConnectionEncoder encoder, @Nullable public void channelActive(ChannelHandlerContext ctx) throws Exception { if (isPingIntervalConfigured()) { isPingAckPending = false; + pingAckDropCount = 0; pingScheduler = ctx.executor() .schedule( new PingChecker(ctx), - pingIntervalNanos, + pingAckTimeoutNanos, NANOSECONDS ); } @@ -118,14 +143,9 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { ctx.fireChannelInactive(); } - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - cancel(); - ctx.fireExceptionCaught(cause); - } - private boolean isPingIntervalConfigured() { - return pingIntervalNanos > 0; + return pingAckTimeoutNanos > 0 + && pingScheduleIntervalNanos > 0; } private void cancel() { @@ -155,6 +175,7 @@ public void run() { } isPingAckPending = false; + pingAckDropCount = 0; pingScheduler = invokeNextSchedule(); return; } @@ -170,15 +191,28 @@ public void run() { } if (isOutOfTimeRange()) { + countPingDrop(); + + if (isExceedAckDropThreshold()) { + if (log.isInfoEnabled()) { + log.info("Closing {} channel due to delayed ping frame response (timeout: {} ns). lastReceivedPingTime: {}, current: {}", channel, pingAckTimeoutNanos, lastReceivedPingTime, System.nanoTime()); + } + + close(channel); + return; + } + if (log.isInfoEnabled()) { - log.info("Closing {} channel due to delayed ping frame response (timeout: {} ns).", channel, pingIntervalNanos); + log.info("Drop ping ack frame in {} channel. (ping: {})", channel, lastSentPingData); } - close(channel); + writePing(ctx); + pingScheduler = invokeNextSchedule(); return; } isPingAckPending = false; + pingAckDropCount = 0; pingScheduler = invokeNextSchedule(); } @@ -192,18 +226,26 @@ private void writePing(ChannelHandlerContext ctx) { } private boolean isIoInProgress() { - return pingIntervalNanos > (System.nanoTime() - lastIoTime); + return pingAckTimeoutNanos >= (System.nanoTime() - lastIoTime); } private boolean isOutOfTimeRange() { - return pingIntervalNanos < (System.nanoTime() - lastReceivedPingTime); + return pingAckTimeoutNanos < Math.abs(lastReceivedPingTime - lastSendingPingTime); + } + + private void countPingDrop() { + pingAckDropCount++; + } + + private boolean isExceedAckDropThreshold() { + return pingAckDropCount > pingAckDropThreshold; } private ScheduledFuture invokeNextSchedule() { return ctx.executor() .schedule( new PingChecker(ctx), - pingIntervalNanos, + pingScheduleIntervalNanos, NANOSECONDS ); } @@ -233,6 +275,7 @@ public void operationComplete(ChannelFuture future) throws Exception { } isPingAckPending = true; + lastSendingPingTime = System.nanoTime(); } else if (log.isDebugEnabled()) { log.debug("Failed to wrote PING frame to {} channel.", future.channel()); diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java index eebca60e3d..5fe20c8947 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java @@ -679,7 +679,8 @@ else if (metricsRecorder instanceof ContextAwareHttpClientMetricsRecorder) { } static void configureHttp2Pipeline(ChannelPipeline p, HttpResponseDecoderSpec decoder, - Http2Settings http2Settings, Duration pingInterval, ConnectionObserver observer) { + Http2Settings http2Settings, Duration pingAckTimeout, Duration pingScheduleInterval, + Integer pingAckDropThreshold, ConnectionObserver observer) { Http2FrameCodecBuilder http2FrameCodecBuilder = Http2FrameCodecBuilder.forClient() .validateHeaders(decoder.validateHeaders()) @@ -694,7 +695,8 @@ static void configureHttp2Pipeline(ChannelPipeline p, HttpResponseDecoderSpec de p.addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2Flush, new FlushConsolidationHandler(1024, true)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpCodec, codec) - .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2LivenessHandler, new Http2ConnectionLivenessHandler(codec.encoder(), pingInterval)) + .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2LivenessHandler, + new Http2ConnectionLivenessHandler(codec.encoder(), pingAckTimeout, pingScheduleInterval, pingAckDropThreshold)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2MultiplexHandler, new Http2MultiplexHandler(H2InboundStreamHandler.INSTANCE)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpTrafficHandler, new HttpTrafficHandler(observer)); } @@ -720,7 +722,9 @@ static void configureHttp11OrH2CleartextPipeline( boolean acceptGzip, HttpResponseDecoderSpec decoder, Http2Settings http2Settings, - Duration pingInterval, + Duration pingAckTimeout, + Duration pingScheduleInterval, + Integer pingAckDropThreshold, @Nullable ChannelMetricsRecorder metricsRecorder, ConnectionObserver observer, ChannelOperations.OnSetup opsFactory, @@ -757,7 +761,8 @@ static void configureHttp11OrH2CleartextPipeline( new ReactorNettyHttpClientUpgradeHandler(httpClientCodec, upgradeCodec, decoder.h2cMaxContentLength()); p.addBefore(NettyPipeline.ReactiveBridge, null, httpClientCodec) - .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2LivenessHandler, new Http2ConnectionLivenessHandler(http2FrameCodec.encoder(), pingInterval)) + .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2LivenessHandler, + new Http2ConnectionLivenessHandler(http2FrameCodec.encoder(), pingAckTimeout, pingScheduleInterval, pingAckDropThreshold)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2CUpgradeHandler, upgradeHandler) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpTrafficHandler, new HttpTrafficHandler(observer)); @@ -1002,7 +1007,9 @@ static final class H2OrHttp11Codec extends ChannelInboundHandlerAdapter { final boolean acceptGzip; final HttpResponseDecoderSpec decoder; final Http2Settings http2Settings; - final Duration pingInterval; + final Duration pingAckTimeout; + final Duration pingScheduleInterval; + final Integer pingAckDropThreshold; final ChannelMetricsRecorder metricsRecorder; final ConnectionObserver observer; final SocketAddress proxyAddress; @@ -1013,7 +1020,9 @@ static final class H2OrHttp11Codec extends ChannelInboundHandlerAdapter { this.acceptGzip = initializer.acceptGzip; this.decoder = initializer.decoder; this.http2Settings = initializer.http2Settings; - this.pingInterval = initializer.pingInterval; + this.pingAckTimeout = initializer.pingAckTimeout; + this.pingScheduleInterval = initializer.pingScheduleInterval; + this.pingAckDropThreshold = initializer.pingAckDropThreshold; this.metricsRecorder = initializer.metricsRecorder; this.observer = observer; this.proxyAddress = initializer.proxyAddress; @@ -1032,7 +1041,7 @@ public void channelActive(ChannelHandlerContext ctx) { log.debug(format(ctx.channel(), "Negotiated application-level protocol [" + protocol + "]")); } if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { - configureHttp2Pipeline(ctx.channel().pipeline(), decoder, http2Settings, pingInterval, observer); + configureHttp2Pipeline(ctx.channel().pipeline(), decoder, http2Settings, pingAckTimeout, pingScheduleInterval, pingAckDropThreshold, observer); } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { configureHttp11Pipeline(ctx.channel().pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); @@ -1056,7 +1065,9 @@ static final class HttpClientChannelInitializer implements ChannelPipelineConfig final boolean acceptGzip; final HttpResponseDecoderSpec decoder; final Http2Settings http2Settings; - final Duration pingInterval; + final Duration pingAckTimeout; + final Duration pingScheduleInterval; + final Integer pingAckDropThreshold; final ChannelMetricsRecorder metricsRecorder; final ChannelOperations.OnSetup opsFactory; final int protocols; @@ -1068,7 +1079,9 @@ static final class HttpClientChannelInitializer implements ChannelPipelineConfig this.acceptGzip = config.acceptGzip; this.decoder = config.decoder; this.http2Settings = config.http2Settings(); - this.pingInterval = config.http2Settings != null ? config.http2Settings.pingInterval() : null; + this.pingAckTimeout = config.http2Settings != null ? config.http2Settings.pingAckTimeout() : null; + this.pingScheduleInterval = config.http2Settings != null ? config.http2Settings.pingScheduleInterval() : null; + this.pingAckDropThreshold = config.http2Settings != null ? config.http2Settings.pingAckDropThreshold() : null; this.metricsRecorder = config.metricsRecorderInternal(); this.opsFactory = config.channelOperationsProvider(); this.protocols = config._protocols; @@ -1093,7 +1106,7 @@ else if ((protocols & h11) == h11) { configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h2) == h2) { - configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, pingInterval, observer); + configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, pingAckTimeout, pingScheduleInterval, pingAckDropThreshold, observer); } else if ((protocols & h3) == h3) { configureHttp3Pipeline(channel.pipeline(), metricsRecorder != null, proxyAddress != null); @@ -1101,13 +1114,13 @@ else if ((protocols & h3) == h3) { } else { if ((protocols & h11orH2C) == h11orH2C) { - configureHttp11OrH2CleartextPipeline(channel.pipeline(), acceptGzip, decoder, http2Settings, pingInterval, metricsRecorder, observer, opsFactory, proxyAddress, remoteAddress, uriTagValue); + configureHttp11OrH2CleartextPipeline(channel.pipeline(), acceptGzip, decoder, http2Settings, pingAckTimeout, pingScheduleInterval, pingAckDropThreshold, metricsRecorder, observer, opsFactory, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h11) == h11) { configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h2c) == h2c) { - configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, pingInterval, observer); + configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, pingAckTimeout, pingScheduleInterval, pingAckDropThreshold, observer); } } } diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java index c7fb3a85b3..95587ba5c5 100644 --- a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java +++ b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java @@ -29,12 +29,12 @@ import io.netty.handler.ssl.util.SelfSignedCertificate; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import reactor.core.publisher.Mono; import reactor.netty.BaseHttpTest; import reactor.netty.NettyPipeline; import reactor.netty.resources.ConnectionProvider; -import reactor.util.Logger; -import reactor.util.Loggers; import javax.net.ssl.SSLException; import java.security.cert.CertificateException; @@ -43,7 +43,6 @@ import java.time.ZoneId; import java.util.ArrayList; import java.util.List; -import java.util.function.BiConsumer; import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; @@ -57,8 +56,6 @@ */ class Http2ConnectionLivenessHandlerTest extends BaseHttpTest { - private static final Logger log = Loggers.getLogger(Http2ConnectionLivenessHandlerTest.class); - static SslContext sslServer; static SslContext sslClient; @@ -92,7 +89,7 @@ void successReceiveResponse() { } @Test - void noPingCheckWhenNotConfigured() { + void maintainConnectionWithoutPingCheckWhenNotConfigured() { Http2PingFrameHandler handler = new Http2PingFrameHandler(); disposableServer = createServer() @@ -121,17 +118,18 @@ void noPingCheckWhenNotConfigured() { .single() .block(); - Mono.delay(Duration.ofSeconds(1)) + Mono.delay(Duration.ofSeconds(5)) .block(); - assertThat(handler.getReceivedPingTimes()).isEmpty(); assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).isEmpty(); } - @Test - void closeConnectionIfPingFrameDelayed() { + @ParameterizedTest + @CsvSource({"100,300,3", "300,100,3"}) + void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame) -> Mono.delay(Duration.ofMillis(150)) + (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) .doOnNext( unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) @@ -160,7 +158,9 @@ void closeConnectionIfPingFrameDelayed() { .keepAlive(true) .secure(spec -> spec.sslContext(sslClient)) .http2Settings(builder -> { - builder.pingInterval(Duration.ofMillis(100)); + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); }) .get() .uri("/") @@ -168,17 +168,18 @@ void closeConnectionIfPingFrameDelayed() { .single() .block(); - Mono.delay(Duration.ofSeconds(2)) + Mono.delay(Duration.ofSeconds(3)) .block(); - assertThat(handler.getReceivedPingTimes()).hasSize(1); assertThat(channel.parent().isOpen()).isFalse(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); } - @Test - void closeConnectionInPoolIfPingFrameDelayed() { + @ParameterizedTest + @CsvSource({"100,300,3", "300,100,3"}) + void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame) -> Mono.delay(Duration.ofMillis(150)) + (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) .doOnNext( unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) @@ -209,7 +210,9 @@ void closeConnectionInPoolIfPingFrameDelayed() { .keepAlive(true) .secure(spec -> spec.sslContext(sslClient)) .http2Settings(builder -> { - builder.pingInterval(Duration.ofMillis(100)); + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); }) .get() .uri("/") @@ -217,17 +220,18 @@ void closeConnectionInPoolIfPingFrameDelayed() { .single() .block(); - Mono.delay(Duration.ofSeconds(2)) + Mono.delay(Duration.ofSeconds(3)) .block(); - assertThat(handler.getReceivedPingTimes()).hasSize(1); assertThat(channel.parent().isOpen()).isFalse(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); pool.dispose(); } - @Test - void ackPingFrameWithinInterval() { + @ParameterizedTest + @CsvSource({"300,600,0", "600,300,0"}) + void ackPingFrameWithinInterval(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { Http2PingFrameHandler handler = new Http2PingFrameHandler(); disposableServer = createServer() @@ -251,7 +255,9 @@ void ackPingFrameWithinInterval() { .keepAlive(true) .secure(spec -> spec.sslContext(sslClient)) .http2Settings(builder -> { - builder.pingInterval(Duration.ofSeconds(1)); + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); }) .get() .uri("/") @@ -259,15 +265,16 @@ void ackPingFrameWithinInterval() { .single() .block(); - Mono.delay(Duration.ofSeconds(10)) + Mono.delay(Duration.ofSeconds(5)) .block(); assertThat(channel.parent().isOpen()).isTrue(); assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); } - @Test - void connectionRetentionInPoolOnPingFrameAck() { + @ParameterizedTest + @CsvSource({"300,600,0", "600,300,0"}) + void connectionRetentionInPoolOnPingFrameAck(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { Http2PingFrameHandler handler = new Http2PingFrameHandler(); disposableServer = createServer() @@ -286,17 +293,16 @@ void connectionRetentionInPoolOnPingFrameAck() { .handle((req, resp) -> resp.sendString(Mono.just("Test"))) .bindNow(); - ConnectionProvider pool = ConnectionProvider.builder("connectionRetentionInPoolOnPingFrameAck") - .maxConnections(10) - .maxIdleTime(Duration.ofSeconds(10)) - .maxLifeTime(Duration.ofSeconds(10)) - .build(); + ConnectionProvider pool = ConnectionProvider.create("connectionRetentionInPoolOnPingFrameAck", 1); + Channel channel = createClient(pool, disposableServer::address) .protocol(H2) .keepAlive(true) .secure(spec -> spec.sslContext(sslClient)) .http2Settings(builder -> { - builder.pingInterval(Duration.ofSeconds(1)); + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); }) .get() .uri("/") @@ -304,7 +310,7 @@ void connectionRetentionInPoolOnPingFrameAck() { .single() .block(); - Mono.delay(Duration.ofSeconds(10)) + Mono.delay(Duration.ofSeconds(5)) .block(); assertThat(channel.parent().isOpen()).isTrue(); @@ -313,29 +319,83 @@ void connectionRetentionInPoolOnPingFrameAck() { pool.dispose(); } + @ParameterizedTest + @CsvSource({"300,600,3", "600,300,3"}) + void ackPingFrameWithinThreshold(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> { + int delayTime = 0; + if (receivedPingTimes.size() % 3 != 0) { + delayTime = 600; + } + + Mono.delay(Duration.ofMillis(delayTime)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe(); + } + ); + + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .keepAlive(true) + .secure(spec -> spec.sslContext(sslClient)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + } + private static final class Http2PingFrameHandler extends SimpleChannelInboundHandler { private final List receivedPingTimes = new ArrayList<>(); - private final BiConsumer consumer; + private final TriConsumer> consumer; private Http2PingFrameHandler() { - this.consumer = (ctx, frame) -> + this.consumer = (ctx, frame, receivedPings) -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener((listener) -> { - log.info("Wrote ping ack. data: {}, result: {}", frame.content(), listener.isSuccess()); - }); + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE); } - private Http2PingFrameHandler(BiConsumer consumer) { + private Http2PingFrameHandler(TriConsumer> consumer) { this.consumer = consumer; } @Override protected void channelRead0(ChannelHandlerContext ctx, Http2PingFrame frame) throws InterruptedException { receivedPingTimes.add(LocalDateTime.now(ZoneId.systemDefault())); - consumer.accept(ctx, frame); - ctx.fireChannelRead(frame); + consumer.accept(ctx, frame, receivedPingTimes); } public List getReceivedPingTimes() { @@ -344,4 +404,9 @@ public List getReceivedPingTimes() { .collect(Collectors.toList()); } } + + @FunctionalInterface + public interface TriConsumer { + void accept(T t, U u, V v); + } } From db7586c6cb78118eb2934aa0757b9c60da8ba35b Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sat, 15 Mar 2025 07:18:58 +0900 Subject: [PATCH 12/27] Support HTTP health check in IdleTimeoutHandler via HTTP/2 PING - Added support for HTTP/2 PING-based health checks in IdleTimeoutHandler - Ensures connections remain active during health checks Signed-off-by: raccoonback --- .../java/reactor/netty/NettyPipeline.java | 1 - ...dler.java => Http2ConnectionLiveness.java} | 171 ++-- .../reactor/netty/http/Http2SettingsSpec.java | 28 + .../http/HttpConnectionImmediateClose.java | 69 ++ .../netty/http/HttpConnectionLiveness.java | 47 + .../netty/http/IdleTimeoutHandler.java | 175 ++++ .../reactor/netty/http/client/HttpClient.java | 20 + .../netty/http/client/HttpClientConfig.java | 154 ++-- .../netty/http/server/HttpServerConfig.java | 86 +- .../netty/http/server/HttpTrafficHandler.java | 16 +- .../netty/http/server/IdleTimeoutHandler.java | 85 -- .../Http2ConnectionLivenessHandlerTest.java | 412 --------- .../client/HttpConnectionLivenessTest.java | 792 +++++++++++++++++ .../http/client/HttpIdleTimeoutTest.java | 252 ++++++ .../server/HttpConnectionLivenessTest.java | 816 ++++++++++++++++++ .../http/server/HttpIdleTimeoutTest.java | 187 ++++ 16 files changed, 2625 insertions(+), 686 deletions(-) rename reactor-netty-http/src/main/java/reactor/netty/http/{client/Http2ConnectionLivenessHandler.java => Http2ConnectionLiveness.java} (59%) create mode 100644 reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionImmediateClose.java create mode 100644 reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionLiveness.java create mode 100644 reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java delete mode 100644 reactor-netty-http/src/main/java/reactor/netty/http/server/IdleTimeoutHandler.java delete mode 100644 reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java create mode 100644 reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java create mode 100644 reactor-netty-http/src/test/java/reactor/netty/http/client/HttpIdleTimeoutTest.java create mode 100644 reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java create mode 100644 reactor-netty-http/src/test/java/reactor/netty/http/server/HttpIdleTimeoutTest.java diff --git a/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java b/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java index f05181c2a1..380c66835d 100644 --- a/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java +++ b/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java @@ -112,7 +112,6 @@ public interface NettyPipeline { String OnChannelReadIdle = LEFT + "onChannelReadIdle"; String OnChannelWriteIdle = LEFT + "onChannelWriteIdle"; String ProxyHandler = LEFT + "proxyHandler"; - String H2LivenessHandler = LEFT + "h2LivenessHandler"; /** * Use to register a special handler which ensures that any {@link io.netty.channel.VoidChannelPromise} * will be converted to "unvoided" promises. diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java similarity index 59% rename from reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java rename to reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java index fc5aa42437..b5d885997c 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/Http2ConnectionLivenessHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java @@ -13,31 +13,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package reactor.netty.http.client; +package reactor.netty.http; import io.netty.channel.Channel; -import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelPromise; -import io.netty.handler.codec.http2.Http2ConnectionEncoder; +import io.netty.handler.codec.http2.Http2FrameCodec; +import io.netty.handler.codec.http2.Http2FrameWriter; import io.netty.handler.codec.http2.Http2PingFrame; import reactor.util.Logger; import reactor.util.Loggers; import reactor.util.annotation.Nullable; import java.time.Duration; -import java.util.Objects; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ThreadLocalRandom; import static java.util.concurrent.TimeUnit.NANOSECONDS; /** - * Handler that supports connection health checks using HTTP/2 Ping Frames. + * Supports connection health checks using HTTP/2 Ping Frames. * - *

This Handler sends a ping frame at the specified interval when no frame is being read or written, + *

Http2ConnectionLiveness sends a ping frame at the specified interval when no frame is being read or written, * ensuring the connection health is monitored. If a ping ACK frame is not received within the configured interval, * the connection will be closed.

* @@ -49,16 +47,16 @@ *

If no interval is specified, no ping frame checking will be performed.

* * @author raccoonback - * @since 1.2.3 + * @since 1.2.5 */ -final class Http2ConnectionLivenessHandler extends ChannelDuplexHandler { +public final class Http2ConnectionLiveness implements HttpConnectionLiveness { - private static final Logger log = Loggers.getLogger(Http2ConnectionLivenessHandler.class); + static final Logger log = Loggers.getLogger(Http2ConnectionLiveness.class); private ScheduledFuture pingScheduler; private final ChannelFutureListener pingWriteListener = new PingWriteListener(); - private final Http2ConnectionEncoder encoder; + private final Http2FrameWriter http2FrameWriter; private final long pingAckTimeoutNanos; private final long pingScheduleIntervalNanos; private final int pingAckDropThreshold; @@ -67,14 +65,24 @@ final class Http2ConnectionLivenessHandler extends ChannelDuplexHandler { private long lastSentPingData; private long lastReceivedPingTime; private long lastSendingPingTime; - private long lastIoTime; private boolean isPingAckPending; - public Http2ConnectionLivenessHandler(Http2ConnectionEncoder encoder, @Nullable Duration pingAckTimeout, - @Nullable Duration pintScheduleInterval, @Nullable Integer pingAckDropThreshold) { - Objects.requireNonNull(encoder, "encoder"); - - this.encoder = encoder; + /** + * Constructs a new {@code Http2ConnectionLiveness} instance. + * + * @param http2FrameCodec the HTTP/2 frame codec + * @param pingAckTimeout the ping ACK timeout duration + * @param pingScheduleInterval the ping schedule interval duration + * @param pingAckDropThreshold the ping ACK drop threshold + */ + public Http2ConnectionLiveness( + Http2FrameCodec http2FrameCodec, + @Nullable Duration pingAckTimeout, + @Nullable Duration pingScheduleInterval, + @Nullable Integer pingAckDropThreshold + ) { + this.http2FrameWriter = http2FrameCodec.encoder() + .frameWriter(); if (pingAckTimeout != null) { this.pingAckTimeoutNanos = pingAckTimeout.toNanos(); @@ -83,8 +91,8 @@ public Http2ConnectionLivenessHandler(Http2ConnectionEncoder encoder, @Nullable this.pingAckTimeoutNanos = 0L; } - if (pintScheduleInterval != null) { - this.pingScheduleIntervalNanos = pintScheduleInterval.toNanos(); + if (pingScheduleInterval != null) { + this.pingScheduleIntervalNanos = pingScheduleInterval.toNanos(); } else { this.pingScheduleIntervalNanos = 0L; @@ -98,67 +106,72 @@ public Http2ConnectionLivenessHandler(Http2ConnectionEncoder encoder, @Nullable } } - @Override - public void channelActive(ChannelHandlerContext ctx) throws Exception { + /** + * Checks the liveness of the connection and schedules a ping if necessary. + * + * @param ctx the {@link ChannelHandlerContext} of the connection + */ + public void check(ChannelHandlerContext ctx) { if (isPingIntervalConfigured()) { - isPingAckPending = false; - pingAckDropCount = 0; - pingScheduler = ctx.executor() - .schedule( - new PingChecker(ctx), - pingAckTimeoutNanos, - NANOSECONDS - ); + if (pingScheduler == null) { + isPingAckPending = false; + pingAckDropCount = 0; + pingScheduler = ctx.executor() + .schedule( + new PingTimeoutTask(ctx), + pingAckTimeoutNanos, + NANOSECONDS + ); + } + + return; } - ctx.fireChannelActive(); + ctx.close(); } - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + /** + * Receives a message from the peer and processes it if it is a ping frame. + * + * @param msg the message received from the peer + */ + public void receive(Object msg) { if (msg instanceof Http2PingFrame) { Http2PingFrame frame = (Http2PingFrame) msg; if (frame.ack() && frame.content() == lastSentPingData) { lastReceivedPingTime = System.nanoTime(); } } - else { - lastIoTime = System.nanoTime(); - } - - ctx.fireChannelRead(msg); - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { - lastIoTime = System.nanoTime(); - - ctx.write(msg, promise); } - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - cancel(); - ctx.fireChannelInactive(); - } - - private boolean isPingIntervalConfigured() { + /** + * Checks if the ping interval is configured. + * + * @return {@code true} if the ping interval is configured, {@code false} otherwise + */ + public boolean isPingIntervalConfigured() { return pingAckTimeoutNanos > 0 && pingScheduleIntervalNanos > 0; } - private void cancel() { + /** + * Cancels the scheduled ping task. + */ + public void cancel() { if (pingScheduler != null) { pingScheduler.cancel(false); + pingScheduler = null; } } - private class PingChecker implements Runnable { + /** + * A task that handles ping timeouts. + */ + class PingTimeoutTask implements Runnable { private final ChannelHandlerContext ctx; - PingChecker(ChannelHandlerContext ctx) { + PingTimeoutTask(ChannelHandlerContext ctx) { this.ctx = ctx; } @@ -169,20 +182,9 @@ public void run() { return; } - if (lastIoTime == 0 || isIoInProgress()) { - if (log.isDebugEnabled()) { - log.debug("{} channel is currently reading or writing data.", channel); - } - - isPingAckPending = false; - pingAckDropCount = 0; - pingScheduler = invokeNextSchedule(); - return; - } - if (!isPingAckPending) { if (log.isDebugEnabled()) { - log.debug("Attempting to send a ping frame to {} channel.", channel); + log.debug("Attempting to send a ping frame to the channel: {}", channel); } writePing(ctx); @@ -195,15 +197,15 @@ public void run() { if (isExceedAckDropThreshold()) { if (log.isInfoEnabled()) { - log.info("Closing {} channel due to delayed ping frame response (timeout: {} ns). lastReceivedPingTime: {}, current: {}", channel, pingAckTimeoutNanos, lastReceivedPingTime, System.nanoTime()); + log.info("Closing the channel due to delayed ping frame response (timeout: {} ns). {}", pingAckTimeoutNanos, channel); } - close(channel); + close(); return; } if (log.isInfoEnabled()) { - log.info("Drop ping ack frame in {} channel. (ping: {})", channel, lastSentPingData); + log.info("Dropping ping ACK frame in channel (ping data: {}). channel: {}", lastSentPingData, channel); } writePing(ctx); @@ -219,16 +221,12 @@ public void run() { private void writePing(ChannelHandlerContext ctx) { lastSentPingData = ThreadLocalRandom.current().nextLong(); - encoder.frameWriter() + http2FrameWriter .writePing(ctx, false, lastSentPingData, ctx.newPromise()) .addListener(pingWriteListener); ctx.flush(); } - private boolean isIoInProgress() { - return pingAckTimeoutNanos >= (System.nanoTime() - lastIoTime); - } - private boolean isOutOfTimeRange() { return pingAckTimeoutNanos < Math.abs(lastReceivedPingTime - lastSendingPingTime); } @@ -244,41 +242,46 @@ private boolean isExceedAckDropThreshold() { private ScheduledFuture invokeNextSchedule() { return ctx.executor() .schedule( - new PingChecker(ctx), + this, pingScheduleIntervalNanos, NANOSECONDS ); } - private void close(Channel channel) { - channel.close() + private void close() { + ctx.close() .addListener(future -> { if (future.isSuccess()) { if (log.isDebugEnabled()) { - log.debug("{} channel closed an channel", channel); + log.debug("Channel closed after liveness check: {}", ctx.channel()); } } else if (log.isDebugEnabled()) { - log.debug("{} channel failed to close an channel", channel, future.cause()); + log.debug("Failed to close the channel: {}. Cause: {}", ctx.channel(), future.cause()); } }); } } + /** + * A listener that handles the completion of ping frame writes. + */ private class PingWriteListener implements ChannelFutureListener { @Override - public void operationComplete(ChannelFuture future) throws Exception { + public void operationComplete(ChannelFuture future) { if (future.isSuccess()) { if (log.isDebugEnabled()) { - log.debug("Wrote PING frame to {} channel.", future.channel()); + log.debug("Successfully wrote PING frame to the channel: {}", future.channel()); } isPingAckPending = true; lastSendingPingTime = System.nanoTime(); } - else if (log.isDebugEnabled()) { - log.debug("Failed to wrote PING frame to {} channel.", future.channel()); + else { + if (log.isDebugEnabled()) { + log.debug("Failed to write PING frame to the channel: {}", future.channel()); + } } } } diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java b/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java index 21d0bbc06b..825099a698 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java @@ -456,4 +456,32 @@ public Builder pushEnabled(boolean pushEnabled) { } */ } + + public Http2Settings http2Settings() { + Http2Settings settings = Http2Settings.defaultSettings(); + + if (headerTableSize != null) { + settings.headerTableSize(headerTableSize); + } + + if (initialWindowSize != null) { + settings.initialWindowSize(initialWindowSize); + } + + if (maxConcurrentStreams != null) { + settings.maxConcurrentStreams(maxConcurrentStreams); + } + + if (maxFrameSize != null) { + settings.maxFrameSize(maxFrameSize); + } + + settings.maxHeaderListSize(maxHeaderListSize); + + if (pushEnabled != null) { + settings.pushEnabled(pushEnabled); + } + + return settings; + } } diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionImmediateClose.java b/reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionImmediateClose.java new file mode 100644 index 0000000000..67dda27029 --- /dev/null +++ b/reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionImmediateClose.java @@ -0,0 +1,69 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.http; + +import io.netty.channel.ChannelHandlerContext; + +/** + * This class implements the {@link HttpConnectionLiveness} interface and provides + * a mechanism to immediately close the HTTP connection without checking its liveness. + * + *

+ * The methods in this class are no-ops except for the {@code check} method, which + * closes the connection. + *

+ * + * @author raccoonback + * @since 1.2.5 + */ +public final class HttpConnectionImmediateClose implements HttpConnectionLiveness { + + /** + * Constructs a new {@code HttpConnectionImmediateClose} instance. + */ + public HttpConnectionImmediateClose() { + } + + /** + * Cancels the HTTP connection. + * This method is a no-op. + */ + @Override + public void cancel() { + // no op + } + + /** + * Receives a message from the peer. + * This method is a no-op. + * + * @param msg the message received from the peer + */ + @Override + public void receive(Object msg) { + // no op + } + + /** + * Checks the liveness of the connection and closes it immediately. + * + * @param ctx the {@link ChannelHandlerContext} of the connection + */ + @Override + public void check(ChannelHandlerContext ctx) { + ctx.close(); + } +} diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionLiveness.java b/reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionLiveness.java new file mode 100644 index 0000000000..09c08e9d70 --- /dev/null +++ b/reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionLiveness.java @@ -0,0 +1,47 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.http; + +import io.netty.channel.ChannelHandlerContext; + +/** + * Interface for checking the liveness of an HTTP connection. + * This interface provides methods to cancel the HTTP connection, + * process messages received from the peer, and start checking the liveness of the connection. + * + * @author raccoonback + * @since 1.2.5 + */ +public interface HttpConnectionLiveness { + + /** + * Cancel Http Connection by protocol. + * + */ + void cancel(); + + /** + * Receive the message received from peer. + * + */ + void receive(Object msg); + + /** + * Start checking liveness. + * + */ + void check(ChannelHandlerContext ctx); +} diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java new file mode 100644 index 0000000000..abdefa9b0c --- /dev/null +++ b/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java @@ -0,0 +1,175 @@ +/* + * Copyright (c) 2022-2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.http; + +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPipeline; +import io.netty.handler.codec.http.HttpClientCodec; +import io.netty.handler.codec.http.HttpClientUpgradeHandler; +import io.netty.handler.codec.http.HttpServerCodec; +import io.netty.handler.codec.http.HttpServerUpgradeHandler; +import io.netty.handler.timeout.IdleState; +import io.netty.handler.timeout.IdleStateEvent; +import io.netty.handler.timeout.IdleStateHandler; +import reactor.netty.NettyPipeline; +import reactor.util.Logger; +import reactor.util.Loggers; +import reactor.util.annotation.Nullable; + +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +import static reactor.netty.ReactorNetty.format; + +/** + * A handler that manages idle timeout for HTTP connections. + * This handler will close the connection if it remains idle for the specified duration. + * It also checks the liveness of the HTTP connection. + * + * @author raccoonback + * @since 1.2.5 + */ +public final class IdleTimeoutHandler extends IdleStateHandler { + + private static final Logger log = Loggers.getLogger(IdleTimeoutHandler.class); + + private final HttpConnectionLiveness httpConnectionLiveness; + + private IdleTimeoutHandler(long idleTimeout, HttpConnectionLiveness httpConnectionLiveness) { + super(0, 0, idleTimeout, TimeUnit.MILLISECONDS); + this.httpConnectionLiveness = httpConnectionLiveness; + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + protected void channelIdle(ChannelHandlerContext ctx, IdleStateEvent evt) { + if (evt.state() == IdleState.ALL_IDLE) { + if (log.isDebugEnabled()) { + log.debug(format(ctx.channel(), + "Connection was idle for [{}ms], as per configuration the connection will be closed."), + getAllIdleTimeInMillis()); + } + // FutureReturnValueIgnored is deliberate + + httpConnectionLiveness.check(ctx); + } + + ctx.fireUserEventTriggered(evt); + } + + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + httpConnectionLiveness.receive(msg); + + super.channelRead(ctx, msg); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + httpConnectionLiveness.cancel(); + + super.channelInactive(ctx); + } + + /** + * Adds an idle timeout handler to the server pipeline. + * + * @param pipeline the channel pipeline + * @param idleTimeout the idle timeout duration + * @param httpConnectionLiveness the HTTP connection liveness checker + */ + public static void addIdleTimeoutServerHandler(ChannelPipeline pipeline, @Nullable Duration idleTimeout, + HttpConnectionLiveness httpConnectionLiveness) { + if (idleTimeout != null && pipeline.get(NettyPipeline.IdleTimeoutHandler) == null) { + String baseName = null; + if (pipeline.get(NettyPipeline.HttpCodec) != null) { + baseName = NettyPipeline.HttpCodec; + } + else { + ChannelHandler httpServerUpgradeHandler = pipeline.get(HttpServerUpgradeHandler.class); + if (httpServerUpgradeHandler != null) { + baseName = pipeline.context(httpServerUpgradeHandler).name(); + } + else { + ChannelHandler httpServerCodec = pipeline.get(HttpServerCodec.class); + if (httpServerCodec != null) { + baseName = pipeline.context(httpServerCodec).name(); + } + } + } + + pipeline.addAfter(baseName, + NettyPipeline.IdleTimeoutHandler, + new IdleTimeoutHandler( + idleTimeout.toMillis(), + httpConnectionLiveness + ) + ); + } + } + + /** + * Adds an idle timeout handler to the client pipeline. + * This handler will close the connection if it remains idle for the specified duration. + * + * @param pipeline the channel pipeline to which the handler will be added + * @param idleTimeout the duration of idle time after which the connection will be closed + * @param httpConnectionLiveness the HTTP connection liveness checker to be used + */ + public static void addIdleTimeoutClientHandler(ChannelPipeline pipeline, @Nullable Duration idleTimeout, + HttpConnectionLiveness httpConnectionLiveness) { + if (idleTimeout != null && pipeline.get(NettyPipeline.IdleTimeoutHandler) == null) { + String baseName = null; + if (pipeline.get(NettyPipeline.HttpCodec) != null) { + baseName = NettyPipeline.HttpCodec; + } + else { + ChannelHandler httpClientUpgradeHandler = pipeline.get(HttpClientUpgradeHandler.class); + if (httpClientUpgradeHandler != null) { + baseName = pipeline.context(httpClientUpgradeHandler).name(); + } + else { + ChannelHandler httpClientCodec = pipeline.get(HttpClientCodec.class); + if (httpClientCodec != null) { + baseName = pipeline.context(httpClientCodec).name(); + } + } + } + + pipeline.addAfter( + baseName, + NettyPipeline.IdleTimeoutHandler, + new IdleTimeoutHandler( + idleTimeout.toMillis(), + httpConnectionLiveness + ) + ); + } + } + + /** + * Removes the idle timeout handler from the pipeline if it exists. + * + * @param pipeline the channel pipeline from which the handler will be removed + */ + public static void removeIdleTimeoutHandler(ChannelPipeline pipeline) { + if (pipeline.get(NettyPipeline.IdleTimeoutHandler) != null) { + pipeline.remove(NettyPipeline.IdleTimeoutHandler); + } + } +} diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java index e5b5f7527a..23d05ebc1f 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java @@ -59,6 +59,7 @@ import reactor.netty.http.HttpProtocol; import reactor.netty.http.logging.HttpMessageLogFactory; import reactor.netty.http.logging.ReactorNettyHttpMessageLogFactory; +import reactor.netty.http.server.HttpServer; import reactor.netty.http.websocket.WebsocketInbound; import reactor.netty.http.websocket.WebsocketOutbound; import reactor.netty.internal.util.Metrics; @@ -1095,6 +1096,25 @@ public final HttpClient headersWhen(FunctionIf an {@code idleTimeout} is not specified, this indicates no timeout (i.e. infinite), + * which means the connection will be closed only if one of the peers decides to close it. + *

If the {@code idleTimeout} is less than {@code 1ms}, then {@code 1ms} will be the idle timeout. + *

By default {@code idleTimeout} is not specified. + * + * @param idleTimeout an idle timeout on the connection when it is waiting for an HTTP request (resolution: ms) + * @return a new {@link HttpClient} + * @since 1.2.5 + */ + public final HttpClient idleTimeout(Duration idleTimeout) { + Objects.requireNonNull(idleTimeout, "idleTimeout"); + HttpClient dup = duplicate(); + dup.configuration().idleTimeout = idleTimeout; + return dup; + } + /** * Apply HTTP/2 configuration. * diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java index 5fe20c8947..e926a2f8ab 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java @@ -78,10 +78,13 @@ import reactor.netty.ReactorNetty; import reactor.netty.channel.ChannelMetricsRecorder; import reactor.netty.channel.ChannelOperations; +import reactor.netty.http.HttpConnectionImmediateClose; +import reactor.netty.http.Http2ConnectionLiveness; import reactor.netty.http.Http2SettingsSpec; import reactor.netty.http.Http3SettingsSpec; import reactor.netty.http.HttpProtocol; import reactor.netty.http.HttpResources; +import reactor.netty.http.IdleTimeoutHandler; import reactor.netty.http.logging.HttpMessageLogFactory; import reactor.netty.http.logging.ReactorNettyHttpMessageLogFactory; import reactor.netty.resources.ConnectionProvider; @@ -361,6 +364,7 @@ public WebsocketClientSpec websocketClientSpec() { BiConsumer doOnResponseError; BiPredicate followRedirectPredicate; HttpHeaders headers; + Duration idleTimeout; Http2SettingsSpec http2Settings; Http3SettingsSpec http3Settings; HttpMessageLogFactory httpMessageLogFactory; @@ -413,6 +417,7 @@ public WebsocketClientSpec websocketClientSpec() { this.doOnResponseError = parent.doOnResponseError; this.followRedirectPredicate = parent.followRedirectPredicate; this.headers = parent.headers; + this.idleTimeout = parent.idleTimeout; this.http2Settings = parent.http2Settings; this.http3Settings = parent.http3Settings; this.httpMessageLogFactory = parent.httpMessageLogFactory; @@ -558,41 +563,6 @@ boolean checkProtocol(int protocol) { return (_protocols & protocol) == protocol; } - Http2Settings http2Settings() { - Http2Settings settings = Http2Settings.defaultSettings(); - - if (http2Settings != null) { - Long headerTableSize = http2Settings.headerTableSize(); - if (headerTableSize != null) { - settings.headerTableSize(headerTableSize); - } - - Integer initialWindowSize = http2Settings.initialWindowSize(); - if (initialWindowSize != null) { - settings.initialWindowSize(initialWindowSize); - } - - Long maxConcurrentStreams = http2Settings.maxConcurrentStreams(); - if (maxConcurrentStreams != null) { - settings.maxConcurrentStreams(maxConcurrentStreams); - } - - Integer maxFrameSize = http2Settings.maxFrameSize(); - if (maxFrameSize != null) { - settings.maxFrameSize(maxFrameSize); - } - - settings.maxHeaderListSize(http2Settings.maxHeaderListSize()); - - Boolean pushEnabled = http2Settings.pushEnabled(); - if (pushEnabled != null) { - settings.pushEnabled(pushEnabled); - } - } - - return settings; - } - static void addStreamHandlers( Channel ch, ConnectionObserver obs, @@ -679,12 +649,11 @@ else if (metricsRecorder instanceof ContextAwareHttpClientMetricsRecorder) { } static void configureHttp2Pipeline(ChannelPipeline p, HttpResponseDecoderSpec decoder, - Http2Settings http2Settings, Duration pingAckTimeout, Duration pingScheduleInterval, - Integer pingAckDropThreshold, ConnectionObserver observer) { + Duration idleTimeout, @Nullable Http2SettingsSpec http2SettingsSpec, ConnectionObserver observer) { Http2FrameCodecBuilder http2FrameCodecBuilder = Http2FrameCodecBuilder.forClient() .validateHeaders(decoder.validateHeaders()) - .initialSettings(http2Settings); + .initialSettings(http2SettingsSpec == null ? Http2Settings.defaultSettings() : http2SettingsSpec.http2Settings()); if (p.get(NettyPipeline.LoggingHandler) != null) { http2FrameCodecBuilder.frameLogger(new Http2FrameLogger(LogLevel.DEBUG, @@ -695,13 +664,22 @@ static void configureHttp2Pipeline(ChannelPipeline p, HttpResponseDecoderSpec de p.addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2Flush, new FlushConsolidationHandler(1024, true)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpCodec, codec) - .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2LivenessHandler, - new Http2ConnectionLivenessHandler(codec.encoder(), pingAckTimeout, pingScheduleInterval, pingAckDropThreshold)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2MultiplexHandler, new Http2MultiplexHandler(H2InboundStreamHandler.INSTANCE)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpTrafficHandler, new HttpTrafficHandler(observer)); + + IdleTimeoutHandler.addIdleTimeoutClientHandler( + p, + idleTimeout, + new Http2ConnectionLiveness( + codec, + http2SettingsSpec != null ? http2SettingsSpec.pingAckTimeout() : null, + http2SettingsSpec != null ? http2SettingsSpec.pingScheduleInterval() : null, + http2SettingsSpec != null ? http2SettingsSpec.pingAckDropThreshold() : null + ) + ); } - static void configureHttp3Pipeline(ChannelPipeline p, boolean removeMetricsRecorder, boolean removeProxyProvider) { + static void configureHttp3Pipeline(ChannelPipeline p, Duration idleTimeout, boolean removeMetricsRecorder, boolean removeProxyProvider) { p.remove(NettyPipeline.ReactiveBridge); p.addLast(NettyPipeline.HttpCodec, newHttp3ClientConnectionHandler()); @@ -714,6 +692,12 @@ static void configureHttp3Pipeline(ChannelPipeline p, boolean removeMetricsRecor if (removeProxyProvider) { p.remove(NettyPipeline.ProxyHandler); } + + IdleTimeoutHandler.addIdleTimeoutClientHandler( + p, + idleTimeout, + new HttpConnectionImmediateClose() + ); } @SuppressWarnings("deprecation") @@ -721,10 +705,8 @@ static void configureHttp11OrH2CleartextPipeline( ChannelPipeline p, boolean acceptGzip, HttpResponseDecoderSpec decoder, - Http2Settings http2Settings, - Duration pingAckTimeout, - Duration pingScheduleInterval, - Integer pingAckDropThreshold, + Duration idleTimeout, + @Nullable Http2SettingsSpec http2SettingsSpec, @Nullable ChannelMetricsRecorder metricsRecorder, ConnectionObserver observer, ChannelOperations.OnSetup opsFactory, @@ -745,7 +727,7 @@ static void configureHttp11OrH2CleartextPipeline( Http2FrameCodecBuilder http2FrameCodecBuilder = Http2FrameCodecBuilder.forClient() .validateHeaders(decoder.validateHeaders()) - .initialSettings(http2Settings); + .initialSettings(http2SettingsSpec == null ? Http2Settings.defaultSettings() : http2SettingsSpec.http2Settings()); if (p.get(NettyPipeline.LoggingHandler) != null) { http2FrameCodecBuilder.frameLogger(new Http2FrameLogger(LogLevel.DEBUG, @@ -755,14 +737,12 @@ static void configureHttp11OrH2CleartextPipeline( Http2FrameCodec http2FrameCodec = http2FrameCodecBuilder.build(); Http2ClientUpgradeCodec upgradeCodec = new Http2ClientUpgradeCodec(http2FrameCodec, - new H2CleartextCodec(http2FrameCodec, opsFactory, acceptGzip, metricsRecorder, proxyAddress, remoteAddress, uriTagValue)); + new H2CleartextCodec(http2FrameCodec, opsFactory, acceptGzip, metricsRecorder, proxyAddress, remoteAddress, uriTagValue, idleTimeout, http2SettingsSpec)); HttpClientUpgradeHandler upgradeHandler = new ReactorNettyHttpClientUpgradeHandler(httpClientCodec, upgradeCodec, decoder.h2cMaxContentLength()); p.addBefore(NettyPipeline.ReactiveBridge, null, httpClientCodec) - .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2LivenessHandler, - new Http2ConnectionLivenessHandler(http2FrameCodec.encoder(), pingAckTimeout, pingScheduleInterval, pingAckDropThreshold)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2CUpgradeHandler, upgradeHandler) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpTrafficHandler, new HttpTrafficHandler(observer)); @@ -786,12 +766,18 @@ else if (metricsRecorder instanceof ContextAwareHttpClientMetricsRecorder) { } } + IdleTimeoutHandler.addIdleTimeoutClientHandler( + p, + idleTimeout, + new HttpConnectionImmediateClose() + ); } @SuppressWarnings("deprecation") static void configureHttp11Pipeline(ChannelPipeline p, boolean acceptGzip, HttpResponseDecoderSpec decoder, + Duration idleTimeout, @Nullable ChannelMetricsRecorder metricsRecorder, @Nullable SocketAddress proxyAddress, SocketAddress remoteAddress, @@ -826,7 +812,14 @@ else if (metricsRecorder instanceof ContextAwareHttpClientMetricsRecorder) { } p.addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpMetricsHandler, handler); } + } + + IdleTimeoutHandler.addIdleTimeoutClientHandler( + p, + idleTimeout, + new HttpConnectionImmediateClose() + ); } static final Pattern FOLLOW_REDIRECT_CODES = Pattern.compile("30[12378]"); @@ -875,6 +868,8 @@ static final class H2CleartextCodec extends ChannelHandlerAdapter { final SocketAddress proxyAddress; final SocketAddress remoteAddress; final Function uriTagValue; + final Duration idleTimeout; + final Http2SettingsSpec http2SettingsSpec; H2CleartextCodec( Http2FrameCodec http2FrameCodec, @@ -883,7 +878,10 @@ static final class H2CleartextCodec extends ChannelHandlerAdapter { @Nullable ChannelMetricsRecorder metricsRecorder, @Nullable SocketAddress proxyAddress, SocketAddress remoteAddress, - @Nullable Function uriTagValue) { + @Nullable Function uriTagValue, + @Nullable Duration idleTimeout, + @Nullable Http2SettingsSpec http2SettingsSpec + ) { this.acceptGzip = acceptGzip; this.http2FrameCodec = http2FrameCodec; this.metricsRecorder = metricsRecorder; @@ -891,6 +889,8 @@ static final class H2CleartextCodec extends ChannelHandlerAdapter { this.proxyAddress = proxyAddress; this.remoteAddress = remoteAddress; this.uriTagValue = uriTagValue; + this.idleTimeout = idleTimeout; + this.http2SettingsSpec = http2SettingsSpec; } @Override @@ -923,6 +923,20 @@ public void handlerAdded(ChannelHandlerContext ctx) { } pipeline.remove(NettyPipeline.ReactiveBridge); pipeline.remove(this); + + if(idleTimeout != null) { + IdleTimeoutHandler.removeIdleTimeoutHandler(pipeline); + IdleTimeoutHandler.addIdleTimeoutClientHandler( + pipeline, + idleTimeout, + new Http2ConnectionLiveness( + http2FrameCodec, + http2SettingsSpec != null ? http2SettingsSpec.pingAckTimeout() : null, + http2SettingsSpec != null ? http2SettingsSpec.pingScheduleInterval() : null, + http2SettingsSpec != null ? http2SettingsSpec.pingAckDropThreshold() : null + ) + ); + } } } @@ -1006,10 +1020,8 @@ public boolean isSharable() { static final class H2OrHttp11Codec extends ChannelInboundHandlerAdapter { final boolean acceptGzip; final HttpResponseDecoderSpec decoder; - final Http2Settings http2Settings; - final Duration pingAckTimeout; - final Duration pingScheduleInterval; - final Integer pingAckDropThreshold; + final Duration idleTimeout; + final Http2SettingsSpec http2SettingsSpec; final ChannelMetricsRecorder metricsRecorder; final ConnectionObserver observer; final SocketAddress proxyAddress; @@ -1019,10 +1031,8 @@ static final class H2OrHttp11Codec extends ChannelInboundHandlerAdapter { H2OrHttp11Codec(HttpClientChannelInitializer initializer, ConnectionObserver observer, SocketAddress remoteAddress) { this.acceptGzip = initializer.acceptGzip; this.decoder = initializer.decoder; - this.http2Settings = initializer.http2Settings; - this.pingAckTimeout = initializer.pingAckTimeout; - this.pingScheduleInterval = initializer.pingScheduleInterval; - this.pingAckDropThreshold = initializer.pingAckDropThreshold; + this.idleTimeout = initializer.idleTimeout; + this.http2SettingsSpec = initializer.http2SettingsSpec; this.metricsRecorder = initializer.metricsRecorder; this.observer = observer; this.proxyAddress = initializer.proxyAddress; @@ -1041,10 +1051,10 @@ public void channelActive(ChannelHandlerContext ctx) { log.debug(format(ctx.channel(), "Negotiated application-level protocol [" + protocol + "]")); } if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { - configureHttp2Pipeline(ctx.channel().pipeline(), decoder, http2Settings, pingAckTimeout, pingScheduleInterval, pingAckDropThreshold, observer); + configureHttp2Pipeline(ctx.channel().pipeline(), decoder, idleTimeout, http2SettingsSpec, observer); } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { - configureHttp11Pipeline(ctx.channel().pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); + configureHttp11Pipeline(ctx.channel().pipeline(), acceptGzip, decoder, idleTimeout, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); } else { throw new IllegalStateException("unknown protocol: " + protocol); @@ -1064,10 +1074,8 @@ static final class HttpClientChannelInitializer implements ChannelPipelineConfig final boolean acceptGzip; final HttpResponseDecoderSpec decoder; - final Http2Settings http2Settings; - final Duration pingAckTimeout; - final Duration pingScheduleInterval; - final Integer pingAckDropThreshold; + final Duration idleTimeout; + final Http2SettingsSpec http2SettingsSpec; final ChannelMetricsRecorder metricsRecorder; final ChannelOperations.OnSetup opsFactory; final int protocols; @@ -1078,10 +1086,8 @@ static final class HttpClientChannelInitializer implements ChannelPipelineConfig HttpClientChannelInitializer(HttpClientConfig config) { this.acceptGzip = config.acceptGzip; this.decoder = config.decoder; - this.http2Settings = config.http2Settings(); - this.pingAckTimeout = config.http2Settings != null ? config.http2Settings.pingAckTimeout() : null; - this.pingScheduleInterval = config.http2Settings != null ? config.http2Settings.pingScheduleInterval() : null; - this.pingAckDropThreshold = config.http2Settings != null ? config.http2Settings.pingAckDropThreshold() : null; + this.idleTimeout = config.idleTimeout; + this.http2SettingsSpec = config.http2SettingsSpec(); this.metricsRecorder = config.metricsRecorderInternal(); this.opsFactory = config.channelOperationsProvider(); this.protocols = config._protocols; @@ -1103,24 +1109,24 @@ public void onChannelInit(ConnectionObserver observer, Channel channel, @Nullabl new H2OrHttp11Codec(this, observer, remoteAddress)); } else if ((protocols & h11) == h11) { - configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); + configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, idleTimeout, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h2) == h2) { - configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, pingAckTimeout, pingScheduleInterval, pingAckDropThreshold, observer); + configureHttp2Pipeline(channel.pipeline(), decoder, idleTimeout, http2SettingsSpec, observer); } else if ((protocols & h3) == h3) { - configureHttp3Pipeline(channel.pipeline(), metricsRecorder != null, proxyAddress != null); + configureHttp3Pipeline(channel.pipeline(), idleTimeout, metricsRecorder != null, proxyAddress != null); } } else { if ((protocols & h11orH2C) == h11orH2C) { - configureHttp11OrH2CleartextPipeline(channel.pipeline(), acceptGzip, decoder, http2Settings, pingAckTimeout, pingScheduleInterval, pingAckDropThreshold, metricsRecorder, observer, opsFactory, proxyAddress, remoteAddress, uriTagValue); + configureHttp11OrH2CleartextPipeline(channel.pipeline(), acceptGzip, decoder, idleTimeout, http2SettingsSpec, metricsRecorder, observer, opsFactory, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h11) == h11) { - configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); + configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, idleTimeout, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h2c) == h2c) { - configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, pingAckTimeout, pingScheduleInterval, pingAckDropThreshold, observer); + configureHttp2Pipeline(channel.pipeline(), decoder, idleTimeout, http2SettingsSpec, observer); } } } diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java index 0ee34d6350..7e98ac61f7 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java @@ -65,10 +65,13 @@ import reactor.netty.channel.AbstractChannelMetricsHandler; import reactor.netty.channel.ChannelMetricsRecorder; import reactor.netty.channel.ChannelOperations; +import reactor.netty.http.HttpConnectionImmediateClose; +import reactor.netty.http.Http2ConnectionLiveness; import reactor.netty.http.Http2SettingsSpec; import reactor.netty.http.Http3SettingsSpec; import reactor.netty.http.HttpProtocol; import reactor.netty.http.HttpResources; +import reactor.netty.http.IdleTimeoutHandler; import reactor.netty.http.logging.HttpMessageLogFactory; import reactor.netty.http.logging.ReactorNettyHttpMessageLogFactory; import reactor.netty.http.server.compression.HttpCompressionOptionsSpec; @@ -641,7 +644,8 @@ static void configureHttp3Pipeline( @Nullable Duration readTimeout, @Nullable Duration requestTimeout, @Nullable Function uriTagValue, - boolean validate) { + boolean validate, + @Nullable Duration idleTimeout) { p.remove(NettyPipeline.ReactiveBridge); p.addLast(NettyPipeline.HttpCodec, newHttp3ServerConnectionHandler(accessLogEnabled, accessLog, errorLogEnabled, errorLog, @@ -653,6 +657,12 @@ static void configureHttp3Pipeline( // Connection metrics are not applicable p.remove(NettyPipeline.ChannelMetricsHandler); } + + IdleTimeoutHandler.addIdleTimeoutServerHandler( + p, + idleTimeout, + new HttpConnectionImmediateClose() + ); } static void configureH2Pipeline(ChannelPipeline p, @@ -714,7 +724,16 @@ static void configureH2Pipeline(ChannelPipeline p, cookieDecoder, cookieEncoder, formDecoderProvider, forwardedHeaderHandler, httpMessageLogFactory, listener, mapHandle, methodTagValue, metricsRecorder, minCompressionSize, opsFactory, readTimeout, requestTimeout, uriTagValue))); - IdleTimeoutHandler.addIdleTimeoutHandler(p, idleTimeout); + IdleTimeoutHandler.addIdleTimeoutServerHandler( + p, + idleTimeout, + new Http2ConnectionLiveness( + http2FrameCodec, + http2SettingsSpec != null ? http2SettingsSpec.pingAckTimeout() : null, + http2SettingsSpec != null ? http2SettingsSpec.pingScheduleInterval() : null, + http2SettingsSpec != null ? http2SettingsSpec.pingAckDropThreshold() : null + ) + ); if (metricsRecorder != null) { if (metricsRecorder instanceof MicrometerHttpServerMetricsRecorder) { @@ -766,10 +785,10 @@ static void configureHttp11OrH2CleartextPipeline(ChannelPipeline p, Http11OrH2CleartextCodec upgrader = new Http11OrH2CleartextCodec(accessLogEnabled, accessLog, errorLogEnabled, errorLog, compressionOptions, compressPredicate, cookieDecoder, cookieEncoder, p.get(NettyPipeline.LoggingHandler) != null, - enableGracefulShutdown, formDecoderProvider, forwardedHeaderHandler, http2SettingsSpec, httpMessageLogFactory, listener, mapHandle, - methodTagValue, metricsRecorder, minCompressionSize, opsFactory, readTimeout, requestTimeout, uriTagValue, decoder.validateHeaders()); + enableGracefulShutdown, formDecoderProvider, forwardedHeaderHandler, idleTimeout, http2SettingsSpec, httpMessageLogFactory, listener, + mapHandle, methodTagValue, metricsRecorder, minCompressionSize, opsFactory, readTimeout, requestTimeout, uriTagValue, decoder.validateHeaders()); - ChannelHandler http2ServerHandler = new H2CleartextCodec(upgrader, http2SettingsSpec != null ? http2SettingsSpec.maxStreams() : null); + ChannelHandler http2ServerHandler = new H2CleartextCodec(upgrader, http2SettingsSpec != null ? http2SettingsSpec.maxStreams() : null, idleTimeout, http2SettingsSpec); HttpServerUpgradeHandler httpServerUpgradeHandler = readTimeout == null && requestTimeout == null ? new HttpServerUpgradeHandler(httpServerCodec, upgrader, decoder.h2cMaxContentLength()) : @@ -783,8 +802,8 @@ static void configureHttp11OrH2CleartextPipeline(ChannelPipeline p, .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpTrafficHandler, new HttpTrafficHandler(compressPredicate, compressionOptions, cookieDecoder, cookieEncoder, formDecoderProvider, - forwardedHeaderHandler, httpMessageLogFactory, idleTimeout, listener, mapHandle, maxKeepAliveRequests, - readTimeout, requestTimeout, decoder.validateHeaders())); + forwardedHeaderHandler, httpMessageLogFactory, idleTimeout, http2SettingsSpec, listener, mapHandle, + maxKeepAliveRequests, readTimeout, requestTimeout, decoder.validateHeaders())); if (accessLogEnabled) { p.addAfter(NettyPipeline.HttpTrafficHandler, NettyPipeline.AccessLogHandler, AccessLogHandlerFactory.H1.create(accessLog)); @@ -820,6 +839,12 @@ else if (metricsRecorder instanceof ContextAwareHttpServerMetricsRecorder) { if (errorLogEnabled) { p.addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.ErrorLogHandler, new DefaultErrorLogHandler(errorLog)); } + + IdleTimeoutHandler.addIdleTimeoutServerHandler( + p, + idleTimeout, + new HttpConnectionImmediateClose() + ); } @SuppressWarnings("deprecation") @@ -861,7 +886,7 @@ static void configureHttp11Pipeline(ChannelPipeline p, .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpTrafficHandler, new HttpTrafficHandler(compressPredicate, compressionOptions, cookieDecoder, cookieEncoder, formDecoderProvider, - forwardedHeaderHandler, httpMessageLogFactory, idleTimeout, listener, mapHandle, maxKeepAliveRequests, + forwardedHeaderHandler, httpMessageLogFactory, idleTimeout, null, listener, mapHandle, maxKeepAliveRequests, readTimeout, requestTimeout, decoder.validateHeaders())); if (accessLogEnabled) { @@ -901,6 +926,12 @@ else if (metricsRecorder instanceof ContextAwareHttpServerMetricsRecorder) { if (errorLogEnabled) { p.addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.ErrorLogHandler, new DefaultErrorLogHandler(errorLog)); } + + IdleTimeoutHandler.addIdleTimeoutServerHandler( + p, + idleTimeout, + new HttpConnectionImmediateClose() + ); } static final boolean ACCESS_LOG = Boolean.parseBoolean(System.getProperty(ACCESS_LOG_ENABLED, "false")); @@ -990,12 +1021,14 @@ static final class H2CleartextCodec extends ChannelHandlerAdapter { final boolean addHttp2FrameCodec; final boolean removeMetricsHandler; final Long maxStreams; + final Duration idleTimeout; + final Http2SettingsSpec http2SettingsSpec; /** * Used when full H2 preface is received. */ - H2CleartextCodec(Http11OrH2CleartextCodec upgrader, @Nullable Long maxStreams) { - this(upgrader, true, true, maxStreams); + H2CleartextCodec(Http11OrH2CleartextCodec upgrader, @Nullable Long maxStreams, @Nullable Duration idleTimeout, @Nullable Http2SettingsSpec http2SettingsSpec) { + this(upgrader, true, true, maxStreams, idleTimeout, http2SettingsSpec); } /** @@ -1003,11 +1036,13 @@ static final class H2CleartextCodec extends ChannelHandlerAdapter { * is added by {@link Http2ServerUpgradeCodec} */ H2CleartextCodec(Http11OrH2CleartextCodec upgrader, boolean addHttp2FrameCodec, boolean removeMetricsHandler, - @Nullable Long maxStreams) { + @Nullable Long maxStreams, @Nullable Duration idleTimeout, @Nullable Http2SettingsSpec http2SettingsSpec) { this.upgrader = upgrader; this.addHttp2FrameCodec = addHttp2FrameCodec; this.removeMetricsHandler = removeMetricsHandler; this.maxStreams = maxStreams; + this.idleTimeout = idleTimeout; + this.http2SettingsSpec = http2SettingsSpec; } @Override @@ -1044,6 +1079,20 @@ public void handlerAdded(ChannelHandlerContext ctx) { } pipeline.remove(NettyPipeline.HttpTrafficHandler); pipeline.remove(NettyPipeline.ReactiveBridge); + + if(idleTimeout != null) { + IdleTimeoutHandler.removeIdleTimeoutHandler(pipeline); + IdleTimeoutHandler.addIdleTimeoutServerHandler( + pipeline, + idleTimeout, + new Http2ConnectionLiveness( + upgrader.http2FrameCodec, + http2SettingsSpec != null ? http2SettingsSpec.pingAckTimeout() : null, + http2SettingsSpec != null ? http2SettingsSpec.pingScheduleInterval() : null, + http2SettingsSpec != null ? http2SettingsSpec.pingAckDropThreshold() : null + ) + ); + } } } @@ -1153,6 +1202,8 @@ static final class Http11OrH2CleartextCodec extends ChannelInitializer final Duration readTimeout; final Duration requestTimeout; final Function uriTagValue; + final Duration idleTimeout; + final Http2SettingsSpec http2SettingsSpec; Http11OrH2CleartextCodec( boolean accessLogEnabled, @@ -1167,6 +1218,7 @@ static final class Http11OrH2CleartextCodec extends ChannelInitializer boolean enableGracefulShutdown, HttpServerFormDecoderProvider formDecoderProvider, @Nullable BiFunction forwardedHeaderHandler, + @Nullable Duration idleTimeout, @Nullable Http2SettingsSpec http2SettingsSpec, HttpMessageLogFactory httpMessageLogFactory, ConnectionObserver listener, @@ -1222,6 +1274,8 @@ static final class Http11OrH2CleartextCodec extends ChannelInitializer this.readTimeout = readTimeout; this.requestTimeout = requestTimeout; this.uriTagValue = uriTagValue; + this.idleTimeout = idleTimeout; + this.http2SettingsSpec = http2SettingsSpec; } /** @@ -1239,7 +1293,7 @@ protected void initChannel(Channel ch) { @Nullable public HttpServerUpgradeHandler.UpgradeCodec newUpgradeCodec(CharSequence protocol) { if (AsciiString.contentEquals(Http2CodecUtil.HTTP_UPGRADE_PROTOCOL_NAME, protocol)) { - return new Http2ServerUpgradeCodec(http2FrameCodec, new H2CleartextCodec(this, false, false, maxStreams)); + return new Http2ServerUpgradeCodec(http2FrameCodec, new H2CleartextCodec(this, false, false, maxStreams, idleTimeout, http2SettingsSpec)); } else { return null; @@ -1355,11 +1409,6 @@ protected void configurePipeline(ChannelHandlerContext ctx, String protocol) { cookieDecoder, cookieEncoder, true, decoder, formDecoderProvider, forwardedHeaderHandler, httpMessageLogFactory, idleTimeout, listener, mapHandle, maxKeepAliveRequests, methodTagValue, metricsRecorder, minCompressionSize, readTimeout, requestTimeout, uriTagValue); - - // When the server is configured with HTTP/1.1 and H2 and HTTP/1.1 is negotiated, - // when channelActive event happens, this HttpTrafficHandler is still not in the pipeline, - // and will not be able to add IdleTimeoutHandler. So in this use case add IdleTimeoutHandler here. - IdleTimeoutHandler.addIdleTimeoutHandler(ctx.pipeline(), idleTimeout); return; } @@ -1541,7 +1590,8 @@ else if ((protocols & h3) == h3) { readTimeout, requestTimeout, uriTagValue, - decoder.validateHeaders()); + decoder.validateHeaders(), + idleTimeout); } } else { diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java index 7079ac2ea2..93f2d8543d 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java @@ -51,6 +51,7 @@ import reactor.netty.ConnectionObserver; import reactor.netty.ReactorNetty; import reactor.netty.channel.ChannelOperations; +import reactor.netty.http.Http2SettingsSpec; import reactor.netty.http.logging.HttpMessageArgProviderFactory; import reactor.netty.http.logging.HttpMessageLogFactory; import reactor.netty.http.server.compression.HttpCompressionOptionsSpec; @@ -85,6 +86,7 @@ final class HttpTrafficHandler extends ChannelDuplexHandler implements Runnable final BiFunction forwardedHeaderHandler; final HttpMessageLogFactory httpMessageLogFactory; final Duration idleTimeout; + final Http2SettingsSpec http2SettingsSpec; final ConnectionObserver listener; final BiFunction, ? super Connection, ? extends Mono> mapHandle; @@ -121,6 +123,7 @@ final class HttpTrafficHandler extends ChannelDuplexHandler implements Runnable @Nullable BiFunction forwardedHeaderHandler, HttpMessageLogFactory httpMessageLogFactory, @Nullable Duration idleTimeout, + @Nullable Http2SettingsSpec http2SettingsSpec, ConnectionObserver listener, @Nullable BiFunction, ? super Connection, ? extends Mono> mapHandle, int maxKeepAliveRequests, @@ -136,6 +139,7 @@ final class HttpTrafficHandler extends ChannelDuplexHandler implements Runnable this.cookieDecoder = decoder; this.httpMessageLogFactory = httpMessageLogFactory; this.idleTimeout = idleTimeout; + this.http2SettingsSpec = http2SettingsSpec; this.mapHandle = mapHandle; this.maxKeepAliveRequests = maxKeepAliveRequests; this.readTimeout = readTimeout; @@ -153,13 +157,6 @@ public void handlerAdded(ChannelHandlerContext ctx) throws Exception { ctx.read(); } - @Override - public void channelActive(ChannelHandlerContext ctx) { - IdleTimeoutHandler.addIdleTimeoutHandler(ctx.pipeline(), idleTimeout); - - ctx.fireChannelActive(); - } - @Override public void channelRead(ChannelHandlerContext ctx, Object msg) { read = true; @@ -175,10 +172,6 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { if (msg instanceof HttpRequest) { finalizingResponse = false; - if (idleTimeout != null) { - IdleTimeoutHandler.removeIdleTimeoutHandler(ctx.pipeline()); - } - final HttpRequest request = (HttpRequest) msg; if (H2.equals(request.protocolVersion())) { @@ -546,7 +539,6 @@ void handleLastHttpContent(Object msg, ChannelPromise promise) { ctx.executor().execute(this); } else { - IdleTimeoutHandler.addIdleTimeoutHandler(ctx.pipeline(), idleTimeout); ctx.read(); } } diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/server/IdleTimeoutHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/server/IdleTimeoutHandler.java deleted file mode 100644 index d31f6310b8..0000000000 --- a/reactor-netty-http/src/main/java/reactor/netty/http/server/IdleTimeoutHandler.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Copyright (c) 2022-2024 VMware, Inc. or its affiliates, All Rights Reserved. - * - * 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 reactor.netty.http.server; - -import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelPipeline; -import io.netty.handler.codec.http.HttpServerCodec; -import io.netty.handler.codec.http.HttpServerUpgradeHandler; -import io.netty.handler.timeout.IdleState; -import io.netty.handler.timeout.IdleStateEvent; -import io.netty.handler.timeout.IdleStateHandler; -import reactor.netty.NettyPipeline; -import reactor.util.annotation.Nullable; - -import java.time.Duration; -import java.util.concurrent.TimeUnit; - -import static reactor.netty.ReactorNetty.format; - -final class IdleTimeoutHandler extends IdleStateHandler { - - IdleTimeoutHandler(long idleTimeout) { - super(idleTimeout, 0, 0, TimeUnit.MILLISECONDS); - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - protected void channelIdle(ChannelHandlerContext ctx, IdleStateEvent evt) { - if (evt.state() == IdleState.READER_IDLE) { - if (HttpServerOperations.log.isDebugEnabled()) { - HttpServerOperations.log.debug(format(ctx.channel(), - "Connection was idle for [{}ms], as per configuration the connection will be closed."), - getReaderIdleTimeInMillis()); - } - // FutureReturnValueIgnored is deliberate - ctx.close(); - } - ctx.fireUserEventTriggered(evt); - } - - static void addIdleTimeoutHandler(ChannelPipeline pipeline, @Nullable Duration idleTimeout) { - if (idleTimeout != null && pipeline.get(NettyPipeline.IdleTimeoutHandler) == null) { - String baseName = null; - if (pipeline.get(NettyPipeline.HttpCodec) != null) { - baseName = NettyPipeline.HttpCodec; - } - else { - ChannelHandler httpServerUpgradeHandler = pipeline.get(HttpServerUpgradeHandler.class); - if (httpServerUpgradeHandler != null) { - baseName = pipeline.context(httpServerUpgradeHandler).name(); - } - else { - ChannelHandler httpServerCodec = pipeline.get(HttpServerCodec.class); - if (httpServerCodec != null) { - baseName = pipeline.context(httpServerCodec).name(); - } - } - } - - pipeline.addAfter(baseName, - NettyPipeline.IdleTimeoutHandler, - new IdleTimeoutHandler(idleTimeout.toMillis())); - } - } - - static void removeIdleTimeoutHandler(ChannelPipeline pipeline) { - if (pipeline.get(NettyPipeline.IdleTimeoutHandler) != null) { - pipeline.remove(NettyPipeline.IdleTimeoutHandler); - } - } -} diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java deleted file mode 100644 index 95587ba5c5..0000000000 --- a/reactor-netty-http/src/test/java/reactor/netty/http/client/Http2ConnectionLivenessHandlerTest.java +++ /dev/null @@ -1,412 +0,0 @@ -/* - * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. - * - * 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 reactor.netty.http.client; - -import io.netty.channel.Channel; -import io.netty.channel.ChannelFutureListener; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; -import io.netty.handler.codec.http2.DefaultHttp2PingFrame; -import io.netty.handler.codec.http2.Http2FrameCodec; -import io.netty.handler.codec.http2.Http2FrameCodecBuilder; -import io.netty.handler.codec.http2.Http2PingFrame; -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslContextBuilder; -import io.netty.handler.ssl.util.InsecureTrustManagerFactory; -import io.netty.handler.ssl.util.SelfSignedCertificate; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; -import reactor.core.publisher.Mono; -import reactor.netty.BaseHttpTest; -import reactor.netty.NettyPipeline; -import reactor.netty.resources.ConnectionProvider; - -import javax.net.ssl.SSLException; -import java.security.cert.CertificateException; -import java.time.Duration; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Collectors; - -import static org.assertj.core.api.Assertions.assertThat; -import static reactor.netty.http.HttpProtocol.H2; - -/** - * This test class verifies {@link Http2ConnectionLivenessHandler}. - * - * @author raccoonback - * @since 1.2.3 - */ -class Http2ConnectionLivenessHandlerTest extends BaseHttpTest { - - static SslContext sslServer; - static SslContext sslClient; - - @BeforeAll - static void createSelfSignedCertificate() throws CertificateException, SSLException { - SelfSignedCertificate ssc = new SelfSignedCertificate(); - sslServer = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey()) - .build(); - sslClient = SslContextBuilder.forClient() - .trustManager(InsecureTrustManagerFactory.INSTANCE) - .build(); - } - - @Test - void successReceiveResponse() { - disposableServer = createServer() - .protocol(H2) - .secure(spec -> spec.sslContext(sslServer)) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - String result = createClient(disposableServer::address) - .protocol(H2) - .secure(spec -> spec.sslContext(sslClient)) - .get() - .uri("/") - .responseSingle((resp, bytes) -> bytes.asString()) - .block(); - - assertThat(result).isEqualTo("Test"); - } - - @Test - void maintainConnectionWithoutPingCheckWhenNotConfigured() { - Http2PingFrameHandler handler = new Http2PingFrameHandler(); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .keepAlive(true) - .secure(spec -> spec.sslContext(sslClient)) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).isEmpty(); - } - - @ParameterizedTest - @CsvSource({"100,300,3", "300,100,3"}) - void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) - .doOnNext( - unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) - ) - .subscribe() - ); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .keepAlive(true) - .secure(spec -> spec.sslContext(sslClient)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(3)) - .block(); - - assertThat(channel.parent().isOpen()).isFalse(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); - } - - @ParameterizedTest - @CsvSource({"100,300,3", "300,100,3"}) - void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) - .doOnNext( - unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) - ) - .subscribe() - ); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - ConnectionProvider pool = ConnectionProvider.create("closeConnectionInPoolIfPingFrameDelayed", 1); - - Channel channel = createClient(pool, disposableServer::address) - .protocol(H2) - .keepAlive(true) - .secure(spec -> spec.sslContext(sslClient)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(3)) - .block(); - - assertThat(channel.parent().isOpen()).isFalse(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); - - pool.dispose(); - } - - @ParameterizedTest - @CsvSource({"300,600,0", "600,300,0"}) - void ackPingFrameWithinInterval(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler(); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .keepAlive(true) - .secure(spec -> spec.sslContext(sslClient)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); - } - - @ParameterizedTest - @CsvSource({"300,600,0", "600,300,0"}) - void connectionRetentionInPoolOnPingFrameAck(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler(); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - ConnectionProvider pool = ConnectionProvider.create("connectionRetentionInPoolOnPingFrameAck", 1); - - Channel channel = createClient(pool, disposableServer::address) - .protocol(H2) - .keepAlive(true) - .secure(spec -> spec.sslContext(sslClient)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); - - pool.dispose(); - } - - @ParameterizedTest - @CsvSource({"300,600,3", "600,300,3"}) - void ackPingFrameWithinThreshold(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame, receivedPingTimes) -> { - int delayTime = 0; - if (receivedPingTimes.size() % 3 != 0) { - delayTime = 600; - } - - Mono.delay(Duration.ofMillis(delayTime)) - .doOnNext( - unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) - ) - .subscribe(); - } - ); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .keepAlive(true) - .secure(spec -> spec.sslContext(sslClient)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); - } - - private static final class Http2PingFrameHandler extends SimpleChannelInboundHandler { - - private final List receivedPingTimes = new ArrayList<>(); - - private final TriConsumer> consumer; - - private Http2PingFrameHandler() { - this.consumer = (ctx, frame, receivedPings) -> - ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE); - } - - private Http2PingFrameHandler(TriConsumer> consumer) { - this.consumer = consumer; - } - - @Override - protected void channelRead0(ChannelHandlerContext ctx, Http2PingFrame frame) throws InterruptedException { - receivedPingTimes.add(LocalDateTime.now(ZoneId.systemDefault())); - consumer.accept(ctx, frame, receivedPingTimes); - } - - public List getReceivedPingTimes() { - return receivedPingTimes.stream() - .sorted() - .collect(Collectors.toList()); - } - } - - @FunctionalInterface - public interface TriConsumer { - void accept(T t, U u, V v); - } -} diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java new file mode 100644 index 0000000000..68251abc45 --- /dev/null +++ b/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java @@ -0,0 +1,792 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.http.client; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.codec.http2.DefaultHttp2PingFrame; +import io.netty.handler.codec.http2.Http2FrameCodec; +import io.netty.handler.codec.http2.Http2FrameCodecBuilder; +import io.netty.handler.codec.http2.Http2FrameWriter; +import io.netty.handler.codec.http2.Http2PingFrame; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslContextBuilder; +import io.netty.handler.ssl.util.InsecureTrustManagerFactory; +import io.netty.handler.ssl.util.SelfSignedCertificate; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import reactor.core.publisher.Mono; +import reactor.netty.BaseHttpTest; +import reactor.netty.NettyPipeline; +import reactor.netty.http.HttpConnectionLiveness; +import reactor.netty.resources.ConnectionProvider; +import reactor.util.Logger; +import reactor.util.Loggers; + +import javax.net.ssl.SSLException; +import java.security.cert.CertificateException; +import java.time.Duration; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static reactor.netty.http.HttpProtocol.H2; +import static reactor.netty.http.HttpProtocol.H2C; +import static reactor.netty.http.HttpProtocol.HTTP11; + +/** + * This test class verifies {@link HttpConnectionLiveness} with client side. + * + * @author raccoonback + * @since 1.2.5 + */ +class HttpConnectionLivenessTest extends BaseHttpTest { + + static final Logger log = Loggers.getLogger(HttpConnectionLivenessTest.class); + + static SslContext sslServer; + static SslContext sslClient; + + @BeforeAll + static void createSelfSignedCertificate() throws CertificateException, SSLException { + SelfSignedCertificate ssc = new SelfSignedCertificate(); + sslServer = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey()) + .build(); + sslClient = SslContextBuilder.forClient() + .trustManager(InsecureTrustManagerFactory.INSTANCE) + .build(); + } + + @Nested + class Http2Test { + + @Test + void successReceiveResponse() { + disposableServer = createServer() + .protocol(H2) + .secure(spec -> spec.sslContext(sslServer)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + String result = createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .get() + .uri("/") + .responseSingle((resp, bytes) -> bytes.asString()) + .block(); + + assertThat(result).isEqualTo("Test"); + } + + @Test + void maintainConnectionWithoutPingCheckWhenNotConfigured() { + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).isEmpty(); + } + + @ParameterizedTest + @CsvSource({ + "100,300,3", "300,100,3", + "100,300,3", "300,100,3", + }) + void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe() + ); + + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .idleTimeout(Duration.ofMillis(300)) + .secure(spec -> spec.sslContext(sslClient)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(3)) + .block(); + + assertThat(channel.parent().isOpen()).isFalse(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); + } + + @ParameterizedTest + @CsvSource({ + "100,300,3", "300,100,3", + "100,300,3", "300,100,3", + }) + void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe() + ); + + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + ConnectionProvider pool = ConnectionProvider.create("closeConnectionInPoolIfPingFrameDelayed", 1); + + Channel channel = createClient(pool, disposableServer::address) + .protocol(H2) + .idleTimeout(Duration.ofMillis(300)) + .secure(spec -> spec.sslContext(sslClient)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(4)) + .block(); + + assertThat(channel.parent().isOpen()).isFalse(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); + + pool.dispose(); + } + + @ParameterizedTest + @CsvSource({ + "300,600,0", "600,300,0", + "300,600,0", "600,300,0" + }) + void ackPingFrameWithinInterval(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .idleTimeout(Duration.ofMillis(300)) + .secure(spec -> spec.sslContext(sslClient)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + } + + @ParameterizedTest + @CsvSource({ + "300,600,0", "600,300,0", + "300,600,0", "600,300,0" + }) + void connectionRetentionInPoolOnPingFrameAck(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + ConnectionProvider pool = ConnectionProvider.create("connectionRetentionInPoolOnPingFrameAck", 1); + + Channel channel = createClient(pool, disposableServer::address) + .protocol(H2) + .idleTimeout(Duration.ofMillis(300)) + .secure(spec -> spec.sslContext(sslClient)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + + pool.dispose(); + } + + @ParameterizedTest + @CsvSource({ + "300,600,3", "600,300,3", + "300,600,3", "600,300,3" + }) + void ackPingFrameWithinThreshold(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> { + int delayTime = 0; + if (receivedPingTimes.size() % 3 != 0) { + delayTime = 600; + } + + Mono.delay(Duration.ofMillis(delayTime)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe(); + } + ); + + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .idleTimeout(Duration.ofMillis(300)) + .secure(spec -> spec.sslContext(sslClient)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + } + } + + @Nested + class H2cTest { + + @Test + void successReceiveResponse() { + disposableServer = createServer() + .protocol(H2C) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + String result = createClient(disposableServer::address) + .protocol(H2C) + .get() + .uri("/") + .responseSingle((resp, bytes) -> bytes.asString()) + .block(); + + assertThat(result).isEqualTo("Test"); + } + + @Test + void maintainConnectionWithoutPingCheckWhenNotConfigured() { + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2C) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).isEmpty(); + } + + @ParameterizedTest + @CsvSource({ + "100,300,3", "300,100,3", + "100,300,3", "300,100,3", + }) + void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe() + ); + + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2C) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(3)) + .block(); + + assertThat(channel.parent().isOpen()).isFalse(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); + } + + @ParameterizedTest + @CsvSource({ + "100,300,3", "300,100,3", + "100,300,3", "300,100,3", + }) + void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe() + ); + + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + ConnectionProvider pool = ConnectionProvider.create("closeConnectionInPoolIfPingFrameDelayed", 1); + + Channel channel = createClient(pool, disposableServer::address) + .protocol(H2C) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(4)) + .block(); + + assertThat(channel.parent().isOpen()).isFalse(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); + + pool.dispose(); + } + + @ParameterizedTest + @CsvSource({ + "300,600,0", "600,300,0", + "300,600,0", "600,300,0" + }) + void ackPingFrameWithinInterval(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2C) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + } + + @ParameterizedTest + @CsvSource({ + "300,600,0", "600,300,0", + "300,600,0", "600,300,0" + }) + void connectionRetentionInPoolOnPingFrameAck(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + ConnectionProvider pool = ConnectionProvider.create("connectionRetentionInPoolOnPingFrameAck", 1); + + Channel channel = createClient(pool, disposableServer::address) + .protocol(H2C) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + + pool.dispose(); + } + + @ParameterizedTest + @CsvSource({ + "300,600,3", "600,300,3", + "300,600,3", "600,300,3" + }) + void ackPingFrameWithinThreshold(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> { + int delayTime = 0; + if (receivedPingTimes.size() % 3 != 0) { + delayTime = 600; + } + + Mono.delay(Duration.ofMillis(delayTime)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe(); + } + ); + + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2C) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + } + } + + @Nested + class Http11Test { + + @Test + void closeWithoutDelay() { + disposableServer = createServer() + .protocol(HTTP11) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11) + .idleTimeout(Duration.ofMillis(100)) + .secure(spec -> spec.sslContext(sslClient)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(1)) + .block(); + + assertThat(channel.isOpen()).isFalse(); + } + } + + private static final class Http2PingFrameHandler extends SimpleChannelInboundHandler { + + private final List receivedPingTimes = new ArrayList<>(); + + private final TriConsumer> consumer; + + private Http2PingFrameHandler() { + this.consumer = (ctx, frame, receivedPings) -> { + Http2FrameCodec channelHandler = ctx.pipeline().get(Http2FrameCodec.class); + Http2FrameWriter http2FrameWriter = channelHandler.encoder() + .frameWriter(); + + http2FrameWriter.writePing(ctx, true, frame.content(), ctx.newPromise()) + .addListener((ChannelFuture future) -> { + if (future.isSuccess()) { + log.debug("[Http2PingFrameHandler] Wrote PING frame to {} channel.", future.channel()); + } + else { + log.debug("[Http2PingFrameHandler] Failed to wrote PING frame to {} channel.", future.channel()); + } + }); + }; + } + + private Http2PingFrameHandler(TriConsumer> consumer) { + this.consumer = consumer; + } + + @Override + protected void channelRead0(ChannelHandlerContext ctx, Http2PingFrame frame) throws InterruptedException { + receivedPingTimes.add(LocalDateTime.now(ZoneId.systemDefault())); + consumer.accept(ctx, frame, receivedPingTimes); + } + + public List getReceivedPingTimes() { + return receivedPingTimes.stream() + .sorted() + .collect(Collectors.toList()); + } + } + + @FunctionalInterface + public interface TriConsumer { + void accept(T t, U u, V v); + } +} diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpIdleTimeoutTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpIdleTimeoutTest.java new file mode 100644 index 0000000000..f16254a6db --- /dev/null +++ b/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpIdleTimeoutTest.java @@ -0,0 +1,252 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.http.client; + +import io.netty.channel.Channel; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslContextBuilder; +import io.netty.handler.ssl.util.InsecureTrustManagerFactory; +import io.netty.handler.ssl.util.SelfSignedCertificate; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import reactor.core.publisher.Mono; +import reactor.netty.BaseHttpTest; + +import javax.net.ssl.SSLException; +import java.security.cert.CertificateException; +import java.time.Duration; + +import static org.assertj.core.api.Assertions.assertThat; +import static reactor.netty.http.HttpProtocol.H2; +import static reactor.netty.http.HttpProtocol.H2C; +import static reactor.netty.http.HttpProtocol.HTTP11; + +class HttpIdleTimeoutTest extends BaseHttpTest { + + static SslContext sslServer; + static SslContext sslClient; + + @BeforeAll + static void createSelfSignedCertificate() throws CertificateException, SSLException { + SelfSignedCertificate ssc = new SelfSignedCertificate(); + sslServer = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey()) + .build(); + sslClient = SslContextBuilder.forClient() + .trustManager(InsecureTrustManagerFactory.INSTANCE) + .build(); + } + + @Test + void maintainConnectionWithoutIdleTimeoutInHttp11() { + disposableServer = createServer() + .protocol(HTTP11) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(2)) + .block(); + + assertThat(channel.isOpen()).isTrue(); + } + + @Test + void maintainConnectionWithoutIdleTimeoutInH2C() { + disposableServer = createServer() + .protocol(HTTP11, H2C) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11, H2C) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(2)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + } + + @Test + void maintainConnectionWithoutIdleTimeoutInHttp2() { + disposableServer = createServer() + .protocol(H2) + .secure(spec -> spec.sslContext(sslServer)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(2)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + } + + @Test + void idleTimeoutInHttp11() { + disposableServer = createServer() + .protocol(HTTP11) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11) + .idleTimeout(Duration.ofSeconds(3)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(2)) + .block(); + + assertThat(channel.isOpen()).isTrue(); + } + + @Test + void idleTimeoutInH2C() { + disposableServer = createServer() + .protocol(HTTP11, H2C) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11, H2C) + .idleTimeout(Duration.ofSeconds(3)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(2)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + } + + @Test + void idleTimeoutInHttp2() { + disposableServer = createServer() + .protocol(H2) + .secure(spec -> spec.sslContext(sslServer)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .idleTimeout(Duration.ofSeconds(3)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(2)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + } + + @Test + void closeAfterIdleTimeoutInHttp11() { + disposableServer = createServer() + .protocol(HTTP11) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11) + .idleTimeout(Duration.ofSeconds(2)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(3)) + .block(); + + assertThat(channel.isOpen()).isFalse(); + } + + @Test + void closeAfterIdleTimeoutInH2C() { + disposableServer = createServer() + .protocol(HTTP11, H2C) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11, H2C) + .idleTimeout(Duration.ofSeconds(2)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(3)) + .block(); + + assertThat(channel.parent().isOpen()).isFalse(); + } + + @Test + void closeAfterIdleTimeoutInHttp2() { + disposableServer = createServer() + .protocol(H2) + .secure(spec -> spec.sslContext(sslServer)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .idleTimeout(Duration.ofSeconds(2)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(3)) + .block(); + + assertThat(channel.parent().isOpen()).isFalse(); + } +} diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java new file mode 100644 index 0000000000..7e5437cb3e --- /dev/null +++ b/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java @@ -0,0 +1,816 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.http.server; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.codec.http2.DefaultHttp2PingFrame; +import io.netty.handler.codec.http2.Http2FrameCodec; +import io.netty.handler.codec.http2.Http2FrameCodecBuilder; +import io.netty.handler.codec.http2.Http2FrameWriter; +import io.netty.handler.codec.http2.Http2PingFrame; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslContextBuilder; +import io.netty.handler.ssl.util.InsecureTrustManagerFactory; +import io.netty.handler.ssl.util.SelfSignedCertificate; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import reactor.core.publisher.Mono; +import reactor.netty.BaseHttpTest; +import reactor.netty.NettyPipeline; +import reactor.netty.http.HttpConnectionLiveness; +import reactor.netty.resources.ConnectionProvider; +import reactor.util.Logger; +import reactor.util.Loggers; + +import javax.net.ssl.SSLException; +import java.security.cert.CertificateException; +import java.time.Duration; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static reactor.netty.http.HttpProtocol.H2; +import static reactor.netty.http.HttpProtocol.H2C; +import static reactor.netty.http.HttpProtocol.HTTP11; + +/** + * This test class verifies {@link HttpConnectionLiveness} with server side. + * + * @author raccoonback + * @since 1.2.5 + */ +class HttpConnectionLivenessTest extends BaseHttpTest { + + static final Logger log = Loggers.getLogger(HttpConnectionLivenessTest.class); + + static SslContext sslServer; + static SslContext sslClient; + + @BeforeAll + static void createSelfSignedCertificate() throws CertificateException, SSLException { + SelfSignedCertificate ssc = new SelfSignedCertificate(); + sslServer = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey()) + .build(); + sslClient = SslContextBuilder.forClient() + .trustManager(InsecureTrustManagerFactory.INSTANCE) + .build(); + } + + @Nested + class Http2Test { + + @Test + void successReceiveResponse() { + disposableServer = createServer() + .protocol(H2) + .secure(spec -> spec.sslContext(sslServer)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + String result = createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .get() + .uri("/") + .responseSingle((resp, bytes) -> bytes.asString()) + .block(); + + assertThat(result).isEqualTo("Test"); + } + + @Test + void maintainConnectionWithoutPingCheckWhenNotConfigured() { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).isEmpty(); + } + + @ParameterizedTest + @CsvSource({ + "100,300,3", "300,100,3", + "100,300,3", "300,100,3", + }) + void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe() + ); + createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(3)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isFalse(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); + } + + @ParameterizedTest + @CsvSource({ + "100,300,3", "300,100,3", + "100,300,3", "300,100,3", + }) + void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe() + ); + + ConnectionProvider pool = ConnectionProvider.create("closeConnectionInPoolIfPingFrameDelayed", 1); + createClient(pool, disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(4)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isFalse(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); + + pool.dispose(); + } + + @ParameterizedTest + @CsvSource({ + "300,600,0", "600,300,0", + "300,600,0", "600,300,0" + }) + void ackPingFrameWithinInterval(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + } + + @ParameterizedTest + @CsvSource({ + "300,600,0", "600,300,0", + "300,600,0", "600,300,0" + }) + void connectionRetentionInPoolOnPingFrameAck(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + ConnectionProvider pool = ConnectionProvider.create("connectionRetentionInPoolOnPingFrameAck", 1); + createClient(pool, disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + + pool.dispose(); + } + + @ParameterizedTest + @CsvSource({ + "300,600,3", "600,300,3", + "300,600,3", "600,300,3" + }) + void ackPingFrameWithinThreshold(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2) + .maxKeepAliveRequests(1) + .secure(spec -> spec.sslContext(sslServer)) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> { + int delayTime = 0; + if (receivedPingTimes.size() % 3 != 0) { + delayTime = 600; + } + + Mono.delay(Duration.ofMillis(delayTime)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe(); + } + ); + createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + } + } + + @Nested + class H2cTest { + + @Test + void successReceiveResponse() { + disposableServer = createServer() + .protocol(H2C) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + String result = createClient(disposableServer::address) + .protocol(H2C) + .get() + .uri("/") + .responseSingle((resp, bytes) -> bytes.asString()) + .block(); + + assertThat(result).isEqualTo("Test"); + } + + @Test + void maintainConnectionWithoutPingCheckWhenNotConfigured() { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + createClient(disposableServer::address) + .protocol(H2C) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).isEmpty(); + } + + @ParameterizedTest + @CsvSource({ + "100,300,3", "300,100,3", + "100,300,3", "300,100,3", + }) + void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe() + ); + createClient(disposableServer::address) + .protocol(H2C) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(3)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isFalse(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); + } + + @ParameterizedTest + @CsvSource({ + "100,300,3", "300,100,3", + "100,300,3", "300,100,3", + }) + void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe() + ); + ConnectionProvider pool = ConnectionProvider.create("closeConnectionInPoolIfPingFrameDelayed", 1); + createClient(pool, disposableServer::address) + .protocol(H2C) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .response() + .block(); + + Mono.delay(Duration.ofSeconds(4)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isFalse(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); + + pool.dispose(); + } + + @ParameterizedTest + @CsvSource({ + "300,600,0", "600,300,0", + "300,600,0", "600,300,0" + }) + void ackPingFrameWithinInterval(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + createClient(disposableServer::address) + .protocol(H2C) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + } + + @ParameterizedTest + @CsvSource({ + "300,600,0", "600,300,0", + "300,600,0", "600,300,0" + }) + void connectionRetentionInPoolOnPingFrameAck(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler(); + ConnectionProvider pool = ConnectionProvider.create("connectionRetentionInPoolOnPingFrameAck", 1); + createClient(pool, disposableServer::address) + .protocol(H2C) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + + pool.dispose(); + } + + @ParameterizedTest + @CsvSource({ + "300,600,3", "600,300,3", + "300,600,3", "600,300,3" + }) + void ackPingFrameWithinThreshold(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(H2C) + .maxKeepAliveRequests(1) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .idleTimeout(Duration.ofMillis(300)) + .http2Settings(builder -> { + builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) + .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) + .pingAckDropThreshold(pingAckDropThreshold); + }) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Http2PingFrameHandler handler = new Http2PingFrameHandler( + (ctx, frame, receivedPingTimes) -> { + int delayTime = 0; + if (receivedPingTimes.size() % 3 != 0) { + delayTime = 600; + } + + Mono.delay(Duration.ofMillis(delayTime)) + .doOnNext( + unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) + .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) + ) + .subscribe(); + } + ); + createClient(disposableServer::address) + .protocol(H2C) + .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { + Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forClient() + .autoAckPingFrame(false) + .autoAckSettingsFrame(true) + .build(); + + channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); + channel.pipeline().addLast(handler); + }) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(5)) + .block(); + + assertThat(connectedServerChannel.get().parent().isOpen()).isTrue(); + assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); + } + } + + @Nested + class Http11Test { + + @Test + void closeWithoutDelay() { + AtomicReference connectedServerChannel = new AtomicReference<>(); + disposableServer = createServer() + .protocol(HTTP11) + .maxKeepAliveRequests(1) + .doOnConnection(connection -> { + connectedServerChannel.set(connection.channel()); + }) + .secure(spec -> spec.sslContext(sslServer)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + createClient(disposableServer::address) + .protocol(HTTP11) + .idleTimeout(Duration.ofMillis(100)) + .secure(spec -> spec.sslContext(sslClient)) + .get() + .uri("/") + .response() + .block(); + + Mono.delay(Duration.ofSeconds(1)) + .block(); + + assertThat(connectedServerChannel.get().isOpen()).isFalse(); + } + } + + private static final class Http2PingFrameHandler extends SimpleChannelInboundHandler { + + private final List receivedPingTimes = new ArrayList<>(); + + private final TriConsumer> consumer; + + private Http2PingFrameHandler() { + this.consumer = (ctx, frame, receivedPings) -> { + Http2FrameCodec channelHandler = ctx.pipeline().get(Http2FrameCodec.class); + Http2FrameWriter http2FrameWriter = channelHandler.encoder() + .frameWriter(); + + http2FrameWriter.writePing(ctx, true, frame.content(), ctx.newPromise()) + .addListener((ChannelFuture future) -> { + if (future.isSuccess()) { + log.debug("[Http2PingFrameHandler] Wrote PING frame to {} channel.", future.channel()); + } + else { + log.debug("[Http2PingFrameHandler] Failed to wrote PING frame to {} channel.", future.channel()); + } + }); + }; + } + + private Http2PingFrameHandler(TriConsumer> consumer) { + this.consumer = consumer; + } + + @Override + protected void channelRead0(ChannelHandlerContext ctx, Http2PingFrame frame) throws InterruptedException { + receivedPingTimes.add(LocalDateTime.now(ZoneId.systemDefault())); + consumer.accept(ctx, frame, receivedPingTimes); + } + + public List getReceivedPingTimes() { + return receivedPingTimes.stream() + .sorted() + .collect(Collectors.toList()); + } + } + + @FunctionalInterface + public interface TriConsumer { + void accept(T t, U u, V v); + } +} diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpIdleTimeoutTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpIdleTimeoutTest.java new file mode 100644 index 0000000000..afa0cfe723 --- /dev/null +++ b/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpIdleTimeoutTest.java @@ -0,0 +1,187 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.http.server; + +import io.netty.channel.Channel; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslContextBuilder; +import io.netty.handler.ssl.util.InsecureTrustManagerFactory; +import io.netty.handler.ssl.util.SelfSignedCertificate; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import reactor.core.publisher.Mono; +import reactor.netty.BaseHttpTest; + +import javax.net.ssl.SSLException; +import java.security.cert.CertificateException; +import java.time.Duration; + +import static org.assertj.core.api.Assertions.assertThat; +import static reactor.netty.http.HttpProtocol.H2; +import static reactor.netty.http.HttpProtocol.H2C; +import static reactor.netty.http.HttpProtocol.HTTP11; + +class HttpIdleTimeoutTest extends BaseHttpTest { + + static SslContext sslServer; + static SslContext sslClient; + + @BeforeAll + static void createSelfSignedCertificate() throws CertificateException, SSLException { + SelfSignedCertificate ssc = new SelfSignedCertificate(); + sslServer = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey()) + .build(); + sslClient = SslContextBuilder.forClient() + .trustManager(InsecureTrustManagerFactory.INSTANCE) + .build(); + } + + @Test + void idleTimeoutInHttp11() { + disposableServer = createServer() + .protocol(HTTP11) + .idleTimeout(Duration.ofSeconds(3)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(2)) + .block(); + + assertThat(channel.isOpen()).isTrue(); + } + + @Test + void idleTimeoutInH2C() { + disposableServer = createServer() + .protocol(HTTP11, H2C) + .idleTimeout(Duration.ofSeconds(3)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11, H2C) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(2)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + } + + @Test + void idleTimeoutInHttp2() { + disposableServer = createServer() + .protocol(H2) + .idleTimeout(Duration.ofSeconds(3)) + .secure(spec -> spec.sslContext(sslServer)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(2)) + .block(); + + assertThat(channel.parent().isOpen()).isTrue(); + } + + @Test + void closeAfterIdleTimeoutInHttp11() { + disposableServer = createServer() + .protocol(HTTP11) + .idleTimeout(Duration.ofSeconds(2)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(3)) + .block(); + + assertThat(channel.isOpen()).isFalse(); + } + + @Test + void closeAfterIdleTimeoutInH2C() { + disposableServer = createServer() + .protocol(HTTP11, H2C) + .idleTimeout(Duration.ofSeconds(2)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(HTTP11, H2C) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(3)) + .block(); + + assertThat(channel.parent().isOpen()).isFalse(); + } + + @Test + void closeAfterIdleTimeoutInHttp2() { + disposableServer = createServer() + .protocol(H2) + .idleTimeout(Duration.ofSeconds(2)) + .secure(spec -> spec.sslContext(sslServer)) + .handle((req, resp) -> resp.sendString(Mono.just("Test"))) + .bindNow(); + + Channel channel = createClient(disposableServer::address) + .protocol(H2) + .secure(spec -> spec.sslContext(sslClient)) + .get() + .uri("/") + .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) + .single() + .block(); + + Mono.delay(Duration.ofSeconds(3)) + .block(); + + assertThat(channel.parent().isOpen()).isFalse(); + } +} From d6c6a97cb1e8f28db812637221e3ce1cfc9b60c8 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sat, 15 Mar 2025 07:23:08 +0900 Subject: [PATCH 13/27] Rollback previous Http2ConnectionLivenessHandler settings Signed-off-by: raccoonback --- .../src/main/java/reactor/netty/NettyPipeline.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java b/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java index 380c66835d..866d007a74 100644 --- a/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java +++ b/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2011-2025 VMware, Inc. or its affiliates, All Rights Reserved. + * Copyright (c) 2011-2024 VMware, Inc. or its affiliates, All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. From 84bb3fc840953704dd3d8e51646800afa4647e43 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sat, 15 Mar 2025 07:39:31 +0900 Subject: [PATCH 14/27] Fix checkstyle Signed-off-by: raccoonback --- .../src/main/java/reactor/netty/http/client/HttpClient.java | 1 - .../main/java/reactor/netty/http/client/HttpClientConfig.java | 2 +- .../main/java/reactor/netty/http/server/HttpServerConfig.java | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java index 23d05ebc1f..121030a12c 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java @@ -59,7 +59,6 @@ import reactor.netty.http.HttpProtocol; import reactor.netty.http.logging.HttpMessageLogFactory; import reactor.netty.http.logging.ReactorNettyHttpMessageLogFactory; -import reactor.netty.http.server.HttpServer; import reactor.netty.http.websocket.WebsocketInbound; import reactor.netty.http.websocket.WebsocketOutbound; import reactor.netty.internal.util.Metrics; diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java index e926a2f8ab..e18ad5d249 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java @@ -924,7 +924,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { pipeline.remove(NettyPipeline.ReactiveBridge); pipeline.remove(this); - if(idleTimeout != null) { + if (idleTimeout != null) { IdleTimeoutHandler.removeIdleTimeoutHandler(pipeline); IdleTimeoutHandler.addIdleTimeoutClientHandler( pipeline, diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java index 7e98ac61f7..1b08178718 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java @@ -1080,7 +1080,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { pipeline.remove(NettyPipeline.HttpTrafficHandler); pipeline.remove(NettyPipeline.ReactiveBridge); - if(idleTimeout != null) { + if (idleTimeout != null) { IdleTimeoutHandler.removeIdleTimeoutHandler(pipeline); IdleTimeoutHandler.addIdleTimeoutServerHandler( pipeline, From 36c039c036322c391af25dcb0b47599769da8948 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sat, 15 Mar 2025 08:00:54 +0900 Subject: [PATCH 15/27] Rollback docs related with HTTP2 Ping Signed-off-by: raccoonback --- docs/modules/ROOT/pages/http-client.adoc | 4 ---- .../http/client/http2/H2Application.java | 11 ++--------- 2 files changed, 2 insertions(+), 13 deletions(-) diff --git a/docs/modules/ROOT/pages/http-client.adoc b/docs/modules/ROOT/pages/http-client.adoc index b9815117ae..9eb4125778 100644 --- a/docs/modules/ROOT/pages/http-client.adoc +++ b/docs/modules/ROOT/pages/http-client.adoc @@ -318,10 +318,6 @@ include::{examples-dir}/http2/H2Application.java[lines=18..42] ---- <1> Configures the client to support only `HTTP/2` <2> Configures `SSL` -<3> Sets the interval for sending `HTTP/2` `PING` frames and receiving `ACK` responses -<4> Sets the execution interval for the scheduler that sends `HTTP/2` `PING frames and periodically checks for `ACK` responses -<5> Sets the threshold for retrying `HTTP/2` `PING` frame transmissions. - The following listing presents a simple `H2C` example: diff --git a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java index 10aca39285..5030e8a2b5 100644 --- a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java +++ b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/http2/H2Application.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2025 VMware, Inc. or its affiliates, All Rights Reserved. + * Copyright (c) 2020-2021 VMware, Inc. or its affiliates, All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,20 +21,13 @@ import reactor.netty.http.client.HttpClient; import reactor.util.function.Tuple2; -import java.time.Duration; - public class H2Application { public static void main(String[] args) { HttpClient client = HttpClient.create() .protocol(HttpProtocol.H2) //<1> - .secure() //<2> - .http2Settings( - builder -> builder.pingAckTimeout(Duration.ofMillis(600)) // <3> - .pingScheduleInterval(Duration.ofMillis(300)) // <4> - .pingAckDropThreshold(2) // <5> - ); + .secure(); //<2> Tuple2 response = client.get() From 76f9828a91ca5a23af95150eac9447e0b8a43729 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Sun, 16 Mar 2025 10:10:21 +0900 Subject: [PATCH 16/27] Fix broken test Signed-off-by: raccoonback --- .../reactor-netty-http/reflect-config.json | 4 +-- .../netty/http/HttpProtocolsTests.java | 26 +++++++++++++------ .../client/HttpConnectionLivenessTest.java | 8 +++--- .../server/HttpConnectionLivenessTest.java | 8 +++--- 4 files changed, 28 insertions(+), 18 deletions(-) diff --git a/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json b/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json index 81f188e8ca..c77b5b4d9b 100644 --- a/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json +++ b/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json @@ -260,9 +260,9 @@ }, { "condition": { - "typeReachable": "reactor.netty.http.server.IdleTimeoutHandler" + "typeReachable": "reactor.netty.http.IdleTimeoutHandler" }, - "name": "reactor.netty.http.server.IdleTimeoutHandler", + "name": "reactor.netty.http.IdleTimeoutHandler", "queryAllPublicMethods": true }, { diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/HttpProtocolsTests.java b/reactor-netty-http/src/test/java/reactor/netty/http/HttpProtocolsTests.java index 6082f15679..1488fafca3 100644 --- a/reactor-netty-http/src/test/java/reactor/netty/http/HttpProtocolsTests.java +++ b/reactor-netty-http/src/test/java/reactor/netty/http/HttpProtocolsTests.java @@ -38,11 +38,13 @@ import io.netty.handler.codec.http.HttpServerCodec; import io.netty.handler.codec.http.LastHttpContent; import io.netty.handler.codec.http2.Http2Connection; +import io.netty.handler.codec.http2.Http2ConnectionAdapter; import io.netty.handler.codec.http2.Http2DataFrame; import io.netty.handler.codec.http2.Http2FrameCodec; import io.netty.handler.codec.http2.Http2HeadersFrame; 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.codec.http2.Http2StreamChannel; import io.netty.handler.ssl.SslHandshakeCompletionEvent; import io.netty.handler.ssl.util.InsecureTrustManagerFactory; @@ -52,7 +54,6 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import org.mockito.Mockito; import org.reactivestreams.Publisher; import org.slf4j.LoggerFactory; import reactor.core.publisher.Flux; @@ -727,13 +728,22 @@ else if (serverProtocols.length == 2 && serverProtocols[1] == HttpProtocol.H2) { } Http2FrameCodec http2FrameCodec = conn.channel().parent().pipeline().get(Http2FrameCodec.class); - Http2Connection.Listener goAwayFrameListener = Mockito.mock(Http2Connection.Listener.class); - Mockito.doAnswer(invocation -> { - goAwayReceived.countDown(); - return null; - }) - .when(goAwayFrameListener) - .onGoAwayReceived(Mockito.anyInt(), Mockito.anyLong(), Mockito.any()); + Http2Connection.Listener goAwayFrameListener = new Http2ConnectionAdapter() { + @Override + public void onGoAwayReceived(int lastStreamId, long errorCode, ByteBuf debugData) { + goAwayReceived.countDown(); + + super.onGoAwayReceived(lastStreamId, errorCode, debugData); + } + + @Override + public void onStreamClosed(Http2Stream stream) { + goAwayReceived.countDown(); + + super.onStreamClosed(stream); + } + }; + http2FrameCodec.connection().addListener(goAwayFrameListener); }) .port(disposableServer.port()) diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java index 68251abc45..108ed33153 100644 --- a/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java +++ b/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java @@ -140,7 +140,7 @@ void maintainConnectionWithoutPingCheckWhenNotConfigured() { @ParameterizedTest @CsvSource({ "100,300,3", "300,100,3", - "100,300,3", "300,100,3", + "100,300,3", "300,100,3" }) void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { Http2PingFrameHandler handler = new Http2PingFrameHandler( @@ -193,7 +193,7 @@ void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingSched @ParameterizedTest @CsvSource({ "100,300,3", "300,100,3", - "100,300,3", "300,100,3", + "100,300,3", "300,100,3" }) void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { Http2PingFrameHandler handler = new Http2PingFrameHandler( @@ -461,7 +461,7 @@ void maintainConnectionWithoutPingCheckWhenNotConfigured() { @ParameterizedTest @CsvSource({ "100,300,3", "300,100,3", - "100,300,3", "300,100,3", + "100,300,3", "300,100,3" }) void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { Http2PingFrameHandler handler = new Http2PingFrameHandler( @@ -512,7 +512,7 @@ void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingSched @ParameterizedTest @CsvSource({ "100,300,3", "300,100,3", - "100,300,3", "300,100,3", + "100,300,3", "300,100,3" }) void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { Http2PingFrameHandler handler = new Http2PingFrameHandler( diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java index 7e5437cb3e..90e4c901d3 100644 --- a/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java +++ b/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java @@ -143,7 +143,7 @@ void maintainConnectionWithoutPingCheckWhenNotConfigured() { @ParameterizedTest @CsvSource({ "100,300,3", "300,100,3", - "100,300,3", "300,100,3", + "100,300,3", "300,100,3" }) void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { AtomicReference connectedServerChannel = new AtomicReference<>(); @@ -198,7 +198,7 @@ void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingSched @ParameterizedTest @CsvSource({ "100,300,3", "300,100,3", - "100,300,3", "300,100,3", + "100,300,3", "300,100,3" }) void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { AtomicReference connectedServerChannel = new AtomicReference<>(); @@ -475,7 +475,7 @@ void maintainConnectionWithoutPingCheckWhenNotConfigured() { @ParameterizedTest @CsvSource({ "100,300,3", "300,100,3", - "100,300,3", "300,100,3", + "100,300,3", "300,100,3" }) void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { AtomicReference connectedServerChannel = new AtomicReference<>(); @@ -528,7 +528,7 @@ void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingSched @ParameterizedTest @CsvSource({ "100,300,3", "300,100,3", - "100,300,3", "300,100,3", + "100,300,3", "300,100,3" }) void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { AtomicReference connectedServerChannel = new AtomicReference<>(); From 8e7e733a47d49098f79c6c503568dd5d61f271c0 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Mon, 17 Mar 2025 09:02:44 +0900 Subject: [PATCH 17/27] Update docs about HTTP2 Ping frame and IdleTimeout Signed-off-by: raccoonback --- docs/modules/ROOT/pages/http-client.adoc | 44 +++++++++++++++++ docs/modules/ROOT/pages/http-server.adoc | 29 +++++++++++ .../http/client/idle/timeout/Application.java | 44 +++++++++++++++++ .../http/client/liveness/Application.java | 49 +++++++++++++++++++ .../http/server/liveness/Application.java | 40 +++++++++++++++ 5 files changed, 206 insertions(+) create mode 100644 reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/idle/timeout/Application.java create mode 100644 reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/liveness/Application.java create mode 100644 reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/server/liveness/Application.java diff --git a/docs/modules/ROOT/pages/http-client.adoc b/docs/modules/ROOT/pages/http-client.adoc index 9eb4125778..35e63c0906 100644 --- a/docs/modules/ROOT/pages/http-client.adoc +++ b/docs/modules/ROOT/pages/http-client.adoc @@ -680,6 +680,50 @@ include::{examples-dir}/channeloptions/Application.java[lines=18..54] <4> Configures the time between individual `keepalive` probes to 1 minute. <5> Configures the maximum number of TCP `keepalive` probes to 8. + +==== IdleTimeout +The maximum time (resolution: ms) that this connection stays opened and waits for HTTP request. Once the timeout is reached, the connection is closed. +By default, `idleTimeout` is not specified, this indicates no timeout (i.e. infinite), which means the connection is closed only if one of the peers decides to close it explicitly. + +NOTE: It is always a good practice to configure an idle timeout. + +To customize the default settings, you can configure `HttpClient` as follows: + +{examples-link}/idle/timeout/Application.java +---- +include::{examples-dir}/idle/timeout/Application.java[lines=26..44] +---- +<1> Configures the default idle timeout to 1 second. + +===== Configuring Ping Frame for HTTP/2 Health Check +When using the HTTP/2 protocol, it is recommended to configure a *PING* frame to maintain the connection and ensure timely health checks. +The HttpClient in Reactor Netty allows setting up a *PING* frame to prevent idle connections from being prematurely closed when *idleTimeout* is configured. + +[NOTE] +==== +To enable HTTP/2 PING frame-based health checks, you must configure `idleTimeout`. +Without `idleTimeout`, the connection may remain open indefinitely, preventing proper detection of inactive or unresponsive connections. +Setting an appropriate `idleTimeout` ensures that PING-based health checks can effectively terminate unresponsive connections. +==== + +*Benefits of Using PING Frames* + +- Actively monitors connection health by checking real-time responses to *PING* frames. +- Ensures that health checks detect unresponsive connections quickly. +- Helps maintain long-lived connections in an efficient manner. + +To enable *PING* frames for HTTP/2 connections, configure the HttpClient as follows: + +{examples-link}/liveness/Application.java +---- +include::{examples-dir}/liveness/Application.java[lines=26..49] +---- +<1> To set up a health check using HTTP2 Ping frame, `idleTimeout` must be set first. +<2> Sets the interval for sending `HTTP/2` `PING` frames and receiving `ACK` responses +<3> Sets the execution interval for the scheduler that sends `HTTP/2` `PING frames and periodically checks for `ACK` responses +<4> Sets the threshold for retrying `HTTP/2` `PING` frame transmissions. + + [[ssl-tls-timeout]] ==== SSL/TLS Timeout `HttpClient` supports the SSL/TLS functionality provided by Netty. diff --git a/docs/modules/ROOT/pages/http-server.adoc b/docs/modules/ROOT/pages/http-server.adoc index 5013ccf29e..c8485f9263 100644 --- a/docs/modules/ROOT/pages/http-server.adoc +++ b/docs/modules/ROOT/pages/http-server.adoc @@ -786,6 +786,35 @@ include::{examples-dir}/idle/timeout/Application.java[lines=18..35] ---- <1> Configures the default idle timeout to 1 second. +==== Configuring Ping Frame for HTTP/2 Health Check +When using the HTTP/2 protocol, it is recommended to configure a *PING* frame to maintain the connection and ensure timely health checks. +The HttpServer in Reactor Netty allows setting up a *PING* frame to prevent idle connections from being prematurely closed when *idleTimeout* is configured. + +[NOTE] +==== +To enable HTTP/2 PING frame-based health checks, you must configure `idleTimeout`. +Without `idleTimeout`, the connection may remain open indefinitely, preventing proper detection of inactive or unresponsive connections. +Setting an appropriate `idleTimeout` ensures that PING-based health checks can effectively terminate unresponsive connections. +==== + +*Benefits of Using PING Frames* + +- Actively monitors connection health by checking real-time responses to *PING* frames. +- Ensures that health checks detect unresponsive connections quickly. +- Helps maintain long-lived connections in an efficient manner. + +To enable *PING* frames for HTTP/2 connections, configure the HttpServer as follows: + +{examples-link}/liveness/Application.java +---- +include::{examples-dir}/liveness/Application.java[lines=23..40] +---- +<1> To set up a health check using HTTP2 Ping frame, `idleTimeout` must be set first. +<2> Sets the interval for sending `HTTP/2` `PING` frames and receiving `ACK` responses +<3> Sets the execution interval for the scheduler that sends `HTTP/2` `PING frames and periodically checks for `ACK` responses +<4> Sets the threshold for retrying `HTTP/2` `PING` frame transmissions. + + [[http-server-ssl-tls-timeout]] === SSL/TLS Timeout `HttpServer` supports the SSL/TLS functionality provided by Netty. diff --git a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/idle/timeout/Application.java b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/idle/timeout/Application.java new file mode 100644 index 0000000000..bce7fc5ddd --- /dev/null +++ b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/idle/timeout/Application.java @@ -0,0 +1,44 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.examples.documentation.http.client.idle.timeout; + +import io.netty.handler.codec.http.HttpHeaders; +import reactor.core.publisher.Mono; +import reactor.netty.http.HttpProtocol; +import reactor.netty.http.client.HttpClient; +import reactor.util.function.Tuple2; + +import java.time.Duration; + +public class Application { + + public static void main(String[] args) { + HttpClient client = + HttpClient.create() + .protocol(HttpProtocol.H2) + .secure() + .idleTimeout(Duration.ofSeconds(1)); //<1> + + Tuple2 response = + client.get() + .uri("https://example.com/") + .responseSingle((res, bytes) -> bytes.asString() + .zipWith(Mono.just(res.responseHeaders()))) + .block(); + System.out.println("Used stream ID: " + response.getT2().get("x-http2-stream-id")); + System.out.println("Response: " + response.getT1()); + } +} diff --git a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/liveness/Application.java b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/liveness/Application.java new file mode 100644 index 0000000000..ecab86e0ec --- /dev/null +++ b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/liveness/Application.java @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.examples.documentation.http.client.liveness; + +import io.netty.handler.codec.http.HttpHeaders; +import reactor.core.publisher.Mono; +import reactor.netty.http.HttpProtocol; +import reactor.netty.http.client.HttpClient; +import reactor.util.function.Tuple2; + +import java.time.Duration; + +public class Application { + + public static void main(String[] args) { + HttpClient client = + HttpClient.create() + .protocol(HttpProtocol.H2) + .secure() + .idleTimeout(Duration.ofSeconds(1)) //<1> + .http2Settings( + builder -> builder.pingAckTimeout(Duration.ofMillis(600)) // <2> + .pingScheduleInterval(Duration.ofMillis(300)) // <3> + .pingAckDropThreshold(2) // <4> + ); + + Tuple2 response = + client.get() + .uri("https://example.com/") + .responseSingle((res, bytes) -> bytes.asString() + .zipWith(Mono.just(res.responseHeaders()))) + .block(); + System.out.println("Used stream ID: " + response.getT2().get("x-http2-stream-id")); + System.out.println("Response: " + response.getT1()); + } +} diff --git a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/server/liveness/Application.java b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/server/liveness/Application.java new file mode 100644 index 0000000000..c2b6922b48 --- /dev/null +++ b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/server/liveness/Application.java @@ -0,0 +1,40 @@ +/* + * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. + * + * 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 reactor.netty.examples.documentation.http.server.liveness; + +import reactor.netty.DisposableServer; +import reactor.netty.http.server.HttpServer; + +import java.time.Duration; + +public class Application { + + public static void main(String[] args) { + DisposableServer server = + HttpServer.create() + .idleTimeout(Duration.ofSeconds(1)) //<1> + .secure() + .http2Settings( + builder -> builder.pingAckTimeout(Duration.ofMillis(600)) // <2> + .pingScheduleInterval(Duration.ofMillis(300)) // <3> + .pingAckDropThreshold(2) // <4> + ); + .bindNow(); + + server.onDispose() + .block(); + } +} From 8664765a88c39c538b2266011bf3bc8865f2d5d3 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Mon, 17 Mar 2025 09:11:16 +0900 Subject: [PATCH 18/27] Add override annotation Signed-off-by: raccoonback --- .../netty/http/Http2ConnectionLiveness.java | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java b/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java index b5d885997c..f1df04567b 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java @@ -111,6 +111,7 @@ public Http2ConnectionLiveness( * * @param ctx the {@link ChannelHandlerContext} of the connection */ + @Override public void check(ChannelHandlerContext ctx) { if (isPingIntervalConfigured()) { if (pingScheduler == null) { @@ -135,6 +136,7 @@ public void check(ChannelHandlerContext ctx) { * * @param msg the message received from the peer */ + @Override public void receive(Object msg) { if (msg instanceof Http2PingFrame) { Http2PingFrame frame = (Http2PingFrame) msg; @@ -144,19 +146,10 @@ public void receive(Object msg) { } } - /** - * Checks if the ping interval is configured. - * - * @return {@code true} if the ping interval is configured, {@code false} otherwise - */ - public boolean isPingIntervalConfigured() { - return pingAckTimeoutNanos > 0 - && pingScheduleIntervalNanos > 0; - } - /** * Cancels the scheduled ping task. */ + @Override public void cancel() { if (pingScheduler != null) { pingScheduler.cancel(false); @@ -164,6 +157,11 @@ public void cancel() { } } + private boolean isPingIntervalConfigured() { + return pingAckTimeoutNanos > 0 + && pingScheduleIntervalNanos > 0; + } + /** * A task that handles ping timeouts. */ From 0d5b0ef45a7ed028fc7c86e53abc9f00b0984d59 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Mon, 17 Mar 2025 09:17:01 +0900 Subject: [PATCH 19/27] Disable checkstyle about channel close Signed-off-by: raccoonback --- .../main/java/reactor/netty/http/Http2ConnectionLiveness.java | 1 + .../java/reactor/netty/http/HttpConnectionImmediateClose.java | 1 + 2 files changed, 2 insertions(+) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java b/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java index f1df04567b..dd6b183624 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java @@ -112,6 +112,7 @@ public Http2ConnectionLiveness( * @param ctx the {@link ChannelHandlerContext} of the connection */ @Override + @SuppressWarnings("FutureReturnValueIgnored") public void check(ChannelHandlerContext ctx) { if (isPingIntervalConfigured()) { if (pingScheduler == null) { diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionImmediateClose.java b/reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionImmediateClose.java index 67dda27029..eb226b2ffc 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionImmediateClose.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/HttpConnectionImmediateClose.java @@ -63,6 +63,7 @@ public void receive(Object msg) { * @param ctx the {@link ChannelHandlerContext} of the connection */ @Override + @SuppressWarnings("FutureReturnValueIgnored") public void check(ChannelHandlerContext ctx) { ctx.close(); } From 33f15aa19e4f0f2aca91afbcf0ff206ceeed5d4d Mon Sep 17 00:00:00 2001 From: raccoonback Date: Mon, 17 Mar 2025 09:28:16 +0900 Subject: [PATCH 20/27] Fix broken docs Signed-off-by: raccoonback --- docs/modules/ROOT/pages/http-server.adoc | 2 +- .../http/server/liveness/Application.java | 16 +++++++++++++--- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/docs/modules/ROOT/pages/http-server.adoc b/docs/modules/ROOT/pages/http-server.adoc index c8485f9263..9712f53494 100644 --- a/docs/modules/ROOT/pages/http-server.adoc +++ b/docs/modules/ROOT/pages/http-server.adoc @@ -807,7 +807,7 @@ To enable *PING* frames for HTTP/2 connections, configure the HttpServer as foll {examples-link}/liveness/Application.java ---- -include::{examples-dir}/liveness/Application.java[lines=23..40] +include::{examples-dir}/liveness/Application.java[lines=26..50] ---- <1> To set up a health check using HTTP2 Ping frame, `idleTimeout` must be set first. <2> Sets the interval for sending `HTTP/2` `PING` frames and receiving `ACK` responses diff --git a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/server/liveness/Application.java b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/server/liveness/Application.java index c2b6922b48..ad75804f73 100644 --- a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/server/liveness/Application.java +++ b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/server/liveness/Application.java @@ -16,23 +16,33 @@ package reactor.netty.examples.documentation.http.server.liveness; import reactor.netty.DisposableServer; +import reactor.netty.http.Http2SslContextSpec; +import reactor.netty.http.HttpProtocol; import reactor.netty.http.server.HttpServer; +import java.io.File; import java.time.Duration; public class Application { public static void main(String[] args) { + File cert = new File("certificate.crt"); + File key = new File("private.key"); + + Http2SslContextSpec http2SslContextSpec = Http2SslContextSpec.forServer(cert, key); + DisposableServer server = HttpServer.create() + .port(8080) + .protocol(HttpProtocol.H2) + .secure(spec -> spec.sslContext(http2SslContextSpec)) .idleTimeout(Duration.ofSeconds(1)) //<1> - .secure() .http2Settings( builder -> builder.pingAckTimeout(Duration.ofMillis(600)) // <2> .pingScheduleInterval(Duration.ofMillis(300)) // <3> .pingAckDropThreshold(2) // <4> - ); - .bindNow(); + ) + .bindNow(); server.onDispose() .block(); From c23df4b53539cdff6ad1ace9017ca7bbbec28d7a Mon Sep 17 00:00:00 2001 From: raccoonback Date: Wed, 19 Mar 2025 13:28:54 +0900 Subject: [PATCH 21/27] Stop scheduler when data is received, considering the connection active Signed-off-by: raccoonback --- .../java/reactor/netty/http/Http2ConnectionLiveness.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java b/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java index dd6b183624..f31cf4e089 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/Http2ConnectionLiveness.java @@ -19,6 +19,7 @@ import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.http2.Http2DataFrame; import io.netty.handler.codec.http2.Http2FrameCodec; import io.netty.handler.codec.http2.Http2FrameWriter; import io.netty.handler.codec.http2.Http2PingFrame; @@ -145,6 +146,10 @@ public void receive(Object msg) { lastReceivedPingTime = System.nanoTime(); } } + + if (msg instanceof Http2DataFrame) { + cancel(); + } } /** From 164297910046238a94451a910c26bff82856422c Mon Sep 17 00:00:00 2001 From: raccoonback Date: Thu, 24 Apr 2025 23:06:55 +0900 Subject: [PATCH 22/27] Revert IdleTimeoutHandler on HttpClient Signed-off-by: raccoonback --- docs/modules/ROOT/pages/http-client.adoc | 44 - .../http/client/idle/timeout/Application.java | 44 - .../http/client/liveness/Application.java | 49 -- .../reactor/netty/http/Http2SettingsSpec.java | 28 - .../netty/http/IdleTimeoutHandler.java | 41 - .../reactor/netty/http/client/HttpClient.java | 19 - .../netty/http/client/HttpClientConfig.java | 138 ++- .../client/HttpConnectionLivenessTest.java | 792 ------------------ .../http/client/HttpIdleTimeoutTest.java | 252 ------ .../server/HttpConnectionLivenessTest.java | 6 +- 10 files changed, 59 insertions(+), 1354 deletions(-) delete mode 100644 reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/idle/timeout/Application.java delete mode 100644 reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/liveness/Application.java delete mode 100644 reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java delete mode 100644 reactor-netty-http/src/test/java/reactor/netty/http/client/HttpIdleTimeoutTest.java diff --git a/docs/modules/ROOT/pages/http-client.adoc b/docs/modules/ROOT/pages/http-client.adoc index 35e63c0906..9eb4125778 100644 --- a/docs/modules/ROOT/pages/http-client.adoc +++ b/docs/modules/ROOT/pages/http-client.adoc @@ -680,50 +680,6 @@ include::{examples-dir}/channeloptions/Application.java[lines=18..54] <4> Configures the time between individual `keepalive` probes to 1 minute. <5> Configures the maximum number of TCP `keepalive` probes to 8. - -==== IdleTimeout -The maximum time (resolution: ms) that this connection stays opened and waits for HTTP request. Once the timeout is reached, the connection is closed. -By default, `idleTimeout` is not specified, this indicates no timeout (i.e. infinite), which means the connection is closed only if one of the peers decides to close it explicitly. - -NOTE: It is always a good practice to configure an idle timeout. - -To customize the default settings, you can configure `HttpClient` as follows: - -{examples-link}/idle/timeout/Application.java ----- -include::{examples-dir}/idle/timeout/Application.java[lines=26..44] ----- -<1> Configures the default idle timeout to 1 second. - -===== Configuring Ping Frame for HTTP/2 Health Check -When using the HTTP/2 protocol, it is recommended to configure a *PING* frame to maintain the connection and ensure timely health checks. -The HttpClient in Reactor Netty allows setting up a *PING* frame to prevent idle connections from being prematurely closed when *idleTimeout* is configured. - -[NOTE] -==== -To enable HTTP/2 PING frame-based health checks, you must configure `idleTimeout`. -Without `idleTimeout`, the connection may remain open indefinitely, preventing proper detection of inactive or unresponsive connections. -Setting an appropriate `idleTimeout` ensures that PING-based health checks can effectively terminate unresponsive connections. -==== - -*Benefits of Using PING Frames* - -- Actively monitors connection health by checking real-time responses to *PING* frames. -- Ensures that health checks detect unresponsive connections quickly. -- Helps maintain long-lived connections in an efficient manner. - -To enable *PING* frames for HTTP/2 connections, configure the HttpClient as follows: - -{examples-link}/liveness/Application.java ----- -include::{examples-dir}/liveness/Application.java[lines=26..49] ----- -<1> To set up a health check using HTTP2 Ping frame, `idleTimeout` must be set first. -<2> Sets the interval for sending `HTTP/2` `PING` frames and receiving `ACK` responses -<3> Sets the execution interval for the scheduler that sends `HTTP/2` `PING frames and periodically checks for `ACK` responses -<4> Sets the threshold for retrying `HTTP/2` `PING` frame transmissions. - - [[ssl-tls-timeout]] ==== SSL/TLS Timeout `HttpClient` supports the SSL/TLS functionality provided by Netty. diff --git a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/idle/timeout/Application.java b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/idle/timeout/Application.java deleted file mode 100644 index bce7fc5ddd..0000000000 --- a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/idle/timeout/Application.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. - * - * 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 reactor.netty.examples.documentation.http.client.idle.timeout; - -import io.netty.handler.codec.http.HttpHeaders; -import reactor.core.publisher.Mono; -import reactor.netty.http.HttpProtocol; -import reactor.netty.http.client.HttpClient; -import reactor.util.function.Tuple2; - -import java.time.Duration; - -public class Application { - - public static void main(String[] args) { - HttpClient client = - HttpClient.create() - .protocol(HttpProtocol.H2) - .secure() - .idleTimeout(Duration.ofSeconds(1)); //<1> - - Tuple2 response = - client.get() - .uri("https://example.com/") - .responseSingle((res, bytes) -> bytes.asString() - .zipWith(Mono.just(res.responseHeaders()))) - .block(); - System.out.println("Used stream ID: " + response.getT2().get("x-http2-stream-id")); - System.out.println("Response: " + response.getT1()); - } -} diff --git a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/liveness/Application.java b/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/liveness/Application.java deleted file mode 100644 index ecab86e0ec..0000000000 --- a/reactor-netty-examples/src/main/java/reactor/netty/examples/documentation/http/client/liveness/Application.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. - * - * 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 reactor.netty.examples.documentation.http.client.liveness; - -import io.netty.handler.codec.http.HttpHeaders; -import reactor.core.publisher.Mono; -import reactor.netty.http.HttpProtocol; -import reactor.netty.http.client.HttpClient; -import reactor.util.function.Tuple2; - -import java.time.Duration; - -public class Application { - - public static void main(String[] args) { - HttpClient client = - HttpClient.create() - .protocol(HttpProtocol.H2) - .secure() - .idleTimeout(Duration.ofSeconds(1)) //<1> - .http2Settings( - builder -> builder.pingAckTimeout(Duration.ofMillis(600)) // <2> - .pingScheduleInterval(Duration.ofMillis(300)) // <3> - .pingAckDropThreshold(2) // <4> - ); - - Tuple2 response = - client.get() - .uri("https://example.com/") - .responseSingle((res, bytes) -> bytes.asString() - .zipWith(Mono.just(res.responseHeaders()))) - .block(); - System.out.println("Used stream ID: " + response.getT2().get("x-http2-stream-id")); - System.out.println("Response: " + response.getT1()); - } -} diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java b/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java index 825099a698..21d0bbc06b 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/Http2SettingsSpec.java @@ -456,32 +456,4 @@ public Builder pushEnabled(boolean pushEnabled) { } */ } - - public Http2Settings http2Settings() { - Http2Settings settings = Http2Settings.defaultSettings(); - - if (headerTableSize != null) { - settings.headerTableSize(headerTableSize); - } - - if (initialWindowSize != null) { - settings.initialWindowSize(initialWindowSize); - } - - if (maxConcurrentStreams != null) { - settings.maxConcurrentStreams(maxConcurrentStreams); - } - - if (maxFrameSize != null) { - settings.maxFrameSize(maxFrameSize); - } - - settings.maxHeaderListSize(maxHeaderListSize); - - if (pushEnabled != null) { - settings.pushEnabled(pushEnabled); - } - - return settings; - } } diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java index abdefa9b0c..48c6cee14d 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java @@ -18,8 +18,6 @@ import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelPipeline; -import io.netty.handler.codec.http.HttpClientCodec; -import io.netty.handler.codec.http.HttpClientUpgradeHandler; import io.netty.handler.codec.http.HttpServerCodec; import io.netty.handler.codec.http.HttpServerUpgradeHandler; import io.netty.handler.timeout.IdleState; @@ -123,45 +121,6 @@ public static void addIdleTimeoutServerHandler(ChannelPipeline pipeline, @Nullab } } - /** - * Adds an idle timeout handler to the client pipeline. - * This handler will close the connection if it remains idle for the specified duration. - * - * @param pipeline the channel pipeline to which the handler will be added - * @param idleTimeout the duration of idle time after which the connection will be closed - * @param httpConnectionLiveness the HTTP connection liveness checker to be used - */ - public static void addIdleTimeoutClientHandler(ChannelPipeline pipeline, @Nullable Duration idleTimeout, - HttpConnectionLiveness httpConnectionLiveness) { - if (idleTimeout != null && pipeline.get(NettyPipeline.IdleTimeoutHandler) == null) { - String baseName = null; - if (pipeline.get(NettyPipeline.HttpCodec) != null) { - baseName = NettyPipeline.HttpCodec; - } - else { - ChannelHandler httpClientUpgradeHandler = pipeline.get(HttpClientUpgradeHandler.class); - if (httpClientUpgradeHandler != null) { - baseName = pipeline.context(httpClientUpgradeHandler).name(); - } - else { - ChannelHandler httpClientCodec = pipeline.get(HttpClientCodec.class); - if (httpClientCodec != null) { - baseName = pipeline.context(httpClientCodec).name(); - } - } - } - - pipeline.addAfter( - baseName, - NettyPipeline.IdleTimeoutHandler, - new IdleTimeoutHandler( - idleTimeout.toMillis(), - httpConnectionLiveness - ) - ); - } - } - /** * Removes the idle timeout handler from the pipeline if it exists. * diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java index 121030a12c..e5b5f7527a 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClient.java @@ -1095,25 +1095,6 @@ public final HttpClient headersWhen(FunctionIf an {@code idleTimeout} is not specified, this indicates no timeout (i.e. infinite), - * which means the connection will be closed only if one of the peers decides to close it. - *

If the {@code idleTimeout} is less than {@code 1ms}, then {@code 1ms} will be the idle timeout. - *

By default {@code idleTimeout} is not specified. - * - * @param idleTimeout an idle timeout on the connection when it is waiting for an HTTP request (resolution: ms) - * @return a new {@link HttpClient} - * @since 1.2.5 - */ - public final HttpClient idleTimeout(Duration idleTimeout) { - Objects.requireNonNull(idleTimeout, "idleTimeout"); - HttpClient dup = duplicate(); - dup.configuration().idleTimeout = idleTimeout; - return dup; - } - /** * Apply HTTP/2 configuration. * diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java index e18ad5d249..5730675907 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/client/HttpClientConfig.java @@ -78,13 +78,10 @@ import reactor.netty.ReactorNetty; import reactor.netty.channel.ChannelMetricsRecorder; import reactor.netty.channel.ChannelOperations; -import reactor.netty.http.HttpConnectionImmediateClose; -import reactor.netty.http.Http2ConnectionLiveness; import reactor.netty.http.Http2SettingsSpec; import reactor.netty.http.Http3SettingsSpec; import reactor.netty.http.HttpProtocol; import reactor.netty.http.HttpResources; -import reactor.netty.http.IdleTimeoutHandler; import reactor.netty.http.logging.HttpMessageLogFactory; import reactor.netty.http.logging.ReactorNettyHttpMessageLogFactory; import reactor.netty.resources.ConnectionProvider; @@ -364,7 +361,6 @@ public WebsocketClientSpec websocketClientSpec() { BiConsumer doOnResponseError; BiPredicate followRedirectPredicate; HttpHeaders headers; - Duration idleTimeout; Http2SettingsSpec http2Settings; Http3SettingsSpec http3Settings; HttpMessageLogFactory httpMessageLogFactory; @@ -417,7 +413,6 @@ public WebsocketClientSpec websocketClientSpec() { this.doOnResponseError = parent.doOnResponseError; this.followRedirectPredicate = parent.followRedirectPredicate; this.headers = parent.headers; - this.idleTimeout = parent.idleTimeout; this.http2Settings = parent.http2Settings; this.http3Settings = parent.http3Settings; this.httpMessageLogFactory = parent.httpMessageLogFactory; @@ -563,6 +558,41 @@ boolean checkProtocol(int protocol) { return (_protocols & protocol) == protocol; } + Http2Settings http2Settings() { + Http2Settings settings = Http2Settings.defaultSettings(); + + if (http2Settings != null) { + Long headerTableSize = http2Settings.headerTableSize(); + if (headerTableSize != null) { + settings.headerTableSize(headerTableSize); + } + + Integer initialWindowSize = http2Settings.initialWindowSize(); + if (initialWindowSize != null) { + settings.initialWindowSize(initialWindowSize); + } + + Long maxConcurrentStreams = http2Settings.maxConcurrentStreams(); + if (maxConcurrentStreams != null) { + settings.maxConcurrentStreams(maxConcurrentStreams); + } + + Integer maxFrameSize = http2Settings.maxFrameSize(); + if (maxFrameSize != null) { + settings.maxFrameSize(maxFrameSize); + } + + settings.maxHeaderListSize(http2Settings.maxHeaderListSize()); + + Boolean pushEnabled = http2Settings.pushEnabled(); + if (pushEnabled != null) { + settings.pushEnabled(pushEnabled); + } + } + + return settings; + } + static void addStreamHandlers( Channel ch, ConnectionObserver obs, @@ -649,37 +679,24 @@ else if (metricsRecorder instanceof ContextAwareHttpClientMetricsRecorder) { } static void configureHttp2Pipeline(ChannelPipeline p, HttpResponseDecoderSpec decoder, - Duration idleTimeout, @Nullable Http2SettingsSpec http2SettingsSpec, ConnectionObserver observer) { + Http2Settings http2Settings, ConnectionObserver observer) { Http2FrameCodecBuilder http2FrameCodecBuilder = Http2FrameCodecBuilder.forClient() .validateHeaders(decoder.validateHeaders()) - .initialSettings(http2SettingsSpec == null ? Http2Settings.defaultSettings() : http2SettingsSpec.http2Settings()); + .initialSettings(http2Settings); if (p.get(NettyPipeline.LoggingHandler) != null) { http2FrameCodecBuilder.frameLogger(new Http2FrameLogger(LogLevel.DEBUG, "reactor.netty.http.client.h2")); } - Http2FrameCodec codec = http2FrameCodecBuilder.build(); - p.addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2Flush, new FlushConsolidationHandler(1024, true)) - .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpCodec, codec) + .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpCodec, http2FrameCodecBuilder.build()) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.H2MultiplexHandler, new Http2MultiplexHandler(H2InboundStreamHandler.INSTANCE)) .addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpTrafficHandler, new HttpTrafficHandler(observer)); - - IdleTimeoutHandler.addIdleTimeoutClientHandler( - p, - idleTimeout, - new Http2ConnectionLiveness( - codec, - http2SettingsSpec != null ? http2SettingsSpec.pingAckTimeout() : null, - http2SettingsSpec != null ? http2SettingsSpec.pingScheduleInterval() : null, - http2SettingsSpec != null ? http2SettingsSpec.pingAckDropThreshold() : null - ) - ); } - static void configureHttp3Pipeline(ChannelPipeline p, Duration idleTimeout, boolean removeMetricsRecorder, boolean removeProxyProvider) { + static void configureHttp3Pipeline(ChannelPipeline p, boolean removeMetricsRecorder, boolean removeProxyProvider) { p.remove(NettyPipeline.ReactiveBridge); p.addLast(NettyPipeline.HttpCodec, newHttp3ClientConnectionHandler()); @@ -692,12 +709,6 @@ static void configureHttp3Pipeline(ChannelPipeline p, Duration idleTimeout, bool if (removeProxyProvider) { p.remove(NettyPipeline.ProxyHandler); } - - IdleTimeoutHandler.addIdleTimeoutClientHandler( - p, - idleTimeout, - new HttpConnectionImmediateClose() - ); } @SuppressWarnings("deprecation") @@ -705,8 +716,7 @@ static void configureHttp11OrH2CleartextPipeline( ChannelPipeline p, boolean acceptGzip, HttpResponseDecoderSpec decoder, - Duration idleTimeout, - @Nullable Http2SettingsSpec http2SettingsSpec, + Http2Settings http2Settings, @Nullable ChannelMetricsRecorder metricsRecorder, ConnectionObserver observer, ChannelOperations.OnSetup opsFactory, @@ -727,7 +737,7 @@ static void configureHttp11OrH2CleartextPipeline( Http2FrameCodecBuilder http2FrameCodecBuilder = Http2FrameCodecBuilder.forClient() .validateHeaders(decoder.validateHeaders()) - .initialSettings(http2SettingsSpec == null ? Http2Settings.defaultSettings() : http2SettingsSpec.http2Settings()); + .initialSettings(http2Settings); if (p.get(NettyPipeline.LoggingHandler) != null) { http2FrameCodecBuilder.frameLogger(new Http2FrameLogger(LogLevel.DEBUG, @@ -737,7 +747,7 @@ static void configureHttp11OrH2CleartextPipeline( Http2FrameCodec http2FrameCodec = http2FrameCodecBuilder.build(); Http2ClientUpgradeCodec upgradeCodec = new Http2ClientUpgradeCodec(http2FrameCodec, - new H2CleartextCodec(http2FrameCodec, opsFactory, acceptGzip, metricsRecorder, proxyAddress, remoteAddress, uriTagValue, idleTimeout, http2SettingsSpec)); + new H2CleartextCodec(http2FrameCodec, opsFactory, acceptGzip, metricsRecorder, proxyAddress, remoteAddress, uriTagValue)); HttpClientUpgradeHandler upgradeHandler = new ReactorNettyHttpClientUpgradeHandler(httpClientCodec, upgradeCodec, decoder.h2cMaxContentLength()); @@ -766,18 +776,12 @@ else if (metricsRecorder instanceof ContextAwareHttpClientMetricsRecorder) { } } - IdleTimeoutHandler.addIdleTimeoutClientHandler( - p, - idleTimeout, - new HttpConnectionImmediateClose() - ); } @SuppressWarnings("deprecation") static void configureHttp11Pipeline(ChannelPipeline p, boolean acceptGzip, HttpResponseDecoderSpec decoder, - Duration idleTimeout, @Nullable ChannelMetricsRecorder metricsRecorder, @Nullable SocketAddress proxyAddress, SocketAddress remoteAddress, @@ -812,14 +816,7 @@ else if (metricsRecorder instanceof ContextAwareHttpClientMetricsRecorder) { } p.addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.HttpMetricsHandler, handler); } - } - - IdleTimeoutHandler.addIdleTimeoutClientHandler( - p, - idleTimeout, - new HttpConnectionImmediateClose() - ); } static final Pattern FOLLOW_REDIRECT_CODES = Pattern.compile("30[12378]"); @@ -868,8 +865,6 @@ static final class H2CleartextCodec extends ChannelHandlerAdapter { final SocketAddress proxyAddress; final SocketAddress remoteAddress; final Function uriTagValue; - final Duration idleTimeout; - final Http2SettingsSpec http2SettingsSpec; H2CleartextCodec( Http2FrameCodec http2FrameCodec, @@ -878,10 +873,7 @@ static final class H2CleartextCodec extends ChannelHandlerAdapter { @Nullable ChannelMetricsRecorder metricsRecorder, @Nullable SocketAddress proxyAddress, SocketAddress remoteAddress, - @Nullable Function uriTagValue, - @Nullable Duration idleTimeout, - @Nullable Http2SettingsSpec http2SettingsSpec - ) { + @Nullable Function uriTagValue) { this.acceptGzip = acceptGzip; this.http2FrameCodec = http2FrameCodec; this.metricsRecorder = metricsRecorder; @@ -889,8 +881,6 @@ static final class H2CleartextCodec extends ChannelHandlerAdapter { this.proxyAddress = proxyAddress; this.remoteAddress = remoteAddress; this.uriTagValue = uriTagValue; - this.idleTimeout = idleTimeout; - this.http2SettingsSpec = http2SettingsSpec; } @Override @@ -923,20 +913,6 @@ public void handlerAdded(ChannelHandlerContext ctx) { } pipeline.remove(NettyPipeline.ReactiveBridge); pipeline.remove(this); - - if (idleTimeout != null) { - IdleTimeoutHandler.removeIdleTimeoutHandler(pipeline); - IdleTimeoutHandler.addIdleTimeoutClientHandler( - pipeline, - idleTimeout, - new Http2ConnectionLiveness( - http2FrameCodec, - http2SettingsSpec != null ? http2SettingsSpec.pingAckTimeout() : null, - http2SettingsSpec != null ? http2SettingsSpec.pingScheduleInterval() : null, - http2SettingsSpec != null ? http2SettingsSpec.pingAckDropThreshold() : null - ) - ); - } } } @@ -1020,8 +996,7 @@ public boolean isSharable() { static final class H2OrHttp11Codec extends ChannelInboundHandlerAdapter { final boolean acceptGzip; final HttpResponseDecoderSpec decoder; - final Duration idleTimeout; - final Http2SettingsSpec http2SettingsSpec; + final Http2Settings http2Settings; final ChannelMetricsRecorder metricsRecorder; final ConnectionObserver observer; final SocketAddress proxyAddress; @@ -1031,8 +1006,7 @@ static final class H2OrHttp11Codec extends ChannelInboundHandlerAdapter { H2OrHttp11Codec(HttpClientChannelInitializer initializer, ConnectionObserver observer, SocketAddress remoteAddress) { this.acceptGzip = initializer.acceptGzip; this.decoder = initializer.decoder; - this.idleTimeout = initializer.idleTimeout; - this.http2SettingsSpec = initializer.http2SettingsSpec; + this.http2Settings = initializer.http2Settings; this.metricsRecorder = initializer.metricsRecorder; this.observer = observer; this.proxyAddress = initializer.proxyAddress; @@ -1051,10 +1025,10 @@ public void channelActive(ChannelHandlerContext ctx) { log.debug(format(ctx.channel(), "Negotiated application-level protocol [" + protocol + "]")); } if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { - configureHttp2Pipeline(ctx.channel().pipeline(), decoder, idleTimeout, http2SettingsSpec, observer); + configureHttp2Pipeline(ctx.channel().pipeline(), decoder, http2Settings, observer); } else if (ApplicationProtocolNames.HTTP_1_1.equals(protocol)) { - configureHttp11Pipeline(ctx.channel().pipeline(), acceptGzip, decoder, idleTimeout, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); + configureHttp11Pipeline(ctx.channel().pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); } else { throw new IllegalStateException("unknown protocol: " + protocol); @@ -1074,8 +1048,7 @@ static final class HttpClientChannelInitializer implements ChannelPipelineConfig final boolean acceptGzip; final HttpResponseDecoderSpec decoder; - final Duration idleTimeout; - final Http2SettingsSpec http2SettingsSpec; + final Http2Settings http2Settings; final ChannelMetricsRecorder metricsRecorder; final ChannelOperations.OnSetup opsFactory; final int protocols; @@ -1086,8 +1059,7 @@ static final class HttpClientChannelInitializer implements ChannelPipelineConfig HttpClientChannelInitializer(HttpClientConfig config) { this.acceptGzip = config.acceptGzip; this.decoder = config.decoder; - this.idleTimeout = config.idleTimeout; - this.http2SettingsSpec = config.http2SettingsSpec(); + this.http2Settings = config.http2Settings(); this.metricsRecorder = config.metricsRecorderInternal(); this.opsFactory = config.channelOperationsProvider(); this.protocols = config._protocols; @@ -1109,24 +1081,24 @@ public void onChannelInit(ConnectionObserver observer, Channel channel, @Nullabl new H2OrHttp11Codec(this, observer, remoteAddress)); } else if ((protocols & h11) == h11) { - configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, idleTimeout, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); + configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h2) == h2) { - configureHttp2Pipeline(channel.pipeline(), decoder, idleTimeout, http2SettingsSpec, observer); + configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, observer); } else if ((protocols & h3) == h3) { - configureHttp3Pipeline(channel.pipeline(), idleTimeout, metricsRecorder != null, proxyAddress != null); + configureHttp3Pipeline(channel.pipeline(), metricsRecorder != null, proxyAddress != null); } } else { if ((protocols & h11orH2C) == h11orH2C) { - configureHttp11OrH2CleartextPipeline(channel.pipeline(), acceptGzip, decoder, idleTimeout, http2SettingsSpec, metricsRecorder, observer, opsFactory, proxyAddress, remoteAddress, uriTagValue); + configureHttp11OrH2CleartextPipeline(channel.pipeline(), acceptGzip, decoder, http2Settings, metricsRecorder, observer, opsFactory, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h11) == h11) { - configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, idleTimeout, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); + configureHttp11Pipeline(channel.pipeline(), acceptGzip, decoder, metricsRecorder, proxyAddress, remoteAddress, uriTagValue); } else if ((protocols & h2c) == h2c) { - configureHttp2Pipeline(channel.pipeline(), decoder, idleTimeout, http2SettingsSpec, observer); + configureHttp2Pipeline(channel.pipeline(), decoder, http2Settings, observer); } } } diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java deleted file mode 100644 index 108ed33153..0000000000 --- a/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpConnectionLivenessTest.java +++ /dev/null @@ -1,792 +0,0 @@ -/* - * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. - * - * 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 reactor.netty.http.client; - -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; -import io.netty.handler.codec.http2.DefaultHttp2PingFrame; -import io.netty.handler.codec.http2.Http2FrameCodec; -import io.netty.handler.codec.http2.Http2FrameCodecBuilder; -import io.netty.handler.codec.http2.Http2FrameWriter; -import io.netty.handler.codec.http2.Http2PingFrame; -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslContextBuilder; -import io.netty.handler.ssl.util.InsecureTrustManagerFactory; -import io.netty.handler.ssl.util.SelfSignedCertificate; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Nested; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; -import reactor.core.publisher.Mono; -import reactor.netty.BaseHttpTest; -import reactor.netty.NettyPipeline; -import reactor.netty.http.HttpConnectionLiveness; -import reactor.netty.resources.ConnectionProvider; -import reactor.util.Logger; -import reactor.util.Loggers; - -import javax.net.ssl.SSLException; -import java.security.cert.CertificateException; -import java.time.Duration; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Collectors; - -import static org.assertj.core.api.Assertions.assertThat; -import static reactor.netty.http.HttpProtocol.H2; -import static reactor.netty.http.HttpProtocol.H2C; -import static reactor.netty.http.HttpProtocol.HTTP11; - -/** - * This test class verifies {@link HttpConnectionLiveness} with client side. - * - * @author raccoonback - * @since 1.2.5 - */ -class HttpConnectionLivenessTest extends BaseHttpTest { - - static final Logger log = Loggers.getLogger(HttpConnectionLivenessTest.class); - - static SslContext sslServer; - static SslContext sslClient; - - @BeforeAll - static void createSelfSignedCertificate() throws CertificateException, SSLException { - SelfSignedCertificate ssc = new SelfSignedCertificate(); - sslServer = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey()) - .build(); - sslClient = SslContextBuilder.forClient() - .trustManager(InsecureTrustManagerFactory.INSTANCE) - .build(); - } - - @Nested - class Http2Test { - - @Test - void successReceiveResponse() { - disposableServer = createServer() - .protocol(H2) - .secure(spec -> spec.sslContext(sslServer)) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - String result = createClient(disposableServer::address) - .protocol(H2) - .secure(spec -> spec.sslContext(sslClient)) - .get() - .uri("/") - .responseSingle((resp, bytes) -> bytes.asString()) - .block(); - - assertThat(result).isEqualTo("Test"); - } - - @Test - void maintainConnectionWithoutPingCheckWhenNotConfigured() { - Http2PingFrameHandler handler = new Http2PingFrameHandler(); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .secure(spec -> spec.sslContext(sslClient)) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).isEmpty(); - } - - @ParameterizedTest - @CsvSource({ - "100,300,3", "300,100,3", - "100,300,3", "300,100,3" - }) - void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) - .doOnNext( - unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) - ) - .subscribe() - ); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .idleTimeout(Duration.ofMillis(300)) - .secure(spec -> spec.sslContext(sslClient)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(3)) - .block(); - - assertThat(channel.parent().isOpen()).isFalse(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); - } - - @ParameterizedTest - @CsvSource({ - "100,300,3", "300,100,3", - "100,300,3", "300,100,3" - }) - void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) - .doOnNext( - unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) - ) - .subscribe() - ); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - ConnectionProvider pool = ConnectionProvider.create("closeConnectionInPoolIfPingFrameDelayed", 1); - - Channel channel = createClient(pool, disposableServer::address) - .protocol(H2) - .idleTimeout(Duration.ofMillis(300)) - .secure(spec -> spec.sslContext(sslClient)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(4)) - .block(); - - assertThat(channel.parent().isOpen()).isFalse(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); - - pool.dispose(); - } - - @ParameterizedTest - @CsvSource({ - "300,600,0", "600,300,0", - "300,600,0", "600,300,0" - }) - void ackPingFrameWithinInterval(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler(); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .idleTimeout(Duration.ofMillis(300)) - .secure(spec -> spec.sslContext(sslClient)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); - } - - @ParameterizedTest - @CsvSource({ - "300,600,0", "600,300,0", - "300,600,0", "600,300,0" - }) - void connectionRetentionInPoolOnPingFrameAck(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler(); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - ConnectionProvider pool = ConnectionProvider.create("connectionRetentionInPoolOnPingFrameAck", 1); - - Channel channel = createClient(pool, disposableServer::address) - .protocol(H2) - .idleTimeout(Duration.ofMillis(300)) - .secure(spec -> spec.sslContext(sslClient)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); - - pool.dispose(); - } - - @ParameterizedTest - @CsvSource({ - "300,600,3", "600,300,3", - "300,600,3", "600,300,3" - }) - void ackPingFrameWithinThreshold(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame, receivedPingTimes) -> { - int delayTime = 0; - if (receivedPingTimes.size() % 3 != 0) { - delayTime = 600; - } - - Mono.delay(Duration.ofMillis(delayTime)) - .doOnNext( - unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) - ) - .subscribe(); - } - ); - - disposableServer = createServer() - .protocol(H2) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .idleTimeout(Duration.ofMillis(300)) - .secure(spec -> spec.sslContext(sslClient)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); - } - } - - @Nested - class H2cTest { - - @Test - void successReceiveResponse() { - disposableServer = createServer() - .protocol(H2C) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - String result = createClient(disposableServer::address) - .protocol(H2C) - .get() - .uri("/") - .responseSingle((resp, bytes) -> bytes.asString()) - .block(); - - assertThat(result).isEqualTo("Test"); - } - - @Test - void maintainConnectionWithoutPingCheckWhenNotConfigured() { - Http2PingFrameHandler handler = new Http2PingFrameHandler(); - - disposableServer = createServer() - .protocol(H2C) - .maxKeepAliveRequests(1) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2C) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).isEmpty(); - } - - @ParameterizedTest - @CsvSource({ - "100,300,3", "300,100,3", - "100,300,3", "300,100,3" - }) - void closeConnectionIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) - .doOnNext( - unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) - ) - .subscribe() - ); - - disposableServer = createServer() - .protocol(H2C) - .maxKeepAliveRequests(1) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2C) - .idleTimeout(Duration.ofMillis(300)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(3)) - .block(); - - assertThat(channel.parent().isOpen()).isFalse(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); - } - - @ParameterizedTest - @CsvSource({ - "100,300,3", "300,100,3", - "100,300,3", "300,100,3" - }) - void closeConnectionInPoolIfPingFrameDelayed(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame, receivedPingTimes) -> Mono.delay(Duration.ofMillis(600)) - .doOnNext( - unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) - ) - .subscribe() - ); - - disposableServer = createServer() - .protocol(H2C) - .maxKeepAliveRequests(1) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - ConnectionProvider pool = ConnectionProvider.create("closeConnectionInPoolIfPingFrameDelayed", 1); - - Channel channel = createClient(pool, disposableServer::address) - .protocol(H2C) - .idleTimeout(Duration.ofMillis(300)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(4)) - .block(); - - assertThat(channel.parent().isOpen()).isFalse(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThan(pingAckDropThreshold); - - pool.dispose(); - } - - @ParameterizedTest - @CsvSource({ - "300,600,0", "600,300,0", - "300,600,0", "600,300,0" - }) - void ackPingFrameWithinInterval(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler(); - - disposableServer = createServer() - .protocol(H2C) - .maxKeepAliveRequests(1) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2C) - .idleTimeout(Duration.ofMillis(300)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); - } - - @ParameterizedTest - @CsvSource({ - "300,600,0", "600,300,0", - "300,600,0", "600,300,0" - }) - void connectionRetentionInPoolOnPingFrameAck(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler(); - - disposableServer = createServer() - .protocol(H2C) - .maxKeepAliveRequests(1) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - ConnectionProvider pool = ConnectionProvider.create("connectionRetentionInPoolOnPingFrameAck", 1); - - Channel channel = createClient(pool, disposableServer::address) - .protocol(H2C) - .idleTimeout(Duration.ofMillis(300)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); - - pool.dispose(); - } - - @ParameterizedTest - @CsvSource({ - "300,600,3", "600,300,3", - "300,600,3", "600,300,3" - }) - void ackPingFrameWithinThreshold(Integer pingAckTimeout, Integer pingScheduleInterval, Integer pingAckDropThreshold) { - Http2PingFrameHandler handler = new Http2PingFrameHandler( - (ctx, frame, receivedPingTimes) -> { - int delayTime = 0; - if (receivedPingTimes.size() % 3 != 0) { - delayTime = 600; - } - - Mono.delay(Duration.ofMillis(delayTime)) - .doOnNext( - unUsed -> ctx.writeAndFlush(new DefaultHttp2PingFrame(frame.content(), true)) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE) - ) - .subscribe(); - } - ); - - disposableServer = createServer() - .protocol(H2C) - .maxKeepAliveRequests(1) - .doOnChannelInit((connectionObserver, channel, remoteAddress) -> { - Http2FrameCodec http2FrameCodec = Http2FrameCodecBuilder.forServer() - .autoAckPingFrame(false) - .autoAckSettingsFrame(true) - .build(); - - channel.pipeline().replace(NettyPipeline.HttpCodec, NettyPipeline.HttpCodec, http2FrameCodec); - channel.pipeline().addLast(handler); - }) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2C) - .idleTimeout(Duration.ofMillis(300)) - .http2Settings(builder -> { - builder.pingAckTimeout(Duration.ofMillis(pingAckTimeout)) - .pingScheduleInterval(Duration.ofMillis(pingScheduleInterval)) - .pingAckDropThreshold(pingAckDropThreshold); - }) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(5)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - assertThat(handler.getReceivedPingTimes()).hasSizeGreaterThanOrEqualTo(2); - } - } - - @Nested - class Http11Test { - - @Test - void closeWithoutDelay() { - disposableServer = createServer() - .protocol(HTTP11) - .maxKeepAliveRequests(1) - .secure(spec -> spec.sslContext(sslServer)) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(HTTP11) - .idleTimeout(Duration.ofMillis(100)) - .secure(spec -> spec.sslContext(sslClient)) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(1)) - .block(); - - assertThat(channel.isOpen()).isFalse(); - } - } - - private static final class Http2PingFrameHandler extends SimpleChannelInboundHandler { - - private final List receivedPingTimes = new ArrayList<>(); - - private final TriConsumer> consumer; - - private Http2PingFrameHandler() { - this.consumer = (ctx, frame, receivedPings) -> { - Http2FrameCodec channelHandler = ctx.pipeline().get(Http2FrameCodec.class); - Http2FrameWriter http2FrameWriter = channelHandler.encoder() - .frameWriter(); - - http2FrameWriter.writePing(ctx, true, frame.content(), ctx.newPromise()) - .addListener((ChannelFuture future) -> { - if (future.isSuccess()) { - log.debug("[Http2PingFrameHandler] Wrote PING frame to {} channel.", future.channel()); - } - else { - log.debug("[Http2PingFrameHandler] Failed to wrote PING frame to {} channel.", future.channel()); - } - }); - }; - } - - private Http2PingFrameHandler(TriConsumer> consumer) { - this.consumer = consumer; - } - - @Override - protected void channelRead0(ChannelHandlerContext ctx, Http2PingFrame frame) throws InterruptedException { - receivedPingTimes.add(LocalDateTime.now(ZoneId.systemDefault())); - consumer.accept(ctx, frame, receivedPingTimes); - } - - public List getReceivedPingTimes() { - return receivedPingTimes.stream() - .sorted() - .collect(Collectors.toList()); - } - } - - @FunctionalInterface - public interface TriConsumer { - void accept(T t, U u, V v); - } -} diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpIdleTimeoutTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpIdleTimeoutTest.java deleted file mode 100644 index f16254a6db..0000000000 --- a/reactor-netty-http/src/test/java/reactor/netty/http/client/HttpIdleTimeoutTest.java +++ /dev/null @@ -1,252 +0,0 @@ -/* - * Copyright (c) 2025 VMware, Inc. or its affiliates, All Rights Reserved. - * - * 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 reactor.netty.http.client; - -import io.netty.channel.Channel; -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslContextBuilder; -import io.netty.handler.ssl.util.InsecureTrustManagerFactory; -import io.netty.handler.ssl.util.SelfSignedCertificate; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import reactor.core.publisher.Mono; -import reactor.netty.BaseHttpTest; - -import javax.net.ssl.SSLException; -import java.security.cert.CertificateException; -import java.time.Duration; - -import static org.assertj.core.api.Assertions.assertThat; -import static reactor.netty.http.HttpProtocol.H2; -import static reactor.netty.http.HttpProtocol.H2C; -import static reactor.netty.http.HttpProtocol.HTTP11; - -class HttpIdleTimeoutTest extends BaseHttpTest { - - static SslContext sslServer; - static SslContext sslClient; - - @BeforeAll - static void createSelfSignedCertificate() throws CertificateException, SSLException { - SelfSignedCertificate ssc = new SelfSignedCertificate(); - sslServer = SslContextBuilder.forServer(ssc.certificate(), ssc.privateKey()) - .build(); - sslClient = SslContextBuilder.forClient() - .trustManager(InsecureTrustManagerFactory.INSTANCE) - .build(); - } - - @Test - void maintainConnectionWithoutIdleTimeoutInHttp11() { - disposableServer = createServer() - .protocol(HTTP11) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(HTTP11) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(2)) - .block(); - - assertThat(channel.isOpen()).isTrue(); - } - - @Test - void maintainConnectionWithoutIdleTimeoutInH2C() { - disposableServer = createServer() - .protocol(HTTP11, H2C) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(HTTP11, H2C) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(2)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - } - - @Test - void maintainConnectionWithoutIdleTimeoutInHttp2() { - disposableServer = createServer() - .protocol(H2) - .secure(spec -> spec.sslContext(sslServer)) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .secure(spec -> spec.sslContext(sslClient)) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(2)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - } - - @Test - void idleTimeoutInHttp11() { - disposableServer = createServer() - .protocol(HTTP11) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(HTTP11) - .idleTimeout(Duration.ofSeconds(3)) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(2)) - .block(); - - assertThat(channel.isOpen()).isTrue(); - } - - @Test - void idleTimeoutInH2C() { - disposableServer = createServer() - .protocol(HTTP11, H2C) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(HTTP11, H2C) - .idleTimeout(Duration.ofSeconds(3)) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(2)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - } - - @Test - void idleTimeoutInHttp2() { - disposableServer = createServer() - .protocol(H2) - .secure(spec -> spec.sslContext(sslServer)) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .secure(spec -> spec.sslContext(sslClient)) - .idleTimeout(Duration.ofSeconds(3)) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(2)) - .block(); - - assertThat(channel.parent().isOpen()).isTrue(); - } - - @Test - void closeAfterIdleTimeoutInHttp11() { - disposableServer = createServer() - .protocol(HTTP11) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(HTTP11) - .idleTimeout(Duration.ofSeconds(2)) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(3)) - .block(); - - assertThat(channel.isOpen()).isFalse(); - } - - @Test - void closeAfterIdleTimeoutInH2C() { - disposableServer = createServer() - .protocol(HTTP11, H2C) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(HTTP11, H2C) - .idleTimeout(Duration.ofSeconds(2)) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(3)) - .block(); - - assertThat(channel.parent().isOpen()).isFalse(); - } - - @Test - void closeAfterIdleTimeoutInHttp2() { - disposableServer = createServer() - .protocol(H2) - .secure(spec -> spec.sslContext(sslServer)) - .handle((req, resp) -> resp.sendString(Mono.just("Test"))) - .bindNow(); - - Channel channel = createClient(disposableServer::address) - .protocol(H2) - .secure(spec -> spec.sslContext(sslClient)) - .idleTimeout(Duration.ofSeconds(2)) - .get() - .uri("/") - .responseConnection((conn, receiver) -> Mono.just(receiver.channel())) - .single() - .block(); - - Mono.delay(Duration.ofSeconds(3)) - .block(); - - assertThat(channel.parent().isOpen()).isFalse(); - } -} diff --git a/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java b/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java index 90e4c901d3..95ac1de551 100644 --- a/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java +++ b/reactor-netty-http/src/test/java/reactor/netty/http/server/HttpConnectionLivenessTest.java @@ -752,9 +752,11 @@ void closeWithoutDelay() { .handle((req, resp) -> resp.sendString(Mono.just("Test"))) .bindNow(); - createClient(disposableServer::address) + ConnectionProvider provider = ConnectionProvider.builder("closeWithoutDelay") + .maxIdleTime(Duration.ofMillis(100)) + .build(); + createClient(provider, disposableServer::address) .protocol(HTTP11) - .idleTimeout(Duration.ofMillis(100)) .secure(spec -> spec.sslContext(sslClient)) .get() .uri("/") From e9ed832e5837565bb1e07d9e29be098d03859353 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Fri, 25 Apr 2025 01:27:27 +0900 Subject: [PATCH 23/27] Cancel HTTP health check scheduler when IdleTimeoutHandler is removed from pipeline Signed-off-by: raccoonback --- .../main/java/reactor/netty/http/IdleTimeoutHandler.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java index 48c6cee14d..367631ce83 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java @@ -84,6 +84,13 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { super.channelInactive(ctx); } + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + httpConnectionLiveness.cancel(); + + super.handlerRemoved(ctx); + } + /** * Adds an idle timeout handler to the server pipeline. * From 90124bc6036962aacdc98592356e8c5d714bf4af Mon Sep 17 00:00:00 2001 From: raccoonback Date: Tue, 29 Apr 2025 22:56:53 +0900 Subject: [PATCH 24/27] Enable IdleTimeoutHandler only during periods without active HTTP message exchange Signed-off-by: raccoonback --- .../netty/http/server/HttpTrafficHandler.java | 66 +++++++++++++++---- 1 file changed, 52 insertions(+), 14 deletions(-) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java index 93f2d8543d..2f714d4ddf 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java @@ -15,17 +15,17 @@ */ package reactor.netty.http.server; -import java.net.SocketAddress; -import java.time.Duration; -import java.time.ZonedDateTime; -import java.util.Optional; -import java.util.Queue; -import java.util.function.BiFunction; -import java.util.function.BiPredicate; +import static io.netty.handler.codec.http.HttpUtil.isContentLengthSet; +import static io.netty.handler.codec.http.HttpUtil.isKeepAlive; +import static io.netty.handler.codec.http.HttpUtil.isTransferEncodingChunked; +import static io.netty.handler.codec.http.HttpUtil.setKeepAlive; +import static io.netty.handler.codec.http.LastHttpContent.EMPTY_LAST_CONTENT; +import static reactor.netty.ReactorNetty.format; import io.netty.channel.ChannelDuplexHandler; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPipeline; import io.netty.channel.ChannelPromise; import io.netty.handler.codec.DecoderResult; import io.netty.handler.codec.DecoderResultProvider; @@ -43,28 +43,32 @@ import io.netty.handler.codec.http.LastHttpContent; import io.netty.handler.codec.http.cookie.ServerCookieDecoder; import io.netty.handler.codec.http.cookie.ServerCookieEncoder; +import io.netty.handler.codec.http2.Http2FrameCodec; import io.netty.handler.ssl.SslHandler; import io.netty.util.ReferenceCountUtil; +import java.net.SocketAddress; +import java.time.Duration; +import java.time.ZonedDateTime; +import java.util.Optional; +import java.util.Queue; +import java.util.function.BiFunction; +import java.util.function.BiPredicate; import reactor.core.Exceptions; import reactor.core.publisher.Mono; import reactor.netty.Connection; import reactor.netty.ConnectionObserver; import reactor.netty.ReactorNetty; import reactor.netty.channel.ChannelOperations; +import reactor.netty.http.Http2ConnectionLiveness; import reactor.netty.http.Http2SettingsSpec; +import reactor.netty.http.HttpConnectionImmediateClose; +import reactor.netty.http.IdleTimeoutHandler; import reactor.netty.http.logging.HttpMessageArgProviderFactory; import reactor.netty.http.logging.HttpMessageLogFactory; import reactor.netty.http.server.compression.HttpCompressionOptionsSpec; import reactor.util.annotation.Nullable; import reactor.util.concurrent.Queues; -import static io.netty.handler.codec.http.HttpUtil.isContentLengthSet; -import static io.netty.handler.codec.http.HttpUtil.isKeepAlive; -import static io.netty.handler.codec.http.HttpUtil.isTransferEncodingChunked; -import static io.netty.handler.codec.http.HttpUtil.setKeepAlive; -import static io.netty.handler.codec.http.LastHttpContent.EMPTY_LAST_CONTENT; -import static reactor.netty.ReactorNetty.format; - /** * Replace {@link io.netty.handler.codec.http.HttpServerKeepAliveHandler} with extra * handler management. @@ -157,6 +161,12 @@ public void handlerAdded(ChannelHandlerContext ctx) throws Exception { ctx.read(); } + @Override + public void channelActive(ChannelHandlerContext ctx) { + setupIdleTimeoutHandler(ctx.pipeline()); + ctx.fireChannelActive(); + } + @Override public void channelRead(ChannelHandlerContext ctx, Object msg) { read = true; @@ -172,6 +182,10 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { if (msg instanceof HttpRequest) { finalizingResponse = false; + if (idleTimeout != null) { + IdleTimeoutHandler.removeIdleTimeoutHandler(ctx.pipeline()); + } + final HttpRequest request = (HttpRequest) msg; if (H2.equals(request.protocolVersion())) { @@ -539,6 +553,7 @@ void handleLastHttpContent(Object msg, ChannelPromise promise) { ctx.executor().execute(this); } else { + setupIdleTimeoutHandler(ctx.pipeline()); ctx.read(); } } @@ -682,6 +697,29 @@ static boolean isMultipart(HttpResponse response) { MULTIPART_PREFIX.length()); } + private void setupIdleTimeoutHandler(ChannelPipeline pipeline) { + Http2FrameCodec httpCodec = pipeline.get(Http2FrameCodec.class); + if(httpCodec != null) { + IdleTimeoutHandler.addIdleTimeoutServerHandler( + pipeline, + idleTimeout, + new Http2ConnectionLiveness( + httpCodec, + http2SettingsSpec != null ? http2SettingsSpec.pingAckTimeout() : null, + http2SettingsSpec != null ? http2SettingsSpec.pingScheduleInterval() : null, + http2SettingsSpec != null ? http2SettingsSpec.pingAckDropThreshold() : null + ) + ); + return; + } + + IdleTimeoutHandler.addIdleTimeoutServerHandler( + pipeline, + idleTimeout, + new HttpConnectionImmediateClose() + ); + } + static final class HttpRequestHolder { final HttpRequest request; final ZonedDateTime timestamp; From fa6b4c305f1aee45d3844cabe5fc3bdad791f8c2 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Tue, 29 Apr 2025 23:22:24 +0900 Subject: [PATCH 25/27] Fix native config data about channel handler test Signed-off-by: raccoonback --- .../reactor-netty-http/reflect-config.json | 7 ------- 1 file changed, 7 deletions(-) diff --git a/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json b/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json index c77b5b4d9b..b4a26c7a4e 100644 --- a/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json +++ b/reactor-netty-http/src/main/resources/META-INF/native-image/io.projectreactor.netty/reactor-netty-http/reflect-config.json @@ -341,12 +341,5 @@ }, "name": "reactor.netty.http.server.logging.error.BaseErrorLogHandler", "queryAllPublicMethods": true - }, - { - "condition": { - "typeReachable": "reactor.netty.http.client.Http2ConnectionLivenessHandler" - }, - "name": "reactor.netty.http.client.Http2ConnectionLivenessHandler", - "queryAllPublicMethods": true } ] \ No newline at end of file From 2aea90fbf1bdb2c7167ae841143adc2d8c2a6d99 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Tue, 29 Apr 2025 23:23:49 +0900 Subject: [PATCH 26/27] addIdleTimeoutServerHandler -> addIdleTimeoutHandler Signed-off-by: raccoonback --- .../java/reactor/netty/http/IdleTimeoutHandler.java | 2 +- .../reactor/netty/http/server/HttpServerConfig.java | 10 +++++----- .../reactor/netty/http/server/HttpTrafficHandler.java | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java index 367631ce83..e2128582b3 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/IdleTimeoutHandler.java @@ -98,7 +98,7 @@ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { * @param idleTimeout the idle timeout duration * @param httpConnectionLiveness the HTTP connection liveness checker */ - public static void addIdleTimeoutServerHandler(ChannelPipeline pipeline, @Nullable Duration idleTimeout, + public static void addIdleTimeoutHandler(ChannelPipeline pipeline, @Nullable Duration idleTimeout, HttpConnectionLiveness httpConnectionLiveness) { if (idleTimeout != null && pipeline.get(NettyPipeline.IdleTimeoutHandler) == null) { String baseName = null; diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java index 1b08178718..19923e765c 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpServerConfig.java @@ -658,7 +658,7 @@ static void configureHttp3Pipeline( p.remove(NettyPipeline.ChannelMetricsHandler); } - IdleTimeoutHandler.addIdleTimeoutServerHandler( + IdleTimeoutHandler.addIdleTimeoutHandler( p, idleTimeout, new HttpConnectionImmediateClose() @@ -724,7 +724,7 @@ static void configureH2Pipeline(ChannelPipeline p, cookieDecoder, cookieEncoder, formDecoderProvider, forwardedHeaderHandler, httpMessageLogFactory, listener, mapHandle, methodTagValue, metricsRecorder, minCompressionSize, opsFactory, readTimeout, requestTimeout, uriTagValue))); - IdleTimeoutHandler.addIdleTimeoutServerHandler( + IdleTimeoutHandler.addIdleTimeoutHandler( p, idleTimeout, new Http2ConnectionLiveness( @@ -840,7 +840,7 @@ else if (metricsRecorder instanceof ContextAwareHttpServerMetricsRecorder) { p.addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.ErrorLogHandler, new DefaultErrorLogHandler(errorLog)); } - IdleTimeoutHandler.addIdleTimeoutServerHandler( + IdleTimeoutHandler.addIdleTimeoutHandler( p, idleTimeout, new HttpConnectionImmediateClose() @@ -927,7 +927,7 @@ else if (metricsRecorder instanceof ContextAwareHttpServerMetricsRecorder) { p.addBefore(NettyPipeline.ReactiveBridge, NettyPipeline.ErrorLogHandler, new DefaultErrorLogHandler(errorLog)); } - IdleTimeoutHandler.addIdleTimeoutServerHandler( + IdleTimeoutHandler.addIdleTimeoutHandler( p, idleTimeout, new HttpConnectionImmediateClose() @@ -1082,7 +1082,7 @@ public void handlerAdded(ChannelHandlerContext ctx) { if (idleTimeout != null) { IdleTimeoutHandler.removeIdleTimeoutHandler(pipeline); - IdleTimeoutHandler.addIdleTimeoutServerHandler( + IdleTimeoutHandler.addIdleTimeoutHandler( pipeline, idleTimeout, new Http2ConnectionLiveness( diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java index 2f714d4ddf..785a521615 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java @@ -700,7 +700,7 @@ static boolean isMultipart(HttpResponse response) { private void setupIdleTimeoutHandler(ChannelPipeline pipeline) { Http2FrameCodec httpCodec = pipeline.get(Http2FrameCodec.class); if(httpCodec != null) { - IdleTimeoutHandler.addIdleTimeoutServerHandler( + IdleTimeoutHandler.addIdleTimeoutHandler( pipeline, idleTimeout, new Http2ConnectionLiveness( @@ -713,7 +713,7 @@ private void setupIdleTimeoutHandler(ChannelPipeline pipeline) { return; } - IdleTimeoutHandler.addIdleTimeoutServerHandler( + IdleTimeoutHandler.addIdleTimeoutHandler( pipeline, idleTimeout, new HttpConnectionImmediateClose() From d446748074a52966ff285faf414ba6a404de0e51 Mon Sep 17 00:00:00 2001 From: raccoonback Date: Tue, 29 Apr 2025 23:41:17 +0900 Subject: [PATCH 27/27] Fix checkstyle Signed-off-by: raccoonback --- .../src/main/java/reactor/netty/NettyPipeline.java | 2 +- .../main/java/reactor/netty/http/server/HttpTrafficHandler.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java b/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java index 866d007a74..380c66835d 100644 --- a/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java +++ b/reactor-netty-core/src/main/java/reactor/netty/NettyPipeline.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2011-2024 VMware, Inc. or its affiliates, All Rights Reserved. + * Copyright (c) 2011-2025 VMware, Inc. or its affiliates, All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java index 785a521615..82ed034ac8 100644 --- a/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java +++ b/reactor-netty-http/src/main/java/reactor/netty/http/server/HttpTrafficHandler.java @@ -699,7 +699,7 @@ static boolean isMultipart(HttpResponse response) { private void setupIdleTimeoutHandler(ChannelPipeline pipeline) { Http2FrameCodec httpCodec = pipeline.get(Http2FrameCodec.class); - if(httpCodec != null) { + if (httpCodec != null) { IdleTimeoutHandler.addIdleTimeoutHandler( pipeline, idleTimeout,