Merging HTTP/2 preface logic into connection handler

Motivation:

The HTTP/2 connection preface logic is currently handled in two places.
Reading/writing the client preface string is handled by
Http2PrefaceHandler while the reading/writing of the initial settings
frame is handled by AbstractHttp2ConnectionHandler. Given that their
isn't much code in Http2PrefaceHandler, it makes sense to just merge it
into the preface handling logic of AbstractHttp2ConnectionHandler. This
will also make configuring the pipeline simpler for HTTP/2.

Modifications:

Removed Http2PrefaceHandler and added it's logic to
AbstractHttp2ConnectionHandler. Updated other classes depending on
Http2PrefaceHandler.

Result:

All of the HTTP/2 connection preface processing logic is now in one
place.
This commit is contained in:
nmittler 2014-05-10 08:11:40 -07:00 committed by Norman Maurer
parent 496bd70e0a
commit 35ab47a5d5
8 changed files with 148 additions and 247 deletions

View File

@ -16,6 +16,7 @@
package io.netty.handler.codec.http2;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.connectionPrefaceBuf;
import static io.netty.handler.codec.http2.Http2CodecUtil.failAndThrow;
import static io.netty.handler.codec.http2.Http2CodecUtil.toByteBuf;
import static io.netty.handler.codec.http2.Http2CodecUtil.toHttp2Exception;
@ -41,17 +42,16 @@ import java.util.Arrays;
import java.util.List;
/**
* Abstract base class for a handler of HTTP/2 frames. Handles reading and writing of HTTP/2
* frames as well as management of connection state and flow control for both inbound and outbound
* data frames.
* Abstract base class for a handler of HTTP/2 frames. Handles reading and writing of HTTP/2 frames
* as well as management of connection state and flow control for both inbound and outbound data
* frames.
* <p>
* Subclasses need to implement the methods defined by the {@link Http2FrameObserver} interface for
* receiving inbound frames. Outbound frames are sent via one of the {@code writeXXX} methods.
* <p>
* It should be noted that the initial SETTINGS frame is sent upon either activation or addition of
* this handler to the pipeline. Subclasses overriding {@link #channelActive} or
* {@link #handlerAdded} must call this class to write the initial SETTINGS frame to the remote
* endpoint.
* It should be noted that the connection preface is sent upon either activation or addition of this
* handler to the pipeline. Subclasses overriding {@link #channelActive} or {@link #handlerAdded}
* must call this class to write the preface to the remote endpoint.
*/
public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecoder implements
Http2FrameObserver {
@ -62,8 +62,9 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
private final Http2Connection connection;
private final Http2InboundFlowController inboundFlow;
private final Http2OutboundFlowController outboundFlow;
private boolean initialSettingsSent;
private boolean initialSettingsReceived;
private ByteBuf clientPrefaceString;
private boolean prefaceSent;
private boolean prefaceReceived;
private ChannelHandlerContext ctx;
private ChannelFutureListener closeListener;
// We prefer ArrayDeque to LinkedList because later will produce more GC.
@ -106,22 +107,31 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
this.frameWriter = frameWriter;
this.inboundFlow = inboundFlow;
this.outboundFlow = outboundFlow;
// Set the expected client preface string. Only servers should receive this.
this.clientPrefaceString = connection.isServer()? connectionPrefaceBuf() : null;
}
@Override
public void channelActive(ChannelHandlerContext ctx) throws Exception {
// The channel just became active - send the initial settings frame to the remote
// The channel just became active - send the connection preface to the remote
// endpoint.
sendInitialSettings(ctx);
sendPreface(ctx);
super.channelActive(ctx);
}
@Override
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
// This handler was just added to the context. In case it was handled after
// the connection became active, send the initial settings frame now.
// the connection became active, send the connection preface now.
this.ctx = ctx;
sendInitialSettings(ctx);
sendPreface(ctx);
}
@Override
protected void handlerRemoved0(ChannelHandlerContext ctx) throws Exception {
// Free any resources associated with this handler.
freeResources();
}
protected final ChannelHandlerContext ctx() {
@ -362,12 +372,55 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
protected final void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out)
throws Exception {
try {
// Read the remaining of the client preface string if we haven't already.
// If this is a client endpoint, always returns true.
if (!readClientPrefaceString(ctx, in)) {
// Still processing the client preface.
return;
}
frameReader.readFrame(ctx, in, internalFrameObserver);
} catch (Http2Exception e) {
processHttp2Exception(ctx, e);
}
}
/**
* Decodes the client connection preface string from the input buffer.
*
* @return {@code true} if processing of the client preface string is complete. Since client
* preface strings can only be received by servers, returns true immediately for client
* endpoints.
*/
private boolean readClientPrefaceString(ChannelHandlerContext ctx, ByteBuf in) throws Http2Exception {
if (clientPrefaceString == null) {
return true;
}
int prefaceRemaining = clientPrefaceString.readableBytes();
int bytesRead = Math.min(in.readableBytes(), prefaceRemaining);
// Read the portion of the input up to the length of the preface, if reached.
ByteBuf sourceSlice = in.readSlice(bytesRead);
// Read the same number of bytes from the preface buffer.
ByteBuf prefaceSlice = clientPrefaceString.readSlice(bytesRead);
// If the input so far doesn't match the preface, break the connection.
if (bytesRead == 0 || !prefaceSlice.equals(sourceSlice)) {
ctx.close();
return false;
}
if (!clientPrefaceString.isReadable()) {
// Entire preface has been read.
clientPrefaceString.release();
clientPrefaceString = null;
return true;
}
return false;
}
/**
* Processes the given exception. Depending on the type of exception, delegates to either
* {@link #processConnectionError} or {@link #processStreamError}.
@ -445,6 +498,10 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
private void freeResources() {
frameReader.close();
frameWriter.close();
if (clientPrefaceString != null) {
clientPrefaceString.release();
clientPrefaceString = null;
}
}
private void closeLocalSide(Http2Stream stream, ChannelHandlerContext ctx, ChannelFuture future) {
@ -487,25 +544,35 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
}
}
private void verifyInitialSettingsReceived() throws Http2Exception {
if (!initialSettingsReceived) {
/**
* Verifies that the HTTP/2 connection preface has been received from the remote endpoint.
*/
private void verifyPrefaceReceived() throws Http2Exception {
if (!prefaceReceived) {
throw protocolError("Received non-SETTINGS as first frame.");
}
}
/**
* Sends the initial settings frame upon establishment of the connection, if not already sent.
* Sends the HTTP/2 connection preface upon establishment of the connection, if not already sent.
*/
private void sendInitialSettings(final ChannelHandlerContext ctx) throws Http2Exception {
if (!initialSettingsSent && ctx.channel().isActive()) {
initialSettingsSent = true;
Http2Settings settings = settings();
outstandingLocalSettingsQueue.add(settings);
frameWriter.writeSettings(ctx, ctx.newPromise(), settings).addListener(
ChannelFutureListener.CLOSE_ON_FAILURE);
private void sendPreface(final ChannelHandlerContext ctx) throws Http2Exception {
if (prefaceSent || !ctx.channel().isActive()) {
return;
}
prefaceSent = true;
if (!connection.isServer()) {
// Clients must send the preface string as the first bytes on the connection.
ctx.write(connectionPrefaceBuf()).addListener(ChannelFutureListener.CLOSE_ON_FAILURE);
}
// Both client and server must send their initial settings.
Http2Settings settings = settings();
outstandingLocalSettingsQueue.add(settings);
frameWriter.writeSettings(ctx, ctx.newPromise(), settings).addListener(
ChannelFutureListener.CLOSE_ON_FAILURE);
}
/**
@ -516,7 +583,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
@Override
public void onDataRead(final ChannelHandlerContext ctx, int streamId, ByteBuf data, int padding,
boolean endOfStream, boolean endOfSegment, boolean compressed) throws Http2Exception {
verifyInitialSettingsReceived();
verifyPrefaceReceived();
if (!connection.local().allowCompressedData() && compressed) {
throw protocolError("compression is disallowed.");
@ -561,7 +628,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
public void onHeadersRead(ChannelHandlerContext ctx, int streamId, Http2Headers headers,
int streamDependency, short weight, boolean exclusive, int padding,
boolean endStream, boolean endSegment) throws Http2Exception {
verifyInitialSettingsReceived();
verifyPrefaceReceived();
if (isInboundStreamAfterGoAway(streamId)) {
return;
@ -613,7 +680,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
@Override
public void onPriorityRead(ChannelHandlerContext ctx, int streamId, int streamDependency,
short weight, boolean exclusive) throws Http2Exception {
verifyInitialSettingsReceived();
verifyPrefaceReceived();
if (isInboundStreamAfterGoAway(streamId)) {
// Ignore frames for any stream created after we sent a go-away.
@ -630,7 +697,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
@Override
public void onRstStreamRead(ChannelHandlerContext ctx, int streamId, long errorCode)
throws Http2Exception {
verifyInitialSettingsReceived();
verifyPrefaceReceived();
if (isInboundStreamAfterGoAway(streamId)) {
// Ignore frames for any stream created after we sent a go-away.
@ -650,7 +717,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
@Override
public void onSettingsAckRead(ChannelHandlerContext ctx) throws Http2Exception {
verifyInitialSettingsReceived();
verifyPrefaceReceived();
// Apply oldest outstanding local settings here. This is a synchronization point
// between endpoints.
Http2Settings settings = outstandingLocalSettingsQueue.poll();
@ -710,14 +777,14 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
frameWriter.writeSettingsAck(ctx, ctx.newPromise());
// We've received at least one non-ack settings frame from the remote endpoint.
initialSettingsReceived = true;
prefaceReceived = true;
AbstractHttp2ConnectionHandler.this.onSettingsRead(ctx, settings);
}
@Override
public void onPingRead(ChannelHandlerContext ctx, ByteBuf data) throws Http2Exception {
verifyInitialSettingsReceived();
verifyPrefaceReceived();
// Send an ack back to the remote client.
frameWriter.writePing(ctx, ctx.newPromise(), true, data);
@ -727,7 +794,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
@Override
public void onPingAckRead(ChannelHandlerContext ctx, ByteBuf data) throws Http2Exception {
verifyInitialSettingsReceived();
verifyPrefaceReceived();
AbstractHttp2ConnectionHandler.this.onPingAckRead(ctx, data);
}
@ -735,7 +802,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
@Override
public void onPushPromiseRead(ChannelHandlerContext ctx, int streamId,
int promisedStreamId, Http2Headers headers, int padding) throws Http2Exception {
verifyInitialSettingsReceived();
verifyPrefaceReceived();
if (isInboundStreamAfterGoAway(streamId)) {
// Ignore frames for any stream created after we sent a go-away.
@ -762,7 +829,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
@Override
public void onWindowUpdateRead(ChannelHandlerContext ctx, int streamId,
int windowSizeIncrement) throws Http2Exception {
verifyInitialSettingsReceived();
verifyPrefaceReceived();
if (isInboundStreamAfterGoAway(streamId)) {
// Ignore frames for any stream created after we sent a go-away.
@ -796,7 +863,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
@Override
public void onBlockedRead(ChannelHandlerContext ctx, int streamId) throws Http2Exception {
verifyInitialSettingsReceived();
verifyPrefaceReceived();
if (isInboundStreamAfterGoAway(streamId)) {
// Ignore frames for any stream created after we sent a go-away.

View File

@ -73,10 +73,8 @@ public abstract class Http2OrHttpChooser extends ByteToMessageDecoder {
}
private final int maxHttpContentLength;
private final boolean server;
protected Http2OrHttpChooser(boolean server, int maxHttpContentLength) {
this.server = server;
protected Http2OrHttpChooser(int maxHttpContentLength) {
this.maxHttpContentLength = maxHttpContentLength;
}
@ -129,7 +127,6 @@ public abstract class Http2OrHttpChooser extends ByteToMessageDecoder {
*/
protected void addHttp2Handlers(ChannelHandlerContext ctx) {
ChannelPipeline pipeline = ctx.pipeline();
pipeline.addLast("http2PrefaceHandler", new Http2PrefaceHandler(server));
pipeline.addLast("http2ConnectionHandler", createHttp2RequestHandler());
}

View File

@ -1,120 +0,0 @@
/*
* Copyright 2014 The Netty Project
*
* The Netty Project licenses this file to you 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:
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
* or implied. See the License for the specific language governing permissions and limitations under
* the License.
*/
package io.netty.handler.codec.http2;
import static io.netty.handler.codec.http2.Http2CodecUtil.connectionPrefaceBuf;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerAdapter;
import io.netty.channel.ChannelHandlerContext;
/**
* Reads and writes the HTTP/2 connection preface, which must be the first bytes sent by both
* endpoints upon successful establishment of an HTTP/2 connection. After receiving the preface from
* the remote endpoint, this handler removes itself from the pipeline.
*
* https://tools.ietf.org/html/draft-ietf-httpbis-http2-12#section-3.5
*/
public class Http2PrefaceHandler extends ChannelHandlerAdapter {
private boolean server;
private final ByteBuf preface = connectionPrefaceBuf();
private boolean prefaceWritten;
public Http2PrefaceHandler(boolean server) {
this.server = server;
}
@Override
public void channelActive(ChannelHandlerContext ctx) throws Exception {
// The channel just became active - send the HTTP2 connection preface to the remote
// endpoint.
sendPreface(ctx);
super.channelActive(ctx);
}
@Override
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
// This handler was just added to the context. In case it was handled after
// the connection became active, send the HTTP2 connection preface now.
sendPreface(ctx);
}
@Override
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
if (server) {
// Only servers receive the preface string.
if (preface.isReadable() && msg instanceof ByteBuf) {
ByteBuf buf = (ByteBuf) msg;
processHttp2Preface(ctx, buf);
if (preface.isReadable()) {
// More preface left to process.
buf.release();
return;
}
}
}
super.channelRead(ctx, msg);
}
/**
* Sends the HTTP2 connection preface to the remote endpoint, if not already sent.
*/
private void sendPreface(final ChannelHandlerContext ctx) {
if (server) {
// The preface string is only written by clients.
return;
}
if (!prefaceWritten && ctx.channel().isActive()) {
prefaceWritten = true;
ctx.writeAndFlush(connectionPrefaceBuf()).addListener(new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
if (!future.isSuccess() && ctx.channel().isOpen()) {
// The write failed, close the connection.
ctx.close();
} else {
ctx.pipeline().remove(Http2PrefaceHandler.this);
}
}
});
}
}
private void processHttp2Preface(ChannelHandlerContext ctx, ByteBuf in) throws Http2Exception {
int prefaceRemaining = preface.readableBytes();
int bytesRead = Math.min(in.readableBytes(), prefaceRemaining);
// Read the portion of the input up to the length of the preface, if reached.
ByteBuf sourceSlice = in.readSlice(bytesRead);
// Read the same number of bytes from the preface buffer.
ByteBuf prefaceSlice = preface.readSlice(bytesRead);
// If the input so far doesn't match the preface, break the connection.
if (bytesRead == 0 || !prefaceSlice.equals(sourceSlice)) {
ctx.close();
return;
}
if (!preface.isReadable()) {
// Entire preface has been read, remove ourselves from the pipeline.
ctx.pipeline().remove(this);
}
}
}

