HTTP/2 Max Header List Size Bug

Motivation:
If the HPACK Decoder detects that SETTINGS_MAX_HEADER_LIST_SIZE has been violated it aborts immediately and sends a RST_STREAM frame for what ever stream caused the issue. Because HPACK is stateful this means that the HPACK state may become out of sync between peers, and the issue won't be detected until the next headers frame. We should make a best effort to keep processing to keep the HPACK state in sync with our peer, or completely close the connection.
If the HPACK Encoder is configured to verify SETTINGS_MAX_HEADER_LIST_SIZE it checks the limit and encodes at the same time. This may result in modifying the HPACK local state but not sending the headers to the peer if SETTINGS_MAX_HEADER_LIST_SIZE is violated. This will also lead to an inconsistency in HPACK state that will be flagged at some later time.

Modifications:
- HPACK Decoder now has 2 levels of limits related to SETTINGS_MAX_HEADER_LIST_SIZE. The first will attempt to keep processing data and send a RST_STREAM after all data is processed. The second will send a GO_AWAY and close the entire connection.
- When the HPACK Encoder enforces SETTINGS_MAX_HEADER_LIST_SIZE it should not modify the HPACK state until the size has been checked.
- https://tools.ietf.org/html/rfc7540#section-6.5.2 states that the initial value of SETTINGS_MAX_HEADER_LIST_SIZE is "unlimited". We currently use 8k as a limit. We should honor the specifications default value so we don't unintentionally close a connection before the remote peer is aware of the local settings.
- Remove unnecessary object allocation in DefaultHttp2HeadersDecoder and DefaultHttp2HeadersEncoder.

Result:
Fixes https://github.com/netty/netty/issues/6209.
This commit is contained in:
Scott Mitchell 2017-01-13 17:09:44 -08:00
parent ead87b7df8
commit 2fd42cfc6b
40 changed files with 618 additions and 261 deletions

View File

