HTTP/2 Ensure default settings are correctly enforced and interfaces clarified

Motivation:
The responsibility for retaining the settings values and enforcing the settings constraints is spread out in different areas of the code and may be initialized with different values than the default specified in the RFC. This should not be allowed by default and interfaces which are responsible for maintaining/enforcing settings state should clearly indicate the restrictions that they should only be set by the codec upon receipt of a SETTINGS ACK frame.

Modifications:
- Encoder, Decoder, and the Headers Encoder/Decoder no longer expose public constructors that allow the default settings to be changed.
- Http2HeadersDecoder#maxHeaderSize() exists to provide some bound when headers/continuation frames are being aggregated. However this is roughly the same as SETTINGS_MAX_HEADER_LIST_SIZE (besides the 32 byte octet for each header field) and can be used instead of attempting to keep the two independent values in sync.
- Encoding headers now enforces SETTINGS_MAX_HEADER_LIST_SIZE at the octect level. Previously the header encoder compared the number of header key/value pairs against SETTINGS_MAX_HEADER_LIST_SIZE instead of the number of octets (plus 32 bytes overhead).
- DefaultHttp2ConnectionDecoder#onData calls shouldIgnoreHeadersOrDataFrame but may swallow exceptions from this method. This means a STREAM_RST frame may not be sent when it should for an unknown stream and thus violate the RFC. The exception is no longer swallowed.

Result:
Default settings state is enforced and interfaces related to settings state are clarified.
This commit is contained in:
Scott Mitchell 2016-09-16 06:57:33 -07:00
parent f010033590
commit 540c26bb56
34 changed files with 552 additions and 402 deletions

View File

