Motivation:
In Http2FrameCodec we made the incorrect assumption that we can only have 1 buffered outboundstream as maximum. This is not correct and we need to account for multiple buffered streams.
Modifications:
- Use a map to allow buffer multiple streams
- Add unit test.
Result:
Fixes https://github.com/netty/netty/issues/8692.
Motiviation:
Http2FrameCodecTest and Http2MultiplexCodecTest were quite fragile and often not went through the whole pipeline which made testing sometimes hard and error-prone.
Modification:
- Refactor tests to have data flow through the whole pipeline and so made the test more robust (by testing the while implementation).
Result:
Easier to write tests for the codecs in the future and more robust testing in general.
Beside this it also fixes https://github.com/netty/netty/issues/6036.
Motivation:
We should always call ctx.read() even when AUTO_READ is false as flow-control is enforced by the HTTP/2 protocol.
See also https://tools.ietf.org/html/rfc7540#section-5.2.2.
We already did this before but not explicit and only did so because of some implementation details of ByteToMessageDecoder. It's better to be explicit here to not risk of breakage later on.
Modifications:
- Ensure we always call ctx.read() when AUTO_READ is false
- Add unit test.
Result:
No risk of staling the connection when HTTP/2 is used.
Motivation:
In windows if the project is in a path that contains whitespace,
resources cannot be accessed and tests fail.
Modifications:
Adds ResourcesUtil.java in netty-common. Tests use ResourcesUtil.java to access a resource.
Result:
Being able to build netty in a path containing whitespace
Motivation:
ByteBuf supports “marker indexes”. The intended use case for these is if a speculative operation (e.g. decode) is in process the user can “mark” and interface and refer to it later if the operation isn’t successful (e.g. not enough data). However this is rarely used in practice,
requires extra memory to maintain, and introduces complexity in the state management for derived/pooled buffer initialization, resizing, and other operations which may modify reader/writer indexes.
Modifications:
Remove support for marking and adjust testcases / code.
Result:
Fixes https://github.com/netty/netty/issues/8535.
Motivation:
We did not correct respect ctx.read() calls while processing a read for a child Channel. This could lead to read stales when auto read is disabled and no other read was requested.
Modifications:
- Keep track of extra read() calls while processing reads
- Add unit tests that verify that read() is respected when triggered either in channelRead(...) or channelReadComplete(...)
Result:
Fixes https://github.com/netty/netty/issues/8209.
Motivation:
When the DefaultHttp2ConnectionEncoder writes the initial headers for a new
locally created stream we create the stream in the half-closed state if the
end-stream flag is set which signals to the life cycle manager that the headers
have been sent. However, if we synchronously fail to write the headers the
life cycle manager then sends a RST_STREAM on our behalf which is a connection
level PROTOCOL_ERROR because the peer sees the stream in an IDLE state.
Modification:
Don't open the stream in the half-closed state if the end-stream flag is
set and let the life cycle manager take care of it.
Result:
Cleaner state management in the DefaultHttp2ConnectionEncoder.
Fixes#8434.
Motivation:
The `Http2StreamFrameToHttpObjectCodec` is marked `@Sharable` but mutates
an internal `HttpScheme` field every time it is added to a pipeline.
Modifications:
Instead of storing the `HttpScheme` in the handler we store it as an
attribute on the parent channel.
Result:
Fixes#8480.
Motivation:
When writing an HTTP/2 HEADERS with END_STREAM=1, the application expects
the stream to be closed afterward. However, the write can fail locally
due to HPACK encoder and similar. When that happens we need to make sure
to issue a RST_STREAM otherwise the stream can be closed locally but
orphaned remotely. The RST_STREAM is typically handled by
Http2ConnectionHandler.onStreamError, which will only send a RST_STREAM
if that stream still exists locally.
There are two possible flows for trailers, one handled immediately and
one going through the flow controller. Previously they behaved
differently, with the immedate code calling the error handler after
closing the stream. The immediate code also used a listener for calling
closeStreamLocal while the flow controlled code did so immediately after
the write.
The two code paths also differed in their VoidChannelPromise handling,
but both were broken. The immediate code path called unvoid() only if
END_STREAM=1, however it could always potentially add a listener via
notifyLifecycleManagerOnError(). And the flow controlled code path
unvoided incorrectly, changing the promise completion behavior. It also
passed the wrong promise to closeStreamLocal() in FlowControlledBase.
Modifications:
Move closeStreamLocal handling after calls to onError. This is the
primary change.
Now call closeStreamLocal immediately instead of when the future
completes. This is the more likely correct behavior as it matches that
of DATA frames.
Fix all the VoidChannelPromise handling.
Result:
Http2ConnectionHandler.onStreamError sees the same state as the remote
and issues a RST_STREAM, properly cleaning up the stream.
Motivation:
Http2MultiplexCodec queues data internally if data is delivered from the
parent channel but the child channel did not request data. If the parent
channel notifies of a stream closure it is possible data in the queue
will be discarded before closing the channel.
Http2MultiplexCodec interacts with RecvByteBufAllocator to control the
child channel's demand for read. However it currently only ever reads a
maximum of one time per loop. This can thrash the read loop and bloat
the call stack if auto read is on, because channelReadComplete will
re-enter the read loop synchronously, and also neglect to deliver data
during the parent's read loop (if it is active). This also meant the
readPendingQueue was not utilized as originally intended (to extend the
child channel's read loop during the parent channel's read loop if
demand for data still existed).
Modifications:
- Modify the child channel's read loop to respect the
RecvByteBufAllocator, and append to the parents readPendingQueue if
appropriate.
- Stream closure notification behaves like EPOLL and KQUEUE transports
and reads all queued data, because the data is already queued in memory
and it is known there will be no more data. This will also replenish the
connection flow control window which may otherwise be constrained by a
closed stream.
Result:
More correct read loop and less risk of dropping data.
Motivation:
When a Http2MultiplexCodec stream channel fails to write the first
HEADERS it will forcibly close, and that will trigger sending a
RST_STREAM, which is commonly a connection level protocol error. This is
because it has what looks like a valid stream id, but didn't check with
the connection as to whether the stream may have actually existed.
Modifications:
Instead of checking if the stream was just a valid looking id ( > 0) we
check with the connection as to whether it may have existed at all.
Result:
We no longer send a RST_STREAM frame from Http2MultiplexCodec for idle
streams.
Motivation:
The Http2Connection state is updated by the DefaultHttp2ConnectionDecoder after the frame listener is notified of the goaway frame. If the listener sends a frame synchronously this means the connection state will not know about the goaway it just received and we may send frames that are not allowed on the connection. This may also mean a stream object is created but it may never get taken out of the stream map unless some other event occurs (e.g. timeout).
Modifications:
- The Http2Connection state should be updated before the listener is notified of the goaway
- The Http2Connection state modification and validation should be self contained when processing a goaway instead of partially in the decoder.
Result:
No more creating streams and sending frames after a goaway has been sent or received.
Motivation:
If the local endpoint receives a GO_AWAY frame and then tries to write a stream with a streamId higher than the last know stream ID we will throw a connection error. This results in the local peer sending a GO_AWAY frame to the remote peer, but this is not necessary as the error can be isolated to the local endpoint and communicated via the ChannelFuture return value.
Modifications:
- Instead of throwing a connection error, throw a stream error that simulates the peer receiving the stream and replying with a RST
Result:
Connections are not closed abruptly when trying to create a stream on the local endpoint after a GO_AWAY frame is received.
Motivation:
If a write fails for a Http2MultiplexChannel stream channel, the channel
may be forcibly closed, but only after the promise has been failed. That
means continuations attached to the promise may see the channel in an
inconsistent state of still being open and active.
Modifications:
Move the satisfaction of the promise to after the channel cleanup logic
runs.
Result:
Listeners attached to the future that resulted in a Failed write will
see the stream channel in the correct state.
Motivation:
The HTTP/2 spec dictates that invalid pseudo-headers should cause the
request/response to be treated as malformed (8.1.2.1), and the recourse
for that is to treat the situation as a stream error of type
PROTOCOL_ERROR (8.1.2.6). However, we're treating them as a connection
error with the connection being immediately torn down and the HPACK
state potentially being corrupted.
Modifications:
The HpackDecoder now throws a StreamException for validation failures
and throwing is deffered until the end of of the decode phase to ensure
that the HPACK state isn't corrupted by returning early.
Result:
Behavior more closely aligned with the HTTP/2 spec.
Fixes#8043.
Motivation:
There is an inconsistency between the order of events in the
StreamChannel implementation in Http2MultiplexCodec and other Channel
implementations that extend AbstractChannel where channelInactive and
channelUnregistered events are not performed 'later'. This can cause an
unexected order of events for ChannelHandler implementations that call
Channel.close() in response to some event.
Modification:
The Http2MultiplexCodec.DefaultHttp2StreamChannel.Http2ChannelUnsafe was
modified to bounce the deregistration and channelInactive events through
the parent channels EventLoop.
Result:
Stream events are now in the proper order.
Fixes#8018.
Motivation:
Http2MultiplexCodec doesn't currently have an API for using the response
of a h2c upgrade request.
Modifications:
Add a new API to the Http2MultiplexCodecBuilder which allows for setting
an upgrade handler and wire it into the Http2MultiplexCodec
implementation.
Result:
When using the Http2MultiplexCodec with h2c upgrades the upgrade handler
will get added to the Http2StreamChannel which represents the
half-closed (local) response of stream 1. It is then up to the user to
manage the transition from the IO channel pipeline configuration
necessary for making the h2c upgrade request to a form where it can read
the response from the new stream channel.
Fixes#7947.
Motivation:
This is a followup for #7860. In the fix for #7860 we only partly fixed the problem as Http2UnknownFrame did not correctly extend HttpStreamFrame and so only worked when using the Http2FrameCodec. We need to have it extend HttpStreamFrame as otherwise Http2MultiplexCodec will reject to handle it correctly.
Modifications:
- Let Http2UnknownFrame extend HttpStreamFrame
- Add unit tests for writing and reading Http2UnkownFrame instances when the Http2MultiplexCodec is used.
Result:
Fixes https://github.com/netty/netty/issues/7969.
Motivation:
When a sender sends too large of headers it should not unnecessarily
kill the connection, as killing the connection is a heavy-handed
solution while SETTINGS_MAX_HEADER_LIST_SIZE is advisory and may be
ignored.
The maxHeaderListSizeGoAway limit in HpackDecoder is unnecessary because
any headers causing the list to exceeding the max size can simply be
thrown away. In addition, DefaultHttp2FrameReader.HeadersBlockBuilder
limits the entire block to maxHeaderListSizeGoAway. Thus individual
literals are limited to maxHeaderListSizeGoAway.
(Technically, literals are limited to 1.6x maxHeaderListSizeGoAway,
since the canonical Huffman code has a maximum compression ratio of
.625. However, the "unnecessary" limit in HpackDecoder was also being
applied to compressed sizes.)
Modifications:
Remove maxHeaderListSizeGoAway checking in HpackDecoder and instead
eagerly throw away any headers causing the list to exceed
maxHeaderListSize.
Result:
Fewer large header cases will trigger connection-killing.
DefaultHttp2FrameReader.HeadersBlockBuilder will still kill the
connection when maxHeaderListSizeGoAway is exceeded, however.
Fixes#7887
Motivation:
Streams can be deregistered so we can't assume their existence in the stream map.
Modifications:
Add a null-check in case a stream has been deregistered.
Result:
Fixes#7898.
Motivation:
We incorrectly called frame.release() in onHttp2GoAwayFrame which could lead to IllegalReferenceCountExceptions. The call of release() is inappropriate because the fireChannelRead() in onHttp2Frame() will handle it.
Modifications:
- Not call frame.release()
- Add a unit test
Result:
Fxies https://github.com/netty/netty/issues/7892.
It is possible to create streams in the half-closed state where the
stream state doesn't reflect that the request headers have been sent by
the client or the server hasn't received the request headers. This
state isn't possible in the H2 spec as a half closed stream must have
either received a full request or have received the headers from a
pushed stream. In the current implementation, this can cause the stream
created as part of an h2c upgrade request to be in this invalid state
and result in the omission of RST frames as the client doesn't believe
it has sent the request to begin with.
Modification:
The `DefaultHttp2Connection.activate` method checks the state and
modifies the status of the request headers as appropriate.
Result:
Fixes#7847.
Motivation:
When connecting to an HTTP/2 server that did not set any value for the
SETTINGS_MAX_HEADER_LIST_SIZE in the settings frame, the netty client was
imposing an arbitrary maximum header list size of 8kB. There should be no need
for the client to enforce such a limit if the server has not specified any
limit. This caused an issue for a grpc-java client that needed to send a large
header to a server via an Envoy proxy server. The error condition is
demonstrated here: https://github.com/JLofgren/demo-grpc-java-bug-4284
Fixes grpc-java issue #4284 - https://github.com/grpc/grpc-java/issues/4284
and netty issue #7825 - https://github.com/netty/netty/issues/7825
Modifications:
In HpackEncoder use MAX_HEADER_LIST_SIZE as default maxHeader list size.
Result:
HpackEncoder will only enforce a max header list size if the server has
specified a limit in its settings frame.
Motivation:
We should allow to write Http2UnkownFrame to allow custom extensions.
Modifications:
Allow to write Http2UnkownFrame
Add unit test
Result:
Fixes https://github.com/netty/netty/issues/7860.
Motivation:
There is a race between both flushing the upgrade response and receiving
more data before the flush ChannelPromise can fire and reshape the
pipeline. Since We have already committed to an upgrade by writing the
upgrade response, we need to be immediately prepared for handling the
next protocol.
Modifications:
The pipeline reshaping logic in HttpServerUpgradeHandler has been moved
out of the ChannelFutureListener attached to the write of the upgrade
response and happens immediately after the writeAndFlush call, but
before the method returns.
Result:
The pipeline is no longer subject to receiving more data before the
pipeline has been reformed.
Motivation:
We did not correctly set the stream id in the headers of HttpMessage when converting a Http2HeadersFrame. This is based on https://github.com/netty/netty/pull/7778 so thanks to @jprante.
Modifications:
- Correctly set the id when possible in the header.
- Add test case
Result:
Correctly include stream id.
Motivation:
Allow the observation of SETTINGS frame by other handlers in the pipeline. For my particular use case this allows me to observe the value of MAX_CONCURRENT_STREAMS for a ChannelPool abstraction that supports HTTP/2 multiplexing. Beside this also forward GOAWAY frames.
Modification:
Always forward SETTINGS and GOAWAY frames
Result:
Settings / Goaway can now be observed in the parent channel. Previously it was not possible (to my knowledge) to capture the settings when using Http2MultiplexCodec.
Motivation:
At the moment we use a ByteBuf as the payload for a http2 frame. This complicates life-time management a lot with no real gain and also may produce more objects then needed. We should just use a long as it is required to be 8 bytes anyway.
Modifications:
Use long for ping payloads.
Result:
Fixes [#7629].
Motivation:
Reflective setAccessible(true) will produce scary warnings on the console when using java9+, while netty still works. That said users may feel uncomfortable with these warnings, we should not try to do it by default when using java9+.
Modifications:
Add io.netty.tryReflectionSetAccessible system property which controls if setAccessible(...) will be used. By default it will bet set to false when using java9+.
Result:
Fixes [#7254].
Motivation:
According to the spec:
All pseudo-header fields MUST appear in the header block before regular
header fields. Any request or response that contains a pseudo-header
field that appears in a header block after
a regular header field MUST be treated as malformed (Section 8.1.2.6).
Pseudo-header fields are only valid in the context in which they are defined.
Pseudo-header fields defined for requests MUST NOT appear in responses;
pseudo-header fields defined for responses MUST NOT appear in requests.
Pseudo-header fields MUST NOT appear in trailers.
Endpoints MUST treat a request or response that contains undefined or
invalid pseudo-header fields as malformed (Section 8.1.2.6).
Clients MUST NOT accept a malformed response. Note that these requirements
are intended to protect against several types of common attacks against HTTP;
they are deliberately strict because being permissive can expose
implementations to these vulnerabilities.
Modifications:
- Introduce validation in HPackDecoder
Result:
- Requests with unknown pseudo-field headers are rejected
- Requests with containing response specific pseudo-headers are rejected
- Requests where pseudo-header appear after regular header are rejected
- h2spec 8.1.2.1 pass
Motivation:
We should convert Http2Exceptions that are produced because of STREAM_CLOSED to ClosedChannelException when hand-over to the child channel to make it more consistent with other transports.
Modifications:
- Check if STREAM_CLOSED is used and if so create a new ClosedChannelException (while preserve the original exception as cause) and use it in the child channel
- Ensure STREAM_CLOSED is used in DefaultHttp2RemoteFlowController when writes are failed because of a closed stream.
- Add testcase
Result:
More consistent and correct exception usage.
Motivation:
When checking if a value is present, ReadOnlyHttp2Headers always ignores
case for values.
RFC 7540 says: https://tools.ietf.org/html/rfc7540#section-8.1.2
"header field names are strings of ASCII characters that are compared in a case-insensitive fashion"
But there is no such constraint on header values
Modifications:
Updated ReadOnlyHttp2Headers.contains to compare header value in a
case-sensitive way.
Result:
ReadOnlyHttp2Headers compares header names in a case-insensitive way,
values in a case-sensitive way.
Motivation:
If you test a header value providing a String, contains() returns false.
This is due to the implementation inherited from DefaultHeaders using
the JAVA_HASHER.
JAVA_HASHER.equals returns false because a is a String and b an
AsciiString.
Modifications:
DefaultHttp2Headers overrides contains and uses CASE_SENSITIVE_HASHER.
Result:
You can test a header value with any CharSequence implementation.
Motivation:
The completion order of promises in Http2MultiplexChannel#close should be consistent with that of AbstractChannel. Otherwise this may result in Future listeners seeing incorrect channel state.
Modifications:
Add tests cases.
Result:
Ensure consistent behavior between Http2MultiplexChannel and AbstractChannel.
Motivation:
Calling DefaultHttp2StreamChannel.Unsafe.close(...) multiple times should not fail.
Modification:
- Correctly handle multiple calls to DefaultHttp2StreamChannel.Unsafe.close(...)
- Complete closePromise and promise that is given to close(...) in the correct order.
- Add unit test
Result:
Fixes [#7628] and [#7641]
Motivation:
When part of a HTTP/2 StreamChannel the Http2StreamChannel.isOpen() / isActive() should report false within a call to a ChannelInboundHandlers channelInactive() method.
Modifications:
Fullfill promise before call fireChannelInactive()
Result:
Correctly update state / promise before notify handlers. Fixes [#7638]
Motivation:
With HTTP1, it's very easy to check if a header is present and has a
given value: you can simply invoke
io.netty.handler.codec.http.HttpHeaders#contains(java.lang.CharSequence, java.lang.CharSequence, boolean)
It is not possible to do the same with HTTP2. You have to get the list
of all headers (returned as String) and then iterate over it invoking
String#equals or String#equalsIgnoreCase
Modifications:
I've added io.netty.handler.codec.http2.Http2Headers#contains and
implemented it in DefaultHttp2Headers, EmptyHttp2Headers and ReadOnlyHttp2Headers.
Result:
You can use AsciiString constants to check if a header is present in a
consice and efficient manner.
Motivation:
Usually when using netty exceptions which happen for outbound operations should not be fired through the pipeline but only the ChannelPromise should be failed.
Modifications:
- Change Http2LifecycleManager.onError(...) to take also an boolean that indicate if the error was caused by an outbound operation
- Channel Http2ConnectionHandler.on*Error(...) methods to also take this boolean
- Change Http2FrameCodec to only fire exceptions through the pipeline if these are not outbound operations related
- Add unit test.
Result:
More consistent error handling when using Http2FrameCodec and Http2MultiplexCodec.
Motivation:
Http2MultiplexCodec swallows Http2PingFrames without releasing the payload, resulting in a memory leak.
Modification:
Send unhandled frames down the pipeline for consumption/disposal by another InboundChannelHandler.
Result:
Fixes#7607.
Motivation:
Http2FrameCodec increases the initialWindowSize when the user attempts to increase the connection flow control window. The initialWindowSize should only be touched as a result of a SETTINGS frame, and otherwise may result in flow control getting out of sync with our peer.
Modifications:
- Http2FrameCodec shouldn't update the initialWindowSize when a WindowUpdateFrame is written on the connection channel
Result:
More correct WindowUpdate processing.
Motivation:
HPackDecoder works on entire header block, we shouldn't encounter
incomplete header fields. If we do we should treat it as
a decoding error and according to the specification:
A decoding error in a header block MUST be treated as
a connection error (Section 5.4.1) of type COMPRESSION_ERROR.
Modifications:
* Check final state in HpackDecoder once we've decoded all the data.
Result:
* Throw a connection error if we receive incomplete header fields
* H2spec 4.3 tests all passes
Motivation:
Http2FrameStream#CONNECTION_STREAM is required to identify the
connection stream. However this leads to inconsistent usage from a user
perspective. When a user creates a Http2Frame for a non-connection
stream, the Http2MultiplexCodec automatically sets the stream, and the
user is never exposed to the Http2FrameStream object. However when the
user writes a Http2Frame for a connection stream they are required to
set the Http2FrameStream object. We can remove the Http2FrameStream#CONNECTION_STREAM
and keep the Http2FrameStream object internal, and therefore consistent
between the connection and non-connection use cases.
Modifications:
- Remove Http2FrameStream#CONNECTION_STREAM
- Update Http2FrameCodec to handle Http2Frame#stream() which returns
null
Result:
More consistent usage on http2 parent channel and http2 child channel.
Motivation:
Http2FrameCodecTest#newOutboundStream has a timeout of 1 second and has been observed to timeout on CI servers.
Modifications:
- Increase the timeout to 5 seconds
Result:
Less false positive test failures on CI servers.
Motivation:
According to the HTTP/2 Spec:
SETTINGS_MAX_HEADER_LIST_SIZE (0x6): This advisory setting informs a
peer of the maximum size of header list that the sender is
prepared to accept, in octets. The value is based on the
uncompressed size of header fields, including the length of the
name and value in octets plus an overhead of 32 octets for each
header field.
We were accounting for the 32 bytes when encoding in HpackEncoder,
but not when decoding in HPackDecoder.
Modifications:
- Add 32 bytes to the header list length for each entry when decoding
with HPackDecoder.
Result:
- We account for the 32 bytes overhead by header entry in HPackDecoder
Motiviation:
In our replace(...) methods we always used validation for the newly created headers while the original headers may not use validation at all.
Modifications:
- Only use validation if the original headers used validation as well.
- Ensure we create a copy of the headers in replace(...).
Result:
Fixes [#5226]
Motivation:
Currently the remote flow controller limits the maximum amount of pending data to Integer.MAX_VALUE. The overflow handling is also not very graceful in that it may lead to infinite loops, or otherwise no progress being made.
Modifications:
- StreamByteDistributor and RemoteFlowController should support pending bytes of type long.
Result:
Fixes https://github.com/netty/netty/issues/4283
Motivation:
http/2 counts header sizes somewhat inconsistently. Sometimes, headers
which are substantively less than the header list size will be measured
as longer than the header list size.
Modifications:
Keep better track of the nameLength of a given name, so that we don't
accidentally end up reusing a nameLength.
Result:
More consistent measurement of header list size.
Fixes#7511.
H2C upgrades should be ineligible for flow control
Motivation:
When the h2c upgrade request is too big, the Http2FrameCodec complains
it's too big for flow control reasons, even though it's ineligible for
flow control.
Modifications:
Specially mark upgrade streams and make Http2FrameCodec know not to try
to flow control on those streams.
Result:
Servers won't barf when they receive an upgrade request with a fat
payload.
[Fixes#7280]
Motivation:
Http2ConnectionHandler uses ctx.fireUserEvent to propagate the Http2ConnectionPrefaceAndSettingsFrameWrittenEvent through the pipeline. This will propagate the event to the next inbound handler in the pipeline. If the user extends Http2ConnectionHandler the Http2ConnectionPrefaceAndSettingsFrameWrittenEvent may be missed and initialization dependent upon this event will not be run.
Modifications:
- Http2ConnectionHandler should use userEventTriggered instead of ctx.fireUserEvent
Result:
Classes that extend Http2ConnectionHandler will see the Http2ConnectionPrefaceAndSettingsFrameWrittenEvent user event.
Motivation:
HttpConversionUtil#toHttp2Headers has special code to filter the TE header name. However this filtering code may result in adding the <TE, TRAILERS> tuple in scenarios that are not appropriate. For example if a value containing trailers is seen it will be added, but the value could not actually be equal to trailers. Also CSV values are not supported.
Modifications:
- Account for CSV header values
- Account for the value containing 'trailers' but not actually being equal to 'trailers'
Result:
More robust parsing of the TE header.
Motivation:
Netty could handle "connection" or "te" headers more gently when
converting from http/1.1 to http/2 headers. Http/2 headers don't
support single-hop headers, so when we convert from http/1.1 to http/2,
we should drop all single-hop headers. This includes headers like
"transfer-encoding" and "connection", but also the headers that
"connection" points to, since "connection" can be used to designate
other headers as single-hop headers. For the "te" header, we can more
permissively convert it by just dropping non-conforming headers (ie
non-"trailers" headers) which is what we do for all other headers when
we convert.
Modifications:
Add a new blacklist to the http/1.1 to http/2 conversion, which is
constructed from the values of the "connection" header, and stop
throwing an exception when a "te" header is passed with a non-"trailers"
value. Instead, drop all values except for "trailers". Add unit tests
for "connection" and "te" headers when converting from http/1.1 to http/2.
Result:
This will improve the h2c upgrade request, and also conversions from
http/1.1 to http/2. This will simplify implementing spec-compliant
http/2 servers that want to share code between their http/1.1 and http/2
implementations.
[Fixes#7355]
Motivation:
HTTP/2 allows writes of 0 length data frames. However in some cases EMPTY_BUFFER is used instead of the actual buffer that was written. This may mask writes of released buffers or otherwise invalid buffer objects. It is also possible that if the buffer is invalid AbstractCoalescingBufferQueue will not release the aggregated buffer nor fail the associated promise.
Modifications:
- DefaultHttp2FrameCodec should take care to fail the promise, even if releasing the data throws
- AbstractCoalescingBufferQueue should release any aggregated data and fail the associated promise if something goes wrong during aggregation
Result:
More correct handling of invalid buffers in HTTP/2 code.
Motivation:
If a child channel's read is triggered outside the parent channel's read
loop then it is possible a WINDOW_UPDATE will be written, but not
flushed.
If a child channel's beginRead processes data from the inboundBuffer and
then readPending is set to false, which will result in data not being
delivered if in the parent's read loop and more data is attempted to be
delievered to that child channel.
Modifications:
- The child channel must force a flush if a frame is written as a result
of reading a frame, and this is not in the parent channel's read loop
- The child channel must allow a transition from dequeueing from
beginRead into the parent channel's read loop to deliver more data
Result:
The child channel flushes data when reading outside the parent's read
loop, and has frames delivered more reliably.
Motivation:
Http2StreamFrameToHttpObjectCodec was not properly encoding and
decoding 100-Continue HttpResponse/Http2SettingsFrame properly. It was
encoding 100-Continue FullHttpResponse as an Http2SettingFrame with
endStream=true, causing the child channel to terminate. It was not
decoding 100-Continue Http2SettingsFrame (endStream=false) as
FullHttpResponse. This should be fixed as it would cause http2 child
stream to prematurely close, and could cause HttpObjectAggregator to
fail if it's in the pipeline.
Modification:
- Fixed encode() to properly encode 100-Continue FullHttpResponse as
Http2SettingsFrame with endStream=false
- Reject 100-Continue HttpResponse that are NOT FullHttpResponse
- Fixed decode() to properly decode 100-Continue Http2SettingsFrame
(endStream=false) as a FullHttpResponse
- made Http2StreamFrameToHttpObjectCodec sharable so that it can b used
among child streams within the same Http2MultiplexCodec
Result:
Now Http2StreamFrameToHttpObjectCodec should be properly handling
100-Continue responses.
Motivation:
Previously client Http2ConnectionHandler trigger a user event
immediately when the HTTP/2 connection preface is sent. Any attempt to
immediately send a new request could cause the server to terminate the
connection, as it might not have received the SETTINGS frame from the
client. Per RFC7540 Section 3.5, the preface "MUST be followed by a
SETTINGS frame (Section 6.5), which MAY be empty."
(https://tools.ietf.org/html/rfc7540#section-3.5)
This event could be made more meaningful if it also indicates that the
initial client SETTINGS frame has been sent to signal that the channel
is ready to send new requests.
Modification:
- Renamed event to Http2ConnectionPrefaceAndSettingsFrameWrittenEvent.
- Modified Http2ConnectionHandler to trigger the user event only if it
is a client and it has sent both the preface and SETTINGS frame.
Result:
It is now safe to use the event as an indicator that the HTTP/2
connection is ready to send new requests.
Motivation:
A regression was introduced in 86e653e which had the effect that the writability was not updated for a Channel while queueing data in the SslHandler.
Modifications:
- Factor out code that will increment / decrement pending bytes and use it in AbstractCoalescingBufferQueue and PendingWriteQueue
- Add test-case
Result:
Channel writability changes are triggered again.
Motivation:
Even if it's a super micro-optimization (most JVM could optimize such
cases in runtime), in theory (and according to some perf tests) it
may help a bit. It also makes a code more clear and allows you to
access such methods in the test scope directly, without instance of
the class.
Modifications:
Add 'static' modifier for all methods, where it possible. Mostly in
test scope.
Result:
Cleaner code with proper 'static' modifiers.
Motivation:
During code read of the Netty codebase I noticed that the Netty
HttpServerUpgradeHandler unconditionally sets a Content-Length: 0
header on 101 Switching Protocols responses. This explicitly
contravenes RFC 7230 Section 3.3.2 (Content-Length), which notes
that:
A server MUST NOT send a Content-Length header field in any
response with a status code of 1xx (Informational) or 204
(No Content).
While it is unlikely that any client will ever be confused by
this behaviour, there is no reason to contravene this part of the
specification.
Modifications:
Removed the line of code setting the header field and changed the
only test that expected it to be there.
Result:
When performing the server portion of HTTP upgrade, the 101
Switching Protocols response will no longer contain a
Content-Length: 0 header field.
Motivation:
Http2ServerDowngrader is specifically built for server channels where
inbound Http2StreamFrames are converted into HttpRequests, and outbound
HttpResponses are converted into Http2StreamFrames. It can be easily
made to be more generic to work with client channels where inbound
Http2StreamFrames are converted into HttpResponses, and outbound
HttpRequests are converted into Http2StreamFrames.
Modification:
- Renamed Http2ServerDowngrader to a more general
Http2StreamFrameToHttpObjectCodec
- Made it take in an "isServer" parameter to determine whether encoding
inbound Http2StreamFrames should create HttpRequests (for server) or
HttpResponses (for client)
- Norman fixed a leak in the unit test. Thanks! :-)
Result:
Now Http2StreamFrameToHttpObjectCodec can be used to translate
Http2StreamFrame to HttpObject for both server and client.
Motivation:
We changed Http2ConnectionHandler to expect the upgrade method to be
called *after* we send the preface (ie add the handler to the pipeline)
but we forgot to change the Http2ClientUpgradeCodec to match the new
policy. This meant that client-side h2c upgrades failed.
Modifications:
Reverse sending the preface and calling the upgrade method to match the
new policy.
Result:
Clients can initiate h2c upgrades successfully.
Motiviation:
At the moment an NPE is thrown if someone tries to use the InboundHttp2ToHttpAdapter.
Modifications:
- Ensure the status was null in "InboundHttp2ToHttpAdapter::onPushPromiseRead" before calling "HttpConversionUtil.parseStatus" methods.
- Fix setting status to OK in "InboundHttp2ToHttpAdapter::onPushPromiseRead".
Result:
Fixes [#7214].
Motivation:
The Headers interface supports an interface to get all the headers values corresponding to a particular name. This API returns a List which requires intermediate storage and increases GC pressure.
Modifications:
- Add a method which returns an iterator over all the values for a specific name
Result:
Ability to iterator over values for a specific name with no intermediate collection.
Motivation:
We missed to mark the Http2StreamChannel as writable in some cases which could lead to the situation that a Channel never becomes writable. Also when a Http2StreamChannel was created we always marked it non-writable at the beginning which means if the user will only start writing once the Channel becomes writable it will never happen as it only became writable after the first header was written.
Modifications:
- Correctly handle updates for writability in all cases
- Change unit tests to cover this.
Result:
Fixes [#7179].
Motiviation:
At the moment an NPE is thrown if someone tries to use the Http2ServerUpgradeCodec with Http2FrameCodec and Http2MultiplexCodec.
Modifications:
- Ensure the handler was added to the pipeline before calling on*Upgrade(...) methods.
- Add tests
- Fix adding of handlers after upgrade.
Result:
Fixes [#7173].
Motivation:
We must not add the inboundStreamHandler for outbound streams creates by Http2MultiplexCodec as the user will specify a handler via Http2StreamChannelBootstrap.
Modifications:
- Check if the stream is for outbound and if so not add the inboundStreamHandler to the pipeline
- Update tests so this is covered.
Result:
Fixes [#7178]
Motivation:
379ac890f4 introduced the usage of the inline mock maker. This unfortunally not work on java7 and java9.
Modifications:
Just use reflection to create the event for now.
Result:
Netty tests pass again on java7 and java9 as well.
Motivation:
When using Http2FrameCodec / Http2MultiplexCodec with HttpServerUpgradeHandler reference count exception will be triggered.
Modifications:
- Correctly retain before calling InboundHttpToHttp2Adapter.handle
- Add unit test
Result:
Fixes [#7172].
Motivation:
The construction `new AsciiString(string.substring(...))` can be replaced with the `new AsciiString(string, start, length)` to avoid extra allocation.
Modifications:
Apply the described replacement in `HttpConversionUtil#setHttp2Authority`.
Result:
Less allocations.
Motivation:
Our http2 child channel implementation was not 100 % complete and had a few bugs. Beside this the performance overhead was non-trivial.
Modifications:
There are a lot of modifications, the most important....
* Http2FrameCodec extends Http2ConnectionHandler and Http2MultiplexCodec extends Http2FrameCodec to reduce performance heads and inter-dependencies on handlers in the pipeline
* Correctly handle outbound flow control for child channels
* Support unknow frame types in Http2FrameCodec and Http2MultiplexCodec
* Use a consistent way how to create Http2ConnectionHandler, Http2FrameCodec and Http2MultiplexCodec (via a builder)
* Remove Http2Codec and Http2CodecBuilder as the user should just use Http2MultipleCodec and Http2MultiplexCodecBuilder now
* Smart handling of flushes from child channels to reduce overhead
* Reduce object allocations
* child channels always use the same EventLoop as the parent Channel to reduce overhead and simplify implementation.
* Not extend AbstractChannel for the child channel implementation to reduce overhead in terms of performance and memory usage
* Remove Http2FrameStream.managedState(...) as the user of the child channel api should just use Channel.attr(...)
Result:
Http2MultiplexCodec (and so child channels) and Http2FrameCodec are more correct, faster and more feature complete.
Motivation:
This PR (unfortunately) does 4 things:
1) Add outbound flow control to the Http2MultiplexCodec:
The HTTP/2 child channel API should interact with HTTP/2 outbound/remote flow control. That is,
if a H2 stream used up all its flow control window, the corresponding child channel should be
marked unwritable and a writability-changed event should be fired. Similarly, a unwritable
child channel should be marked writable and a writability-event should be fired, once a
WINDOW_UPDATE frame has been received. The changes are (mostly) contained in ChannelOutboundBuffer,
AbstractHttp2StreamChannel and Http2MultiplexCodec.
2) Introduce a Http2Stream2 object, that is used instead of stream identifiers on stream frames. A
Http2Stream2 object allows an application to attach state to it, and so a application handler
no longer needs to maintain stream state (i.e. in a map(id -> state)) himself.
3) Remove stream state events, which are no longer necessary due to the introduction of Http2Stream2.
Also those stream state events have been found hard and complex to work with, when porting gRPC
to the Http2FrameCodec.
4) Add support for HTTP/2 frames that have not yet been implemented, like PING and SETTINGS. Also add
a Http2FrameCodecBuilder that exposes options from the Http2ConnectionHandler API that couldn't else
be used with the frame codec, like buffering outbound streams, window update ratio, frame logger, etc.
Modifications:
1) A child channel's writability and a H2 stream's outbound flow control window interact, as described
in the motivation. A channel handler is free to ignore the channel's writability, in which case the
parent channel is reponsible for buffering writes until a WINDOW_UPDATE is received.
The connection-level flow control window is ignored for now. That is, a child channel's writability
is only affected by the stream-level flow control window. So a child channel could be marked writable,
even though the connection-level flow control window is zero.
2) Modify Http2StreamFrame and the Http2FrameCodec to take a Http2Stream2 object intstead of a primitive
integer. Introduce a special Http2ChannelDuplexHandler that has newStream() and forEachActiveStream()
methods. It's recommended for a user to extend from this handler, to use those advanced features.
3) As explained in the documentation, a new inbound stream active can be detected by checking if the
Http2Stream2.managedState() of a Http2HeadersFrame is null. An outbound stream active can be detected
by adding a listener to the ChannelPromise of the write of the first Http2HeadersFrame. A stream
closed event can be listened to by adding a listener to the Http2Stream2.closeFuture().
4) Add a simple Http2FrameCodecBuilder and implement the missing frame types.
Result:
1) The Http2MultiplexCodec supports outbound flow control.
2) The Http2FrameCodec API makes it easy for a user to manage custom stream specific state and to create
new outbound streams.
3) The Http2FrameCodec API is much cleaner and easier to work with. Hacks like the ChannelCarryingHeadersFrame
are no longer necessary.
4) The Http2FrameCodec now also supports PING and SETTINGS frames. The Http2FrameCodecBuilder allows the Http2FrameCodec
to use some of the rich features of the Http2ConnectionHandler API.
Motivation:
When we use pipeline.replace and we still had ongoing inbound, then
there will be some problem that inbound message would go to wrong
handlers. So we add handler first, and remove self after add, so that
the next handler will be the correct one.
Modifications:
Uses remove after addAfter instead of replace.
Result:
Fixed#6881
Motivation:
There should be a way to allow graceful shutdown to wait for all open streams to close without a timeout. Using gracefulShutdownTimeoutMillis with a large value is a bit of a hack, and has a gotcha that sufficiently large values will overflow the long, resulting in a ClosingChannelFutureListener that executes immediately.
Modification:
Allow to use gracefulShutdownTimeoutMillis(-1) to express waiting until all streams are closed.
Result:
We can now shutdown the connection without a forced timeout.
Motivation:
codec-http2 currently does not strictly enforce the HTTP/1.x semantics with respect to the number of headers defined in RFC 7540 Section 8.1 [1]. We currently don't validate the number of headers nor do we validate that the trailing headers should indicate EOS.
[1] https://tools.ietf.org/html/rfc7540#section-8.1
Modifications:
- DefaultHttp2ConnectionDecoder should only allow decoding of a single headers and a single trailers
- DefaultHttp2ConnectionEncoder should only allow encoding of a single headers and optionally a single trailers
Result:
Constraints of RFC 7540 restricting the number of headers/trailers is enforced.
Motivation:
DefaultHttp2ConnectionEncoder#writeHeaders attempts to find a stream object, and if one doesn't exist it tries to create one. However in the event that the local endpoint has received a RST_STREAM frame before writing the response headers we attempt to create a stream. Since this stream ID is for the incorrect endpoint we then generate a GO_AWAY for what appears to be a protocol error, but can instead be failed locally.
Modifications:
- Just fail the local promise in the above situation instead of sending a GO_AWAY
Result:
Less severe consequences if the server asynchronously sends headers after a RST_STREAM has been received.
Fixes https://github.com/netty/netty/issues/6906.
Motivation:
PR #6811 introduced a public utility methods to decode hex dump and its parts, but they are not visible from netty-common.
Modifications:
1. Move the `decodeHexByte`, `decodeHexDump` and `decodeHexNibble` methods into `StringUtils`.
2. Apply these methods where applicable.
3. Remove similar methods from other locations (e.g. `HpackHex` test class).
Result:
Less code duplication.
Motivation:
Some JUnit assert calls can be replaced by simpler.
Modifications:
Replacement with a more suitable methods.
Result:
More informative JUnit reports.
Motivation:
DefaultHttp2FrameWriter has constructors that it would be a hassle to
expose as configuration parameters on Http2Codec. We should instead
make a builder for Http2Codec.
Modifications:
Get rid of the public constructors on Http2Codec and instead make sure
you can always use the builder where you would have used the constructor
before.
Result:
Http2Codec can be configured more flexibly, and the SensitivityDetector
can be configured.
Motivation:
Each StreamByteDistributor may allow for priority in different ways, but there are certain characteristics which are invalid regardless of the distribution algorithm. We should validate these invalid characteristics at the flow controller level.
Modifications:
- Disallow negative stream IDs from being used. These streams may be accepted by the WeightedFairQueueByteDistributor and cause state for other valid streams to be evicted.
- Improve unit tests to verify limits are enforced.
Result:
Boundary conditions related to the priority parameters are validated more strictly.
Motivation:
Chrome was randomly getting stuck loading the tiles examples.
Investigation showed that the Netty flow controller thought it had
nothing to send for the connection even though some streams has queued
data and window available.
Modifications:
Fixed an accounting error where an implicitly created parent was not
being added to the dependency tree, thus it and all of its children were
orphaned from the connection's tree and would never have data written.
Result:
Fixes#6621
Motivation:
The CI servers have reported leaks while building the HTTP/2 unit tests. The unit tests attempt to wait for the channels to be closed before exiting the test, but we should wait in case there are any tasks pending on the EventLoopGroup's task queues.
Modifications:
- Change the Future.sync() operations to Future.syncUninterruptibly()
- HTTP/2 unit tests which use local channel should wait for 5 seconds before shutting down the EventLoopGroups
Result:
More likely that any cleanup related tasks will execute before the unit tests are shutdown.
Motivation:
codec-http2 has some helper methods to write to ByteBuf in a big endian fashion. This is the default memory structure for ByteBuf so these helper methods are not necessary.
Modifications:
- remove writeUnsignedInt and writeUnsignedShort
Result:
codec-http2 has less ByteBuf helper methods which are not necessary.
Motivation:
HTTP/2 support two ways to start on a no-tls tcp connection,
http/1.1 upgrade and prior knowlege methodology to start HTTP/2.
Currently, the http2-server from example only support
starting by upgrade. I think we can do a simple dispatch by peek first
bytes from inbound that match to prior knowledge preface or not and
determine which handlers to set into pipeline.
Modifications:
Add ClearTextHttp2ServerUpgradeHandler to support start HTTP/2 via clear
text with two approach. And update example/http2-server to support
this functionality.
Result:
netty HTTP/2 and the example http2-server accept for two ways to start
HTTP/2 over clear text.
Fixed memory leak problem
Update fields to final
Rename ClearText to cleartext
Addressed comments for code improvement
- Always prefer static, final, and private if possible
- Add UnstableApi annotation
- Used EmbeddedChannel.readInbound instead of unhandled inbound handler
- More assertion
Update javadoc for CleartextHttp2ServerUpgradeHandler
Rename ClearTextHttp2ServerUpgradeHandler to CleartextHttp2ServerUpgradeHandler
Removed redundant code about configure pipeline
nit: PriorKnowledgeHandler
Removed Mockito.spy, investigate conn state instead
Add Http2UpgradeEvent
Check null of the constructor arguments
Rename Http2UpgradeEvent to PriorKnowledgeUpgradeEvent
Update unit test
Motivation
Http2StreamChannel ignores options of its parent channel when being created. That leads to surprising results when, for example, unpooled allocator could be silently replaced with pooled allocator (default setting).
Modification
Copy parent channel's options over to the Http2StreamChannel.
Result
Channel options are now consistent between Http2StreamChannel and its parent channel. Newly added test passes on this branch and fails on master. Fixes#6551.
Motivation:
We used some deprecated Mockito methods.
Modifications:
- Replace deprecated method usage
- Some cleanup
Result:
No more usage of deprecated Mockito methods. Fixes [#6482].
Motivation:
DefaultHttp2ConnectionDecoder#onSettingsRead processes the settings, and then sends a SETTINGS ACK to the remote peer. Processing the settings may result in frames which violate the previous settings being send to the remote peer. The remote peer will not apply the new settings until it has received the SETTINGS ACK, and therefore we may violate the settings from the remote peer's perspective and the connection will be shutdown.
Modifications:
- We should send the SETTINGS ACK before we process the settings to ensure the peer receives the SETTINGS ACK before other frames which assume the settings have already been applied
Result:
Fixes https://github.com/netty/netty/issues/6520.
Motivation:
The internal.hpack classes are no longer exposed in our public APIs and can be made package private in the http2 package.
Modifications:
- Make the hpack classes package private in the http2 package
Result:
Less APIs exposed as public.
Motivation:
The timeouts used in the Http2ConnectionRoundtripTest seems to be too low when leak-detection is enabled so we sometimes get failed tests due timeout.
Modifications:
Increase timeouts.
Result:
Fixes [#6442].
Motivation:
DefaultHttp2FrameWriter contains important logic for how http2
frame encode into binary, currently, netty had no unit test
just for DefaultHttp2FrameWriter.
Modifications:
Write DefaultHttp2FrameWriterTest to test DefaultHttp2FrameWriter
Result:
The coverage of DefaultHttp2FrameWriter is increased, and the
class was more reliable
Motivation:
Some unit HTTP/2 unit tests use LocalChannel. LocalChannel's doClose method will ensure any pending items in the queue will be released, but it may execute a Runnable on the peer's EventLoop to ensure the peer's queue is also cleaned up. The HTTP/2 unit tests close the event loop groups with no wait time so that unit tests will execute quickly, but if the doClose Runnable is in the EventLoop's queue it will not run and thus the items in the queue will not be released.
Modifications:
- Ensure all HTTP/2 unit tests which use LocalChannel wait for both client and server channels to be closed before closing the EventLoop.
Result:
Related to https://github.com/netty/netty/issues/5850.
Motivation:
DefaultHttp2FrameReader contains the logic for how it parsed the network
traffic from http2 client,
it also validate the content is legal or not.
So keep high coverage rate on it will increase the stability of api.
Modifications:
Add unit test on DefaultHttp2FrameReader
Result:
Coverage rate increased
Motivation:
We used various mocking frameworks. We should only use one...
Modifications:
Make usage of mocking framework consistent by only using Mockito.
Result:
Less dependencies and more consistent mocking usage.
Motivation:
codec-http2 couples the dependency tree state with the remainder of the stream state (Http2Stream). This makes implementing constraints where stream state and dependency tree state diverge in the RFC challenging. For example the RFC recommends retaining dependency tree state after a stream transitions to closed [1]. Dependency tree state can be exchanged on streams in IDLE. In practice clients may use stream IDs for the purpose of establishing QoS classes and therefore retaining this dependency tree state can be important to client perceived performance. It is difficult to limit the total amount of state we retain when stream state and dependency tree state is combined.
Modifications:
- Remove dependency tree, priority, and weight related items from public facing Http2Connection and Http2Stream APIs. This information is optional to track and depends on the flow controller implementation.
- Move all dependency tree, priority, and weight related code from DefaultHttp2Connection to WeightedFairQueueByteDistributor. This is currently the only place which cares about priority. We can pull out the dependency tree related code in the future if it is generally useful to expose for other implementations.
- DefaultHttp2Connection should explicitly limit the number of reserved streams now that IDLE streams are no longer created.
Result:
More compliant with the HTTP/2 RFC.
Fixes https://github.com/netty/netty/issues/6206.
[1] https://tools.ietf.org/html/rfc7540#section-5.3.4
Motivation:
If an HTTP/2 client writes data before the connection preface the peer will shutdown the socket. Depending on what is in the pipeline (SslHandler) may require different evaluation criteria to infer when the codec-http2 has written the connection preface on behalf of the client. This can lead to unnecessarily complexity and error prone/racy application code.
Modifications:
- Introduce a user event that is fired up the pipeline when codec-http2 writes the connection preface
Result:
Reliable mechanism for applications to use to know when connection preface has been written (related to https://github.com/netty/netty/issues/6272).