From 804d8434dc7026d8ae9df1a3a2431fd3034c2594 Mon Sep 17 00:00:00 2001 From: Scott Mitchell Date: Tue, 3 Jul 2018 19:51:16 -0700 Subject: [PATCH] HTTP/2 goaway connection state update sequencing (#8080) Motivation: The Http2Connection state is updated by the DefaultHttp2ConnectionDecoder after the frame listener is notified of the goaway frame. If the listener sends a frame synchronously this means the connection state will not know about the goaway it just received and we may send frames that are not allowed on the connection. This may also mean a stream object is created but it may never get taken out of the stream map unless some other event occurs (e.g. timeout). Modifications: - The Http2Connection state should be updated before the listener is notified of the goaway - The Http2Connection state modification and validation should be self contained when processing a goaway instead of partially in the decoder. Result: No more creating streams and sending frames after a goaway has been sent or received. --- .../codec/http2/DefaultHttp2Connection.java | 69 ++++++++-------- .../http2/DefaultHttp2ConnectionDecoder.java | 12 +-- .../handler/codec/http2/Http2Connection.java | 21 ++++- .../codec/http2/Http2ConnectionHandler.java | 64 +++++++-------- .../DefaultHttp2ConnectionDecoderTest.java | 7 -- .../DefaultHttp2ConnectionEncoderTest.java | 6 +- .../http2/DefaultHttp2ConnectionTest.java | 22 +++++- .../http2/Http2ConnectionHandlerTest.java | 7 ++ .../http2/Http2ConnectionRoundtripTest.java | 78 +++++++++++++++++++ .../http2/StreamBufferingEncoderTest.java | 4 +- 10 files changed, 200 insertions(+), 90 deletions(-) diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/DefaultHttp2Connection.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/DefaultHttp2Connection.java index 8dd0a31cd2..8590e06fa2 100644 --- a/codec-http2/src/main/java/io/netty/handler/codec/http2/DefaultHttp2Connection.java +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/DefaultHttp2Connection.java @@ -25,7 +25,6 @@ import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; import io.netty.util.concurrent.UnaryPromiseNotifier; import io.netty.util.internal.EmptyArrays; -import io.netty.util.internal.PlatformDependent; import io.netty.util.internal.UnstableApi; import io.netty.util.internal.logging.InternalLogger; import io.netty.util.internal.logging.InternalLoggerFactory; @@ -225,7 +224,12 @@ public class DefaultHttp2Connection implements Http2Connection { } @Override - public void goAwayReceived(final int lastKnownStream, long errorCode, ByteBuf debugData) { + public void goAwayReceived(final int lastKnownStream, long errorCode, ByteBuf debugData) throws Http2Exception { + if (localEndpoint.lastStreamKnownByPeer() >= 0 && localEndpoint.lastStreamKnownByPeer() < lastKnownStream) { + throw connectionError(PROTOCOL_ERROR, "lastStreamId MUST NOT increase. Current value: %d new value: %d", + localEndpoint.lastStreamKnownByPeer(), lastKnownStream); + } + localEndpoint.lastStreamKnownByPeer(lastKnownStream); for (int i = 0; i < listeners.size(); ++i) { try { @@ -235,19 +239,7 @@ public class DefaultHttp2Connection implements Http2Connection { } } - try { - forEachActiveStream(new Http2StreamVisitor() { - @Override - public boolean visit(Http2Stream stream) { - if (stream.id() > lastKnownStream && localEndpoint.isValidStreamId(stream.id())) { - stream.close(); - } - return true; - } - }); - } catch (Http2Exception e) { - PlatformDependent.throwException(e); - } + closeStreamsGreaterThanLastKnownStreamId(lastKnownStream, localEndpoint); } @Override @@ -256,7 +248,20 @@ public class DefaultHttp2Connection implements Http2Connection { } @Override - public void goAwaySent(final int lastKnownStream, long errorCode, ByteBuf debugData) { + public boolean goAwaySent(final int lastKnownStream, long errorCode, ByteBuf debugData) throws Http2Exception { + if (remoteEndpoint.lastStreamKnownByPeer() >= 0) { + // Protect against re-entrancy. Could happen if writing the frame fails, and error handling + // treating this is a connection handler and doing a graceful shutdown... + if (lastKnownStream == remoteEndpoint.lastStreamKnownByPeer()) { + return false; + } + if (lastKnownStream > remoteEndpoint.lastStreamKnownByPeer()) { + throw connectionError(PROTOCOL_ERROR, "Last stream identifier must not increase between " + + "sending multiple GOAWAY frames (was '%d', is '%d').", + remoteEndpoint.lastStreamKnownByPeer(), lastKnownStream); + } + } + remoteEndpoint.lastStreamKnownByPeer(lastKnownStream); for (int i = 0; i < listeners.size(); ++i) { try { @@ -266,19 +271,21 @@ public class DefaultHttp2Connection implements Http2Connection { } } - try { - forEachActiveStream(new Http2StreamVisitor() { - @Override - public boolean visit(Http2Stream stream) { - if (stream.id() > lastKnownStream && remoteEndpoint.isValidStreamId(stream.id())) { - stream.close(); - } - return true; + closeStreamsGreaterThanLastKnownStreamId(lastKnownStream, remoteEndpoint); + return true; + } + + private void closeStreamsGreaterThanLastKnownStreamId(final int lastKnownStream, + final DefaultEndpoint endpoint) throws Http2Exception { + forEachActiveStream(new Http2StreamVisitor() { + @Override + public boolean visit(Http2Stream stream) { + if (stream.id() > lastKnownStream && endpoint.isValidStreamId(stream.id())) { + stream.close(); } - }); - } catch (Http2Exception e) { - PlatformDependent.throwException(e); - } + return true; + } + }); } /** @@ -863,10 +870,10 @@ public class DefaultHttp2Connection implements Http2Connection { private void checkNewStreamAllowed(int streamId, State state) throws Http2Exception { assert state != IDLE; - if (goAwayReceived() && streamId > localEndpoint.lastStreamKnownByPeer()) { + if (lastStreamKnownByPeer >= 0 && streamId > lastStreamKnownByPeer) { throw streamError(streamId, REFUSED_STREAM, - "Cannot create stream %d since this endpoint has received a GOAWAY frame with last stream id %d.", - streamId, localEndpoint.lastStreamKnownByPeer()); + "Cannot create stream %d greater than Last-Stream-ID %d from GOAWAY.", + streamId, lastStreamKnownByPeer); } if (!isValidStreamId(streamId)) { if (streamId < 0) { diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionDecoder.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionDecoder.java index 49335e95e7..9114f05ff0 100644 --- a/codec-http2/src/main/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionDecoder.java +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionDecoder.java @@ -158,12 +158,8 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder { void onGoAwayRead0(ChannelHandlerContext ctx, int lastStreamId, long errorCode, ByteBuf debugData) throws Http2Exception { - if (connection.goAwayReceived() && connection.local().lastStreamKnownByPeer() < lastStreamId) { - throw connectionError(PROTOCOL_ERROR, "lastStreamId MUST NOT increase. Current value: %d new value: %d", - connection.local().lastStreamKnownByPeer(), lastStreamId); - } - listener.onGoAwayRead(ctx, lastStreamId, errorCode, debugData); connection.goAwayReceived(lastStreamId, errorCode, debugData); + listener.onGoAwayRead(ctx, lastStreamId, errorCode, debugData); } void onUnknownFrame0(ChannelHandlerContext ctx, byte frameType, int streamId, Http2Flags flags, @@ -535,12 +531,18 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder { throw streamError(streamId, STREAM_CLOSED, "Received %s frame for an unknown stream %d", frameName, streamId); } else if (stream.isResetSent() || streamCreatedAfterGoAwaySent(streamId)) { + // If we have sent a reset stream it is assumed the stream will be closed after the write completes. + // If we have not sent a reset, but the stream was created after a GoAway this is not supported by + // DefaultHttp2Connection and if a custom Http2Connection is used it is assumed the lifetime is managed + // elsewhere so we don't close the stream or otherwise modify the stream's state. + if (logger.isInfoEnabled()) { logger.info("{} ignoring {} frame for stream {} {}", ctx.channel(), frameName, stream.isResetSent() ? "RST_STREAM sent." : ("Stream created after GOAWAY sent. Last known stream by peer " + connection.remote().lastStreamKnownByPeer())); } + return true; } return false; diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2Connection.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2Connection.java index f43820bf23..7639e08406 100644 --- a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2Connection.java +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2Connection.java @@ -326,8 +326,15 @@ public interface Http2Connection { /** * Indicates that a {@code GOAWAY} was received from the remote endpoint and sets the last known stream. + * @param lastKnownStream The Last-Stream-ID in the + * GOAWAY frame. + * @param errorCode the Error Code in the + * GOAWAY frame. + * @param message The Additional Debug Data in the + * GOAWAY frame. Note that reference count ownership + * belongs to the caller (ownership is not transferred to this method). */ - void goAwayReceived(int lastKnownStream, long errorCode, ByteBuf message); + void goAwayReceived(int lastKnownStream, long errorCode, ByteBuf message) throws Http2Exception; /** * Indicates whether or not a {@code GOAWAY} was sent to the remote endpoint. @@ -335,7 +342,15 @@ public interface Http2Connection { boolean goAwaySent(); /** - * Indicates that a {@code GOAWAY} was sent to the remote endpoint and sets the last known stream. + * Updates the local state of this {@link Http2Connection} as a result of a {@code GOAWAY} to send to the remote + * endpoint. + * @param lastKnownStream The Last-Stream-ID in the + * GOAWAY frame. + * @param errorCode the Error Code in the + * GOAWAY frame. + * GOAWAY frame. Note that reference count ownership + * belongs to the caller (ownership is not transferred to this method). + * @return {@code true} if the corresponding {@code GOAWAY} frame should be sent to the remote endpoint. */ - void goAwaySent(int lastKnownStream, long errorCode, ByteBuf message); + boolean goAwaySent(int lastKnownStream, long errorCode, ByteBuf message) throws Http2Exception; } diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2ConnectionHandler.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2ConnectionHandler.java index e44149de5b..3151f1ccb2 100644 --- a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2ConnectionHandler.java +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2ConnectionHandler.java @@ -794,47 +794,37 @@ public class Http2ConnectionHandler extends ByteToMessageDecoder implements Http @Override public ChannelFuture goAway(final ChannelHandlerContext ctx, final int lastStreamId, final long errorCode, final ByteBuf debugData, ChannelPromise promise) { + promise = promise.unvoid(); + final Http2Connection connection = connection(); try { - promise = promise.unvoid(); - final Http2Connection connection = connection(); - if (connection().goAwaySent()) { - // Protect against re-entrancy. Could happen if writing the frame fails, and error handling - // treating this is a connection handler and doing a graceful shutdown... - if (lastStreamId == connection().remote().lastStreamKnownByPeer()) { - // Release the data and notify the promise - debugData.release(); - return promise.setSuccess(); - } - if (lastStreamId > connection.remote().lastStreamKnownByPeer()) { - throw connectionError(PROTOCOL_ERROR, "Last stream identifier must not increase between " + - "sending multiple GOAWAY frames (was '%d', is '%d').", - connection.remote().lastStreamKnownByPeer(), lastStreamId); - } + if (!connection.goAwaySent(lastStreamId, errorCode, debugData)) { + debugData.release(); + promise.trySuccess(); + return promise; } - - connection.goAwaySent(lastStreamId, errorCode, debugData); - - // Need to retain before we write the buffer because if we do it after the refCnt could already be 0 and - // result in an IllegalRefCountException. - debugData.retain(); - ChannelFuture future = frameWriter().writeGoAway(ctx, lastStreamId, errorCode, debugData, promise); - - if (future.isDone()) { - processGoAwayWriteResult(ctx, lastStreamId, errorCode, debugData, future); - } else { - future.addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - processGoAwayWriteResult(ctx, lastStreamId, errorCode, debugData, future); - } - }); - } - - return future; - } catch (Throwable cause) { // Make sure to catch Throwable because we are doing a retain() in this method. + } catch (Throwable cause) { debugData.release(); - return promise.setFailure(cause); + promise.tryFailure(cause); + return promise; } + + // Need to retain before we write the buffer because if we do it after the refCnt could already be 0 and + // result in an IllegalRefCountException. + debugData.retain(); + ChannelFuture future = frameWriter().writeGoAway(ctx, lastStreamId, errorCode, debugData, promise); + + if (future.isDone()) { + processGoAwayWriteResult(ctx, lastStreamId, errorCode, debugData, future); + } else { + future.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + processGoAwayWriteResult(ctx, lastStreamId, errorCode, debugData, future); + } + }); + } + + return future; } /** diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionDecoderTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionDecoderTest.java index 8d71ddeaff..7e87d52893 100644 --- a/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionDecoderTest.java +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionDecoderTest.java @@ -674,13 +674,6 @@ public class DefaultHttp2ConnectionDecoderTest { verify(listener).onRstStreamRead(eq(ctx), anyInt(), anyLong()); } - @Test(expected = Http2Exception.class) - public void goawayIncreasedLastStreamIdShouldThrow() throws Exception { - when(local.lastStreamKnownByPeer()).thenReturn(1); - when(connection.goAwayReceived()).thenReturn(true); - decode().onGoAwayRead(ctx, 3, 2L, EMPTY_BUFFER); - } - @Test(expected = Http2Exception.class) public void rstStreamReadForUnknownStreamShouldThrow() throws Exception { when(connection.streamMayHaveExisted(STREAM_ID)).thenReturn(false); diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionEncoderTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionEncoderTest.java index 5f9a6b1871..cdbfaa4ebd 100644 --- a/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionEncoderTest.java +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionEncoderTest.java @@ -770,7 +770,7 @@ public class DefaultHttp2ConnectionEncoderTest { } @Test - public void canWriteHeaderFrameAfterGoAwayReceived() { + public void canWriteHeaderFrameAfterGoAwayReceived() throws Http2Exception { writeAllFlowControlledFrames(); goAwayReceived(STREAM_ID); ChannelPromise promise = newPromise(); @@ -803,11 +803,11 @@ public class DefaultHttp2ConnectionEncoderTest { return connection.stream(streamId); } - private void goAwayReceived(int lastStreamId) { + private void goAwayReceived(int lastStreamId) throws Http2Exception { connection.goAwayReceived(lastStreamId, 0, EMPTY_BUFFER); } - private void goAwaySent(int lastStreamId) { + private void goAwaySent(int lastStreamId) throws Http2Exception { connection.goAwaySent(lastStreamId, 0, EMPTY_BUFFER); } diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionTest.java index ef385beed6..69183e0843 100644 --- a/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionTest.java +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/DefaultHttp2ConnectionTest.java @@ -47,8 +47,8 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.anyLong; -import static org.mockito.Mockito.eq; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; @@ -423,11 +423,29 @@ public class DefaultHttp2ConnectionTest { } @Test(expected = Http2Exception.class) - public void goAwayReceivedShouldDisallowCreation() throws Http2Exception { + public void goAwayReceivedShouldDisallowLocalCreation() throws Http2Exception { + server.goAwayReceived(0, 1L, Unpooled.EMPTY_BUFFER); + server.local().createStream(3, true); + } + + @Test + public void goAwayReceivedShouldAllowRemoteCreation() throws Http2Exception { server.goAwayReceived(0, 1L, Unpooled.EMPTY_BUFFER); server.remote().createStream(3, true); } + @Test(expected = Http2Exception.class) + public void goAwaySentShouldDisallowRemoteCreation() throws Http2Exception { + server.goAwaySent(0, 1L, Unpooled.EMPTY_BUFFER); + server.remote().createStream(2, true); + } + + @Test + public void goAwaySentShouldAllowLocalCreation() throws Http2Exception { + server.goAwaySent(0, 1L, Unpooled.EMPTY_BUFFER); + server.local().createStream(2, true); + } + @Test public void closeShouldSucceed() throws Http2Exception { Http2Stream stream = server.remote().createStream(3, true); diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ConnectionHandlerTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ConnectionHandlerTest.java index be5c41b0e5..223d6005f4 100644 --- a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ConnectionHandlerTest.java +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ConnectionHandlerTest.java @@ -189,6 +189,7 @@ public class Http2ConnectionHandlerTest { when(connection.stream(NON_EXISTANT_STREAM_ID)).thenReturn(null); when(connection.numActiveStreams()).thenReturn(1); when(connection.stream(STREAM_ID)).thenReturn(stream); + when(connection.goAwaySent(anyInt(), anyLong(), any(ByteBuf.class))).thenReturn(true); when(stream.open(anyBoolean())).thenReturn(stream); when(encoder.writeSettings(eq(ctx), any(Http2Settings.class), eq(promise))).thenReturn(future); when(ctx.alloc()).thenReturn(UnpooledByteBufAllocator.DEFAULT); @@ -638,6 +639,12 @@ public class Http2ConnectionHandlerTest { when(connection.goAwaySent()).thenReturn(true); when(remote.lastStreamKnownByPeer()).thenReturn(STREAM_ID); + doAnswer(new Answer() { + @Override + public Boolean answer(InvocationOnMock invocationOnMock) { + throw new IllegalStateException(); + } + }).when(connection).goAwaySent(anyInt(), anyLong(), any(ByteBuf.class)); handler.goAway(ctx, STREAM_ID + 2, errorCode, data, promise); assertTrue(promise.isDone()); assertFalse(promise.isSuccess()); diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ConnectionRoundtripTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ConnectionRoundtripTest.java index a909b9eb1c..f4a4b6c94b 100644 --- a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ConnectionRoundtripTest.java +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ConnectionRoundtripTest.java @@ -889,6 +889,84 @@ public class Http2ConnectionRoundtripTest { setServerGracefulShutdownTime(0); } + @Test + public void createStreamSynchronouslyAfterGoAwayReceivedShouldFailLocally() throws Exception { + bootstrapEnv(1, 1, 2, 1, 1); + + final CountDownLatch clientGoAwayLatch = new CountDownLatch(1); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocationOnMock) throws Throwable { + clientGoAwayLatch.countDown(); + return null; + } + }).when(clientListener).onGoAwayRead(any(ChannelHandlerContext.class), anyInt(), anyLong(), any(ByteBuf.class)); + + // We want both sides to do graceful shutdown during the test. + setClientGracefulShutdownTime(10000); + setServerGracefulShutdownTime(10000); + + final Http2Headers headers = dummyHeaders(); + final AtomicReference clientWriteAfterGoAwayFutureRef = new AtomicReference(); + final CountDownLatch clientWriteAfterGoAwayLatch = new CountDownLatch(1); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocationOnMock) throws Throwable { + ChannelFuture f = http2Client.encoder().writeHeaders(ctx(), 5, headers, 0, (short) 16, false, 0, + true, newPromise()); + clientWriteAfterGoAwayFutureRef.set(f); + f.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + clientWriteAfterGoAwayLatch.countDown(); + } + }); + http2Client.flush(ctx()); + return null; + } + }).when(clientListener).onGoAwayRead(any(ChannelHandlerContext.class), anyInt(), anyLong(), any(ByteBuf.class)); + + runInChannel(clientChannel, new Http2Runnable() { + @Override + public void run() throws Http2Exception { + http2Client.encoder().writeHeaders(ctx(), 3, headers, 0, (short) 16, false, 0, + true, newPromise()); + http2Client.flush(ctx()); + } + }); + + assertTrue(serverSettingsAckLatch.await(DEFAULT_AWAIT_TIMEOUT_SECONDS, SECONDS)); + + // Server has received the headers, so the stream is open + assertTrue(requestLatch.await(DEFAULT_AWAIT_TIMEOUT_SECONDS, SECONDS)); + + runInChannel(serverChannel, new Http2Runnable() { + @Override + public void run() throws Http2Exception { + http2Server.encoder().writeGoAway(serverCtx(), 3, NO_ERROR.code(), EMPTY_BUFFER, serverNewPromise()); + http2Server.flush(serverCtx()); + } + }); + + // Wait for the client's write operation to complete. + assertTrue(clientWriteAfterGoAwayLatch.await(DEFAULT_AWAIT_TIMEOUT_SECONDS, SECONDS)); + + ChannelFuture clientWriteAfterGoAwayFuture = clientWriteAfterGoAwayFutureRef.get(); + assertNotNull(clientWriteAfterGoAwayFuture); + Throwable clientCause = clientWriteAfterGoAwayFuture.cause(); + assertThat(clientCause, is(instanceOf(Http2Exception.StreamException.class))); + assertEquals(Http2Error.REFUSED_STREAM.code(), ((Http2Exception.StreamException) clientCause).error().code()); + + // Wait for the server to receive a GO_AWAY, but this is expected to timeout! + assertFalse(goAwayLatch.await(1, SECONDS)); + verify(serverListener, never()).onGoAwayRead(any(ChannelHandlerContext.class), anyInt(), anyLong(), + any(ByteBuf.class)); + + // Shutdown shouldn't wait for the server to close streams + setClientGracefulShutdownTime(0); + setServerGracefulShutdownTime(0); + } + @Test public void flowControlProperlyChunksLargeMessage() throws Exception { final Http2Headers headers = dummyHeaders(); diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/StreamBufferingEncoderTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/StreamBufferingEncoderTest.java index a1768cb857..15dc168b4d 100644 --- a/codec-http2/src/test/java/io/netty/handler/codec/http2/StreamBufferingEncoderTest.java +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/StreamBufferingEncoderTest.java @@ -222,7 +222,7 @@ public class StreamBufferingEncoderTest { } @Test - public void bufferingNewStreamFailsAfterGoAwayReceived() { + public void bufferingNewStreamFailsAfterGoAwayReceived() throws Http2Exception { encoder.writeSettingsAck(ctx, newPromise()); setMaxConcurrentStreams(0); connection.goAwayReceived(1, 8, EMPTY_BUFFER); @@ -235,7 +235,7 @@ public class StreamBufferingEncoderTest { } @Test - public void receivingGoAwayFailsBufferedStreams() { + public void receivingGoAwayFailsBufferedStreams() throws Http2Exception { encoder.writeSettingsAck(ctx, newPromise()); setMaxConcurrentStreams(5);