@ -19,6 +19,7 @@ package io.netty.handler.codec.http2;
import io.netty.handler.codec.http2.Http2HeadersEncoder.SensitivityDetector;
import io.netty.util.internal.UnstableApi;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_LIST_SIZE;
import static io.netty.util.internal.ObjectUtil.checkNotNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.SECONDS;
@ -76,7 +77,7 @@ public abstract class AbstractHttp2ConnectionHandlerBuilder<T extends Http2Conne
private static final SensitivityDetector DEFAULT_HEADER_SENSITIVITY_DETECTOR = Http2HeadersEncoder.NEVER_SENSITIVE;
// The properties that can always be set.
private Http2Settings initialSettings = new Http2Settings();
private Http2Settings initialSettings = new Http2Settings().maxHeaderListSize(DEFAULT_HEADER_LIST_SIZE);
private Http2FrameListener frameListener;
private long gracefulShutdownTimeoutMillis = DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT_MILLIS;
@ -334,7 +335,10 @@ public abstract class AbstractHttp2ConnectionHandlerBuilder<T extends Http2Conne
}
private T buildFromConnection(Http2Connection connection) {
Http2FrameReader reader = new DefaultHttp2FrameReader(isValidateHeaders());
Long maxHeaderListSize = initialSettings.maxHeaderListSize();
Http2FrameReader reader = new DefaultHttp2FrameReader(maxHeaderListSize == null ?
new DefaultHttp2HeadersDecoder(isValidateHeaders()) :
new DefaultHttp2HeadersDecoder(isValidateHeaders(), maxHeaderListSize));
Http2FrameWriter writer = encoderIgnoreMaxHeaderListSize == null ?
new DefaultHttp2FrameWriter(headerSensitivityDetector()) :
new DefaultHttp2FrameWriter(headerSensitivityDetector(), encoderIgnoreMaxHeaderListSize);

View File

@ -14,14 +14,14 @@
*/
package io.netty.handler.codec.http2;
import static io.netty.util.internal.ObjectUtil.checkNotNull;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPromise;
import io.netty.util.internal.UnstableApi;
import static io.netty.util.internal.ObjectUtil.checkNotNull;
/**
* Decorator around another {@link Http2FrameWriter} instance.
*/
@ -41,14 +41,14 @@ public class DecoratingHttp2FrameWriter implements Http2FrameWriter {
@Override
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, int streamId, Http2Headers headers, int padding,
boolean endStream, ChannelPromise promise) {
boolean endStream, ChannelPromise promise) throws Http2Exception {
return delegate.writeHeaders(ctx, streamId, headers, padding, endStream, promise);
}
@Override
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, int streamId, Http2Headers headers,
int streamDependency, short weight, boolean exclusive, int padding,
boolean endStream, ChannelPromise promise) {
boolean endStream, ChannelPromise promise) throws Http2Exception {
return delegate
.writeHeaders(ctx, streamId, headers, streamDependency, weight, exclusive, padding, endStream, promise);
}
@ -82,7 +82,8 @@ public class DecoratingHttp2FrameWriter implements Http2FrameWriter {
@Override
public ChannelFuture writePushPromise(ChannelHandlerContext ctx, int streamId, int promisedStreamId,
Http2Headers headers, int padding, ChannelPromise promise) {
Http2Headers headers, int padding, ChannelPromise promise)
throws Http2Exception {
return delegate.writePushPromise(ctx, streamId, promisedStreamId, headers, padding, promise);
}

View File

@ -369,14 +369,16 @@ public class DefaultHttp2Connection implements Http2Connection {
* Simple stream implementation. Streams can be compared to each other by priority.
*/
private class DefaultStream implements Http2Stream {
private static final byte SENT_STATE_RST = 0x1;
private static final byte SENT_STATE_HEADERS = 0x2;
private static final byte SENT_STATE_PUSHPROMISE = 0x4;
private final int id;
private final PropertyMap properties = new PropertyMap();
private State state;
private short weight = DEFAULT_PRIORITY_WEIGHT;
private DefaultStream parent;
private IntObjectMap<DefaultStream> children = IntCollections.emptyMap();
private boolean resetSent;
private boolean headersSent;
private byte sentState;
DefaultStream(int id, State state) {
this.id = id;
@ -395,24 +397,35 @@ public class DefaultHttp2Connection implements Http2Connection {
@Override
public boolean isResetSent() {
return resetSent;
return (sentState & SENT_STATE_RST) != 0;
}
@Override
public Http2Stream resetSent() {
resetSent = true;
sentState |= SENT_STATE_RST;
return this;
}
@Override
public Http2Stream headersSent() {
headersSent = true;
sentState |= SENT_STATE_HEADERS;
return this;
}
@Override
public boolean isHeadersSent() {
return headersSent;
return (sentState & SENT_STATE_HEADERS) != 0;
}
@Override
public Http2Stream pushPromiseSent() {
sentState |= SENT_STATE_PUSHPROMISE;
return this;
}
@Override
public boolean isPushPromiseSent() {
return (sentState & SENT_STATE_PUSHPROMISE) != 0;
}
@Override
@ -837,6 +850,16 @@ public class DefaultHttp2Connection implements Http2Connection {
public boolean isHeadersSent() {
throw new UnsupportedOperationException();
}
@Override
public Http2Stream pushPromiseSent() {
throw new UnsupportedOperationException();
}
@Override
public boolean isPushPromiseSent() {
throw new UnsupportedOperationException();
}
}
/**

View File

@ -122,13 +122,13 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
public Http2Settings localSettings() {
Http2Settings settings = new Http2Settings();
Http2FrameReader.Configuration config = frameReader.configuration();
Http2HeaderTable headerTable = config.headerTable();
Http2HeadersDecoder.Configuration headersConfig = config.headersConfiguration();
Http2FrameSizePolicy frameSizePolicy = config.frameSizePolicy();
settings.initialWindowSize(flowController().initialWindowSize());
settings.maxConcurrentStreams(connection.remote().maxActiveStreams());
settings.headerTableSize(headerTable.maxHeaderTableSize());
settings.headerTableSize(headersConfig.maxHeaderTableSize());
settings.maxFrameSize(frameSizePolicy.maxFrameSize());
settings.maxHeaderListSize(headerTable.maxHeaderListSize());
settings.maxHeaderListSize(headersConfig.maxHeaderListSize());
if (!connection.isServer()) {
// Only set the pushEnabled flag if this is a client endpoint.
settings.pushEnabled(connection.local().allowPushTo());
@ -141,6 +141,18 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
frameReader.close();
}
/**
* Calculate the threshold in bytes which should trigger a {@code GO_AWAY} if a set of headers exceeds this amount.
* @param maxHeaderListSize
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a> for the local
* endpoint.
* @return the threshold in bytes which should trigger a {@code GO_AWAY} if a set of headers exceeds this amount.
*/
protected long calculateMaxHeaderListSizeGoAway(long maxHeaderListSize) {
// This is equivalent to `maxHeaderListSize * 1.25` but we avoid floating point multiplication.
return maxHeaderListSize + (maxHeaderListSize >>> 2);
}
private int unconsumedBytes(Http2Stream stream) {
return flowController().unconsumedBytes(stream);
}
@ -383,11 +395,13 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
/**
* Applies settings sent from the local endpoint.
* <p>
* This method is only called after the local settings have been acknowledged from the remote endpoint.
*/
private void applyLocalSettings(Http2Settings settings) throws Http2Exception {
Boolean pushEnabled = settings.pushEnabled();
final Http2FrameReader.Configuration config = frameReader.configuration();
final Http2HeaderTable headerTable = config.headerTable();
final Http2HeadersDecoder.Configuration headerConfig = config.headersConfiguration();
final Http2FrameSizePolicy frameSizePolicy = config.frameSizePolicy();
if (pushEnabled != null) {
if (connection.isServer()) {
@ -404,12 +418,12 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
Long headerTableSize = settings.headerTableSize();
if (headerTableSize != null) {
headerTable.maxHeaderTableSize(headerTableSize);
headerConfig.maxHeaderTableSize(headerTableSize);
}
Long maxHeaderListSize = settings.maxHeaderListSize();
if (maxHeaderListSize != null) {
headerTable.maxHeaderListSize(maxHeaderListSize);
headerConfig.maxHeaderListSize(maxHeaderListSize, calculateMaxHeaderListSizeGoAway(maxHeaderListSize));
}
Integer maxFrameSize = settings.maxFrameSize();

View File

@ -77,7 +77,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
public void remoteSettings(Http2Settings settings) throws Http2Exception {
Boolean pushEnabled = settings.pushEnabled();
Http2FrameWriter.Configuration config = configuration();
Http2HeaderTable outboundHeaderTable = config.headerTable();
Http2HeadersEncoder.Configuration outboundHeaderConfig = config.headersConfiguration();
Http2FrameSizePolicy outboundFrameSizePolicy = config.frameSizePolicy();
if (pushEnabled != null) {
if (!connection.isServer() && pushEnabled) {
@ -95,12 +95,12 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
Long headerTableSize = settings.headerTableSize();
if (headerTableSize != null) {
outboundHeaderTable.maxHeaderTableSize((int) min(headerTableSize, MAX_VALUE));
outboundHeaderConfig.maxHeaderTableSize((int) min(headerTableSize, MAX_VALUE));
}
Long maxHeaderListSize = settings.maxHeaderListSize();
if (maxHeaderListSize != null) {
outboundHeaderTable.maxHeaderListSize(maxHeaderListSize);
outboundHeaderConfig.maxHeaderListSize(maxHeaderListSize);
}
Integer maxFrameSize = settings.maxFrameSize();
@ -197,14 +197,12 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
flowController.addFlowControlled(stream,
new FlowControlledHeaders(stream, headers, streamDependency, weight, exclusive, padding,
endOfStream, promise));
stream.headersSent();
return promise;
}
} catch (Http2NoMoreStreamIdsException e) {
lifecycleManager.onError(ctx, e);
return promise.setFailure(e);
} catch (Throwable e) {
return promise.setFailure(e);
} catch (Throwable t) {
lifecycleManager.onError(ctx, t);
promise.tryFailure(t);
return promise;
}
}
@ -275,11 +273,16 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
Http2Stream stream = requireStream(streamId);
// Reserve the promised stream.
connection.local().reservePushStream(promisedStreamId, stream);
} catch (Throwable e) {
return promise.setFailure(e);
}
return frameWriter.writePushPromise(ctx, streamId, promisedStreamId, headers, padding, promise);
ChannelFuture future = frameWriter.writePushPromise(ctx, streamId, promisedStreamId, headers, padding,
promise);
stream.pushPromiseSent();
return future;
} catch (Throwable t) {
lifecycleManager.onError(ctx, t);
promise.tryFailure(t);
return promise;
}
}
@Override
@ -447,7 +450,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
}
@Override
public void write(ChannelHandlerContext ctx, int allowedBytes) {
public void write(ChannelHandlerContext ctx, int allowedBytes) throws Http2Exception {
if (promise.isVoid()) {
promise = ctx.newPromise();
}
@ -455,6 +458,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
frameWriter.writeHeaders(ctx, stream.id(), headers, streamDependency, weight, exclusive,
padding, endOfStream, promise);
stream.headersSent();
}
@Override

View File

@ -95,8 +95,8 @@ public class DefaultHttp2FrameReader implements Http2FrameReader, Http2FrameSize
}
@Override
public Http2HeaderTable headerTable() {
return headersDecoder.configuration().headerTable();
public Http2HeadersDecoder.Configuration headersConfiguration() {
return headersDecoder.configuration();
}
@Override
@ -673,7 +673,7 @@ public class DefaultHttp2FrameReader implements Http2FrameReader, Http2FrameSize
*/
private void headerSizeExceeded() throws Http2Exception {
close();
headerListSizeExceeded(streamId, headersDecoder.configuration().headerTable().maxHeaderListSize(), true);
headerListSizeExceeded(headersDecoder.configuration().maxHeaderListSizeGoAway());
}
/**
@ -687,7 +687,7 @@ public class DefaultHttp2FrameReader implements Http2FrameReader, Http2FrameSize
*/
final void addFragment(ByteBuf fragment, ByteBufAllocator alloc, boolean endOfHeaders) throws Http2Exception {
if (headerBlock == null) {
if (fragment.readableBytes() > headersDecoder.configuration().headerTable().maxHeaderListSize()) {
if (fragment.readableBytes() > headersDecoder.configuration().maxHeaderListSizeGoAway()) {
headerSizeExceeded();
}
if (endOfHeaders) {
@ -700,7 +700,7 @@ public class DefaultHttp2FrameReader implements Http2FrameReader, Http2FrameSize
}
return;
}
if (headersDecoder.configuration().headerTable().maxHeaderListSize() - fragment.readableBytes() <
if (headersDecoder.configuration().maxHeaderListSizeGoAway() - fragment.readableBytes() <
headerBlock.readableBytes()) {
headerSizeExceeded();
}

View File

@ -22,6 +22,7 @@ import io.netty.channel.ChannelPromise;
import io.netty.handler.codec.http2.Http2CodecUtil.SimpleChannelPromiseAggregator;
import io.netty.handler.codec.http2.Http2FrameWriter.Configuration;
import io.netty.handler.codec.http2.Http2HeadersEncoder.SensitivityDetector;
import io.netty.util.internal.PlatformDependent;
import io.netty.util.internal.UnstableApi;
import static io.netty.buffer.Unpooled.directBuffer;
@ -106,8 +107,8 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter, Http2FrameSize
}
@Override
public Http2HeaderTable headerTable() {
return headersEncoder.configuration().headerTable();
public Http2HeadersEncoder.Configuration headersConfiguration() {
return headersEncoder.configuration();
}
@Override
@ -187,7 +188,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter, Http2FrameSize
@Override
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, int streamId,
Http2Headers headers, int padding, boolean endStream, ChannelPromise promise) {
Http2Headers headers, int padding, boolean endStream, ChannelPromise promise) throws Http2Exception {
return writeHeadersInternal(ctx, streamId, headers, padding, endStream,
false, 0, (short) 0, false, promise);
}
@ -195,7 +196,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter, Http2FrameSize
@Override
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, int streamId,
Http2Headers headers, int streamDependency, short weight, boolean exclusive,
int padding, boolean endStream, ChannelPromise promise) {
int padding, boolean endStream, ChannelPromise promise) throws Http2Exception {
return writeHeadersInternal(ctx, streamId, headers, padding, endStream,
true, streamDependency, weight, exclusive, promise);
}
@ -292,7 +293,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter, Http2FrameSize
@Override
public ChannelFuture writePushPromise(ChannelHandlerContext ctx, int streamId,
int promisedStreamId, Http2Headers headers, int padding, ChannelPromise promise) {
int promisedStreamId, Http2Headers headers, int padding, ChannelPromise promise) throws Http2Exception {
ByteBuf headerBlock = null;
SimpleChannelPromiseAggregator promiseAggregator =
new SimpleChannelPromiseAggregator(promise, ctx.channel(), ctx.executor());
@ -334,8 +335,14 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter, Http2FrameSize
if (!flags.endOfHeaders()) {
writeContinuationFrames(ctx, streamId, headerBlock, padding, promiseAggregator);
}
} catch (Http2Exception e) {
promiseAggregator.setFailure(e);
promiseAggregator.doneAllocatingPromises();
throw e;
} catch (Throwable t) {
promiseAggregator.setFailure(t);
promiseAggregator.doneAllocatingPromises();
PlatformDependent.throwException(t);
} finally {
if (headerBlock != null) {
headerBlock.release();
@ -413,7 +420,8 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter, Http2FrameSize
private ChannelFuture writeHeadersInternal(ChannelHandlerContext ctx,
int streamId, Http2Headers headers, int padding, boolean endStream,
boolean hasPriority, int streamDependency, short weight, boolean exclusive, ChannelPromise promise) {
boolean hasPriority, int streamDependency, short weight, boolean exclusive, ChannelPromise promise)
throws Http2Exception {
ByteBuf headerBlock = null;
SimpleChannelPromiseAggregator promiseAggregator =
new SimpleChannelPromiseAggregator(promise, ctx.channel(), ctx.executor());
@ -465,8 +473,14 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter, Http2FrameSize
if (!flags.endOfHeaders()) {
writeContinuationFrames(ctx, streamId, headerBlock, padding, promiseAggregator);
}
} catch (Http2Exception e) {
promiseAggregator.setFailure(e);
promiseAggregator.doneAllocatingPromises();
throw e;
} catch (Throwable t) {
promiseAggregator.setFailure(t);
promiseAggregator.doneAllocatingPromises();
PlatformDependent.throwException(t);
} finally {
if (headerBlock != null) {
headerBlock.release();

View File

@ -20,6 +20,7 @@ import io.netty.handler.codec.http2.internal.hpack.Decoder;
import io.netty.util.internal.ObjectUtil;
import io.netty.util.internal.UnstableApi;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2Error.COMPRESSION_ERROR;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
@ -29,7 +30,6 @@ public class DefaultHttp2HeadersDecoder implements Http2HeadersDecoder, Http2Hea
private static final float HEADERS_COUNT_WEIGHT_HISTORICAL = 1 - HEADERS_COUNT_WEIGHT_NEW;
private final Decoder decoder;
private final Http2HeaderTable headerTable;
private final boolean validateHeaders;
/**
* Used to calculate an exponential moving average of header sizes to get an estimate of how large the data
@ -42,11 +42,33 @@ public class DefaultHttp2HeadersDecoder implements Http2HeadersDecoder, Http2Hea
}
public DefaultHttp2HeadersDecoder(boolean validateHeaders) {
this(validateHeaders, new Decoder());
this(validateHeaders, DEFAULT_HEADER_LIST_SIZE);
}
public DefaultHttp2HeadersDecoder(boolean validateHeaders, int initialHuffmanDecodeCapacity) {
this(validateHeaders, new Decoder(initialHuffmanDecodeCapacity));
/**
* Create a new instance.
* @param validateHeaders {@code true} to validate headers are valid according to the RFC.
* @param maxHeaderListSize This is the only setting that can be configured before notifying the peer.
* This is because <a href="https://tools.ietf.org/html/rfc7540#section-6.5.1">SETTINGS_MAX_HEADER_LIST_SIZE</a>
* allows a lower than advertised limit from being enforced, and the default limit is unlimited
* (which is dangerous).
*/
public DefaultHttp2HeadersDecoder(boolean validateHeaders, long maxHeaderListSize) {
this(validateHeaders, maxHeaderListSize, 32);
}
/**
* Create a new instance.
* @param validateHeaders {@code true} to validate headers are valid according to the RFC.
* @param maxHeaderListSize This is the only setting that can be configured before notifying the peer.
* This is because <a href="https://tools.ietf.org/html/rfc7540#section-6.5.1">SETTINGS_MAX_HEADER_LIST_SIZE</a>
* allows a lower than advertised limit from being enforced, and the default limit is unlimited
* (which is dangerous).
* @param initialHuffmanDecodeCapacity Size of an intermediate buffer used during huffman decode.
*/
public DefaultHttp2HeadersDecoder(boolean validateHeaders, long maxHeaderListSize,
int initialHuffmanDecodeCapacity) {
this(validateHeaders, new Decoder(maxHeaderListSize, initialHuffmanDecodeCapacity));
}
/**
@ -55,13 +77,32 @@ public class DefaultHttp2HeadersDecoder implements Http2HeadersDecoder, Http2Hea
*/
DefaultHttp2HeadersDecoder(boolean validateHeaders, Decoder decoder) {
this.decoder = ObjectUtil.checkNotNull(decoder, "decoder");
headerTable = new Http2HeaderTableDecoder();
this.validateHeaders = validateHeaders;
}
@Override
public Http2HeaderTable headerTable() {
return headerTable;
public void maxHeaderTableSize(long max) throws Http2Exception {
decoder.setMaxHeaderTableSize(max);
}
@Override
public long maxHeaderTableSize() {
return decoder.getMaxHeaderTableSize();
}
@Override
public void maxHeaderListSize(long max, long goAwayMax) throws Http2Exception {
decoder.setMaxHeaderListSize(max, goAwayMax);
}
@Override
public long maxHeaderListSize() {
return decoder.getMaxHeaderListSize();
}
@Override
public long maxHeaderListSizeGoAway() {
return decoder.getMaxHeaderListSizeGoAway();
}
@Override
@ -86,29 +127,4 @@ public class DefaultHttp2HeadersDecoder implements Http2HeadersDecoder, Http2Hea
throw connectionError(COMPRESSION_ERROR, e, e.getMessage());
}
}
/**
* {@link Http2HeaderTable} implementation to support {@link Http2HeadersDecoder}
*/
private final class Http2HeaderTableDecoder implements Http2HeaderTable {
@Override
public void maxHeaderTableSize(long max) throws Http2Exception {
decoder.setMaxHeaderTableSize(max);
}
@Override
public long maxHeaderTableSize() {
return decoder.getMaxHeaderTableSize();
}
@Override
public void maxHeaderListSize(long max) throws Http2Exception {
decoder.setMaxHeaderListSize(max);
}
@Override
public long maxHeaderListSize() {
return decoder.getMaxHeaderListSize();
}
}
}

View File

@ -28,7 +28,6 @@ import static io.netty.util.internal.ObjectUtil.checkNotNull;
public class DefaultHttp2HeadersEncoder implements Http2HeadersEncoder, Http2HeadersEncoder.Configuration {
private final Encoder encoder;
private final SensitivityDetector sensitivityDetector;
private final Http2HeaderTable headerTable;
private final ByteBuf tableSizeChangeOutput = Unpooled.buffer();
public DefaultHttp2HeadersEncoder() {
@ -55,7 +54,6 @@ public class DefaultHttp2HeadersEncoder implements Http2HeadersEncoder, Http2Hea
DefaultHttp2HeadersEncoder(SensitivityDetector sensitivityDetector, Encoder encoder) {
this.sensitivityDetector = checkNotNull(sensitivityDetector, "sensitiveDetector");
this.encoder = checkNotNull(encoder, "encoder");
headerTable = new Http2HeaderTableEncoder();
}
@Override
@ -77,37 +75,27 @@ public class DefaultHttp2HeadersEncoder implements Http2HeadersEncoder, Http2Hea
}
@Override
public Http2HeaderTable headerTable() {
return headerTable;
public void maxHeaderTableSize(long max) throws Http2Exception {
encoder.setMaxHeaderTableSize(tableSizeChangeOutput, max);
}
@Override
public long maxHeaderTableSize() {
return encoder.getMaxHeaderTableSize();
}
@Override
public void maxHeaderListSize(long max) throws Http2Exception {
encoder.setMaxHeaderListSize(max);
}
@Override
public long maxHeaderListSize() {
return encoder.getMaxHeaderListSize();
}
@Override
public Configuration configuration() {
return this;
}
/**
* {@link Http2HeaderTable} implementation to support {@link Http2HeadersEncoder}
*/
private final class Http2HeaderTableEncoder implements Http2HeaderTable {
@Override
public void maxHeaderTableSize(long max) throws Http2Exception {
encoder.setMaxHeaderTableSize(tableSizeChangeOutput, max);
}
@Override
public long maxHeaderTableSize() {
return encoder.getMaxHeaderTableSize();
}
@Override
public void maxHeaderListSize(long max) throws Http2Exception {
encoder.setMaxHeaderListSize(max);
}
@Override
public long maxHeaderListSize() {
return encoder.getMaxHeaderListSize();
}
}
}

View File

@ -30,6 +30,7 @@ import io.netty.util.internal.UnstableApi;
import static io.netty.buffer.Unpooled.directBuffer;
import static io.netty.buffer.Unpooled.unreleasableBuffer;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
import static io.netty.handler.codec.http2.Http2Exception.headerListSizeError;
import static io.netty.util.CharsetUtil.UTF_8;
import static java.lang.Math.max;
@ -53,7 +54,6 @@ public final class Http2CodecUtil {
* pad length field.
*/
public static final int MAX_PADDING = 256;
public static final int MAX_UNSIGNED_SHORT = 0xFFFF;
public static final long MAX_UNSIGNED_INT = 0xFFFFFFFFL;
public static final int FRAME_HEADER_LENGTH = 9;
public static final int SETTING_ENTRY_LENGTH = 6;
@ -102,10 +102,14 @@ public final class Http2CodecUtil {
public static final long MIN_HEADER_LIST_SIZE = 0;
public static final int DEFAULT_WINDOW_SIZE = 65535;
public static final boolean DEFAULT_ENABLE_PUSH = true;
public static final short DEFAULT_PRIORITY_WEIGHT = 16;
public static final int DEFAULT_HEADER_TABLE_SIZE = 4096;
public static final int DEFAULT_HEADER_LIST_SIZE = 8192;
/**
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">The initial value of this setting is unlimited</a>.
* However in practice we don't want to allow our peers to use unlimited memory by default. So we take advantage
* of the <q>For any given request, a lower limit than what is advertised MAY be enforced.</q> loophole.
*/
public static final long DEFAULT_HEADER_LIST_SIZE = 8192;
public static final int DEFAULT_MAX_FRAME_SIZE = MAX_FRAME_SIZE_LOWER_BOUND;
/**
@ -223,12 +227,32 @@ public final class Http2CodecUtil {
return max(0, min(state.pendingBytes(), state.windowSize()));
}
/**
* Results in a RST_STREAM being sent for {@code streamId} due to violating
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a>.
* @param streamId The stream ID that was being processed when the exceptional condition occurred.
* @param maxHeaderListSize The max allowed size for a list of headers in bytes which was exceeded.
* @param onDecode {@code true} if the exception was encountered during decoder. {@code false} for encode.
* @throws Http2Exception a stream error.
*/
public static void headerListSizeExceeded(int streamId, long maxHeaderListSize,
boolean onDecode) throws Http2Exception {
throw headerListSizeError(streamId, PROTOCOL_ERROR, onDecode, "Header size exceeded max " +
"allowed size (%d)", maxHeaderListSize);
}
/**
* Results in a GO_AWAY being sent due to violating
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a> in an unrecoverable
* manner.
* @param maxHeaderListSize The max allowed size for a list of headers in bytes which was exceeded.
* @throws Http2Exception a connection error.
*/
public static void headerListSizeExceeded(long maxHeaderListSize) throws Http2Exception {
throw connectionError(PROTOCOL_ERROR, "Header size exceeded max " +
"allowed size (%d)", maxHeaderListSize);
}
static void writeFrameHeaderInternal(ByteBuf out, int payloadLength, byte type,
Http2Flags flags, int streamId) {
out.writeMedium(payloadLength);

View File

@ -66,8 +66,8 @@ public class Http2ConnectionHandler extends ByteToMessageDecoder implements Http
private static final InternalLogger logger = InternalLoggerFactory.getInstance(Http2ConnectionHandler.class);
private static final Http2Headers headersTooLarge = new DefaultHttp2Headers().status(
HttpResponseStatus.REQUEST_HEADER_FIELDS_TOO_LARGE.codeAsText());
private static final Http2Headers HEADERS_TOO_LARGE_HEADERS = ReadOnlyHttp2Headers.serverHeaders(false,
HttpResponseStatus.REQUEST_HEADER_FIELDS_TOO_LARGE.codeAsText());
private final Http2ConnectionDecoder decoder;
private final Http2ConnectionEncoder encoder;
@ -161,12 +161,14 @@ public class Http2ConnectionHandler extends ByteToMessageDecoder implements Http
@Override
public void flush(ChannelHandlerContext ctx) throws Http2Exception {
// Trigger pending writes in the remote flow controller.
encoder.flowController().writePendingBytes();
try {
// Trigger pending writes in the remote flow controller.
encoder.flowController().writePendingBytes();
ctx.flush();
} catch (Throwable t) {
throw new Http2Exception(INTERNAL_ERROR, "Error flushing" , t);
} catch (Http2Exception e) {
onError(ctx, e);
} catch (Throwable cause) {
onError(ctx, connectionError(INTERNAL_ERROR, cause, "Error flushing"));
}
}
@ -629,7 +631,13 @@ public class Http2ConnectionHandler extends ByteToMessageDecoder implements Http
// ensure that we have not already sent headers on this stream
if (stream != null && !stream.isHeadersSent()) {
handleServerHeaderDecodeSizeError(ctx, stream);
try {
handleServerHeaderDecodeSizeError(ctx, stream);
} catch (Http2Exception e) {
onError(ctx, e);
} catch (Throwable cause2) {
onError(ctx, connectionError(INTERNAL_ERROR, cause2, "Error DecodeSizeError"));
}
}
}
@ -646,9 +654,11 @@ public class Http2ConnectionHandler extends ByteToMessageDecoder implements Http
*
* @param ctx the channel context
* @param stream the Http2Stream on which the header was received
* @throws Http2Exception if an exception occurs while processing the decode error.
*/
protected void handleServerHeaderDecodeSizeError(ChannelHandlerContext ctx, Http2Stream stream) {
encoder().writeHeaders(ctx, stream.id(), headersTooLarge, 0, true, ctx.newPromise());
protected void handleServerHeaderDecodeSizeError(ChannelHandlerContext ctx, Http2Stream stream)
throws Http2Exception {
encoder().writeHeaders(ctx, stream.id(), HEADERS_TOO_LARGE_HEADERS, 0, true, ctx.newPromise());
}
protected Http2FrameWriter frameWriter() {
@ -695,8 +705,10 @@ public class Http2ConnectionHandler extends ByteToMessageDecoder implements Http
return promise.setSuccess();
}
final ChannelFuture future;
if (stream.state() == IDLE) {
// We cannot write RST_STREAM frames on IDLE streams https://tools.ietf.org/html/rfc7540#section-6.4.
// If the remote peer is not aware of the steam, then we are not allowed to send a RST_STREAM
// https://tools.ietf.org/html/rfc7540#section-6.4.
if (stream.state() == IDLE ||
connection().local().created(stream) && !stream.isHeadersSent() && !stream.isPushPromiseSent()) {
future = promise.setSuccess();
} else {
future = frameWriter().writeRstStream(ctx, stream.id(), errorCode, promise);

View File

@ -136,7 +136,10 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
Http2Connection connection = new DefaultHttp2Connection(server);
frameWriter = new Http2OutboundFrameLogger(frameWriter, frameLogger);
Http2ConnectionEncoder encoder = new DefaultHttp2ConnectionEncoder(connection, frameWriter);
Http2FrameReader frameReader = new DefaultHttp2FrameReader();
Long maxHeaderListSize = initialSettings.maxHeaderListSize();
Http2FrameReader frameReader = new DefaultHttp2FrameReader(maxHeaderListSize == null ?
new DefaultHttp2HeadersDecoder(true) :
new DefaultHttp2HeadersDecoder(true, maxHeaderListSize));
Http2FrameReader reader = new Http2InboundFrameLogger(frameReader, frameLogger);
Http2ConnectionDecoder decoder = new DefaultHttp2ConnectionDecoder(connection, encoder, reader);
decoder.frameListener(new FrameListener());
@ -206,7 +209,7 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
* streams.
*/
@Override
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) {
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Http2Exception {
try {
if (msg instanceof Http2WindowUpdateFrame) {
Http2WindowUpdateFrame frame = (Http2WindowUpdateFrame) msg;
@ -249,7 +252,7 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
http2HandlerCtx, lastStreamId, frame.errorCode(), frame.content().retain(), promise);
}
private void writeStreamFrame(Http2StreamFrame frame, ChannelPromise promise) {
private void writeStreamFrame(Http2StreamFrame frame, ChannelPromise promise) throws Http2Exception {
if (frame instanceof Http2DataFrame) {
Http2DataFrame dataFrame = (Http2DataFrame) frame;
http2Handler.encoder().writeData(http2HandlerCtx, frame.streamId(), dataFrame.content().retain(),
@ -264,7 +267,7 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
}
}
private void writeHeadersFrame(Http2HeadersFrame headersFrame, ChannelPromise promise) {
private void writeHeadersFrame(Http2HeadersFrame headersFrame, ChannelPromise promise) throws Http2Exception {
int streamId = headersFrame.streamId();
if (!isStreamIdValid(streamId)) {
final Endpoint<Http2LocalFlowController> localEndpoint = http2Handler.connection().local();

View File

@ -32,9 +32,9 @@ public interface Http2FrameReader extends Closeable {
*/
interface Configuration {
/**
* Get the {@link Http2HeaderTable} for this {@link Http2FrameReader}
* Get the {@link Http2HeadersDecoder.Configuration} for this {@link Http2FrameReader}
*/
Http2HeaderTable headerTable();
Http2HeadersDecoder.Configuration headersConfiguration();
/**
* Get the {@link Http2FrameSizePolicy} for this {@link Http2FrameReader}

View File

@ -35,9 +35,9 @@ public interface Http2FrameWriter extends Http2DataWriter, Closeable {
*/
interface Configuration {
/**
* Get the {@link Http2HeaderTable} for this {@link Http2FrameWriter}
* Get the {@link Http2HeadersEncoder.Configuration} for this {@link Http2FrameWriter}
*/
Http2HeaderTable headerTable();
Http2HeadersEncoder.Configuration headersConfiguration();
/**
* Get the {@link Http2FrameSizePolicy} for this {@link Http2FrameWriter}
@ -56,9 +56,17 @@ public interface Http2FrameWriter extends Http2DataWriter, Closeable {
* @param endStream indicates if this is the last frame to be sent for the stream.
* @param promise the promise for the write.
* @return the future for the write.
* @throws Http2Exception if an exception occurs while encoding headers.
* <a href="https://tools.ietf.org/html/rfc7540#section-10.5.1">Section 10.5.1</a> states the following:
* <pre>
* The header block MUST be processed to ensure a consistent connection state, unless the connection is closed.
* </pre>
* If this call has modified the HPACK header state you <strong>MUST</strong> throw a connection error.
* <p>
* If this call has <strong>NOT</strong> modified the HPACK header state you are free to throw a stream error.
*/
ChannelFuture writeHeaders(ChannelHandlerContext ctx, int streamId, Http2Headers headers,
int padding, boolean endStream, ChannelPromise promise);
int padding, boolean endStream, ChannelPromise promise) throws Http2Exception;
/**
* Writes a HEADERS frame with priority specified to the remote endpoint.
@ -75,10 +83,18 @@ public interface Http2FrameWriter extends Http2DataWriter, Closeable {
* @param endStream indicates if this is the last frame to be sent for the stream.
* @param promise the promise for the write.
* @return the future for the write.
* @throws Http2Exception if an exception occurs while encoding headers.
* <a href="https://tools.ietf.org/html/rfc7540#section-10.5.1">Section 10.5.1</a> states the following:
* <pre>
* The header block MUST be processed to ensure a consistent connection state, unless the connection is closed.
* </pre>
* If this call has modified the HPACK header state you <strong>MUST</strong> throw a connection error.
* <p>
* If this call has <strong>NOT</strong> modified the HPACK header state you are free to throw a stream error.
*/
ChannelFuture writeHeaders(ChannelHandlerContext ctx, int streamId, Http2Headers headers,
int streamDependency, short weight, boolean exclusive, int padding, boolean endStream,
ChannelPromise promise);
ChannelPromise promise) throws Http2Exception;
/**
* Writes a PRIORITY frame to the remote endpoint.
@ -151,9 +167,17 @@ public interface Http2FrameWriter extends Http2DataWriter, Closeable {
* 256 (inclusive).
* @param promise the promise for the write.
* @return the future for the write.
* @throws Http2Exception if an exception occurs while encoding headers.
* <a href="https://tools.ietf.org/html/rfc7540#section-10.5.1">Section 10.5.1</a> states the following:
* <pre>
* The header block MUST be processed to ensure a consistent connection state, unless the connection is closed.
* </pre>
* If this call has modified the HPACK header state you <strong>MUST</strong> throw a connection error.
* <p>
* If this call has <strong>NOT</strong> modified the HPACK header state you are free to throw a stream error.
*/
ChannelFuture writePushPromise(ChannelHandlerContext ctx, int streamId, int promisedStreamId,
Http2Headers headers, int padding, ChannelPromise promise);
Http2Headers headers, int padding, ChannelPromise promise) throws Http2Exception;
/**
* Writes a GO_AWAY frame to the remote endpoint.

View File

@ -1,51 +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 io.netty.util.internal.UnstableApi;
/**
* Extracts a common interface for encoding and processing HPACK header constraints
*/
@UnstableApi
public interface Http2HeaderTable {
/**
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_HEADER_TABLE_SIZE</a>. This method should
* only be called by Netty (not users) as a result of a receiving a {@code SETTINGS} frame.
*/
void maxHeaderTableSize(long max) throws Http2Exception;
/**
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_HEADER_TABLE_SIZE</a>. The initial value
* returned by this method must be {@link Http2CodecUtil#DEFAULT_HEADER_TABLE_SIZE}.
*/
long maxHeaderTableSize();
/**
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a>. This method should
* only be called by Netty (not users) as a result of a receiving a {@code SETTINGS} frame.
*/
void maxHeaderListSize(long max) throws Http2Exception;
/**
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a>. The initial value
* returned by this method must be {@link Http2CodecUtil#DEFAULT_HEADER_LIST_SIZE}.
*/
long maxHeaderListSize();
}

View File

@ -28,9 +28,44 @@ public interface Http2HeadersDecoder {
*/
interface Configuration {
/**
* Access the Http2HeaderTable for this {@link Http2HeadersDecoder}
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_HEADER_TABLE_SIZE</a>.
* This method should only be called by Netty (not users) as a result of a receiving a {@code SETTINGS} frame.
*/
Http2HeaderTable headerTable();
void maxHeaderTableSize(long max) throws Http2Exception;
/**
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_HEADER_TABLE_SIZE</a>. The initial value
* returned by this method must be {@link Http2CodecUtil#DEFAULT_HEADER_TABLE_SIZE}.
*/
long maxHeaderTableSize();
/**
* Configure the maximum allowed size in bytes of each set of headers.
* <p>
* This method should only be called by Netty (not users) as a result of a receiving a {@code SETTINGS} frame.
* @param max <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a>.
* If this limit is exceeded the implementation should attempt to keep the HPACK header tables up to date
* by processing data from the peer, but a {@code RST_STREAM} frame will be sent for the offending stream.
* @param goAwayMax Must be {@code >= max}. A {@code GO_AWAY} frame will be generated if this limit is exceeded
* for any particular stream.
* @throws Http2Exception if limits exceed the RFC's boundaries or {@code max > goAwayMax}.
*/
void maxHeaderListSize(long max, long goAwayMax) throws Http2Exception;
/**
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a>.
*/
long maxHeaderListSize();
/**
* Represents the upper bound in bytes for a set of headers before a {@code GO_AWAY} should be sent.
* This will be {@code <=}
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a>.
*/
long maxHeaderListSizeGoAway();
}
/**

View File

@ -28,9 +28,31 @@ public interface Http2HeadersEncoder {
*/
interface Configuration {
/**
* Access the Http2HeaderTable for this {@link Http2HeadersEncoder}
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_HEADER_TABLE_SIZE</a>.
* This method should only be called by Netty (not users) as a result of a receiving a {@code SETTINGS} frame.
*/
Http2HeaderTable headerTable();
void maxHeaderTableSize(long max) throws Http2Exception;
/**
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_HEADER_TABLE_SIZE</a>.
* The initial value returned by this method must be {@link Http2CodecUtil#DEFAULT_HEADER_TABLE_SIZE}.
*/
long maxHeaderTableSize();
/**
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a>.
* This method should only be called by Netty (not users) as a result of a receiving a {@code SETTINGS} frame.
*/
void maxHeaderListSize(long max) throws Http2Exception;
/**
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a>.
*/
long maxHeaderListSize();
}
/**

View File

@ -46,7 +46,7 @@ public class Http2OutboundFrameLogger implements Http2FrameWriter {
@Override
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, int streamId,
Http2Headers headers, int padding, boolean endStream, ChannelPromise promise) {
Http2Headers headers, int padding, boolean endStream, ChannelPromise promise) throws Http2Exception {
logger.logHeaders(OUTBOUND, ctx, streamId, headers, padding, endStream);
return writer.writeHeaders(ctx, streamId, headers, padding, endStream, promise);
}
@ -54,7 +54,7 @@ public class Http2OutboundFrameLogger implements Http2FrameWriter {
@Override
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, int streamId,
Http2Headers headers, int streamDependency, short weight, boolean exclusive,
int padding, boolean endStream, ChannelPromise promise) {
int padding, boolean endStream, ChannelPromise promise) throws Http2Exception {
logger.logHeaders(OUTBOUND, ctx, streamId, headers, streamDependency, weight, exclusive,
padding, endStream);
return writer.writeHeaders(ctx, streamId, headers, streamDependency, weight,
@ -101,7 +101,7 @@ public class Http2OutboundFrameLogger implements Http2FrameWriter {
@Override
public ChannelFuture writePushPromise(ChannelHandlerContext ctx, int streamId,
int promisedStreamId, Http2Headers headers, int padding, ChannelPromise promise) {
int promisedStreamId, Http2Headers headers, int padding, ChannelPromise promise) throws Http2Exception {
logger.logPushPromise(OUTBOUND, ctx, streamId, promisedStreamId, headers, padding);
return writer.writePushPromise(ctx, streamId, promisedStreamId, headers, padding, promise);
}

View File

@ -130,8 +130,9 @@ public interface Http2RemoteFlowController extends Http2FlowController {
*
* @param ctx The context to use for writing.
* @param allowedBytes an upper bound on the number of bytes the payload can write at this time.
* @throws Http2Exception if an error occurs during the encode or write.
*/
void write(ChannelHandlerContext ctx, int allowedBytes);
void write(ChannelHandlerContext ctx, int allowedBytes) throws Http2Exception;
/**
* Merge the contents of the {@code next} message into this message so they can be written out as one unit.

View File

@ -193,4 +193,14 @@ public interface Http2Stream {
* Indicates whether or not headers was sent to the remote endpoint.
*/
boolean isHeadersSent();
/**
* Indicates that a push promise was sent to the remote endpoint.
*/
Http2Stream pushPromiseSent();
/**
* Indicates whether or not a push promise was sent to the remote endpoint.
*/
boolean isPushPromiseSent();
}

View File

@ -112,6 +112,7 @@ public class HttpToHttp2ConnectionHandler extends Http2ConnectionHandler {
}
}
} catch (Throwable t) {
onError(ctx, t);
promiseAggregator.setFailure(t);
} finally {
if (release) {
@ -123,7 +124,7 @@ public class HttpToHttp2ConnectionHandler extends Http2ConnectionHandler {
private static void writeHeaders(ChannelHandlerContext ctx, Http2ConnectionEncoder encoder, int streamId,
HttpHeaders headers, Http2Headers http2Headers, boolean endStream,
SimpleChannelPromiseAggregator promiseAggregator) {
SimpleChannelPromiseAggregator promiseAggregator) throws Http2Exception {
int dependencyId = headers.getInt(
HttpConversionUtil.ExtensionHeaderNames.STREAM_DEPENDENCY_ID.text(), 0);
short weight = headers.getShort(

View File

@ -137,7 +137,7 @@ public class StreamBufferingEncoder extends DecoratingHttp2ConnectionEncoder {
@Override
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, int streamId, Http2Headers headers,
int padding, boolean endStream, ChannelPromise promise) {
int padding, boolean endStream, ChannelPromise promise) throws Http2Exception {
return writeHeaders(ctx, streamId, headers, 0, Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT,
false, padding, endStream, promise);
}
@ -145,7 +145,7 @@ public class StreamBufferingEncoder extends DecoratingHttp2ConnectionEncoder {
@Override
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, int streamId, Http2Headers headers,
int streamDependency, short weight, boolean exclusive,
int padding, boolean endOfStream, ChannelPromise promise) {
int padding, boolean endOfStream, ChannelPromise promise) throws Http2Exception {
if (closed) {
return promise.setFailure(new Http2ChannelClosedException());
}
@ -240,7 +240,11 @@ public class StreamBufferingEncoder extends DecoratingHttp2ConnectionEncoder {
while (!pendingStreams.isEmpty() && canCreateStream()) {
Map.Entry<Integer, PendingStream> entry = pendingStreams.pollFirstEntry();
PendingStream pendingStream = entry.getValue();
pendingStream.sendFrames();
try {
pendingStream.sendFrames();
} catch (Throwable t) {
pendingStream.close(t);
}
}
}
@ -277,7 +281,7 @@ public class StreamBufferingEncoder extends DecoratingHttp2ConnectionEncoder {
this.streamId = streamId;
}
void sendFrames() {
void sendFrames() throws Http2Exception {
for (Frame frame : frames) {
frame.send(ctx, streamId);
}
@ -308,7 +312,7 @@ public class StreamBufferingEncoder extends DecoratingHttp2ConnectionEncoder {
}
}
abstract void send(ChannelHandlerContext ctx, int streamId);
abstract void send(ChannelHandlerContext ctx, int streamId) throws Http2Exception;
}
private final class HeadersFrame extends Frame {
@ -331,7 +335,7 @@ public class StreamBufferingEncoder extends DecoratingHttp2ConnectionEncoder {
}
@Override
void send(ChannelHandlerContext ctx, int streamId) {
void send(ChannelHandlerContext ctx, int streamId) throws Http2Exception {
writeHeaders(ctx, streamId, headers, streamDependency, weight, exclusive, padding,
endOfStream, promise);
}

View File

@ -37,7 +37,6 @@ import io.netty.handler.codec.http2.Http2Headers;
import io.netty.handler.codec.http2.internal.hpack.HpackUtil.IndexType;
import io.netty.util.AsciiString;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_TABLE_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_TABLE_SIZE;
@ -45,9 +44,11 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.MIN_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MIN_HEADER_TABLE_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.headerListSizeExceeded;
import static io.netty.handler.codec.http2.Http2Error.COMPRESSION_ERROR;
import static io.netty.handler.codec.http2.Http2Error.INTERNAL_ERROR;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
import static io.netty.util.AsciiString.EMPTY_STRING;
import static io.netty.util.internal.ObjectUtil.checkPositive;
import static io.netty.util.internal.ThrowableUtil.unknownStackTrace;
public final class Decoder {
@ -82,25 +83,30 @@ public final class Decoder {
private final DynamicTable dynamicTable;
private final HuffmanDecoder huffmanDecoder;
private long maxHeaderListSizeGoAway;
private long maxHeaderListSize;
private long maxDynamicTableSize;
private long encoderMaxDynamicTableSize;
private boolean maxDynamicTableSizeChangeRequired;
public Decoder() {
this(32);
}
public Decoder(int initialHuffmanDecodeCapacity) {
this(initialHuffmanDecodeCapacity, DEFAULT_HEADER_TABLE_SIZE);
/**
* Create a new instance.
* @param maxHeaderListSize This is the only setting that can be configured before notifying the peer.
* This is because <a href="https://tools.ietf.org/html/rfc7540#section-6.5.1">SETTINGS_MAX_HEADER_LIST_SIZE</a>
* allows a lower than advertised limit from being enforced, and the default limit is unlimited
* (which is dangerous).
* @param initialHuffmanDecodeCapacity Size of an intermediate buffer used during huffman decode.
*/
public Decoder(long maxHeaderListSize, int initialHuffmanDecodeCapacity) {
this(maxHeaderListSize, initialHuffmanDecodeCapacity, DEFAULT_HEADER_TABLE_SIZE);
}
/**
* Exposed Used for testing only! Default values used in the initial settings frame are overriden intentionally
* for testing but violate the RFC if used outside the scope of testing.
*/
Decoder(int initialHuffmanDecodeCapacity, int maxHeaderTableSize) {
maxHeaderListSize = DEFAULT_HEADER_LIST_SIZE;
Decoder(long maxHeaderListSize, int initialHuffmanDecodeCapacity, int maxHeaderTableSize) {
this.maxHeaderListSize = maxHeaderListSizeGoAway = checkPositive(maxHeaderListSize, "maxHeaderListSize");
maxDynamicTableSize = encoderMaxDynamicTableSize = maxHeaderTableSize;
maxDynamicTableSizeChangeRequired = false;
dynamicTable = new DynamicTable(maxHeaderTableSize);
@ -208,8 +214,8 @@ public final class Decoder {
if (index == 0x7f) {
state = READ_LITERAL_HEADER_NAME_LENGTH;
} else {
if (index > maxHeaderListSize - headersLength) {
headerListSizeExceeded(streamId, maxHeaderListSize, true);
if (index > maxHeaderListSizeGoAway - headersLength) {
headerListSizeExceeded(maxHeaderListSizeGoAway);
}
nameLength = index;
state = READ_LITERAL_HEADER_NAME;
@ -220,8 +226,8 @@ public final class Decoder {
// Header Name is a Literal String
nameLength = decodeULE128(in, index);
if (nameLength > maxHeaderListSize - headersLength) {
headerListSizeExceeded(streamId, maxHeaderListSize, true);
if (nameLength > maxHeaderListSizeGoAway - headersLength) {
headerListSizeExceeded(maxHeaderListSizeGoAway);
}
state = READ_LITERAL_HEADER_NAME;
break;
@ -252,8 +258,8 @@ public final class Decoder {
break;
default:
// Check new header size against max header size
if ((long) index + nameLength > maxHeaderListSize - headersLength) {
headerListSizeExceeded(streamId, maxHeaderListSize, true);
if ((long) index + nameLength > maxHeaderListSizeGoAway - headersLength) {
headerListSizeExceeded(maxHeaderListSizeGoAway);
}
valueLength = index;
state = READ_LITERAL_HEADER_VALUE;
@ -266,8 +272,8 @@ public final class Decoder {
valueLength = decodeULE128(in, index);
// Check new header size against max header size
if ((long) valueLength + nameLength > maxHeaderListSize - headersLength) {
headerListSizeExceeded(streamId, maxHeaderListSize, true);
if ((long) valueLength + nameLength > maxHeaderListSizeGoAway - headersLength) {
headerListSizeExceeded(maxHeaderListSizeGoAway);
}
state = READ_LITERAL_HEADER_VALUE;
break;
@ -286,6 +292,10 @@ public final class Decoder {
default:
throw new Error("should not reach here state: " + state);
}
if (headersLength > maxHeaderListSize) {
headerListSizeExceeded(streamId, maxHeaderListSize, true);
}
}
}
@ -307,18 +317,27 @@ public final class Decoder {
}
}
public void setMaxHeaderListSize(long maxHeaderListSize) throws Http2Exception {
public void setMaxHeaderListSize(long maxHeaderListSize, long maxHeaderListSizeGoAway) throws Http2Exception {
if (maxHeaderListSizeGoAway < maxHeaderListSize || maxHeaderListSizeGoAway < 0) {
throw connectionError(INTERNAL_ERROR, "Header List Size GO_AWAY %d must be positive and >= %d",
maxHeaderListSizeGoAway, maxHeaderListSize);
}
if (maxHeaderListSize < MIN_HEADER_LIST_SIZE || maxHeaderListSize > MAX_HEADER_LIST_SIZE) {
throw connectionError(PROTOCOL_ERROR, "Header List Size must be >= %d and <= %d but was %d",
MIN_HEADER_TABLE_SIZE, MAX_HEADER_TABLE_SIZE, maxHeaderListSize);
}
this.maxHeaderListSize = maxHeaderListSize;
this.maxHeaderListSizeGoAway = maxHeaderListSizeGoAway;
}
public long getMaxHeaderListSize() {
return maxHeaderListSize;
}
public long getMaxHeaderListSizeGoAway() {
return maxHeaderListSizeGoAway;
}
/**
* Return the maximum table size. This is the maximum size allowed by both the encoder and the
* decoder.
@ -404,8 +423,8 @@ public final class Decoder {
private long addHeader(int streamId, Http2Headers headers, CharSequence name, CharSequence value,
long headersLength) throws Http2Exception {
headersLength += name.length() + value.length();
if (headersLength > maxHeaderListSize) {
headerListSizeExceeded(streamId, maxHeaderListSize, true);
if (headersLength > maxHeaderListSizeGoAway) {
headerListSizeExceeded(maxHeaderListSizeGoAway);
}
headers.add(name, value);
return headersLength;

View File

@ -50,6 +50,7 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.MIN_HEADER_TABLE_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.headerListSizeExceeded;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
import static io.netty.handler.codec.http2.internal.hpack.HeaderField.sizeOf;
import static io.netty.handler.codec.http2.internal.hpack.HpackUtil.IndexType.INCREMENTAL;
import static io.netty.handler.codec.http2.internal.hpack.HpackUtil.IndexType.NEVER;
import static io.netty.handler.codec.http2.internal.hpack.HpackUtil.IndexType.NONE;
@ -116,18 +117,18 @@ public final class Encoder {
SensitivityDetector sensitivityDetector)
throws Http2Exception {
long headerSize = 0;
// To ensure we stay consistent with our peer check the size is valid before we potentially modify HPACK state.
for (Map.Entry<CharSequence, CharSequence> header : headers) {
CharSequence name = header.getKey();
CharSequence value = header.getValue();
long currHeaderSize = HeaderField.sizeOf(name, value);
// OK to increment now and check for bounds after because this value is limited to unsigned int and will not
// overflow.
headerSize += currHeaderSize;
headerSize += sizeOf(name, value);
if (headerSize > maxHeaderListSize) {
headerListSizeExceeded(streamId, maxHeaderListSize, false);
}
encodeHeader(out, name, value, sensitivityDetector.isSensitive(name, value), currHeaderSize);
}
encodeHeadersIgnoreMaxHeaderListSize(out, headers, sensitivityDetector);
}
private void encodeHeadersIgnoreMaxHeaderListSize(ByteBuf out, Http2Headers headers,
@ -135,8 +136,7 @@ public final class Encoder {
for (Map.Entry<CharSequence, CharSequence> header : headers) {
CharSequence name = header.getKey();
CharSequence value = header.getValue();
encodeHeader(out, name, value, sensitivityDetector.isSensitive(name, value),
HeaderField.sizeOf(name, value));
encodeHeader(out, name, value, sensitivityDetector.isSensitive(name, value), sizeOf(name, value));
}
}

View File

@ -565,7 +565,7 @@ public class DefaultHttp2ConnectionEncoderTest {
}
@Test
public void canWriteHeaderFrameAfterGoAwayReceived() {
public void canWriteHeaderFrameAfterGoAwayReceived() throws Http2Exception {
writeAllFlowControlledFrames();
goAwayReceived(STREAM_ID);
ChannelPromise promise = newPromise();

View File

@ -22,7 +22,8 @@ import io.netty.util.AsciiString;
import org.junit.Before;
import org.junit.Test;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MIN_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2HeadersEncoder.NEVER_SENSITIVE;
import static io.netty.handler.codec.http2.Http2TestUtil.newTestEncoder;
import static io.netty.handler.codec.http2.Http2TestUtil.randomBytes;
@ -57,7 +58,9 @@ public class DefaultHttp2HeadersDecoderTest {
@Test(expected = Http2Exception.class)
public void testExceedHeaderSize() throws Exception {
ByteBuf buf = encode(randomBytes(DEFAULT_HEADER_LIST_SIZE), randomBytes(1));
final int maxListSize = 100;
decoder.configuration().maxHeaderListSize(maxListSize, maxListSize);
ByteBuf buf = encode(randomBytes(maxListSize), randomBytes(1));
try {
decoder.decodeHeaders(0, buf);
fail();
@ -66,6 +69,36 @@ public class DefaultHttp2HeadersDecoderTest {
}
}
@Test
public void decodeLargerThanHeaderListSizeButLessThanGoAway() throws Exception {
decoder.maxHeaderListSize(MIN_HEADER_LIST_SIZE, MAX_HEADER_LIST_SIZE);
ByteBuf buf = encode(b(":method"), b("GET"));
final int streamId = 1;
try {
decoder.decodeHeaders(streamId, buf);
fail();
} catch (Http2Exception.HeaderListSizeException e) {
assertEquals(streamId, e.streamId());
} finally {
buf.release();
}
}
@Test
public void decodeLargerThanHeaderListSizeGoAway() throws Exception {
decoder.maxHeaderListSize(MIN_HEADER_LIST_SIZE, MIN_HEADER_LIST_SIZE);
ByteBuf buf = encode(b(":method"), b("GET"));
final int streamId = 1;
try {
decoder.decodeHeaders(streamId, buf);
fail();
} catch (Http2Exception e) {
assertEquals(Http2Error.PROTOCOL_ERROR, e.error());
} finally {
buf.release();
}
}
private static byte[] b(String string) {
return string.getBytes(UTF_8);
}

View File

@ -52,7 +52,7 @@ public class DefaultHttp2HeadersEncoderTest {
@Test(expected = StreamException.class)
public void headersExceedMaxSetSizeShouldFail() throws Http2Exception {
Http2Headers headers = headers();
encoder.headerTable().maxHeaderListSize(2);
encoder.maxHeaderListSize(2);
encoder.encodeHeaders(3 /* randomly chosen */, headers, Unpooled.buffer());
}

View File

@ -446,6 +446,7 @@ public class Http2ConnectionHandlerTest {
when(frameWriter.writeRstStream(eq(ctx), eq(STREAM_ID),
anyLong(), any(ChannelPromise.class))).thenReturn(future);
when(stream.state()).thenReturn(CLOSED);
when(stream.isHeadersSent()).thenReturn(true);
// The stream is "closed" but is still known about by the connection (connection().stream(..)
// will return the stream). We should still write a RST_STREAM frame in this scenario.
handler.resetStream(ctx, STREAM_ID, STREAM_CLOSED.code(), promise);

View File

@ -59,6 +59,8 @@ import static java.util.concurrent.TimeUnit.SECONDS;
import static org.junit.Assert.assertArrayEquals;
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 static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyBoolean;
@ -211,6 +213,137 @@ public class Http2ConnectionRoundtripTest {
http2Client.gracefulShutdownTimeoutMillis(0);
}
@Test
public void encodeViolatesMaxHeaderListSizeCanStillUseConnection() throws Exception {
bootstrapEnv(1, 2, 1, 0, 0);
final CountDownLatch serverSettingsAckLatch1 = new CountDownLatch(2);
final CountDownLatch serverSettingsAckLatch2 = new CountDownLatch(3);
final CountDownLatch clientSettingsLatch1 = new CountDownLatch(3);
final CountDownLatch serverRevHeadersLatch = new CountDownLatch(1);
final CountDownLatch clientHeadersLatch = new CountDownLatch(1);
final CountDownLatch clientDataWrite = new CountDownLatch(1);
final AtomicReference<Throwable> clientHeadersWriteException = new AtomicReference<Throwable>();
final AtomicReference<Throwable> clientHeadersWriteException2 = new AtomicReference<Throwable>();
final AtomicReference<Throwable> clientDataWriteException = new AtomicReference<Throwable>();
final Http2Headers headers = dummyHeaders();
doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
serverSettingsAckLatch1.countDown();
serverSettingsAckLatch2.countDown();
return null;
}
}).when(serverListener).onSettingsAckRead(any(ChannelHandlerContext.class));
doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
clientSettingsLatch1.countDown();
return null;
}
}).when(clientListener).onSettingsRead(any(ChannelHandlerContext.class), any(Http2Settings.class));
// Manually add a listener for when we receive the expected headers on the server.
doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
serverRevHeadersLatch.countDown();
return null;
}
}).when(serverListener).onHeadersRead(any(ChannelHandlerContext.class), eq(5), eq(headers),
anyInt(), anyShort(), anyBoolean(), eq(0), eq(true));
// Set the maxHeaderListSize to 100 so we may be able to write some headers, but not all. We want to verify
// that we don't corrupt state if some can be written but not all.
runInChannel(serverConnectedChannel, new Http2Runnable() {
@Override
public void run() throws Http2Exception {
http2Server.encoder().writeSettings(serverCtx(),
new Http2Settings().copyFrom(http2Server.decoder().localSettings())
.maxHeaderListSize(100),
serverNewPromise());
http2Server.flush(serverCtx());
}
});
assertTrue(serverSettingsAckLatch1.await(5, SECONDS));
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() throws Http2Exception {
http2Client.encoder().writeHeaders(ctx(), 3, headers, 0, false, newPromise())
.addListener(new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
clientHeadersWriteException.set(future.cause());
}
});
// It is expected that this write should fail locally and the remote peer will never see this.
http2Client.encoder().writeData(ctx(), 3, Unpooled.buffer(), 0, true, newPromise())
.addListener(new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
clientDataWriteException.set(future.cause());
clientDataWrite.countDown();
}
});
http2Client.flush(ctx());
}
});
assertTrue(clientDataWrite.await(5, SECONDS));
assertNotNull("Header encode should have exceeded maxHeaderListSize!", clientHeadersWriteException.get());
assertNotNull("Data on closed stream should fail!", clientDataWriteException.get());
// Set the maxHeaderListSize to the max value so we can send the headers.
runInChannel(serverConnectedChannel, new Http2Runnable() {
@Override
public void run() throws Http2Exception {
http2Server.encoder().writeSettings(serverCtx(),
new Http2Settings().copyFrom(http2Server.decoder().localSettings())
.maxHeaderListSize(Http2CodecUtil.MAX_HEADER_LIST_SIZE),
serverNewPromise());
http2Server.flush(serverCtx());
}
});
assertTrue(clientSettingsLatch1.await(5, SECONDS));
assertTrue(serverSettingsAckLatch2.await(5, SECONDS));
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() throws Http2Exception {
http2Client.encoder().writeHeaders(ctx(), 5, headers, 0, true,
newPromise()).addListener(new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
clientHeadersWriteException2.set(future.cause());
clientHeadersLatch.countDown();
}
});
http2Client.flush(ctx());
}
});
assertTrue(clientHeadersLatch.await(5, SECONDS));
assertNull("Client write of headers should succeed with increased header list size!",
clientHeadersWriteException2.get());
assertTrue(serverRevHeadersLatch.await(5, SECONDS));
verify(serverListener, never()).onDataRead(any(ChannelHandlerContext.class), anyInt(), any(ByteBuf.class),
anyInt(), anyBoolean());
// Verify that no errors have been received.
verify(serverListener, never()).onGoAwayRead(any(ChannelHandlerContext.class), anyInt(), anyLong(),
any(ByteBuf.class));
verify(serverListener, never()).onRstStreamRead(any(ChannelHandlerContext.class), anyInt(), anyLong());
verify(clientListener, never()).onGoAwayRead(any(ChannelHandlerContext.class), anyInt(), anyLong(),
any(ByteBuf.class));
verify(clientListener, never()).onRstStreamRead(any(ChannelHandlerContext.class), anyInt(), anyLong());
}
@Test
public void http2ExceptionInPipelineShouldCloseConnection() throws Exception {
bootstrapEnv(1, 1, 2, 1);
@ -586,10 +719,18 @@ public class Http2ConnectionRoundtripTest {
return clientChannel.pipeline().firstContext();
}
private ChannelHandlerContext serverCtx() {
return serverConnectedChannel.pipeline().firstContext();
}
private ChannelPromise newPromise() {
return ctx().newPromise();
}
private ChannelPromise serverNewPromise() {
return serverCtx().newPromise();
}
private static Http2Headers dummyHeaders() {
return new DefaultHttp2Headers(false).method(new AsciiString("GET")).scheme(new AsciiString("https"))
.authority(new AsciiString("example.org")).path(new AsciiString("/some/path/resource2"))

View File

@ -40,7 +40,6 @@ import java.util.LinkedList;
import java.util.List;
import static io.netty.buffer.Unpooled.EMPTY_BUFFER;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_PADDING;
import static io.netty.handler.codec.http2.Http2HeadersEncoder.NEVER_SENSITIVE;
import static io.netty.handler.codec.http2.Http2TestUtil.newTestDecoder;
@ -283,7 +282,9 @@ public class Http2FrameRoundtripTest {
@Test
public void headersThatAreTooBigShouldFail() throws Exception {
reader = new DefaultHttp2FrameReader(false);
final Http2Headers headers = headersOfSize(DEFAULT_HEADER_LIST_SIZE + 1);
final int maxListSize = 100;
reader.configuration().headersConfiguration().maxHeaderListSize(maxListSize, maxListSize);
final Http2Headers headers = headersOfSize(maxListSize + 1);
writer.writeHeaders(ctx, STREAM_ID, headers, 2, (short) 3, true, MAX_PADDING, true, ctx.newPromise());
try {
readFrames();

View File

@ -74,10 +74,10 @@ public class Http2HeaderBlockIOTest {
@Test
public void setMaxHeaderSizeShouldBeSuccessful() throws Http2Exception {
encoder.headerTable().maxHeaderTableSize(10);
encoder.maxHeaderTableSize(10);
Http2Headers in = headers();
assertRoundtripSuccessful(in);
assertEquals(10, decoder.headerTable().maxHeaderTableSize());
assertEquals(10, decoder.maxHeaderTableSize());
}
private void assertRoundtripSuccessful(Http2Headers in) throws Http2Exception {

View File

@ -122,9 +122,8 @@ public final class Http2TestUtil {
}
public static Decoder newTestDecoder(long maxHeaderListSize, long maxHeaderTableSize) throws Http2Exception {
Decoder decoder = new Decoder();
Decoder decoder = new Decoder(maxHeaderListSize, 32);
decoder.setMaxHeaderTableSize(maxHeaderTableSize);
decoder.setMaxHeaderListSize(maxHeaderListSize);
return decoder;
}

View File

@ -147,7 +147,7 @@ public class InboundHttp2ToHttpAdapterTest {
.path(new AsciiString("/some/path/resource2"));
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, true, newPromiseClient());
clientChannel.flush();
}
@ -183,7 +183,7 @@ public class InboundHttp2ToHttpAdapterTest {
.add(HttpHeaderNames.COOKIE, "e=f");
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, true, newPromiseClient());
clientChannel.flush();
}
@ -218,7 +218,7 @@ public class InboundHttp2ToHttpAdapterTest {
.add(HttpHeaderNames.COOKIE, "e=f");
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, true, newPromiseClient());
clientChannel.flush();
}
@ -245,7 +245,7 @@ public class InboundHttp2ToHttpAdapterTest {
new AsciiString("Ãã".getBytes(CharsetUtil.UTF_8)));
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, true, newPromiseClient());
clientChannel.flush();
}
@ -270,7 +270,7 @@ public class InboundHttp2ToHttpAdapterTest {
new AsciiString("/some/path/resource2"));
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, false, newPromiseClient());
clientHandler.encoder().writeData(ctxClient(), 3, content.retainedDuplicate(), 0, true,
newPromiseClient());
@ -304,7 +304,7 @@ public class InboundHttp2ToHttpAdapterTest {
final int midPoint = text.length() / 2;
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, false, newPromiseClient());
clientHandler.encoder().writeData(
ctxClient(), 3, content.retainedSlice(0, midPoint), 0, false, newPromiseClient());
@ -340,7 +340,7 @@ public class InboundHttp2ToHttpAdapterTest {
new AsciiString("/some/path/resource2"));
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, false, newPromiseClient());
clientHandler.encoder().writeData(ctxClient(), 3, content.retain(), 0, false, newPromiseClient());
clientHandler.encoder().writeData(ctxClient(), 3, content.retain(), 0, false, newPromiseClient());
@ -384,7 +384,7 @@ public class InboundHttp2ToHttpAdapterTest {
.add(new AsciiString("foo2"), new AsciiString("goo3"));
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, false, newPromiseClient());
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers2, 0, false, newPromiseClient());
clientHandler.encoder().writeData(ctxClient(), 3, content.retain(), 0, true, newPromiseClient());
@ -425,7 +425,7 @@ public class InboundHttp2ToHttpAdapterTest {
.add(new AsciiString("foo2"), new AsciiString("goo3"));
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, false, newPromiseClient());
clientHandler.encoder().writeData(ctxClient(), 3, content.retainedDuplicate(), 0, false,
newPromiseClient());
@ -470,7 +470,7 @@ public class InboundHttp2ToHttpAdapterTest {
new AsciiString("/some/path/resource2"));
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, false, newPromiseClient());
clientHandler.encoder().writeHeaders(ctxClient(), 5, http2Headers2, 0, false, newPromiseClient());
clientChannel.flush(); // Headers are queued in the flow controller and so flush them.
@ -528,7 +528,7 @@ public class InboundHttp2ToHttpAdapterTest {
httpHeaders3.setInt(HttpHeaderNames.CONTENT_LENGTH, 0);
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, false, newPromiseClient());
clientHandler.encoder().writeHeaders(ctxClient(), 5, http2Headers2, 0, false, newPromiseClient());
clientHandler.encoder().writeData(ctxClient(), 3, content.retainedDuplicate(), 0, true,
@ -587,7 +587,7 @@ public class InboundHttp2ToHttpAdapterTest {
.path(new AsciiString("/push/test"));
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers3, 0, true, newPromiseClient());
clientChannel.flush();
}
@ -604,7 +604,7 @@ public class InboundHttp2ToHttpAdapterTest {
.authority(new AsciiString("example.org"));
runInChannel(serverConnectedChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
serverHandler.encoder().writeHeaders(ctxServer(), 3, http2Headers, 0, false, newPromiseServer());
serverHandler.encoder().writePushPromise(ctxServer(), 3, 2, http2Headers2, 0, newPromiseServer());
serverHandler.encoder().writeData(ctxServer(), 3, content.retainedDuplicate(), 0, true,
@ -650,7 +650,7 @@ public class InboundHttp2ToHttpAdapterTest {
try {
runInChannel(clientChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
clientHandler.encoder().writeHeaders(ctxClient(), 3, http2Headers, 0, false, newPromiseClient());
clientChannel.flush();
}
@ -663,7 +663,7 @@ public class InboundHttp2ToHttpAdapterTest {
final Http2Headers http2HeadersResponse = new DefaultHttp2Headers().status(new AsciiString("100"));
runInChannel(serverConnectedChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
serverHandler.encoder().writeHeaders(ctxServer(), 3, http2HeadersResponse, 0, false,
newPromiseServer());
serverConnectedChannel.flush();
@ -692,7 +692,7 @@ public class InboundHttp2ToHttpAdapterTest {
final Http2Headers http2HeadersResponse2 = new DefaultHttp2Headers().status(new AsciiString("200"));
runInChannel(serverConnectedChannel, new Http2Runnable() {
@Override
public void run() {
public void run() throws Http2Exception {
serverHandler.encoder().writeHeaders(ctxServer(), 3, http2HeadersResponse2, 0, true,
newPromiseServer());
serverConnectedChannel.flush();

View File

@ -487,9 +487,9 @@ public class StreamBufferingEncoderTest {
}
private ChannelFuture encoderWriteHeaders(int streamId, ChannelPromise promise) {
encoder.writeHeaders(ctx, streamId, new DefaultHttp2Headers(), 0, DEFAULT_PRIORITY_WEIGHT,
false, 0, false, promise);
try {
encoder.writeHeaders(ctx, streamId, new DefaultHttp2Headers(), 0, DEFAULT_PRIORITY_WEIGHT,
false, 0, false, promise);
encoder.flowController().writePendingBytes();
return promise;
} catch (Http2Exception e) {
@ -498,9 +498,13 @@ public class StreamBufferingEncoderTest {
}
private void writeVerifyWriteHeaders(VerificationMode mode, int streamId) {
verify(writer, mode).writeHeaders(eq(ctx), eq(streamId), any(Http2Headers.class), eq(0),
eq(DEFAULT_PRIORITY_WEIGHT), eq(false), eq(0),
eq(false), any(ChannelPromise.class));
try {
verify(writer, mode).writeHeaders(eq(ctx), eq(streamId), any(Http2Headers.class), eq(0),
eq(DEFAULT_PRIORITY_WEIGHT), eq(false), eq(0),
eq(false), any(ChannelPromise.class));
} catch (Http2Exception e) {
throw new RuntimeException(e);
}
}
private Answer<ChannelFuture> successAnswer() {

View File

@ -69,7 +69,7 @@ public class DecoderTest {
@Before
public void setUp() throws Http2Exception {
decoder = new Decoder();
decoder = new Decoder(8192, 32);
mockHeaders = mock(Http2Headers.class);
}

View File

@ -22,6 +22,7 @@ import io.netty.handler.codec.http2.Http2Headers;
import org.junit.Before;
import org.junit.Test;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_TABLE_SIZE;
import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.mock;
@ -34,7 +35,7 @@ public class EncoderTest {
@Before
public void setUp() throws Http2Exception {
encoder = new Encoder();
decoder = new Decoder();
decoder = new Decoder(DEFAULT_HEADER_LIST_SIZE, 32);
mockHeaders = mock(Http2Headers.class);
}

View File

@ -56,6 +56,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2TestUtil.newTestEncoder;
@ -177,7 +178,7 @@ final class TestCase {
maxHeaderTableSize = Integer.MAX_VALUE;
}
return new Decoder(32, maxHeaderTableSize);
return new Decoder(DEFAULT_HEADER_LIST_SIZE, 32, maxHeaderTableSize);
}
private static byte[] encode(Encoder encoder, List<HeaderField> headers, int maxHeaderTableSize,

View File

@ -23,6 +23,7 @@ import io.netty.handler.codec.http2.DefaultHttp2Headers;
import io.netty.handler.codec.http2.Http2ConnectionDecoder;
import io.netty.handler.codec.http2.Http2ConnectionEncoder;
import io.netty.handler.codec.http2.Http2ConnectionHandler;
import io.netty.handler.codec.http2.Http2Exception;
import io.netty.handler.codec.http2.Http2Flags;
import io.netty.handler.codec.http2.Http2FrameListener;
import io.netty.handler.codec.http2.Http2Headers;
@ -73,7 +74,7 @@ public final class HelloWorldHttp2Handler extends Http2ConnectionHandler impleme
/**
* Sends a "Hello World" DATA frame to the client.
*/
private void sendResponse(ChannelHandlerContext ctx, int streamId, ByteBuf payload) {
private void sendResponse(ChannelHandlerContext ctx, int streamId, ByteBuf payload) throws Http2Exception {
// Send a frame for the response status
Http2Headers headers = new DefaultHttp2Headers().status(OK.codeAsText());
encoder().writeHeaders(ctx, streamId, headers, 0, false, ctx.newPromise());
@ -86,7 +87,8 @@ public final class HelloWorldHttp2Handler extends Http2ConnectionHandler impleme
}
@Override
public int onDataRead(ChannelHandlerContext ctx, int streamId, ByteBuf data, int padding, boolean endOfStream) {
public int onDataRead(ChannelHandlerContext ctx, int streamId, ByteBuf data, int padding, boolean endOfStream)
throws Http2Exception {
int processed = data.readableBytes() + padding;
if (endOfStream) {
sendResponse(ctx, streamId, data.retain());
@ -96,7 +98,7 @@ public final class HelloWorldHttp2Handler extends Http2ConnectionHandler impleme
@Override
public void onHeadersRead(ChannelHandlerContext ctx, int streamId,
Http2Headers headers, int padding, boolean endOfStream) {
Http2Headers headers, int padding, boolean endOfStream) throws Http2Exception {
if (endOfStream) {
ByteBuf content = ctx.alloc().buffer();
content.writeBytes(RESPONSE_BYTES.duplicate());
@ -107,7 +109,7 @@ public final class HelloWorldHttp2Handler extends Http2ConnectionHandler impleme
@Override
public void onHeadersRead(ChannelHandlerContext ctx, int streamId, Http2Headers headers, int streamDependency,
short weight, boolean exclusive, int padding, boolean endOfStream) {
short weight, boolean exclusive, int padding, boolean endOfStream) throws Http2Exception {
onHeadersRead(ctx, streamId, headers, padding, endOfStream);
}

View File

@ -20,6 +20,7 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.http2.Http2Exception;
import io.netty.handler.codec.http2.Http2RemoteFlowController;
import io.netty.handler.codec.http2.Http2Stream;
import io.netty.util.internal.PlatformDependent;
public final class NoopHttp2RemoteFlowController implements Http2RemoteFlowController {
public static final NoopHttp2RemoteFlowController INSTANCE = new NoopHttp2RemoteFlowController();
@ -62,7 +63,11 @@ public final class NoopHttp2RemoteFlowController implements Http2RemoteFlowContr
public void addFlowControlled(Http2Stream stream, FlowControlled payload) {
// Don't check size beforehand because Headers payload returns 0 all the time.
do {
payload.write(ctx, MAX_INITIAL_WINDOW_SIZE);
try {
payload.write(ctx, MAX_INITIAL_WINDOW_SIZE);
} catch (Http2Exception e) {
PlatformDependent.throwException(e);
}
} while (payload.size() > 0);
}

View File

@ -49,6 +49,7 @@ import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.infra.Blackhole;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_TABLE_SIZE;
import static io.netty.microbench.http2.internal.hpack.HpackUtilBenchmark.newTestEncoder;
@ -79,7 +80,7 @@ public class DecoderBenchmark extends AbstractMicrobenchmark {
@Benchmark
@BenchmarkMode(Mode.Throughput)
public void decode(final Blackhole bh) throws Http2Exception {
Decoder decoder = new Decoder();
Decoder decoder = new Decoder(DEFAULT_HEADER_LIST_SIZE, 32);
@SuppressWarnings("unchecked")
Http2Headers headers =
new DefaultHttp2Headers() {