@ -58,6 +58,7 @@ import static java.util.concurrent.TimeUnit.SECONDS;
* <li>{@link #frameLogger(Http2FrameLogger)}</li>
* <li>{@link #headerSensitivityDetector(SensitivityDetector)}</li>
* <li>{@link #encoderEnforceMaxConcurrentStreams(boolean)}</li>
* <li>{@link #encoderIgnoreMaxHeaderListSize(boolean)}</li>
* </ul>
*
* <h3>Exposing necessary methods in a subclass</h3>
@ -97,6 +98,7 @@ public abstract class AbstractHttp2ConnectionHandlerBuilder<T extends Http2Conne
private Http2FrameLogger frameLogger;
private SensitivityDetector headerSensitivityDetector;
private Boolean encoderEnforceMaxConcurrentStreams;
private Boolean encoderIgnoreMaxHeaderListSize;
/**
* Sets the {@link Http2Settings} to use for the initial connection settings exchange.
@ -301,6 +303,19 @@ public abstract class AbstractHttp2ConnectionHandlerBuilder<T extends Http2Conne
return self();
}
/**
* Sets if the <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a>
* should be ignored when encoding headers.
* @param ignoreMaxHeaderListSize {@code true} to ignore
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_HEADER_LIST_SIZE</a>.
* @return this.
*/
protected B encoderIgnoreMaxHeaderListSize(boolean ignoreMaxHeaderListSize) {
enforceNonCodecConstraints("encoderIgnoreMaxHeaderListSize");
this.encoderIgnoreMaxHeaderListSize = ignoreMaxHeaderListSize;
return self();
}
/**
* Create a new {@link Http2ConnectionHandler}.
*/
@ -320,7 +335,9 @@ public abstract class AbstractHttp2ConnectionHandlerBuilder<T extends Http2Conne
private T buildFromConnection(Http2Connection connection) {
Http2FrameReader reader = new DefaultHttp2FrameReader(isValidateHeaders());
Http2FrameWriter writer = new DefaultHttp2FrameWriter(headerSensitivityDetector());
Http2FrameWriter writer = encoderIgnoreMaxHeaderListSize == null ?
new DefaultHttp2FrameWriter(headerSensitivityDetector()) :
new DefaultHttp2FrameWriter(headerSensitivityDetector(), encoderIgnoreMaxHeaderListSize);
if (frameLogger != null) {
reader = new Http2InboundFrameLogger(reader, frameLogger);

View File

@ -26,6 +26,7 @@ import java.util.List;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
import static io.netty.handler.codec.http2.Http2CodecUtil.SMALLEST_MAX_CONCURRENT_STREAMS;
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.Http2Error.STREAM_CLOSED;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
@ -170,22 +171,30 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
Http2LocalFlowController flowController = flowController();
int bytesToReturn = data.readableBytes() + padding;
boolean shouldIgnore = true;
final boolean shouldIgnore;
try {
shouldIgnore = shouldIgnoreHeadersOrDataFrame(ctx, streamId, stream, "DATA");
} finally {
if (shouldIgnore) {
// Ignoring this frame. We still need to count the frame towards the connection flow control
// window, but we immediately mark all bytes as consumed.
flowController.receiveFlowControlledFrame(stream, data, padding, endOfStream);
flowController.consumeBytes(stream, bytesToReturn);
} catch (Http2Exception e) {
// Ignoring this frame. We still need to count the frame towards the connection flow control
// window, but we immediately mark all bytes as consumed.
flowController.receiveFlowControlledFrame(stream, data, padding, endOfStream);
flowController.consumeBytes(stream, bytesToReturn);
throw e;
} catch (Throwable t) {
throw connectionError(INTERNAL_ERROR, t, "Unhandled error on data stream id %d", streamId);
}
// Verify that the stream may have existed after we apply flow control.
verifyStreamMayHaveExisted(streamId);
if (shouldIgnore) {
// Ignoring this frame. We still need to count the frame towards the connection flow control
// window, but we immediately mark all bytes as consumed.
flowController.receiveFlowControlledFrame(stream, data, padding, endOfStream);
flowController.consumeBytes(stream, bytesToReturn);
// All bytes have been consumed.
return bytesToReturn;
}
// Verify that the stream may have existed after we apply flow control.
verifyStreamMayHaveExisted(streamId);
// All bytes have been consumed.
return bytesToReturn;
}
Http2Exception error = null;
@ -395,10 +404,10 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
Long headerTableSize = settings.headerTableSize();
if (headerTableSize != null) {
headerTable.maxHeaderTableSize((int) min(headerTableSize, MAX_VALUE));
headerTable.maxHeaderTableSize(headerTableSize);
}
Integer maxHeaderListSize = settings.maxHeaderListSize();
Long maxHeaderListSize = settings.maxHeaderListSize();
if (maxHeaderListSize != null) {
headerTable.maxHeaderListSize(maxHeaderListSize);
}
@ -415,13 +424,12 @@ public class DefaultHttp2ConnectionDecoder implements Http2ConnectionDecoder {
}
/**
* Calculate the {@code maxStreams} paramter for the {@link Endpoint#maxStreams(int, int)} method based upon
* SETTINGS_MAX_CONCURRENT_STREAMS.
* @param maxConcurrentStreams SETTINGS_MAX_CONCURRENT_STREAMS
* @return the {@code maxStreams} paramter for the {@link Endpoint#maxStreams(int, int)} method.
* Calculate the {@code maxStreams} parameter for the {@link Endpoint#maxStreams(int, int)} method based upon
* {@code SETTINGS_MAX_CONCURRENT_STREAMS}.
* @param maxConcurrentStreams {@code SETTINGS_MAX_CONCURRENT_STREAMS}.
* @return the {@code maxStreams} parameter for the {@link Endpoint#maxStreams(int, int)} method.
*/
@UnstableApi
protected int calculateMaxStreams(int maxConcurrentStreams) {
private int calculateMaxStreams(int maxConcurrentStreams) {
int maxStreams = maxConcurrentStreams + SMALLEST_MAX_CONCURRENT_STREAMS;
return maxStreams < 0 ? MAX_VALUE : maxStreams;
}

View File

@ -98,7 +98,7 @@ public class DefaultHttp2ConnectionEncoder implements Http2ConnectionEncoder {
outboundHeaderTable.maxHeaderTableSize((int) min(headerTableSize, MAX_VALUE));
}
Integer maxHeaderListSize = settings.maxHeaderListSize();
Long maxHeaderListSize = settings.maxHeaderListSize();
if (maxHeaderListSize != null) {
outboundHeaderTable.maxHeaderListSize(maxHeaderListSize);
}

View File

@ -29,9 +29,9 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.PRIORITY_ENTRY_LENGTH;
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_INITIAL_WINDOW_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_MAX_FRAME_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTING_ENTRY_LENGTH;
import static io.netty.handler.codec.http2.Http2CodecUtil.headerListSizeExceeded;
import static io.netty.handler.codec.http2.Http2CodecUtil.isMaxFrameSizeValid;
import static io.netty.handler.codec.http2.Http2CodecUtil.readUnsignedInt;
import static io.netty.handler.codec.http2.Http2Error.ENHANCE_YOUR_CALM;
import static io.netty.handler.codec.http2.Http2Error.FLOW_CONTROL_ERROR;
import static io.netty.handler.codec.http2.Http2Error.FRAME_SIZE_ERROR;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
@ -673,8 +673,7 @@ public class DefaultHttp2FrameReader implements Http2FrameReader, Http2FrameSize
*/
private void headerSizeExceeded() throws Http2Exception {
close();
throw connectionError(ENHANCE_YOUR_CALM, "Header size exceeded max allowed size (%d)",
headersDecoder.configuration().maxHeaderSize());
headerListSizeExceeded(streamId, headersDecoder.configuration().headerTable().maxHeaderListSize());
}
/**
@ -688,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().maxHeaderSize()) {
if (fragment.readableBytes() > headersDecoder.configuration().headerTable().maxHeaderListSize()) {
headerSizeExceeded();
}
if (endOfHeaders) {
@ -701,7 +700,7 @@ public class DefaultHttp2FrameReader implements Http2FrameReader, Http2FrameSize
}
return;
}
if (headersDecoder.configuration().maxHeaderSize() - fragment.readableBytes() <
if (headersDecoder.configuration().headerTable().maxHeaderListSize() - fragment.readableBytes() <
headerBlock.readableBytes()) {
headerSizeExceeded();
}
@ -724,7 +723,7 @@ public class DefaultHttp2FrameReader implements Http2FrameReader, Http2FrameSize
*/
Http2Headers headers() throws Http2Exception {
try {
return headersDecoder.decodeHeaders(headerBlock);
return headersDecoder.decodeHeaders(streamId, headerBlock);
} finally {
close();
}

View File

@ -21,6 +21,7 @@ import io.netty.channel.ChannelHandlerContext;
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.UnstableApi;
import static io.netty.buffer.Unpooled.directBuffer;
@ -44,9 +45,9 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.RST_STREAM_FRAME_LENGT
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTING_ENTRY_LENGTH;
import static io.netty.handler.codec.http2.Http2CodecUtil.WINDOW_UPDATE_FRAME_LENGTH;
import static io.netty.handler.codec.http2.Http2CodecUtil.isMaxFrameSizeValid;
import static io.netty.handler.codec.http2.Http2CodecUtil.verifyPadding;
import static io.netty.handler.codec.http2.Http2CodecUtil.writeFrameHeaderInternal;
import static io.netty.handler.codec.http2.Http2CodecUtil.writeUnsignedInt;
import static io.netty.handler.codec.http2.Http2CodecUtil.verifyPadding;
import static io.netty.handler.codec.http2.Http2CodecUtil.writeUnsignedShort;
import static io.netty.handler.codec.http2.Http2Error.FRAME_SIZE_ERROR;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
@ -86,8 +87,12 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter, Http2FrameSize
this(new DefaultHttp2HeadersEncoder());
}
public DefaultHttp2FrameWriter(Http2HeadersEncoder.SensitivityDetector headersSensativityDetector) {
this(new DefaultHttp2HeadersEncoder(Http2CodecUtil.DEFAULT_HEADER_TABLE_SIZE, headersSensativityDetector));
public DefaultHttp2FrameWriter(SensitivityDetector headersSensativityDetector) {
this(new DefaultHttp2HeadersEncoder(headersSensativityDetector));
}
public DefaultHttp2FrameWriter(SensitivityDetector headersSensativityDetector, boolean ignoreMaxHeaderListSize) {
this(new DefaultHttp2HeadersEncoder(headersSensativityDetector, ignoreMaxHeaderListSize));
}
public DefaultHttp2FrameWriter(Http2HeadersEncoder headersEncoder) {

View File

@ -1,35 +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;
/**
* Provides common functionality for {@link Http2HeaderTable}
*/
class DefaultHttp2HeaderTableListSize {
private int maxHeaderListSize = Integer.MAX_VALUE;
public void maxHeaderListSize(int max) throws Http2Exception {
if (max < 0) {
// Over 2^31 - 1 (minus in integer) size is set to the maximun value
maxHeaderListSize = Integer.MAX_VALUE;
} else {
maxHeaderListSize = max;
}
}
public int maxHeaderListSize() {
return maxHeaderListSize;
}
}

View File

@ -17,14 +17,10 @@ package io.netty.handler.codec.http2;
import io.netty.buffer.ByteBuf;
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_TABLE_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_MAX_HEADER_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_TABLE_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MIN_HEADER_TABLE_SIZE;
import static io.netty.handler.codec.http2.Http2Error.COMPRESSION_ERROR;
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
@UnstableApi
@ -32,7 +28,6 @@ public class DefaultHttp2HeadersDecoder implements Http2HeadersDecoder, Http2Hea
private static final float HEADERS_COUNT_WEIGHT_NEW = 1 / 5f;
private static final float HEADERS_COUNT_WEIGHT_HISTORICAL = 1 - HEADERS_COUNT_WEIGHT_NEW;
private final int maxHeaderSize;
private final Decoder decoder;
private final Http2HeaderTable headerTable;
private final boolean validateHeaders;
@ -47,17 +42,20 @@ public class DefaultHttp2HeadersDecoder implements Http2HeadersDecoder, Http2Hea
}
public DefaultHttp2HeadersDecoder(boolean validateHeaders) {
this(DEFAULT_MAX_HEADER_SIZE, DEFAULT_HEADER_TABLE_SIZE, validateHeaders, 32);
this(validateHeaders, new Decoder());
}
public DefaultHttp2HeadersDecoder(int maxHeaderSize, int maxHeaderTableSize, boolean validateHeaders,
int initialHuffmanDecodeCapacity) {
if (maxHeaderSize <= 0) {
throw new IllegalArgumentException("maxHeaderSize must be positive: " + maxHeaderSize);
}
decoder = new Decoder(maxHeaderSize, maxHeaderTableSize, initialHuffmanDecodeCapacity);
public DefaultHttp2HeadersDecoder(boolean validateHeaders, int initialHuffmanDecodeCapacity) {
this(validateHeaders, new Decoder(initialHuffmanDecodeCapacity));
}
/**
* 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.
*/
DefaultHttp2HeadersDecoder(boolean validateHeaders, Decoder decoder) {
this.decoder = ObjectUtil.checkNotNull(decoder, "decoder");
headerTable = new Http2HeaderTableDecoder();
this.maxHeaderSize = maxHeaderSize;
this.validateHeaders = validateHeaders;
}
@ -66,21 +64,16 @@ public class DefaultHttp2HeadersDecoder implements Http2HeadersDecoder, Http2Hea
return headerTable;
}
@Override
public int maxHeaderSize() {
return maxHeaderSize;
}
@Override
public Configuration configuration() {
return this;
}
@Override
public Http2Headers decodeHeaders(ByteBuf headerBlock) throws Http2Exception {
public Http2Headers decodeHeaders(int streamId, ByteBuf headerBlock) throws Http2Exception {
try {
final Http2Headers headers = new DefaultHttp2Headers(validateHeaders, (int) headerArraySizeAccumulator);
decoder.decode(headerBlock, headers);
decoder.decode(streamId, headerBlock, headers);
headerArraySizeAccumulator = HEADERS_COUNT_WEIGHT_NEW * headers.size() +
HEADERS_COUNT_WEIGHT_HISTORICAL * headerArraySizeAccumulator;
return headers;
@ -97,23 +90,25 @@ public class DefaultHttp2HeadersDecoder implements Http2HeadersDecoder, Http2Hea
/**
* {@link Http2HeaderTable} implementation to support {@link Http2HeadersDecoder}
*/
private final class Http2HeaderTableDecoder extends DefaultHttp2HeaderTableListSize implements Http2HeaderTable {
private final class Http2HeaderTableDecoder implements Http2HeaderTable {
@Override
public void maxHeaderTableSize(long max) throws Http2Exception {
if (max < MIN_HEADER_TABLE_SIZE || max > MAX_HEADER_TABLE_SIZE) {
throw connectionError(PROTOCOL_ERROR, "Header Table Size must be >= %d and <= %d but was %d",
MIN_HEADER_TABLE_SIZE, MAX_HEADER_TABLE_SIZE, max);
}
try {
decoder.setMaxHeaderTableSize(max);
} catch (Throwable t) {
throw connectionError(PROTOCOL_ERROR, t.getMessage(), t);
}
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

@ -20,13 +20,7 @@ import io.netty.buffer.Unpooled;
import io.netty.handler.codec.http2.internal.hpack.Encoder;
import io.netty.util.internal.UnstableApi;
import java.util.Map.Entry;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_TABLE_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_TABLE_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MIN_HEADER_TABLE_SIZE;
import static io.netty.handler.codec.http2.Http2Error.COMPRESSION_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.internal.ObjectUtil.checkNotNull;
@ -38,23 +32,35 @@ public class DefaultHttp2HeadersEncoder implements Http2HeadersEncoder, Http2Hea
private final ByteBuf tableSizeChangeOutput = Unpooled.buffer();
public DefaultHttp2HeadersEncoder() {
this(DEFAULT_HEADER_TABLE_SIZE, NEVER_SENSITIVE);
this(NEVER_SENSITIVE);
}
public DefaultHttp2HeadersEncoder(int maxHeaderTableSize, SensitivityDetector sensitivityDetector) {
public DefaultHttp2HeadersEncoder(SensitivityDetector sensitivityDetector) {
this(sensitivityDetector, new Encoder());
}
public DefaultHttp2HeadersEncoder(SensitivityDetector sensitivityDetector, boolean ignoreMaxHeaderListSize) {
this(sensitivityDetector, new Encoder(ignoreMaxHeaderListSize));
}
public DefaultHttp2HeadersEncoder(SensitivityDetector sensitivityDetector, boolean ignoreMaxHeaderListSize,
int dynamicTableArraySizeHint) {
this(sensitivityDetector, new Encoder(ignoreMaxHeaderListSize, dynamicTableArraySizeHint));
}
/**
* 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.
*/
DefaultHttp2HeadersEncoder(SensitivityDetector sensitivityDetector, Encoder encoder) {
this.sensitivityDetector = checkNotNull(sensitivityDetector, "sensitiveDetector");
encoder = new Encoder(maxHeaderTableSize);
this.encoder = checkNotNull(encoder, "encoder");
headerTable = new Http2HeaderTableEncoder();
}
@Override
public void encodeHeaders(Http2Headers headers, ByteBuf buffer) throws Http2Exception {
try {
if (headers.size() > headerTable.maxHeaderListSize()) {
throw connectionError(PROTOCOL_ERROR, "Number of headers (%d) exceeds maxHeaderListSize (%d)",
headers.size(), headerTable.maxHeaderListSize());
}
// If there was a change in the table size, serialize the output from the encoder
// resulting from that change.
if (tableSizeChangeOutput.isReadable()) {
@ -62,9 +68,7 @@ public class DefaultHttp2HeadersEncoder implements Http2HeadersEncoder, Http2Hea
tableSizeChangeOutput.clear();
}
for (Entry<CharSequence, CharSequence> header : headers) {
encodeHeader(buffer, header.getKey(), header.getValue());
}
encoder.encodeHeaders(buffer, headers, sensitivityDetector);
} catch (Http2Exception e) {
throw e;
} catch (Throwable t) {
@ -82,31 +86,28 @@ public class DefaultHttp2HeadersEncoder implements Http2HeadersEncoder, Http2Hea
return this;
}
private void encodeHeader(ByteBuf out, CharSequence key, CharSequence value) {
encoder.encodeHeader(out, key, value, sensitivityDetector.isSensitive(key, value));
}
/**
* {@link Http2HeaderTable} implementation to support {@link Http2HeadersEncoder}
*/
private final class Http2HeaderTableEncoder extends DefaultHttp2HeaderTableListSize implements Http2HeaderTable {
private final class Http2HeaderTableEncoder implements Http2HeaderTable {
@Override
public void maxHeaderTableSize(long max) throws Http2Exception {
if (max < MIN_HEADER_TABLE_SIZE || max > MAX_HEADER_TABLE_SIZE) {
throw connectionError(PROTOCOL_ERROR, "Header Table Size must be >= %d and <= %d but was %d",
MIN_HEADER_TABLE_SIZE, MAX_HEADER_TABLE_SIZE, max);
}
try {
// No headers should be emitted. If they are, we throw.
encoder.setMaxHeaderTableSize(tableSizeChangeOutput, max);
} catch (Throwable t) {
throw new Http2Exception(PROTOCOL_ERROR, t.getMessage(), t);
}
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

@ -29,6 +29,8 @@ 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.streamError;
import static io.netty.util.CharsetUtil.UTF_8;
import static java.lang.Math.max;
import static java.lang.Math.min;
@ -92,7 +94,7 @@ public final class Http2CodecUtil {
public static final int MAX_INITIAL_WINDOW_SIZE = Integer.MAX_VALUE;
public static final int MAX_FRAME_SIZE_LOWER_BOUND = 0x4000;
public static final int MAX_FRAME_SIZE_UPPER_BOUND = 0xFFFFFF;
public static final long MAX_HEADER_LIST_SIZE = Long.MAX_VALUE;
public static final long MAX_HEADER_LIST_SIZE = MAX_UNSIGNED_INT;
public static final long MIN_HEADER_TABLE_SIZE = 0;
public static final long MIN_CONCURRENT_STREAMS = 0;
@ -103,7 +105,7 @@ public final class Http2CodecUtil {
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_MAX_HEADER_SIZE = 8192;
public static final int DEFAULT_HEADER_LIST_SIZE = 8192;
public static final int DEFAULT_MAX_FRAME_SIZE = MAX_FRAME_SIZE_LOWER_BOUND;
/**
@ -221,6 +223,10 @@ public final class Http2CodecUtil {
return max(0, min(state.pendingBytes(), state.windowSize()));
}
public static void headerListSizeExceeded(int streamId, long maxHeaderListSize) throws Http2Exception {
throw streamError(streamId, 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

@ -232,12 +232,15 @@ public interface Http2Connection {
boolean isServer();
/**
* Sets whether server push is allowed to this endpoint.
* This is the <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_ENABLE_PUSH</a> value sent
* from the opposite endpoint. This method should only be called by Netty (not users) as a result of a
* receiving a {@code SETTINGS} frame.
*/
void allowPushTo(boolean allow);
/**
* Gets whether or not server push is allowed to this endpoint. This is always false
* This is the <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_ENABLE_PUSH</a> value sent
* from the opposite endpoint. The initial value must be {@code true} for the client endpoint and always false
* for a server endpoint.
*/
boolean allowPushTo();
@ -250,8 +253,11 @@ public interface Http2Connection {
/**
* Gets the maximum number of streams (created by this endpoint) that are allowed to be active at
* the same time. This is the {@code SETTINGS_MAX_CONCURRENT_STREAMS} value sent from the opposite endpoint to
* restrict stream creation by this endpoint.
* the same time. This is the
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_CONCURRENT_STREAMS</a>
* value sent from the opposite endpoint to restrict stream creation by this endpoint.
* <p>
* The default value returned by this method must be "unlimited".
*/
int maxActiveStreams();
@ -265,8 +271,9 @@ public interface Http2Connection {
/**
* Sets the limit for {@code SETTINGS_MAX_CONCURRENT_STREAMS} and the limit for {@link #maxStreams()}.
* @param maxActiveStreams The maximum number of streams (created by this endpoint) that are allowed to be
* active at once. This is the {@code SETTINGS_MAX_CONCURRENT_STREAMS} value sent from the opposite endpoint to
* restrict stream creation by this endpoint.
* active at once. This is the
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_CONCURRENT_STREAMS</a> value sent
* from the opposite endpoint to restrict stream creation by this endpoint.
* @param maxStreams The limit imposed by {@link #maxActiveStreams()} does not apply to streams in the IDLE
* state. Since IDLE streams can still consume resources this limit will include streams in all states.
* @throws Http2Exception if {@code maxStreams < maxActiveStream}.

View File

@ -36,8 +36,9 @@ public interface Http2FlowController {
* Sets the connection-wide initial flow control window and updates all stream windows (but not the connection
* stream window) by the delta.
* <p>
* This method is used to apply the {@code SETTINGS_INITIAL_WINDOW_SIZE} value for an
* {@code SETTINGS} frame.
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_INITIAL_WINDOW_SIZE</a>. This method should
* only be called by Netty (not users) as a result of a receiving a {@code SETTINGS} frame.
*
* @param newWindowSize the new initial window size.
* @throws Http2Exception thrown if any protocol-related error occurred.
@ -47,6 +48,10 @@ public interface Http2FlowController {
/**
* Gets the connection-wide initial flow control window size that is used as the basis for new stream flow
* control windows.
* <p>
* Represents the value for
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_INITIAL_WINDOW_SIZE</a>. The initial value
* returned by this method must be {@link Http2CodecUtil#DEFAULT_WINDOW_SIZE}.
*/
int initialWindowSize();

View File

@ -20,11 +20,20 @@ import io.netty.util.internal.UnstableApi;
public interface Http2FrameSizePolicy {
/**
* Sets the maximum allowed frame size. Attempts to write frames longer than this maximum will fail.
* <p>
* This value is used to represent
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_FRAME_SIZE</a>. This method should
* only be called by Netty (not users) as a result of a receiving a {@code SETTINGS} frame.
*/
void maxFrameSize(int max) throws Http2Exception;
/**
* Gets the maximum allowed frame size.
* <p>
* This value is used to represent
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_MAX_FRAME_SIZE</a>. The initial value
* defined by the RFC is unlimited but enforcing a lower limit is generally permitted.
* {@link Http2CodecUtil#DEFAULT_MAX_FRAME_SIZE} can be used as a more conservative default.
*/
int maxFrameSize();
}

View File

@ -22,22 +22,30 @@ import io.netty.util.internal.UnstableApi;
@UnstableApi
public interface Http2HeaderTable {
/**
* Sets the maximum size of the HPACK header table used for decoding HTTP/2 headers.
* 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;
/**
* Gets the maximum size of the HPACK header table used for decoding HTTP/2 headers.
* 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();
/**
* Sets the maximum allowed header elements.
* 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(int max) throws Http2Exception;
void maxHeaderListSize(long max) throws Http2Exception;
/**
* Gets the maximum allowed header elements.
* 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}.
*/
int maxHeaderListSize();
long maxHeaderListSize();
}

View File

@ -31,17 +31,12 @@ public interface Http2HeadersDecoder {
* Access the Http2HeaderTable for this {@link Http2HeadersDecoder}
*/
Http2HeaderTable headerTable();
/**
* Get the maximum number of bytes that is allowed before truncation occurs.
*/
int maxHeaderSize();
}
/**
* Decodes the given headers block and returns the headers.
*/
Http2Headers decodeHeaders(ByteBuf headerBlock) throws Http2Exception;
Http2Headers decodeHeaders(int streamId, ByteBuf headerBlock) throws Http2Exception;
/**
* Get the {@link Configuration} for this {@link Http2HeadersDecoder}

View File

@ -15,6 +15,9 @@
package io.netty.handler.codec.http2;
import io.netty.util.collection.CharObjectHashMap;
import io.netty.util.internal.UnstableApi;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_CONCURRENT_STREAMS;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_TABLE_SIZE;
@ -33,9 +36,6 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_MAX_HEADER_LI
import static io.netty.handler.codec.http2.Http2CodecUtil.isMaxFrameSizeValid;
import static io.netty.util.internal.ObjectUtil.checkNotNull;
import io.netty.util.collection.CharObjectHashMap;
import io.netty.util.internal.UnstableApi;
/**
* Settings for one endpoint in an HTTP/2 connection. Each of the values are optional as defined in
* the spec for the SETTINGS frame. Permits storage of arbitrary key/value pairs but provides helper
@ -165,15 +165,8 @@ public final class Http2Settings extends CharObjectHashMap<Long> {
/**
* Gets the {@code SETTINGS_MAX_HEADER_LIST_SIZE} value. If unavailable, returns {@code null}.
*/
public Integer maxHeaderListSize() {
Integer value = getIntValue(SETTINGS_MAX_HEADER_LIST_SIZE);
// Over 2^31 - 1 (minus in integer) size is set to the maximun value
if (value != null && value < 0) {
value = Integer.MAX_VALUE;
}
return value;
public Long maxHeaderListSize() {
return get(SETTINGS_MAX_HEADER_LIST_SIZE);
}
/**
@ -181,12 +174,7 @@ public final class Http2Settings extends CharObjectHashMap<Long> {
*
* @throws IllegalArgumentException if verification of the setting fails.
*/
public Http2Settings maxHeaderListSize(int value) {
// Over 2^31 - 1 (minus in integer) size is set to the maximun value
if (value < 0) {
value = Integer.MAX_VALUE;
}
public Http2Settings maxHeaderListSize(long value) {
put(SETTINGS_MAX_HEADER_LIST_SIZE, Long.valueOf(value));
return this;
}

View File

@ -37,8 +37,15 @@ 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;
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.ENHANCE_YOUR_CALM;
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.ThrowableUtil.unknownStackTrace;
@ -73,24 +80,28 @@ public final class Decoder {
private final DynamicTable dynamicTable;
private final HuffmanDecoder huffmanDecoder;
private final int maxHeadersLength;
private long maxHeaderListSize;
private long maxDynamicTableSize;
private int encoderMaxDynamicTableSize;
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 maxHeadersLength The maximum size (in bytes) that is allowed for a single header decode operation.
* @param maxHeaderTableSize
* <a href="https://tools.ietf.org/html/rfc7540#section-6.5.2">SETTINGS_HEADER_TABLE_SIZE</a>.
* @param initialHuffmanDecodeCapacity The initial size of the byte array used to do huffman decoding.
* 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.
*/
public Decoder(int maxHeadersLength, int maxHeaderTableSize, int initialHuffmanDecodeCapacity) {
dynamicTable = new DynamicTable(maxHeaderTableSize);
this.maxHeadersLength = maxHeadersLength;
maxDynamicTableSize = maxHeaderTableSize;
encoderMaxDynamicTableSize = maxHeaderTableSize;
Decoder(int initialHuffmanDecodeCapacity, int maxHeaderTableSize) {
maxHeaderListSize = DEFAULT_HEADER_LIST_SIZE;
maxDynamicTableSize = encoderMaxDynamicTableSize = maxHeaderTableSize;
maxDynamicTableSizeChangeRequired = false;
dynamicTable = new DynamicTable(maxHeaderTableSize);
huffmanDecoder = new HuffmanDecoder(initialHuffmanDecodeCapacity);
}
@ -99,9 +110,9 @@ public final class Decoder {
* <p>
* This method assumes the entire header block is contained in {@code in}.
*/
public void decode(ByteBuf in, Http2Headers headers) throws Http2Exception {
public void decode(int streamId, ByteBuf in, Http2Headers headers) throws Http2Exception {
int index = 0;
int headersLength = 0;
long headersLength = 0;
int nameLength = 0;
int valueLength = 0;
byte state = READ_HEADER_REPRESENTATION;
@ -126,7 +137,7 @@ public final class Decoder {
state = READ_INDEXED_HEADER;
break;
default:
headersLength = indexHeader(index, headers, headersLength);
headersLength = indexHeader(streamId, index, headers, headersLength);
}
} else if ((b & 0x40) == 0x40) {
// Literal Header Field with Incremental Indexing
@ -178,7 +189,7 @@ public final class Decoder {
break;
case READ_INDEXED_HEADER:
headersLength = indexHeader(decodeULE128(in, index), headers, headersLength);
headersLength = indexHeader(streamId, decodeULE128(in, index), headers, headersLength);
state = READ_HEADER_REPRESENTATION;
break;
@ -195,8 +206,8 @@ public final class Decoder {
if (index == 0x7f) {
state = READ_LITERAL_HEADER_NAME_LENGTH;
} else {
if (index > maxHeadersLength - headersLength) {
maxHeaderSizeExceeded();
if (index > maxHeaderListSize - headersLength) {
headerListSizeExceeded(streamId, maxHeaderListSize);
}
nameLength = index;
state = READ_LITERAL_HEADER_NAME;
@ -207,8 +218,8 @@ public final class Decoder {
// Header Name is a Literal String
nameLength = decodeULE128(in, index);
if (nameLength > maxHeadersLength - headersLength) {
maxHeaderSizeExceeded();
if (nameLength > maxHeaderListSize - headersLength) {
headerListSizeExceeded(streamId, maxHeaderListSize);
}
state = READ_LITERAL_HEADER_NAME;
break;
@ -233,13 +244,14 @@ public final class Decoder {
state = READ_LITERAL_HEADER_VALUE_LENGTH;
break;
case 0:
headersLength = insertHeader(headers, name, EMPTY_STRING, indexType, headersLength);
headersLength = insertHeader(streamId, headers, name, EMPTY_STRING, indexType,
headersLength);
state = READ_HEADER_REPRESENTATION;
break;
default:
// Check new header size against max header size
if ((long) index + nameLength > maxHeadersLength - headersLength) {
maxHeaderSizeExceeded();
if ((long) index + nameLength > maxHeaderListSize - headersLength) {
headerListSizeExceeded(streamId, maxHeaderListSize);
}
valueLength = index;
state = READ_LITERAL_HEADER_VALUE;
@ -252,8 +264,8 @@ public final class Decoder {
valueLength = decodeULE128(in, index);
// Check new header size against max header size
if ((long) valueLength + nameLength > maxHeadersLength - headersLength) {
maxHeaderSizeExceeded();
if ((long) valueLength + nameLength > maxHeaderListSize - headersLength) {
headerListSizeExceeded(streamId, maxHeaderListSize);
}
state = READ_LITERAL_HEADER_VALUE;
break;
@ -265,7 +277,7 @@ public final class Decoder {
}
CharSequence value = readStringLiteral(in, valueLength, huffmanEncoded);
headersLength = insertHeader(headers, name, value, indexType, headersLength);
headersLength = insertHeader(streamId, headers, name, value, indexType, headersLength);
state = READ_HEADER_REPRESENTATION;
break;
@ -279,7 +291,11 @@ public final class Decoder {
* Set the maximum table size. If this is below the maximum size of the dynamic table used by
* the encoder, the beginning of the next header block MUST signal this change.
*/
public void setMaxHeaderTableSize(long maxHeaderTableSize) {
public void setMaxHeaderTableSize(long maxHeaderTableSize) throws Http2Exception {
if (maxHeaderTableSize < MIN_HEADER_TABLE_SIZE || maxHeaderTableSize > MAX_HEADER_TABLE_SIZE) {
throw connectionError(PROTOCOL_ERROR, "Header Table Size must be >= %d and <= %d but was %d",
MIN_HEADER_TABLE_SIZE, MAX_HEADER_TABLE_SIZE, maxHeaderTableSize);
}
maxDynamicTableSize = maxHeaderTableSize;
if (maxDynamicTableSize < encoderMaxDynamicTableSize) {
// decoder requires less space than encoder
@ -289,6 +305,18 @@ public final class Decoder {
}
}
public void setMaxHeaderListSize(long maxHeaderListSize) throws Http2Exception {
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;
}
public long getMaxHeaderListSize() {
return maxHeaderListSize;
}
/**
* Return the maximum table size. This is the maximum size allowed by both the encoder and the
* decoder.
@ -339,21 +367,21 @@ public final class Decoder {
throw READ_NAME_ILLEGAL_INDEX_VALUE;
}
private int indexHeader(int index, Http2Headers headers, int headersLength) throws Http2Exception {
private long indexHeader(int streamId, int index, Http2Headers headers, long headersLength) throws Http2Exception {
if (index <= StaticTable.length) {
HeaderField headerField = StaticTable.getEntry(index);
return addHeader(headers, headerField.name, headerField.value, headersLength);
return addHeader(streamId, headers, headerField.name, headerField.value, headersLength);
}
if (index - StaticTable.length <= dynamicTable.length()) {
HeaderField headerField = dynamicTable.getEntry(index - StaticTable.length);
return addHeader(headers, headerField.name, headerField.value, headersLength);
return addHeader(streamId, headers, headerField.name, headerField.value, headersLength);
}
throw INDEX_HEADER_ILLEGAL_INDEX_VALUE;
}
private int insertHeader(Http2Headers headers, CharSequence name, CharSequence value, IndexType indexType,
int headerSize) throws Http2Exception {
headerSize = addHeader(headers, name, value, headerSize);
private long insertHeader(int streamId, Http2Headers headers, CharSequence name, CharSequence value,
IndexType indexType, long headerSize) throws Http2Exception {
headerSize = addHeader(streamId, headers, name, value, headerSize);
switch (indexType) {
case NONE:
@ -371,22 +399,14 @@ public final class Decoder {
return headerSize;
}
private int addHeader(Http2Headers headers, CharSequence name, CharSequence value, int headersLength)
throws Http2Exception {
long newHeadersLength = (long) headersLength + name.length() + value.length();
if (newHeadersLength > maxHeadersLength) {
maxHeaderSizeExceeded();
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);
}
headers.add(name, value);
return (int) newHeadersLength;
}
/**
* Respond to headers block resulting in the maximum header size being exceeded.
* @throws Http2Exception If we can not recover from the truncation.
*/
private void maxHeaderSizeExceeded() throws Http2Exception {
throw connectionError(ENHANCE_YOUR_CALM, "Header size exceeded max allowed bytes (%d)", maxHeadersLength);
return headersLength;
}
private CharSequence readStringLiteral(ByteBuf in, int length, boolean huffmanEncoded) throws Http2Exception {

View File

@ -41,13 +41,13 @@ final class DynamicTable {
HeaderField[] headerFields;
int head;
int tail;
private int size;
private long size;
private long capacity = -1; // ensure setCapacity creates the array
/**
* Creates a new dynamic table with the specified initial capacity.
*/
DynamicTable(int initialCapacity) {
DynamicTable(long initialCapacity) {
setCapacity(initialCapacity);
}
@ -106,7 +106,7 @@ final class DynamicTable {
clear();
return;
}
while (size + headerSize > capacity) {
while (capacity - size < headerSize) {
remove();
}
headerFields[head++] = header;

View File

@ -32,13 +32,23 @@
package io.netty.handler.codec.http2.internal.hpack;
import io.netty.buffer.ByteBuf;
import io.netty.handler.codec.http2.Http2Exception;
import io.netty.handler.codec.http2.Http2Headers;
import io.netty.handler.codec.http2.Http2HeadersEncoder.SensitivityDetector;
import io.netty.util.AsciiString;
import io.netty.util.CharsetUtil;
import java.util.Arrays;
import java.util.Map;
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;
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.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.Http2Exception.connectionError;
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;
@ -54,24 +64,32 @@ public final class Encoder {
AsciiString.EMPTY_STRING, Integer.MAX_VALUE, null);
private final HuffmanEncoder huffmanEncoder = new HuffmanEncoder();
private final byte hashMask;
private final boolean ignoreMaxHeaderListSize;
private long size;
private long capacity;
private long maxHeaderTableSize;
private long maxHeaderListSize;
/**
* Creates a new encoder.
*/
public Encoder(long maxHeaderTableSize) {
this(maxHeaderTableSize, 16);
public Encoder() {
this(false);
}
/**
* Creates a new encoder.
*/
public Encoder(long maxHeaderTableSize, int arraySizeHint) {
if (maxHeaderTableSize < MIN_HEADER_TABLE_SIZE || maxHeaderTableSize > MAX_HEADER_TABLE_SIZE) {
throw new IllegalArgumentException("maxHeaderTableSize is invalid: " + maxHeaderTableSize);
}
capacity = maxHeaderTableSize;
public Encoder(boolean ignoreMaxHeaderListSize) {
this(ignoreMaxHeaderListSize, 16);
}
/**
* Creates a new encoder.
*/
public Encoder(boolean ignoreMaxHeaderListSize, int arraySizeHint) {
this.ignoreMaxHeaderListSize = ignoreMaxHeaderListSize;
maxHeaderTableSize = DEFAULT_HEADER_TABLE_SIZE;
maxHeaderListSize = DEFAULT_HEADER_LIST_SIZE;
// Enforce a bound of [2, 128] because hashMask is a byte. The max possible value of hashMask is one less
// than the length of this array, and we want the mask to be > 0.
headerFields = new HeaderEntry[findNextPositivePowerOfTwo(max(2, min(arraySizeHint, 128)))];
@ -84,8 +102,50 @@ public final class Encoder {
*
* <strong>The given {@link CharSequence}s must be immutable!</strong>
*/
public void encodeHeader(ByteBuf out, CharSequence name, CharSequence value, boolean sensitive) {
public void encodeHeaders(ByteBuf out, Http2Headers headers, SensitivityDetector sensitivityDetector)
throws Http2Exception {
if (ignoreMaxHeaderListSize) {
encodeHeadersIgnoreMaxHeaderListSize(out, headers, sensitivityDetector);
} else {
encodeHeadersEnforceMaxHeaderListSize(out, headers, sensitivityDetector);
}
}
private void encodeHeadersEnforceMaxHeaderListSize(ByteBuf out, Http2Headers headers,
SensitivityDetector sensitivityDetector)
throws Http2Exception {
long headerSize = 0;
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;
if (headerSize > maxHeaderListSize) {
throw connectionError(PROTOCOL_ERROR, "Header list size octets (%d) exceeds maxHeaderListSize (%d)",
headerSize, maxHeaderListSize);
}
encodeHeader(out, name, value, sensitivityDetector.isSensitive(name, value), currHeaderSize);
}
}
private void encodeHeadersIgnoreMaxHeaderListSize(ByteBuf out, Http2Headers headers,
SensitivityDetector sensitivityDetector) throws Http2Exception {
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));
}
}
/**
* Encode the header field into the header block.
*
* <strong>The given {@link CharSequence}s must be immutable!</strong>
*/
private void encodeHeader(ByteBuf out, CharSequence name, CharSequence value, boolean sensitive, long headerSize) {
// If the header value is sensitive then it must never be indexed
if (sensitive) {
int nameIndex = getNameIndex(name);
@ -94,7 +154,7 @@ public final class Encoder {
}
// If the peer will only use the static table
if (capacity == 0) {
if (maxHeaderTableSize == 0) {
int staticTableIndex = StaticTable.getIndex(name, value);
if (staticTableIndex == -1) {
int nameIndex = StaticTable.getIndex(name);
@ -105,10 +165,8 @@ public final class Encoder {
return;
}
int headerSize = HeaderField.sizeOf(name, value);
// If the headerSize is greater than the max table size then it must be encoded literally
if (headerSize > capacity) {
if (headerSize > maxHeaderTableSize) {
int nameIndex = getNameIndex(name);
encodeLiteral(out, name, value, NONE, nameIndex);
return;
@ -127,7 +185,7 @@ public final class Encoder {
} else {
ensureCapacity(headerSize);
encodeLiteral(out, name, value, INCREMENTAL, getNameIndex(name));
add(name, value);
add(name, value, headerSize);
}
}
}
@ -135,14 +193,15 @@ public final class Encoder {
/**
* Set the maximum table size.
*/
public void setMaxHeaderTableSize(ByteBuf out, long maxHeaderTableSize) {
public void setMaxHeaderTableSize(ByteBuf out, long maxHeaderTableSize) throws Http2Exception {
if (maxHeaderTableSize < MIN_HEADER_TABLE_SIZE || maxHeaderTableSize > MAX_HEADER_TABLE_SIZE) {
throw new IllegalArgumentException("maxHeaderTableSize is invalid: " + maxHeaderTableSize);
throw connectionError(PROTOCOL_ERROR, "Header Table Size must be >= %d and <= %d but was %d",
MIN_HEADER_TABLE_SIZE, MAX_HEADER_TABLE_SIZE, maxHeaderTableSize);
}
if (capacity == maxHeaderTableSize) {
if (this.maxHeaderTableSize == maxHeaderTableSize) {
return;
}
capacity = maxHeaderTableSize;
this.maxHeaderTableSize = maxHeaderTableSize;
ensureCapacity(0);
// Casting to integer is safe as we verified the maxHeaderTableSize is a valid unsigned int.
encodeInteger(out, 0x20, 5, (int) maxHeaderTableSize);
@ -152,7 +211,19 @@ public final class Encoder {
* Return the maximum table size.
*/
public long getMaxHeaderTableSize() {
return capacity;
return maxHeaderTableSize;
}
public void setMaxHeaderListSize(long maxHeaderListSize) throws Http2Exception {
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_LIST_SIZE, MAX_HEADER_LIST_SIZE, maxHeaderListSize);
}
this.maxHeaderListSize = maxHeaderListSize;
}
public long getMaxHeaderListSize() {
return maxHeaderListSize;
}
/**
@ -235,8 +306,8 @@ public final class Encoder {
* Ensure that the dynamic table has enough room to hold 'headerSize' more bytes. Removes the
* oldest entry from the dynamic table until sufficient space is available.
*/
private void ensureCapacity(int headerSize) {
while (size + headerSize > capacity) {
private void ensureCapacity(long headerSize) {
while (maxHeaderTableSize - size < headerSize) {
int index = length();
if (index == 0) {
break;
@ -316,20 +387,18 @@ public final class Encoder {
/**
* Add the header field to the dynamic table. Entries are evicted from the dynamic table until
* the size of the table and the new header field is less than the table's capacity. If the size
* of the new entry is larger than the table's capacity, the dynamic table will be cleared.
* the size of the table and the new header field is less than the table's maxHeaderTableSize. If the size
* of the new entry is larger than the table's maxHeaderTableSize, the dynamic table will be cleared.
*/
private void add(CharSequence name, CharSequence value) {
int headerSize = HeaderField.sizeOf(name, value);
// Clear the table if the header field size is larger than the capacity.
if (headerSize > capacity) {
private void add(CharSequence name, CharSequence value, long headerSize) {
// Clear the table if the header field size is larger than the maxHeaderTableSize.
if (headerSize > maxHeaderTableSize) {
clear();
return;
}
// Evict oldest entries until we have enough capacity.
while (size + headerSize > capacity) {
// Evict oldest entries until we have enough maxHeaderTableSize.
while (maxHeaderTableSize - size < headerSize) {
remove();
}

View File

@ -41,7 +41,7 @@ class HeaderField {
// overhead associated with the structure.
static final int HEADER_ENTRY_OVERHEAD = 32;
static int sizeOf(CharSequence name, CharSequence value) {
static long sizeOf(CharSequence name, CharSequence value) {
return name.length() + value.length() + HEADER_ENTRY_OVERHEAD;
}

View File

@ -238,15 +238,19 @@ public class DefaultHttp2ConnectionDecoderTest {
int padding = 10;
int processedBytes = data.readableBytes() + padding;
try {
decode().onDataRead(ctx, STREAM_ID, data, padding, true);
verify(localFlow)
.receiveFlowControlledFrame(eq((Http2Stream) null), eq(data), eq(padding), eq(true));
verify(localFlow).consumeBytes(eq((Http2Stream) null), eq(processedBytes));
verify(localFlow).frameWriter(any(Http2FrameWriter.class));
verifyNoMoreInteractions(localFlow);
try {
decode().onDataRead(ctx, STREAM_ID, data, padding, true);
fail();
} catch (Http2Exception e) {
verify(localFlow)
.receiveFlowControlledFrame(eq((Http2Stream) null), eq(data), eq(padding), eq(true));
verify(localFlow).consumeBytes(eq((Http2Stream) null), eq(processedBytes));
verify(localFlow).frameWriter(any(Http2FrameWriter.class));
verifyNoMoreInteractions(localFlow);
// Verify that the event was absorbed and not propagated to the observer.
verify(listener, never()).onDataRead(eq(ctx), anyInt(), any(ByteBuf.class), anyInt(), anyBoolean());
// Verify that the event was absorbed and not propagated to the observer.
verify(listener, never()).onDataRead(eq(ctx), anyInt(), any(ByteBuf.class), anyInt(), anyBoolean());
}
} finally {
data.release();
}

View File

@ -1,55 +0,0 @@
/*
* Copyright 2015 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 org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
/**
* Tests for {@link DefaultHttp2HeaderTableListSize}.
*/
public class DefaultHttp2HeaderTableListSizeTest {
private DefaultHttp2HeaderTableListSize headerTable;
@Before
public void setup() {
headerTable = new DefaultHttp2HeaderTableListSize();
}
@Test
public void defaultMaxHeaderListSizeShouldSucceed() {
assertEquals(Integer.MAX_VALUE, (long) headerTable.maxHeaderListSize());
}
@Test
public void standardMaxHeaderListSizeShouldSucceed() throws Http2Exception {
headerTable.maxHeaderListSize(123);
assertEquals(123L, (long) headerTable.maxHeaderListSize());
}
@Test
public void boundaryMaxHeaderListSizeShouldSucceed() throws Http2Exception {
headerTable.maxHeaderListSize(Integer.MAX_VALUE);
assertEquals(Integer.MAX_VALUE, (long) headerTable.maxHeaderListSize());
final long settingsValueUpperBound = (1L << 32) - 1L;
headerTable.maxHeaderListSize((int) settingsValueUpperBound);
assertEquals(Integer.MAX_VALUE, (long) headerTable.maxHeaderListSize());
}
}

View File

@ -22,8 +22,9 @@ import io.netty.util.AsciiString;
import org.junit.Before;
import org.junit.Test;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_MAX_HEADER_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_TABLE_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_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;
import static io.netty.util.CharsetUtil.UTF_8;
import static org.junit.Assert.assertEquals;
@ -45,7 +46,7 @@ public class DefaultHttp2HeadersDecoderTest {
public void decodeShouldSucceed() throws Exception {
ByteBuf buf = encode(b(":method"), b("GET"), b("akey"), b("avalue"), randomBytes(), randomBytes());
try {
Http2Headers headers = decoder.decodeHeaders(buf);
Http2Headers headers = decoder.decodeHeaders(0, buf);
assertEquals(3, headers.size());
assertEquals("GET", headers.method().toString());
assertEquals("avalue", headers.get(new AsciiString("akey")).toString());
@ -56,9 +57,9 @@ public class DefaultHttp2HeadersDecoderTest {
@Test(expected = Http2Exception.class)
public void testExceedHeaderSize() throws Exception {
ByteBuf buf = encode(randomBytes(DEFAULT_MAX_HEADER_SIZE), randomBytes(1));
ByteBuf buf = encode(randomBytes(DEFAULT_HEADER_LIST_SIZE), randomBytes(1));
try {
decoder.decodeHeaders(buf);
decoder.decodeHeaders(0, buf);
fail();
} finally {
buf.release();
@ -70,13 +71,13 @@ public class DefaultHttp2HeadersDecoderTest {
}
private static ByteBuf encode(byte[]... entries) throws Exception {
Encoder encoder = new Encoder(MAX_HEADER_TABLE_SIZE);
Encoder encoder = newTestEncoder();
ByteBuf out = Unpooled.buffer();
Http2Headers http2Headers = new DefaultHttp2Headers(false);
for (int ix = 0; ix < entries.length;) {
byte[] key = entries[ix++];
byte[] value = entries[ix++];
encoder.encodeHeader(out, new AsciiString(key, false), new AsciiString(value, false), false);
http2Headers.add(new AsciiString(entries[ix++], false), new AsciiString(entries[ix++], false));
}
encoder.encodeHeaders(out, http2Headers, NEVER_SENSITIVE);
return out;
}
}

View File

@ -21,6 +21,7 @@ import io.netty.util.AsciiString;
import org.junit.Before;
import org.junit.Test;
import static io.netty.handler.codec.http2.Http2TestUtil.newTestEncoder;
import static org.junit.Assert.assertTrue;
/**
@ -32,7 +33,7 @@ public class DefaultHttp2HeadersEncoderTest {
@Before
public void setup() {
encoder = new DefaultHttp2HeadersEncoder();
encoder = new DefaultHttp2HeadersEncoder(Http2HeadersEncoder.NEVER_SENSITIVE, newTestEncoder());
}
@Test

View File

@ -26,6 +26,7 @@ import io.netty.channel.ChannelPromise;
import io.netty.channel.DefaultChannelPromise;
import io.netty.util.AsciiString;
import io.netty.util.concurrent.EventExecutor;
import io.netty.util.concurrent.GlobalEventExecutor;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -39,7 +40,11 @@ import java.util.LinkedList;
import java.util.List;
import static io.netty.buffer.Unpooled.EMPTY_BUFFER;
import static io.netty.handler.codec.http2.Http2CodecUtil.*;
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;
import static io.netty.handler.codec.http2.Http2TestUtil.newTestEncoder;
import static io.netty.handler.codec.http2.Http2TestUtil.randomString;
import static io.netty.util.CharsetUtil.UTF_8;
import static java.lang.Math.min;
@ -108,12 +113,12 @@ public class Http2FrameRoundtripTest {
doAnswer(new Answer<ChannelPromise>() {
@Override
public ChannelPromise answer(InvocationOnMock invocation) throws Throwable {
return new DefaultChannelPromise(channel);
return new DefaultChannelPromise(channel, GlobalEventExecutor.INSTANCE);
}
}).when(ctx).newPromise();
writer = new DefaultHttp2FrameWriter();
reader = new DefaultHttp2FrameReader(false);
writer = new DefaultHttp2FrameWriter(new DefaultHttp2HeadersEncoder(NEVER_SENSITIVE, newTestEncoder()));
reader = new DefaultHttp2FrameReader(new DefaultHttp2HeadersDecoder(false, newTestDecoder()));
}
@After
@ -277,7 +282,8 @@ public class Http2FrameRoundtripTest {
@Test
public void headersThatAreTooBigShouldFail() throws Exception {
final Http2Headers headers = headersOfSize(DEFAULT_MAX_HEADER_SIZE + 1);
reader = new DefaultHttp2FrameReader(false);
final Http2Headers headers = headersOfSize(DEFAULT_HEADER_LIST_SIZE + 1);
writer.writeHeaders(ctx, STREAM_ID, headers, 2, (short) 3, true, MAX_PADDING, true, ctx.newPromise());
try {
readFrames();

View File

@ -83,7 +83,7 @@ public class Http2HeaderBlockIOTest {
private void assertRoundtripSuccessful(Http2Headers in) throws Http2Exception {
encoder.encodeHeaders(in, buffer);
Http2Headers out = decoder.decodeHeaders(buffer);
Http2Headers out = decoder.decodeHeaders(0, buffer);
assertEquals(in, out);
}

View File

@ -15,14 +15,15 @@
package io.netty.handler.codec.http2;
import org.junit.Before;
import org.junit.Test;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_FRAME_SIZE_UPPER_BOUND;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_UNSIGNED_INT;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import org.junit.Before;
import org.junit.Test;
/**
* Tests for {@link Http2Settings}.
*/
@ -77,21 +78,20 @@ public class Http2SettingsTest {
}
@Test
public void boundarySettingsShouldBeSet() {
final long overIntegerMaxValue = 1L << 31;
settings.maxHeaderListSize((int) overIntegerMaxValue);
assertEquals(Integer.MAX_VALUE, (long) settings.maxHeaderListSize());
public void headerListSizeUnsignedInt() {
settings.maxHeaderListSize(MAX_UNSIGNED_INT);
assertEquals(MAX_UNSIGNED_INT, (long) settings.maxHeaderListSize());
}
final long settingsValueUpperBound = (1L << 32) - 1L;
settings.maxHeaderListSize((int) settingsValueUpperBound);
assertEquals(Integer.MAX_VALUE, (long) settings.maxHeaderListSize());
@Test(expected = IllegalArgumentException.class)
public void headerListSizeBoundCheck() {
settings.maxHeaderListSize(Long.MAX_VALUE);
}
@Test
public void headerTableSizeUnsignedInt() {
final long value = 1L << 31;
settings.put(Http2CodecUtil.SETTINGS_HEADER_TABLE_SIZE, (Long) value);
assertEquals(value, (long) settings.get(Http2CodecUtil.SETTINGS_HEADER_TABLE_SIZE));
settings.put(Http2CodecUtil.SETTINGS_HEADER_TABLE_SIZE, (Long) MAX_UNSIGNED_INT);
assertEquals(MAX_UNSIGNED_INT, (long) settings.get(Http2CodecUtil.SETTINGS_HEADER_TABLE_SIZE));
}
@Test(expected = IllegalArgumentException.class)

View File

@ -15,6 +15,7 @@
package io.netty.handler.codec.http2;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
@ -22,6 +23,8 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPromise;
import io.netty.channel.DefaultChannelPromise;
import io.netty.handler.codec.ByteToMessageDecoder;
import io.netty.handler.codec.http2.internal.hpack.Decoder;
import io.netty.handler.codec.http2.internal.hpack.Encoder;
import io.netty.util.AsciiString;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.GenericFutureListener;
@ -32,10 +35,13 @@ import java.util.List;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_TABLE_SIZE;
/**
* Utilities for the integration tests.
*/
final class Http2TestUtil {
public final class Http2TestUtil {
/**
* Interface that allows for running a operation that throws a {@link Http2Exception}.
*/
@ -86,6 +92,42 @@ final class Http2TestUtil {
return s;
}
public static Encoder newTestEncoder() {
try {
return newTestEncoder(true, MAX_HEADER_LIST_SIZE, MAX_HEADER_TABLE_SIZE);
} catch (Http2Exception e) {
throw new Error("max size not allowed?", e);
}
}
public static Encoder newTestEncoder(boolean ignoreMaxHeaderListSize,
long maxHeaderListSize, long maxHeaderTableSize) throws Http2Exception {
Encoder encoder = new Encoder();
ByteBuf buf = Unpooled.buffer();
try {
encoder.setMaxHeaderTableSize(buf, maxHeaderTableSize);
encoder.setMaxHeaderListSize(maxHeaderListSize);
} finally {
buf.release();
}
return encoder;
}
public static Decoder newTestDecoder() {
try {
return newTestDecoder(MAX_HEADER_LIST_SIZE, MAX_HEADER_TABLE_SIZE);
} catch (Http2Exception e) {
throw new Error("max size not allowed?", e);
}
}
public static Decoder newTestDecoder(long maxHeaderListSize, long maxHeaderTableSize) throws Http2Exception {
Decoder decoder = new Decoder();
decoder.setMaxHeaderTableSize(maxHeaderTableSize);
decoder.setMaxHeaderListSize(maxHeaderListSize);
return decoder;
}
private Http2TestUtil() {
}

View File

@ -38,6 +38,7 @@ import io.netty.handler.codec.http2.Http2Headers;
import org.junit.Before;
import org.junit.Test;
import static io.netty.handler.codec.http2.Http2TestUtil.newTestDecoder;
import static io.netty.util.AsciiString.EMPTY_STRING;
import static io.netty.util.AsciiString.of;
import static java.lang.Integer.MAX_VALUE;
@ -50,7 +51,7 @@ import static org.mockito.Mockito.verifyNoMoreInteractions;
public class DecoderTest {
private static final int MAX_HEADER_SIZE = 8192;
private static final int MAX_HEADER_LIST_SIZE = 8192;
private static final int MAX_HEADER_TABLE_SIZE = 4096;
private Decoder decoder;
@ -64,15 +65,15 @@ public class DecoderTest {
byte[] b = Hex.decodeHex(encoded.toCharArray());
ByteBuf in = Unpooled.wrappedBuffer(b);
try {
decoder.decode(in, mockHeaders);
decoder.decode(0, in, mockHeaders);
} finally {
in.release();
}
}
@Before
public void setUp() {
decoder = new Decoder(MAX_HEADER_SIZE, MAX_HEADER_TABLE_SIZE, 32);
public void setUp() throws Http2Exception {
decoder = new Decoder();
mockHeaders = mock(Http2Headers.class);
}
@ -81,7 +82,7 @@ public class DecoderTest {
byte[] input = {0, (byte) 0x80, 0};
ByteBuf in = Unpooled.wrappedBuffer(input);
try {
decoder.decode(in, mockHeaders);
decoder.decode(0, in, mockHeaders);
verify(mockHeaders, times(1)).add(EMPTY_STRING, EMPTY_STRING);
} finally {
in.release();
@ -93,7 +94,7 @@ public class DecoderTest {
byte[] input = {0, (byte) 0x81, -1};
ByteBuf in = Unpooled.wrappedBuffer(input);
try {
decoder.decode(in, mockHeaders);
decoder.decode(0, in, mockHeaders);
} finally {
in.release();
}
@ -104,7 +105,7 @@ public class DecoderTest {
byte[] input = {0, (byte) 0x84, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF, (byte) 0xFF};
ByteBuf in = Unpooled.wrappedBuffer(input);
try {
decoder.decode(in, mockHeaders);
decoder.decode(0, in, mockHeaders);
} finally {
in.release();
}
@ -115,7 +116,7 @@ public class DecoderTest {
byte[] input = {0, (byte) 0x81, 0};
ByteBuf in = Unpooled.wrappedBuffer(input);
try {
decoder.decode(in, mockHeaders);
decoder.decode(0, in, mockHeaders);
} finally {
in.release();
}
@ -126,9 +127,9 @@ public class DecoderTest {
byte[] compressed = Hex.decodeHex("FFF0".toCharArray());
ByteBuf in = Unpooled.wrappedBuffer(compressed);
try {
decoder.decode(in, mockHeaders);
decoder.decode(0, in, mockHeaders);
assertEquals(1, in.readableBytes());
decoder.decode(in, mockHeaders);
decoder.decode(0, in, mockHeaders);
} finally {
in.release();
}

View File

@ -41,7 +41,10 @@ import com.google.gson.JsonObject;
import com.google.gson.JsonParseException;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.util.AsciiString;
import io.netty.handler.codec.http2.DefaultHttp2Headers;
import io.netty.handler.codec.http2.Http2Exception;
import io.netty.handler.codec.http2.Http2Headers;
import io.netty.handler.codec.http2.Http2HeadersEncoder;
import java.io.IOException;
import java.io.InputStream;
@ -53,6 +56,9 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2TestUtil.newTestEncoder;
final class TestCase {
private static final Gson GSON = new GsonBuilder()
@ -158,7 +164,11 @@ final class TestCase {
maxHeaderTableSize = Integer.MAX_VALUE;
}
return new Encoder(maxHeaderTableSize);
try {
return newTestEncoder(true, MAX_HEADER_LIST_SIZE, maxHeaderTableSize);
} catch (Http2Exception e) {
throw new Error("invalid initial values!", e);
}
}
private Decoder createDecoder() {
@ -167,20 +177,25 @@ final class TestCase {
maxHeaderTableSize = Integer.MAX_VALUE;
}
return new Decoder(8192, maxHeaderTableSize, 32);
return new Decoder(32, maxHeaderTableSize);
}
private static byte[] encode(Encoder encoder, List<HeaderField> headers, int maxHeaderTableSize,
boolean sensitive) {
final boolean sensitive) throws Http2Exception {
Http2Headers http2Headers = toHttp2Headers(headers);
Http2HeadersEncoder.SensitivityDetector sensitivityDetector = new Http2HeadersEncoder.SensitivityDetector() {
@Override
public boolean isSensitive(CharSequence name, CharSequence value) {
return sensitive;
}
};
ByteBuf buffer = Unpooled.buffer();
try {
if (maxHeaderTableSize != -1) {
encoder.setMaxHeaderTableSize(buffer, maxHeaderTableSize);
}
for (HeaderField e : headers) {
encoder.encodeHeader(buffer, AsciiString.of(e.name), AsciiString.of(e.value), sensitive);
}
encoder.encodeHeaders(buffer, http2Headers, sensitivityDetector);
byte[] bytes = new byte[buffer.readableBytes()];
buffer.readBytes(bytes);
return bytes;
@ -189,12 +204,20 @@ final class TestCase {
}
}
private static Http2Headers toHttp2Headers(List<HeaderField> inHeaders) {
Http2Headers headers = new DefaultHttp2Headers(false);
for (HeaderField e : inHeaders) {
headers.add(e.name, e.value);
}
return headers;
}
private static List<HeaderField> decode(Decoder decoder, byte[] expected) throws Exception {
ByteBuf in = Unpooled.wrappedBuffer(expected);
try {
List<HeaderField> headers = new ArrayList<HeaderField>();
TestHeaderListener listener = new TestHeaderListener(headers);
decoder.decode(in, listener);
decoder.decode(0, in, listener);
return headers;
} finally {
in.release();

View File

@ -41,7 +41,6 @@ public final class HelloWorldHttp2HandlerBuilder
@Override
protected HelloWorldHttp2Handler build(Http2ConnectionDecoder decoder, Http2ConnectionEncoder encoder,
Http2Settings initialSettings) {
HelloWorldHttp2Handler handler = new HelloWorldHttp2Handler(decoder, encoder, initialSettings);
frameListener(handler);
return handler;

View File

@ -36,6 +36,7 @@ import io.netty.buffer.Unpooled;
import io.netty.handler.codec.http2.DefaultHttp2Headers;
import io.netty.handler.codec.http2.Http2Exception;
import io.netty.handler.codec.http2.Http2Headers;
import io.netty.handler.codec.http2.Http2HeadersEncoder;
import io.netty.handler.codec.http2.internal.hpack.Decoder;
import io.netty.handler.codec.http2.internal.hpack.Encoder;
import io.netty.microbench.util.AbstractMicrobenchmark;
@ -48,19 +49,15 @@ import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.infra.Blackhole;
import java.util.List;
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;
public class DecoderBenchmark extends AbstractMicrobenchmark {
@Param
public HeadersSize size;
@Param({ "4096" })
public int maxTableSize;
@Param({ "8192" })
public int maxHeaderSize;
@Param({ "true", "false" })
public boolean sensitive;
@ -70,8 +67,8 @@ public class DecoderBenchmark extends AbstractMicrobenchmark {
private ByteBuf input;
@Setup(Level.Trial)
public void setup() {
input = Unpooled.wrappedBuffer(getSerializedHeaders(Util.headers(size, limitToAscii), sensitive));
public void setup() throws Http2Exception {
input = Unpooled.wrappedBuffer(getSerializedHeaders(Util.http2Headers(size, limitToAscii), sensitive));
}
@TearDown(Level.Trial)
@ -82,7 +79,7 @@ public class DecoderBenchmark extends AbstractMicrobenchmark {
@Benchmark
@BenchmarkMode(Mode.Throughput)
public void decode(final Blackhole bh) throws Http2Exception {
Decoder decoder = new Decoder(maxHeaderSize, maxTableSize, 32);
Decoder decoder = new Decoder();
@SuppressWarnings("unchecked")
Http2Headers headers =
new DefaultHttp2Headers() {
@ -92,18 +89,15 @@ public class DecoderBenchmark extends AbstractMicrobenchmark {
return this;
}
};
decoder.decode(input.duplicate(), headers);
decoder.decode(0, input.duplicate(), headers);
}
private byte[] getSerializedHeaders(List<Header> headers, boolean sensitive) {
Encoder encoder = new Encoder(4096);
private byte[] getSerializedHeaders(Http2Headers headers, boolean sensitive) throws Http2Exception {
Encoder encoder = newTestEncoder();
ByteBuf out = size.newOutBuffer();
try {
for (int i = 0; i < headers.size(); ++i) {
Header header = headers.get(i);
encoder.encodeHeader(out, header.name, header.value, sensitive);
}
encoder.encodeHeaders(out, headers, sensitive ? Http2HeadersEncoder.ALWAYS_SENSITIVE
: Http2HeadersEncoder.NEVER_SENSITIVE);
byte[] bytes = new byte[out.readableBytes()];
out.readBytes(bytes);
return bytes;

View File

@ -32,6 +32,8 @@
package io.netty.microbench.http2.internal.hpack;
import io.netty.buffer.ByteBuf;
import io.netty.handler.codec.http2.Http2Headers;
import io.netty.handler.codec.http2.Http2HeadersEncoder;
import io.netty.handler.codec.http2.internal.hpack.Encoder;
import io.netty.microbench.util.AbstractMicrobenchmark;
import org.openjdk.jmh.annotations.Benchmark;
@ -50,10 +52,12 @@ import org.openjdk.jmh.annotations.Threads;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
import java.io.IOException;
import java.util.List;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import static io.netty.microbench.http2.internal.hpack.HpackUtilBenchmark.newTestEncoder;
@Fork(1)
@Threads(1)
@State(Scope.Benchmark)
@ -65,9 +69,6 @@ public class EncoderBenchmark extends AbstractMicrobenchmark {
@Param
public HeadersSize size;
@Param({ "4096" })
public int maxTableSize;
@Param({ "true", "false" })
public boolean sensitive;
@ -77,13 +78,26 @@ public class EncoderBenchmark extends AbstractMicrobenchmark {
@Param({ "true", "false" })
public boolean limitToAscii;
private List<Header> headers;
private Http2Headers http2Headers;
private ByteBuf output;
private Http2HeadersEncoder.SensitivityDetector sensitivityDetector;
@Setup(Level.Trial)
public void setup() {
headers = Util.headers(size, limitToAscii);
http2Headers = Util.http2Headers(size, limitToAscii);
if (duplicates) {
int size = http2Headers.size();
if (size > 0) {
Iterator<Map.Entry<CharSequence, CharSequence>> itr = http2Headers.iterator();
Map.Entry<CharSequence, CharSequence> entry = itr.next();
http2Headers.clear();
for (int i = 0; i < size; ++i) {
http2Headers.add(entry.getKey(), entry.getValue());
}
}
}
output = size.newOutBuffer();
sensitivityDetector = sensitive ? Http2HeadersEncoder.ALWAYS_SENSITIVE : Http2HeadersEncoder.NEVER_SENSITIVE;
}
@TearDown(Level.Trial)
@ -93,21 +107,10 @@ public class EncoderBenchmark extends AbstractMicrobenchmark {
@Benchmark
@BenchmarkMode(Mode.AverageTime)
public void encode(Blackhole bh) throws IOException {
Encoder encoder = new Encoder(maxTableSize);
public void encode(Blackhole bh) throws Exception {
Encoder encoder = newTestEncoder();
output.clear();
if (duplicates) {
// If duplicates is set, re-add the same header each time.
Header header = headers.get(0);
for (int i = 0; i < headers.size(); ++i) {
encoder.encodeHeader(output, header.name, header.value, sensitive);
}
} else {
for (int i = 0; i < headers.size(); ++i) {
Header header = headers.get(i);
encoder.encodeHeader(output, header.name, header.value, sensitive);
}
}
encoder.encodeHeaders(output, http2Headers, sensitivityDetector);
bh.consume(output);
}
}

View File

@ -15,6 +15,10 @@
*/
package io.netty.microbench.http2.internal.hpack;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.handler.codec.http2.Http2Exception;
import io.netty.handler.codec.http2.internal.hpack.Encoder;
import io.netty.microbench.util.AbstractMicrobenchmark;
import io.netty.util.AsciiString;
import io.netty.util.internal.ConstantTimeUtils;
@ -29,6 +33,9 @@ import org.openjdk.jmh.annotations.Warmup;
import java.util.List;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_LIST_SIZE;
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_HEADER_TABLE_SIZE;
@Threads(1)
@Warmup(iterations = 5)
@Measurement(iterations = 5)
@ -91,4 +98,18 @@ public class HpackUtilBenchmark extends AbstractMicrobenchmark {
return ConstantTimeUtils.equalsConstantTime(s1, s2) != 0;
}
static Encoder newTestEncoder() {
Encoder encoder = new Encoder();
ByteBuf buf = Unpooled.buffer();
try {
encoder.setMaxHeaderTableSize(buf, MAX_HEADER_TABLE_SIZE);
encoder.setMaxHeaderListSize(MAX_HEADER_LIST_SIZE);
} catch (Http2Exception e) {
throw new Error("max size not allowed?", e);
} finally {
buf.release();
}
return encoder;
}
}

View File

@ -31,6 +31,9 @@
*/
package io.netty.microbench.http2.internal.hpack;
import io.netty.handler.codec.http2.DefaultHttp2Headers;
import io.netty.handler.codec.http2.Http2Headers;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -103,4 +106,14 @@ public final class Util {
static List<Header> headers(HeadersSize size, boolean limitToAscii) {
return headersMap.get(new HeadersKey(size, limitToAscii));
}
static Http2Headers http2Headers(HeadersSize size, boolean limitToAscii) {
List<Header> headers = headersMap.get(new HeadersKey(size, limitToAscii));
Http2Headers http2Headers = new DefaultHttp2Headers(false);
for (int i = 0; i < headers.size(); ++i) {
Header header = headers.get(i);
http2Headers.add(header.name, header.value);
}
return http2Headers;
}
}