View File

@ -17,6 +17,7 @@ package io.netty.handler.codec.http2;
import static io.netty.buffer.Unpooled.EMPTY_BUFFER;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.connectionPrefaceBuf;
import static io.netty.handler.codec.http2.Http2CodecUtil.emptyPingBuf;
import static io.netty.handler.codec.http2.Http2Error.NO_ERROR;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
@ -40,6 +41,7 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.buffer.UnpooledByteBufAllocator;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandlerContext;
@ -161,6 +163,45 @@ public class DelegatingHttp2ConnectionHandlerTest {
handler.handlerAdded(ctx);
}
@Test
public void clientShouldSendClientPrefaceStringWhenActive() throws Exception {
when(connection.isServer()).thenReturn(false);
handler = new DelegatingHttp2ConnectionHandler(connection, reader, writer, inboundFlow,
outboundFlow, observer);
handler.channelActive(ctx);
verify(ctx).write(eq(connectionPrefaceBuf()));
}
@Test
public void serverShouldNotSendClientPrefaceStringWhenActive() throws Exception {
when(connection.isServer()).thenReturn(true);
handler = new DelegatingHttp2ConnectionHandler(connection, reader, writer, inboundFlow,
outboundFlow, observer);
handler.channelActive(ctx);
verify(ctx, never()).write(eq(connectionPrefaceBuf()));
}
@Test
public void serverReceivingInvalidClientPrefaceStringShouldCloseConnection() throws Exception {
when(connection.isServer()).thenReturn(true);
handler = new DelegatingHttp2ConnectionHandler(connection, reader, writer, inboundFlow,
outboundFlow, observer);
handler.decode(ctx, Unpooled.copiedBuffer("BAD_PREFACE", UTF_8), Collections.emptyList());
verify(ctx).close();
}
@Test
public void serverReceivingValidClientPrefaceStringShouldContinueReadingFrames() throws Exception {
Mockito.reset(observer);
when(connection.isServer()).thenReturn(true);
handler = new DelegatingHttp2ConnectionHandler(connection, reader, writer, inboundFlow,
outboundFlow, observer);
handler.decode(ctx, connectionPrefaceBuf(), Collections.emptyList());
verify(ctx, never()).close();
decode().onSettingsRead(ctx, new Http2Settings());
verify(observer).onSettingsRead(eq(ctx), eq(new Http2Settings()));
}
@Test
public void closeShouldSendGoAway() throws Exception {
handler.close(ctx, promise);
@ -637,9 +678,11 @@ public class DelegatingHttp2ConnectionHandlerTest {
private void mockContext() {
Mockito.reset(ctx);
when(ctx.alloc()).thenReturn(UnpooledByteBufAllocator.DEFAULT);
when(ctx.channel()).thenReturn(channel);
when(ctx.newSucceededFuture()).thenReturn(future);
when(ctx.newPromise()).thenReturn(promise);
when(ctx.write(any())).thenReturn(future);
}
/**

View File

@ -78,7 +78,6 @@ public class Http2ConnectionRoundtripTest {
@Override
protected void initChannel(Channel ch) throws Exception {
ChannelPipeline p = ch.pipeline();
p.addLast(new Http2PrefaceHandler(true));
p.addLast(new DelegatingHttp2ConnectionHandler(true, new FrameCountDown()));
}
});
@ -89,7 +88,6 @@ public class Http2ConnectionRoundtripTest {
@Override
protected void initChannel(Channel ch) throws Exception {
ChannelPipeline p = ch.pipeline();
p.addLast(new Http2PrefaceHandler(false));
p.addLast(new DelegatingHttp2ConnectionHandler(false, serverObserver));
}
});

View File

@ -1,82 +0,0 @@
/*
* Copyright 2014 The Netty Project
*
* The Netty Project licenses this file to you 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:
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
* or implied. See the License for the specific language governing permissions and limitations under
* the License.
*/
package io.netty.handler.codec.http2;
import static io.netty.handler.codec.http2.Http2CodecUtil.connectionPrefaceBuf;
import static io.netty.util.CharsetUtil.UTF_8;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import io.netty.buffer.Unpooled;
import io.netty.channel.embedded.EmbeddedChannel;
import java.util.Queue;
import org.junit.Test;
/**
* Tests for {@link Http2PrefaceHandler}.
*/
public class Http2PrefaceHandlerTest {
@Test
public void clientShouldWritePrefaceAtStartup() {
EmbeddedChannel channel = createChannel(false);
// Ensure that the preface was automatically written at startup.
Queue<Object> outboundMessages = channel.outboundMessages();
assertTrue(channel.isOpen());
assertNull(channel.pipeline().get(Http2PrefaceHandler.class));
assertTrue(channel.finish());
assertEquals(1, outboundMessages.size());
assertEquals(connectionPrefaceBuf(), outboundMessages.peek());
}
@Test
public void serverShouldNotWritePrefaceAtStartup() {
EmbeddedChannel channel = createChannel(true);
// Ensure that the preface was automatically written at startup.
Queue<Object> outboundMessages = channel.outboundMessages();
assertTrue(channel.isOpen());
assertNotNull(channel.pipeline().get(Http2PrefaceHandler.class));
assertFalse(channel.finish());
assertTrue(outboundMessages.isEmpty());
}
@Test
public void serverShouldBeRemovedAfterReceivingPreface() {
EmbeddedChannel channel = createChannel(true);
// Simulate receiving the preface.
assertTrue(channel.writeInbound(connectionPrefaceBuf()));
assertNull(channel.pipeline().get(Http2PrefaceHandler.class));
assertTrue(channel.isOpen());
assertTrue(channel.finish());
}
@Test
public void serverReceivingBadPrefaceShouldCloseTheConnection() {
EmbeddedChannel channel = createChannel(true);
// Simulate receiving the bad preface.
assertFalse(channel.writeInbound(Unpooled.copiedBuffer("BAD_PREFACE", UTF_8)));
assertFalse(channel.isOpen());
assertFalse(channel.finish());
}
private EmbeddedChannel createChannel(boolean server) {
return new EmbeddedChannel(new Http2PrefaceHandler(server));
}
}

View File

@ -19,7 +19,6 @@ import io.netty.channel.ChannelPipeline;
import io.netty.channel.socket.SocketChannel;
import io.netty.example.securechat.SecureChatSslContextFactory;
import io.netty.handler.codec.http2.AbstractHttp2ConnectionHandler;
import io.netty.handler.codec.http2.Http2PrefaceHandler;
import io.netty.handler.ssl.SslHandler;
import javax.net.ssl.SSLEngine;
@ -47,7 +46,6 @@ public class Http2ClientInitializer extends ChannelInitializer<SocketChannel> {
ChannelPipeline pipeline = ch.pipeline();
pipeline.addLast("ssl", new SslHandler(engine));
pipeline.addLast("http2PrefaceHandler", new Http2PrefaceHandler(false));
pipeline.addLast("http2ConnectionHandler", connectionHandler);
}
}

View File

@ -36,7 +36,7 @@ public class Http2OrHttpHandler extends Http2OrHttpChooser {
}
public Http2OrHttpHandler(int maxHttpContentLength) {
super(true, maxHttpContentLength);
super(maxHttpContentLength);
}
@Override