Finish work on http2 child channel implementation and http2 frame api.
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.
This commit is contained in:
parent
3a2b462a67
commit
74f24a5c19
@ -1,488 +0,0 @@
|
||||
/*
|
||||
* Copyright 2016 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid;
|
||||
import static io.netty.util.internal.ObjectUtil.checkNotNull;
|
||||
import static java.lang.Math.max;
|
||||
import static java.lang.Math.min;
|
||||
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.channel.AbstractChannel;
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelConfig;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelFutureListener;
|
||||
import io.netty.channel.ChannelMetadata;
|
||||
import io.netty.channel.ChannelOutboundBuffer;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
import io.netty.channel.DefaultChannelConfig;
|
||||
import io.netty.channel.EventLoop;
|
||||
import io.netty.channel.MessageSizeEstimator;
|
||||
import io.netty.channel.RecvByteBufAllocator;
|
||||
import io.netty.channel.WriteBufferWaterMark;
|
||||
import io.netty.util.ReferenceCountUtil;
|
||||
import io.netty.util.internal.ThrowableUtil;
|
||||
|
||||
import java.net.SocketAddress;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
|
||||
|
||||
/**
|
||||
* Child {@link Channel} of another channel, for use for modeling streams as channels.
|
||||
*/
|
||||
abstract class AbstractHttp2StreamChannel extends AbstractChannel {
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
private static final AtomicLongFieldUpdater<AbstractHttp2StreamChannel> OUTBOUND_FLOW_CONTROL_WINDOW_UPDATER;
|
||||
|
||||
/**
|
||||
* Used by subclasses to queue a close channel within the read queue. When read, it will close
|
||||
* the channel (using Unsafe) instead of notifying handlers of the message with {@code
|
||||
* channelRead()}. Additional inbound messages must not arrive after this one.
|
||||
*/
|
||||
protected static final Object CLOSE_MESSAGE = new Object();
|
||||
/**
|
||||
* Used to add a message to the {@link ChannelOutboundBuffer}, so as to have it re-evaluate its writability state.
|
||||
*/
|
||||
private static final Object REEVALUATE_WRITABILITY_MESSAGE = new Object();
|
||||
private static final ChannelMetadata METADATA = new ChannelMetadata(false, 16);
|
||||
private static final ClosedChannelException CLOSED_CHANNEL_EXCEPTION = ThrowableUtil.unknownStackTrace(
|
||||
new ClosedChannelException(), AbstractHttp2StreamChannel.class, "doWrite(...)");
|
||||
/**
|
||||
* Number of bytes to consider non-payload messages, to determine when to stop reading. 9 is
|
||||
* arbitrary, but also the minimum size of an HTTP/2 frame. Primarily is non-zero.
|
||||
*/
|
||||
private static final int ARBITRARY_MESSAGE_SIZE = 9;
|
||||
|
||||
private final Http2StreamChannelConfig config = new Http2StreamChannelConfig(this);
|
||||
private final Queue<Object> inboundBuffer = new ArrayDeque<Object>(4);
|
||||
private final Runnable fireChildReadCompleteTask = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
if (readInProgress) {
|
||||
readInProgress = false;
|
||||
unsafe().recvBufAllocHandle().readComplete();
|
||||
pipeline().fireChannelReadComplete();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
private final Http2Stream2 stream;
|
||||
private boolean closed;
|
||||
private boolean readInProgress;
|
||||
|
||||
/**
|
||||
* The flow control window of the remote side i.e. the number of bytes this channel is allowed to send to the remote
|
||||
* peer. The window can become negative if a channel handler ignores the channel's writability. We are using a long
|
||||
* so that we realistically don't have to worry about underflow.
|
||||
*/
|
||||
@SuppressWarnings("UnusedDeclaration")
|
||||
private volatile long outboundFlowControlWindow;
|
||||
|
||||
static {
|
||||
@SuppressWarnings("rawtypes")
|
||||
AtomicLongFieldUpdater<AbstractHttp2StreamChannel> updater = AtomicLongFieldUpdater.newUpdater(
|
||||
AbstractHttp2StreamChannel.class, "outboundFlowControlWindow");
|
||||
if (updater == null) {
|
||||
updater = AtomicLongFieldUpdater.newUpdater(AbstractHttp2StreamChannel.class, "outboundFlowControlWindow");
|
||||
}
|
||||
OUTBOUND_FLOW_CONTROL_WINDOW_UPDATER = updater;
|
||||
}
|
||||
|
||||
protected AbstractHttp2StreamChannel(Channel parent, Http2Stream2 stream) {
|
||||
super(parent);
|
||||
this.stream = stream;
|
||||
}
|
||||
|
||||
protected Http2Stream2 stream() {
|
||||
return stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelMetadata metadata() {
|
||||
return METADATA;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelConfig config() {
|
||||
return config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isOpen() {
|
||||
return !closed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isActive() {
|
||||
return isOpen();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isWritable() {
|
||||
return isStreamIdValid(stream.id())
|
||||
// So that the channel doesn't become active before the initial flow control window has been set.
|
||||
&& outboundFlowControlWindow > 0
|
||||
// Could be null if channel closed.
|
||||
&& unsafe().outboundBuffer() != null
|
||||
&& unsafe().outboundBuffer().isWritable();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AbstractUnsafe newUnsafe() {
|
||||
return new Unsafe();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean isCompatible(EventLoop loop) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SocketAddress localAddress0() {
|
||||
return parent().localAddress();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SocketAddress remoteAddress0() {
|
||||
return parent().remoteAddress();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doBind(SocketAddress localAddress) throws Exception {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doDisconnect() throws Exception {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doClose() throws Exception {
|
||||
closed = true;
|
||||
while (!inboundBuffer.isEmpty()) {
|
||||
ReferenceCountUtil.release(inboundBuffer.poll());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doBeginRead() {
|
||||
if (readInProgress) {
|
||||
return;
|
||||
}
|
||||
|
||||
final RecvByteBufAllocator.Handle allocHandle = unsafe().recvBufAllocHandle();
|
||||
allocHandle.reset(config());
|
||||
if (inboundBuffer.isEmpty()) {
|
||||
readInProgress = true;
|
||||
return;
|
||||
}
|
||||
|
||||
do {
|
||||
Object m = inboundBuffer.poll();
|
||||
if (m == null) {
|
||||
break;
|
||||
}
|
||||
if (!doRead0(m, allocHandle)) {
|
||||
// Channel closed, and already cleaned up.
|
||||
return;
|
||||
}
|
||||
} while (allocHandle.continueReading());
|
||||
|
||||
allocHandle.readComplete();
|
||||
pipeline().fireChannelReadComplete();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final void doWrite(ChannelOutboundBuffer in) throws Exception {
|
||||
if (closed) {
|
||||
throw CLOSED_CHANNEL_EXCEPTION;
|
||||
}
|
||||
final MessageSizeEstimator.Handle sizeEstimator = config().getMessageSizeEstimator().newHandle();
|
||||
for (;;) {
|
||||
final Object msg = in.current();
|
||||
if (msg == null) {
|
||||
break;
|
||||
}
|
||||
// TODO(buchgr): Detecting cancellation relies on ChannelOutboundBuffer internals. NOT COOL!
|
||||
if (msg == Unpooled.EMPTY_BUFFER /* The write was cancelled. */
|
||||
|| msg == REEVALUATE_WRITABILITY_MESSAGE /* Write to trigger writability after window update. */) {
|
||||
in.remove();
|
||||
continue;
|
||||
}
|
||||
final int bytes = sizeEstimator.size(msg);
|
||||
/**
|
||||
* The flow control window needs to be decrement before stealing the message from the buffer (and thereby
|
||||
* decrementing the number of pending bytes). Else, when calling steal() the number of pending bytes could
|
||||
* be less than the writebuffer watermark (=flow control window) and thus trigger a writability change.
|
||||
*
|
||||
* This code must never trigger a writability change. Only reading window updates or channel writes may
|
||||
* change the channel's writability.
|
||||
*/
|
||||
incrementOutboundFlowControlWindow(-bytes);
|
||||
final ChannelPromise promise = in.steal();
|
||||
if (bytes > 0) {
|
||||
promise.addListener(new ReturnFlowControlWindowOnFailureListener(bytes));
|
||||
}
|
||||
// TODO(buchgr): Should we also the change the writability if END_STREAM is set?
|
||||
try {
|
||||
doWrite(msg, promise);
|
||||
} catch (Throwable t) {
|
||||
promise.tryFailure(t);
|
||||
}
|
||||
}
|
||||
doWriteComplete();
|
||||
}
|
||||
|
||||
/**
|
||||
* Process a single write. Guaranteed to eventually be followed by a {@link #doWriteComplete()},
|
||||
* which denotes the end of the batch of writes. May be called from any thread.
|
||||
*/
|
||||
protected abstract void doWrite(Object msg, ChannelPromise promise) throws Exception;
|
||||
|
||||
/**
|
||||
* Process end of batch of {@link #doWrite(ChannelOutboundBuffer)}s. May be called from any thread.
|
||||
*/
|
||||
protected abstract void doWriteComplete();
|
||||
|
||||
/**
|
||||
* {@code bytes}-count of bytes provided to {@link #fireChildRead} have been read. May be called
|
||||
* from any thread. Must not throw an exception.
|
||||
*/
|
||||
protected abstract void bytesConsumed(int bytes);
|
||||
|
||||
/**
|
||||
* Receive a read message. This does not notify handlers unless a read is in progress on the
|
||||
* channel. May be called from any thread.
|
||||
*/
|
||||
protected void fireChildRead(final Object msg) {
|
||||
if (eventLoop().inEventLoop()) {
|
||||
fireChildRead0(msg);
|
||||
} else {
|
||||
eventLoop().execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
fireChildRead0(msg);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
private void fireChildRead0(Object msg) {
|
||||
if (closed) {
|
||||
ReferenceCountUtil.release(msg);
|
||||
return;
|
||||
}
|
||||
if (readInProgress) {
|
||||
assert inboundBuffer.isEmpty();
|
||||
// Check for null because inboundBuffer doesn't support null; we want to be consistent
|
||||
// for what values are supported.
|
||||
RecvByteBufAllocator.Handle allocHandle = unsafe().recvBufAllocHandle();
|
||||
readInProgress = doRead0(checkNotNull(msg, "msg"), allocHandle);
|
||||
if (!allocHandle.continueReading()) {
|
||||
fireChildReadCompleteTask.run();
|
||||
}
|
||||
} else {
|
||||
inboundBuffer.add(msg);
|
||||
}
|
||||
}
|
||||
|
||||
protected void fireChildReadComplete() {
|
||||
if (eventLoop().inEventLoop()) {
|
||||
fireChildReadCompleteTask.run();
|
||||
} else {
|
||||
eventLoop().execute(fireChildReadCompleteTask);
|
||||
}
|
||||
}
|
||||
|
||||
protected void incrementOutboundFlowControlWindow(int bytes) {
|
||||
if (bytes == 0) {
|
||||
return;
|
||||
}
|
||||
OUTBOUND_FLOW_CONTROL_WINDOW_UPDATER.addAndGet(this, bytes);
|
||||
}
|
||||
|
||||
// Visible for testing
|
||||
long getOutboundFlowControlWindow() {
|
||||
return outboundFlowControlWindow;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether reads should continue. The only reason reads shouldn't continue is that the
|
||||
* channel was just closed.
|
||||
*/
|
||||
private boolean doRead0(Object msg, RecvByteBufAllocator.Handle allocHandle) {
|
||||
if (msg == CLOSE_MESSAGE) {
|
||||
allocHandle.readComplete();
|
||||
pipeline().fireChannelReadComplete();
|
||||
close();
|
||||
return false;
|
||||
}
|
||||
if (msg instanceof Http2WindowUpdateFrame) {
|
||||
Http2WindowUpdateFrame windowUpdate = (Http2WindowUpdateFrame) msg;
|
||||
incrementOutboundFlowControlWindow(windowUpdate.windowSizeIncrement());
|
||||
reevaluateWritability();
|
||||
return true;
|
||||
}
|
||||
int numBytesToBeConsumed = 0;
|
||||
if (msg instanceof Http2DataFrame) {
|
||||
numBytesToBeConsumed = dataFrameFlowControlBytes((Http2DataFrame) msg);
|
||||
allocHandle.lastBytesRead(numBytesToBeConsumed);
|
||||
} else {
|
||||
allocHandle.lastBytesRead(ARBITRARY_MESSAGE_SIZE);
|
||||
}
|
||||
allocHandle.incMessagesRead(1);
|
||||
pipeline().fireChannelRead(msg);
|
||||
if (numBytesToBeConsumed != 0) {
|
||||
bytesConsumed(numBytesToBeConsumed);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private void reevaluateWritability() {
|
||||
ChannelOutboundBuffer buffer = unsafe().outboundBuffer();
|
||||
// If the buffer is not writable but should be writable, then write and flush a dummy object
|
||||
// to trigger a writability change.
|
||||
if (!buffer.isWritable() && buffer.totalPendingWriteBytes() < config.getWriteBufferHighWaterMark()) {
|
||||
unsafe().outboundBuffer().addMessage(REEVALUATE_WRITABILITY_MESSAGE, 1, voidPromise());
|
||||
unsafe().flush();
|
||||
}
|
||||
}
|
||||
|
||||
private static int dataFrameFlowControlBytes(Http2DataFrame frame) {
|
||||
return frame.content().readableBytes()
|
||||
+ frame.padding()
|
||||
// +1 to account for the pad length field. See http://httpwg.org/specs/rfc7540.html#DATA
|
||||
+ (frame.padding() & 1);
|
||||
}
|
||||
|
||||
private final class Unsafe extends AbstractUnsafe {
|
||||
@Override
|
||||
public void connect(final SocketAddress remoteAddress,
|
||||
SocketAddress localAddress, final ChannelPromise promise) {
|
||||
promise.setFailure(new UnsupportedOperationException());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the flow-control size for DATA frames, and 0 for all other frames.
|
||||
*/
|
||||
private static final class FlowControlledFrameSizeEstimator implements MessageSizeEstimator {
|
||||
|
||||
private static final FlowControlledFrameSizeEstimator INSTANCE = new FlowControlledFrameSizeEstimator();
|
||||
|
||||
private static final class EstimatorHandle implements MessageSizeEstimator.Handle {
|
||||
|
||||
private static final EstimatorHandle INSTANCE = new EstimatorHandle();
|
||||
|
||||
@Override
|
||||
public int size(Object msg) {
|
||||
if (msg instanceof Http2DataFrame) {
|
||||
return dataFrameFlowControlBytes((Http2DataFrame) msg);
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Handle newHandle() {
|
||||
return EstimatorHandle.INSTANCE;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link ChannelConfig} so that the high and low writebuffer watermarks can reflect the outbound flow control
|
||||
* window, without having to create a new {@link WriteBufferWaterMark} object whenever the flow control window
|
||||
* changes.
|
||||
*/
|
||||
private final class Http2StreamChannelConfig extends DefaultChannelConfig {
|
||||
|
||||
// TODO(buchgr): Overwrite the RecvByteBufAllocator. We only need it to implement max messages per read.
|
||||
Http2StreamChannelConfig(Channel channel) {
|
||||
super(channel);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public int getWriteBufferHighWaterMark() {
|
||||
int window = (int) min(Integer.MAX_VALUE, outboundFlowControlWindow);
|
||||
return max(0, window);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public int getWriteBufferLowWaterMark() {
|
||||
return getWriteBufferHighWaterMark();
|
||||
}
|
||||
|
||||
@Override
|
||||
public MessageSizeEstimator getMessageSizeEstimator() {
|
||||
return FlowControlledFrameSizeEstimator.INSTANCE;
|
||||
}
|
||||
|
||||
// TODO(buchgr): Throwing exceptions is not ideal. Maybe NO-OP and log a warning?
|
||||
@Override
|
||||
public WriteBufferWaterMark getWriteBufferWaterMark() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelConfig setMessageSizeEstimator(MessageSizeEstimator estimator) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public ChannelConfig setWriteBufferHighWaterMark(int writeBufferHighWaterMark) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public ChannelConfig setWriteBufferLowWaterMark(int writeBufferLowWaterMark) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public ChannelConfig setWriteBufferWaterMark(WriteBufferWaterMark writeBufferWaterMark) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
private class ReturnFlowControlWindowOnFailureListener implements ChannelFutureListener {
|
||||
private final int bytes;
|
||||
|
||||
ReturnFlowControlWindowOnFailureListener(int bytes) {
|
||||
this.bytes = bytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
if (!future.isSuccess()) {
|
||||
/**
|
||||
* Return the flow control window of the failed data frame. We expect this code to be rarely executed
|
||||
* and by implementing it as a window update, we don't have to worry about thread-safety.
|
||||
*/
|
||||
fireChildRead(new DefaultHttp2WindowUpdateFrame(bytes).stream(stream));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -23,16 +23,16 @@ import io.netty.util.internal.UnstableApi;
|
||||
@UnstableApi
|
||||
public abstract class AbstractHttp2StreamFrame implements Http2StreamFrame {
|
||||
|
||||
private volatile Http2Stream2 stream;
|
||||
private Http2FrameStream stream;
|
||||
|
||||
@Override
|
||||
public AbstractHttp2StreamFrame stream(Http2Stream2 stream) {
|
||||
public AbstractHttp2StreamFrame stream(Http2FrameStream stream) {
|
||||
this.stream = stream;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream2 stream() {
|
||||
public Http2FrameStream stream() {
|
||||
return stream;
|
||||
}
|
||||
|
||||
@ -50,6 +50,10 @@ public abstract class AbstractHttp2StreamFrame implements Http2StreamFrame {
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
Http2FrameStream stream = this.stream;
|
||||
if (stream == null) {
|
||||
return super.hashCode();
|
||||
}
|
||||
return stream.hashCode();
|
||||
}
|
||||
}
|
||||
|
@ -18,6 +18,7 @@ package io.netty.handler.codec.http2;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.util.IllegalReferenceCountException;
|
||||
import io.netty.util.internal.StringUtil;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.verifyPadding;
|
||||
@ -31,6 +32,7 @@ public final class DefaultHttp2DataFrame extends AbstractHttp2StreamFrame implem
|
||||
private final ByteBuf content;
|
||||
private final boolean endStream;
|
||||
private final int padding;
|
||||
private final int initialFlowControlledBytes;
|
||||
|
||||
/**
|
||||
* Equivalent to {@code new DefaultHttp2DataFrame(content, false)}.
|
||||
@ -73,10 +75,14 @@ public final class DefaultHttp2DataFrame extends AbstractHttp2StreamFrame implem
|
||||
this.endStream = endStream;
|
||||
verifyPadding(padding);
|
||||
this.padding = padding;
|
||||
if (content().readableBytes() + (long) padding > Integer.MAX_VALUE) {
|
||||
throw new IllegalArgumentException("content + padding must be <= Integer.MAX_VALUE");
|
||||
}
|
||||
initialFlowControlledBytes = content().readableBytes() + padding;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2DataFrame stream(Http2Stream2 stream) {
|
||||
public DefaultHttp2DataFrame stream(Http2FrameStream stream) {
|
||||
super.stream(stream);
|
||||
return this;
|
||||
}
|
||||
@ -87,7 +93,7 @@ public final class DefaultHttp2DataFrame extends AbstractHttp2StreamFrame implem
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean endStream() {
|
||||
public boolean isEndStream() {
|
||||
return endStream;
|
||||
}
|
||||
|
||||
@ -104,6 +110,11 @@ public final class DefaultHttp2DataFrame extends AbstractHttp2StreamFrame implem
|
||||
return content;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int initialFlowControlledBytes() {
|
||||
return initialFlowControlledBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2DataFrame copy() {
|
||||
return replace(content().copy());
|
||||
@ -153,8 +164,8 @@ public final class DefaultHttp2DataFrame extends AbstractHttp2StreamFrame implem
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DefaultHttp2DataFrame(stream=" + stream() + ", content=" + content
|
||||
+ ", endStream=" + endStream + ", padding=" + padding + ")";
|
||||
return StringUtil.simpleClassName(this) + "(stream=" + stream() + ", content=" + content
|
||||
+ ", endStream=" + endStream + ", padding=" + padding + ')';
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -18,6 +18,7 @@ package io.netty.handler.codec.http2;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.DefaultByteBufHolder;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.util.internal.StringUtil;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
/**
|
||||
@ -174,7 +175,7 @@ public final class DefaultHttp2GoAwayFrame extends DefaultByteBufHolder implemen
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DefaultHttp2GoAwayFrame(errorCode=" + errorCode + ", content=" + content()
|
||||
+ ", extraStreamIds=" + extraStreamIds + ", lastStreamId=" + lastStreamId + ")";
|
||||
return StringUtil.simpleClassName(this) + "(errorCode=" + errorCode + ", content=" + content()
|
||||
+ ", extraStreamIds=" + extraStreamIds + ", lastStreamId=" + lastStreamId + ')';
|
||||
}
|
||||
}
|
||||
|
@ -15,11 +15,11 @@
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.util.internal.StringUtil;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.verifyPadding;
|
||||
import static io.netty.util.internal.ObjectUtil.checkNotNull;
|
||||
import static io.netty.util.internal.ObjectUtil.checkPositive;
|
||||
|
||||
/**
|
||||
* The default {@link Http2HeadersFrame} implementation.
|
||||
@ -64,7 +64,7 @@ public final class DefaultHttp2HeadersFrame extends AbstractHttp2StreamFrame imp
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2HeadersFrame stream(Http2Stream2 stream) {
|
||||
public DefaultHttp2HeadersFrame stream(Http2FrameStream stream) {
|
||||
super.stream(stream);
|
||||
return this;
|
||||
}
|
||||
@ -79,7 +79,8 @@ public final class DefaultHttp2HeadersFrame extends AbstractHttp2StreamFrame imp
|
||||
return headers;
|
||||
}
|
||||
|
||||
public boolean endStream() {
|
||||
@Override
|
||||
public boolean isEndStream() {
|
||||
return endStream;
|
||||
}
|
||||
|
||||
@ -90,8 +91,8 @@ public final class DefaultHttp2HeadersFrame extends AbstractHttp2StreamFrame imp
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DefaultHttp2HeadersFrame(stream=" + stream() + ", headers=" + headers
|
||||
+ ", endStream=" + endStream + ", padding=" + padding + ")";
|
||||
return StringUtil.simpleClassName(this) + "(stream=" + stream() + ", headers=" + headers
|
||||
+ ", endStream=" + endStream + ", padding=" + padding + ')';
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -18,7 +18,7 @@ package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.DefaultByteBufHolder;
|
||||
import io.netty.util.internal.ObjectUtil;
|
||||
import io.netty.util.internal.StringUtil;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
/**
|
||||
@ -53,17 +53,17 @@ public class DefaultHttp2PingFrame extends DefaultByteBufHolder implements Http2
|
||||
|
||||
@Override
|
||||
public DefaultHttp2PingFrame copy() {
|
||||
return new DefaultHttp2PingFrame(content().copy(), ack);
|
||||
return replace(content().copy());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2PingFrame duplicate() {
|
||||
return (DefaultHttp2PingFrame) super.duplicate();
|
||||
return replace(content().duplicate());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2PingFrame retainedDuplicate() {
|
||||
return (DefaultHttp2PingFrame) super.retainedDuplicate();
|
||||
return replace(content().retainedDuplicate());
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -112,7 +112,6 @@ public class DefaultHttp2PingFrame extends DefaultByteBufHolder implements Http2
|
||||
}
|
||||
|
||||
private static ByteBuf mustBeEightBytes(ByteBuf content) {
|
||||
ObjectUtil.checkNotNull(content, "content must not be null.");
|
||||
if (content.readableBytes() != 8) {
|
||||
throw new IllegalArgumentException("PING frames require 8 bytes of content. Was " +
|
||||
content.readableBytes() + " bytes.");
|
||||
@ -122,6 +121,6 @@ public class DefaultHttp2PingFrame extends DefaultByteBufHolder implements Http2
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DefaultHttp2PingFrame(content=" + contentToString() + ", ack=" + ack + ')';
|
||||
return StringUtil.simpleClassName(this) + "(content=" + contentToString() + ", ack=" + ack + ')';
|
||||
}
|
||||
}
|
||||
|
@ -15,10 +15,10 @@
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.util.internal.StringUtil;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
import static io.netty.util.internal.ObjectUtil.checkNotNull;
|
||||
import static io.netty.util.internal.ObjectUtil.checkPositive;
|
||||
|
||||
/**
|
||||
* The default {@link Http2ResetFrame} implementation.
|
||||
@ -34,7 +34,7 @@ public final class DefaultHttp2ResetFrame extends AbstractHttp2StreamFrame imple
|
||||
* @param error the non-{@code null} reason for reset
|
||||
*/
|
||||
public DefaultHttp2ResetFrame(Http2Error error) {
|
||||
this.errorCode = checkNotNull(error, "error").code();
|
||||
errorCode = checkNotNull(error, "error").code();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -47,7 +47,7 @@ public final class DefaultHttp2ResetFrame extends AbstractHttp2StreamFrame imple
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2ResetFrame stream(Http2Stream2 stream) {
|
||||
public DefaultHttp2ResetFrame stream(Http2FrameStream stream) {
|
||||
super.stream(stream);
|
||||
return this;
|
||||
}
|
||||
@ -64,7 +64,7 @@ public final class DefaultHttp2ResetFrame extends AbstractHttp2StreamFrame imple
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DefaultHttp2ResetFrame(stream=" + stream() + ", errorCode=" + errorCode + ')';
|
||||
return StringUtil.simpleClassName(this) + "(stream=" + stream() + ", errorCode=" + errorCode + ')';
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -16,15 +16,20 @@
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.util.internal.ObjectUtil;
|
||||
import io.netty.util.internal.StringUtil;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
/**
|
||||
* The default {@link Http2SettingsFrame} implementation.
|
||||
*/
|
||||
@UnstableApi
|
||||
public class DefaultHttp2SettingsFrame implements Http2SettingsFrame {
|
||||
|
||||
private final Http2Settings settings;
|
||||
|
||||
public DefaultHttp2SettingsFrame(Http2Settings settings) {
|
||||
this.settings = settings;
|
||||
this.settings = ObjectUtil.checkNotNull(settings, "settings");
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -39,6 +44,6 @@ public class DefaultHttp2SettingsFrame implements Http2SettingsFrame {
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DefaultHttp2SettingsFrame(settings=" + settings + ')';
|
||||
return StringUtil.simpleClassName(this) + "(settings=" + settings + ')';
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,138 @@
|
||||
/*
|
||||
* Copyright 2017 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.DefaultByteBufHolder;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.util.internal.StringUtil;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
@UnstableApi
|
||||
public final class DefaultHttp2UnknownFrame extends DefaultByteBufHolder implements Http2UnknownFrame {
|
||||
private final byte frameType;
|
||||
private final Http2Flags flags;
|
||||
private Http2FrameStream stream;
|
||||
|
||||
public DefaultHttp2UnknownFrame(byte frameType, Http2Flags flags) {
|
||||
this(frameType, flags, Unpooled.EMPTY_BUFFER);
|
||||
}
|
||||
|
||||
public DefaultHttp2UnknownFrame(byte frameType, Http2Flags flags, ByteBuf data) {
|
||||
super(data);
|
||||
this.frameType = frameType;
|
||||
this.flags = flags;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameStream stream() {
|
||||
return stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2UnknownFrame stream(Http2FrameStream stream) {
|
||||
this.stream = stream;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte frameType() {
|
||||
return frameType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Flags flags() {
|
||||
return flags;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return "UNKNOWN";
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2UnknownFrame copy() {
|
||||
return replace(content().copy());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2UnknownFrame duplicate() {
|
||||
return replace(content().duplicate());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2UnknownFrame retainedDuplicate() {
|
||||
return replace(content().retainedDuplicate());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2UnknownFrame replace(ByteBuf content) {
|
||||
return new DefaultHttp2UnknownFrame(frameType, flags, content).stream(stream());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2UnknownFrame retain() {
|
||||
super.retain();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2UnknownFrame retain(int increment) {
|
||||
super.retain(increment);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return StringUtil.simpleClassName(this) + "(frameType=" + frameType() + ", stream=" + stream() +
|
||||
", flags=" + flags() + ", content=" + contentToString() + ')';
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2UnknownFrame touch() {
|
||||
super.touch();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2UnknownFrame touch(Object hint) {
|
||||
super.touch(hint);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (!(o instanceof DefaultHttp2UnknownFrame)) {
|
||||
return false;
|
||||
}
|
||||
DefaultHttp2UnknownFrame other = (DefaultHttp2UnknownFrame) o;
|
||||
return super.equals(other) && flags().equals(other.flags())
|
||||
&& frameType() == other.frameType() && (stream() == null && other.stream() == null) ||
|
||||
stream().equals(other.stream());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int hash = super.hashCode();
|
||||
hash = hash * 31 + frameType();
|
||||
hash = hash * 31 + flags().hashCode();
|
||||
if (stream() != null) {
|
||||
hash = hash * 31 + stream().hashCode();
|
||||
}
|
||||
|
||||
return hash;
|
||||
}
|
||||
}
|
@ -17,8 +17,6 @@ package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
import static io.netty.util.internal.ObjectUtil.checkPositive;
|
||||
|
||||
/**
|
||||
* The default {@link Http2WindowUpdateFrame} implementation.
|
||||
*/
|
||||
@ -28,11 +26,11 @@ public class DefaultHttp2WindowUpdateFrame extends AbstractHttp2StreamFrame impl
|
||||
private final int windowUpdateIncrement;
|
||||
|
||||
public DefaultHttp2WindowUpdateFrame(int windowUpdateIncrement) {
|
||||
this.windowUpdateIncrement = checkPositive(windowUpdateIncrement, "windowUpdateIncrement");
|
||||
this.windowUpdateIncrement = windowUpdateIncrement;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2WindowUpdateFrame stream(Http2Stream2 stream) {
|
||||
public DefaultHttp2WindowUpdateFrame stream(Http2FrameStream stream) {
|
||||
super.stream(stream);
|
||||
return this;
|
||||
}
|
||||
|
@ -19,38 +19,59 @@ package io.netty.handler.codec.http2;
|
||||
import io.netty.channel.ChannelDuplexHandler;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPipeline;
|
||||
import io.netty.util.internal.StringUtil;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
/**
|
||||
* A {@link ChannelDuplexHandler} providing additional functionality for HTTP/2. Specifically it allows to:
|
||||
* <ul>
|
||||
* <li>Create new outbound streams using {@link #newStream()}.</li>
|
||||
* <li>Iterate over all active streams using {@link #forEachActiveStream(Http2Stream2Visitor)}.</li>
|
||||
* <li>Iterate over all active streams using {@link #forEachActiveStream(Http2FrameStreamVisitor)}.</li>
|
||||
* </ul>
|
||||
*
|
||||
* <p>The {@link Http2FrameCodec} is required to be part of the {@link ChannelPipeline} before this handler is added,
|
||||
* or else an {@link IllegalStateException} will be thrown.
|
||||
*/
|
||||
public class Http2ChannelDuplexHandler extends ChannelDuplexHandler {
|
||||
@UnstableApi
|
||||
public abstract class Http2ChannelDuplexHandler extends ChannelDuplexHandler {
|
||||
|
||||
private Http2FrameCodec frameCodec;
|
||||
private volatile Http2FrameCodec frameCodec;
|
||||
|
||||
@Override
|
||||
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
public final void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
frameCodec = requireHttp2FrameCodec(ctx);
|
||||
handlerAdded0(ctx);
|
||||
}
|
||||
|
||||
protected void handlerAdded0(@SuppressWarnings("unused") ChannelHandlerContext ctx) throws Exception {
|
||||
// NOOP
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
|
||||
frameCodec = null;
|
||||
public final void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
|
||||
try {
|
||||
handlerRemoved0(ctx);
|
||||
} finally {
|
||||
frameCodec = null;
|
||||
}
|
||||
}
|
||||
|
||||
protected void handlerRemoved0(@SuppressWarnings("unused") ChannelHandlerContext ctx) throws Exception {
|
||||
// NOOP
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link Http2Stream2} object.
|
||||
* Creates a new {@link Http2FrameStream} object.
|
||||
*
|
||||
* <p>This method is <em>thread-safe</em>.
|
||||
*/
|
||||
public final Http2Stream2 newStream() {
|
||||
return newStream0();
|
||||
public final Http2FrameStream newStream() {
|
||||
Http2FrameCodec codec = frameCodec;
|
||||
if (codec == null) {
|
||||
throw new IllegalStateException(StringUtil.simpleClassName(Http2FrameCodec.class) + " not found." +
|
||||
" Has the handler been added to a pipeline?");
|
||||
}
|
||||
return codec.newStream();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -58,23 +79,10 @@ public class Http2ChannelDuplexHandler extends ChannelDuplexHandler {
|
||||
*
|
||||
* <p>This method may only be called from the eventloop thread.
|
||||
*/
|
||||
protected final void forEachActiveStream(Http2Stream2Visitor streamVisitor) throws Http2Exception {
|
||||
forEachActiveStream0(streamVisitor);
|
||||
}
|
||||
|
||||
// So that it can be overwritten by tests, without being visible to the public.
|
||||
void forEachActiveStream0(Http2Stream2Visitor streamVisitor) throws Http2Exception {
|
||||
protected final void forEachActiveStream(Http2FrameStreamVisitor streamVisitor) throws Http2Exception {
|
||||
frameCodec.forEachActiveStream(streamVisitor);
|
||||
}
|
||||
|
||||
// So that it can be overwritten by tests, without being visible to the public.
|
||||
Http2Stream2 newStream0() {
|
||||
if (frameCodec == null) {
|
||||
throw new IllegalStateException("Frame codec not found. Has the handler been added to a pipeline?");
|
||||
}
|
||||
return frameCodec.newStream();
|
||||
}
|
||||
|
||||
private static Http2FrameCodec requireHttp2FrameCodec(ChannelHandlerContext ctx) {
|
||||
ChannelHandlerContext frameCodecCtx = ctx.pipeline().context(Http2FrameCodec.class);
|
||||
if (frameCodecCtx == null) {
|
||||
|
@ -53,7 +53,7 @@ public class Http2ClientUpgradeCodec implements HttpClientUpgradeHandler.Upgrade
|
||||
}
|
||||
|
||||
public Http2ClientUpgradeCodec(String handlerName, Http2FrameCodec frameCodec, ChannelHandler upgradeToHandler) {
|
||||
this(handlerName, frameCodec.connectionHandler(), upgradeToHandler);
|
||||
this(handlerName, (Http2ConnectionHandler) frameCodec, upgradeToHandler);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1,52 +0,0 @@
|
||||
/*
|
||||
* Copyright 2016 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.channel.ChannelDuplexHandler;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
/**
|
||||
* An HTTP/2 channel handler that adds a {@link Http2FrameCodec} and {@link Http2MultiplexCodec} to the pipeline before
|
||||
* removing itself.
|
||||
*/
|
||||
@UnstableApi
|
||||
public final class Http2Codec extends ChannelDuplexHandler {
|
||||
private final Http2FrameCodec frameCodec;
|
||||
private final Http2MultiplexCodec multiplexCodec;
|
||||
|
||||
Http2Codec(boolean server, Http2StreamChannelBootstrap bootstrap, Http2FrameWriter frameWriter,
|
||||
Http2FrameLogger frameLogger, Http2Settings initialSettings) {
|
||||
Http2FrameCodecBuilder frameBuilder = server
|
||||
? Http2FrameCodecBuilder.forServer()
|
||||
: Http2FrameCodecBuilder.forClient();
|
||||
frameBuilder.frameWriter(frameWriter).frameLogger(frameLogger).initialSettings(initialSettings);
|
||||
frameCodec = frameBuilder.build();
|
||||
multiplexCodec = new Http2MultiplexCodec(server, bootstrap);
|
||||
}
|
||||
|
||||
Http2FrameCodec frameCodec() {
|
||||
return frameCodec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
ctx.pipeline().addBefore(ctx.executor(), ctx.name(), null, frameCodec);
|
||||
ctx.pipeline().addBefore(ctx.executor(), ctx.name(), null, multiplexCodec);
|
||||
|
||||
ctx.pipeline().remove(this);
|
||||
}
|
||||
}
|
@ -1,130 +0,0 @@
|
||||
/*
|
||||
* Copyright 2017 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.channel.ChannelHandler;
|
||||
import io.netty.handler.codec.http2.Http2HeadersEncoder.SensitivityDetector;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
import static io.netty.handler.logging.LogLevel.INFO;
|
||||
import static io.netty.util.internal.ObjectUtil.checkNotNull;
|
||||
|
||||
/**
|
||||
* A builder for {@link Http2Codec}.
|
||||
*/
|
||||
@UnstableApi
|
||||
public final class Http2CodecBuilder {
|
||||
private static final Http2FrameLogger HTTP2_FRAME_LOGGER = new Http2FrameLogger(INFO, Http2Codec.class);
|
||||
|
||||
private final Http2StreamChannelBootstrap bootstrap;
|
||||
private final boolean server;
|
||||
private Http2Settings initialSettings;
|
||||
private Http2FrameLogger frameLogger;
|
||||
private SensitivityDetector headersSensitivityDetector;
|
||||
|
||||
/**
|
||||
* Creates a new {@link Http2Codec} builder.
|
||||
*
|
||||
* @param server {@code true} this is a server
|
||||
* @param streamHandler the handler added to channels for remotely-created streams. It must be
|
||||
* {@link ChannelHandler.Sharable}. {@code null} if the event loop from the parent channel should be used.
|
||||
*/
|
||||
public Http2CodecBuilder(boolean server, ChannelHandler streamHandler) {
|
||||
this(server, new Http2StreamChannelBootstrap().handler(streamHandler));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link Http2Codec} builder.
|
||||
*
|
||||
* @param server {@code true} this is a server
|
||||
* @param bootstrap bootstrap used to instantiate child channels for remotely-created streams.
|
||||
*/
|
||||
public Http2CodecBuilder(boolean server, Http2StreamChannelBootstrap bootstrap) {
|
||||
this.bootstrap = checkNotNull(bootstrap, "bootstrap");
|
||||
this.server = server;
|
||||
this.initialSettings = Http2Settings.defaultSettings();
|
||||
this.frameLogger = HTTP2_FRAME_LOGGER;
|
||||
this.headersSensitivityDetector = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Specifies the initial settings to send to peer.
|
||||
*
|
||||
* @param initialSettings non default initial settings to send to peer
|
||||
* @return {@link Http2CodecBuilder} the builder for the {@link Http2Codec}
|
||||
*/
|
||||
public Http2CodecBuilder initialSettings(Http2Settings initialSettings) {
|
||||
this.initialSettings = initialSettings;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the initial settings to send to peer.
|
||||
*/
|
||||
public Http2Settings initialSettings() {
|
||||
return initialSettings;
|
||||
}
|
||||
|
||||
/**
|
||||
* Specifies the frame logger to log messages with.
|
||||
*
|
||||
* @param frameLogger handler used to log all frames
|
||||
* @return {@link Http2CodecBuilder} the builder for the {@link Http2Codec}
|
||||
*/
|
||||
public Http2CodecBuilder frameLogger(Http2FrameLogger frameLogger) {
|
||||
this.frameLogger = frameLogger;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the frame logger to log messages with.
|
||||
*/
|
||||
public Http2FrameLogger frameLogger() {
|
||||
return frameLogger;
|
||||
}
|
||||
|
||||
/**
|
||||
* Specifies the headers sensitivity detector.
|
||||
*
|
||||
* @param headersSensitivityDetector decides whether headers should be considered sensitive or not
|
||||
* @return {@link Http2CodecBuilder} the builder for the {@link Http2Codec}
|
||||
*/
|
||||
public Http2CodecBuilder headersSensitivityDetector(SensitivityDetector headersSensitivityDetector) {
|
||||
this.headersSensitivityDetector = headersSensitivityDetector;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the headers sensitivity detector.
|
||||
*/
|
||||
public SensitivityDetector headersSensitivityDetector() {
|
||||
return headersSensitivityDetector;
|
||||
}
|
||||
|
||||
private Http2FrameWriter frameWriter() {
|
||||
return headersSensitivityDetector() == null ?
|
||||
new DefaultHttp2FrameWriter() :
|
||||
new DefaultHttp2FrameWriter(headersSensitivityDetector());
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds/creates a new {@link Http2Codec} instance using this builder's current settings.
|
||||
*/
|
||||
public Http2Codec build() {
|
||||
return new Http2Codec(server, bootstrap,
|
||||
frameWriter(), frameLogger(), initialSettings());
|
||||
}
|
||||
}
|
@ -90,6 +90,25 @@ public class Http2ConnectionHandler extends ByteToMessageDecoder implements Http
|
||||
}
|
||||
}
|
||||
|
||||
Http2ConnectionHandler(boolean server, Http2FrameWriter frameWriter, Http2FrameLogger frameLogger,
|
||||
Http2Settings initialSettings) {
|
||||
this.initialSettings = checkNotNull(initialSettings, "initialSettings");
|
||||
|
||||
Http2Connection connection = new DefaultHttp2Connection(server);
|
||||
|
||||
Long maxHeaderListSize = initialSettings.maxHeaderListSize();
|
||||
Http2FrameReader frameReader = new DefaultHttp2FrameReader(maxHeaderListSize == null ?
|
||||
new DefaultHttp2HeadersDecoder(true) :
|
||||
new DefaultHttp2HeadersDecoder(true, maxHeaderListSize));
|
||||
|
||||
if (frameLogger != null) {
|
||||
frameWriter = new Http2OutboundFrameLogger(frameWriter, frameLogger);
|
||||
frameReader = new Http2InboundFrameLogger(frameReader, frameLogger);
|
||||
}
|
||||
encoder = new DefaultHttp2ConnectionEncoder(connection, frameWriter);
|
||||
decoder = new DefaultHttp2ConnectionDecoder(connection, encoder, frameReader);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the amount of time (in milliseconds) this endpoint will wait for all streams to be closed before closing
|
||||
* the connection during the graceful shutdown process. Returns -1 if this connection is configured to wait
|
||||
@ -165,7 +184,7 @@ public class Http2ConnectionHandler extends ByteToMessageDecoder implements Http
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush(ChannelHandlerContext ctx) throws Http2Exception {
|
||||
public void flush(ChannelHandlerContext ctx) {
|
||||
try {
|
||||
// Trigger pending writes in the remote flow controller.
|
||||
encoder.flowController().writePendingBytes();
|
||||
@ -484,13 +503,17 @@ public class Http2ConnectionHandler extends ByteToMessageDecoder implements Http
|
||||
// Trigger flush after read on the assumption that flush is cheap if there is nothing to write and that
|
||||
// for flow-control the read may release window that causes data to be written that can now be flushed.
|
||||
try {
|
||||
// First call channelReadComplete(...) as this may produce more data that we want to flush
|
||||
super.channelReadComplete(ctx);
|
||||
// First call channelReadComplete0(...) as this may produce more data that we want to flush
|
||||
channelReadComplete0(ctx);
|
||||
} finally {
|
||||
flush(ctx);
|
||||
}
|
||||
}
|
||||
|
||||
void channelReadComplete0(ChannelHandlerContext ctx) throws Exception {
|
||||
super.channelReadComplete(ctx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Handles {@link Http2Exception} objects that were thrown from other handlers. Ignores all other exceptions.
|
||||
*/
|
||||
|
@ -36,10 +36,16 @@ public interface Http2DataFrame extends Http2StreamFrame, ByteBufHolder {
|
||||
@Override
|
||||
ByteBuf content();
|
||||
|
||||
/**
|
||||
* Returns the number of bytes that are flow-controlled initialy, so even if the {@link #content()} is consumed
|
||||
* this will not change.
|
||||
*/
|
||||
int initialFlowControlledBytes();
|
||||
|
||||
/**
|
||||
* Returns {@code true} if the END_STREAM flag ist set.
|
||||
*/
|
||||
boolean endStream();
|
||||
boolean isEndStream();
|
||||
|
||||
@Override
|
||||
Http2DataFrame copy();
|
||||
|
@ -16,15 +16,11 @@
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelDuplexHandler;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelFutureListener;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelInboundHandler;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
import io.netty.channel.ChannelPromiseNotifier;
|
||||
import io.netty.channel.DefaultChannelPromise;
|
||||
import io.netty.handler.codec.http2.Http2Connection.PropertyKey;
|
||||
import io.netty.handler.codec.http2.Http2Stream.State;
|
||||
import io.netty.handler.codec.http2.StreamBufferingEncoder.Http2ChannelClosedException;
|
||||
@ -38,8 +34,8 @@ import io.netty.util.internal.UnstableApi;
|
||||
import io.netty.util.internal.logging.InternalLogger;
|
||||
import io.netty.util.internal.logging.InternalLoggerFactory;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.isOutboundStream;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.HTTP_UPGRADE_STREAM_ID;
|
||||
|
||||
/**
|
||||
* <p><em>This API is very immature.</em> The Http2Connection-based API is currently preferred over this API.
|
||||
@ -57,20 +53,15 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid;
|
||||
*
|
||||
* The frame codec delivers and writes frames for active streams. An active stream is closed when either side sends a
|
||||
* {@code RST_STREAM} frame or both sides send a frame with the {@code END_STREAM} flag set. Each
|
||||
* {@link Http2StreamFrame} has a {@link Http2Stream2} object attached that uniquely identifies a particular stream.
|
||||
* {@link Http2StreamFrame} has a {@link Http2FrameStream} object attached that uniquely identifies a particular stream.
|
||||
*
|
||||
* <p>Application specific state can be maintained by attaching a custom object to a stream via
|
||||
* {@link Http2Stream2#managedState(Object)}. As the name suggests, the state object is cleaned up automatically when a
|
||||
* stream or the channel is closed.
|
||||
*
|
||||
* <p>{@link Http2StreamFrame}s read from the channel always a {@link Http2Stream2} object set, while when writing a
|
||||
* {@link Http2StreamFrame} the application code needs to set a {@link Http2Stream2} object using
|
||||
* {@link Http2StreamFrame#stream(Http2Stream2)}.
|
||||
* <p>{@link Http2StreamFrame}s read from the channel always a {@link Http2FrameStream} object set, while when writing a
|
||||
* {@link Http2StreamFrame} the application code needs to set a {@link Http2FrameStream} object using
|
||||
* {@link Http2StreamFrame#stream(Http2FrameStream)}.
|
||||
*
|
||||
* <h3>Flow control</h3>
|
||||
*
|
||||
* The frame codec automatically increments stream and connection flow control windows. It's possible to customize
|
||||
* when flow control windows are updated via {@link Http2FrameCodecBuilder#windowUpdateRatio(float)}.
|
||||
* The frame codec automatically increments stream and connection flow control windows.
|
||||
*
|
||||
* <p>Incoming flow controlled frames need to be consumed by writing a {@link Http2WindowUpdateFrame} with the consumed
|
||||
* number of bytes and the corresponding stream identifier set to the frame codec.
|
||||
@ -84,29 +75,12 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid;
|
||||
*
|
||||
* <h3>New inbound Streams</h3>
|
||||
*
|
||||
* The first frame of a HTTP/2 stream must be a {@link Http2HeadersFrame}, which will have a {@link Http2Stream2} object
|
||||
* attached. An application can detect if it's a new stream by inspecting the {@link Http2Stream2#managedState()} for
|
||||
* {@code null}, and if so attach application specific state via {@link Http2Stream2#managedState(Object)}.
|
||||
*
|
||||
* <pre>
|
||||
* public class MyChannelHandler extends Http2ChannelDuplexHandler {
|
||||
*
|
||||
* @Override
|
||||
* public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
|
||||
* if (msg instanceof Http2HeadersFrame) {
|
||||
* Http2HeadersFrame headersFrame = (Http2HeadersFrame) msg;
|
||||
* if (msg.stream().managedState() == null) {
|
||||
* // A new inbound stream.
|
||||
* msg.stream().managedState(new ApplicationState());
|
||||
* }
|
||||
* }
|
||||
* }
|
||||
* }
|
||||
* </pre>
|
||||
* The first frame of a HTTP/2 stream must be a {@link Http2HeadersFrame}, which will have a {@link Http2FrameStream}
|
||||
* object attached.
|
||||
*
|
||||
* <h3>New outbound Streams</h3>
|
||||
*
|
||||
* A outbound HTTP/2 stream can be created by first instantiating a new {@link Http2Stream2} object via
|
||||
* A outbound HTTP/2 stream can be created by first instantiating a new {@link Http2FrameStream} object via
|
||||
* {@link Http2ChannelDuplexHandler#newStream()}, and then writing a {@link Http2HeadersFrame} object with the stream
|
||||
* attached.
|
||||
*
|
||||
@ -138,9 +112,10 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid;
|
||||
*
|
||||
* <p>The HTTP/2 standard allows for an endpoint to limit the maximum number of concurrently active streams via the
|
||||
* {@code SETTINGS_MAX_CONCURRENT_STREAMS} setting. When this limit is reached, no new streams can be created. However,
|
||||
* the {@link Http2FrameCodec} can be build with {@link Http2FrameCodecBuilder#bufferOutboundStreams} enabled, in which
|
||||
* case a new stream and its associated frames will be buffered until either the limit is increased or an active
|
||||
* stream is closed. It's, however, possible that a buffered stream will never become active. That is, the channel might
|
||||
* the {@link Http2FrameCodec} can be build with
|
||||
* {@link Http2FrameCodecBuilder#encoderEnforceMaxConcurrentStreams(boolean)} enabled, in which case a new stream and
|
||||
* its associated frames will be buffered until either the limit is increased or an active stream is closed. It's,
|
||||
* however, possible that a buffered stream will never become active. That is, the channel might
|
||||
* get closed or a GO_AWAY frame might be received. In the first case, all writes of buffered streams will fail with a
|
||||
* {@link Http2ChannelClosedException}. In the second case, all writes of buffered streams with an identifier less than
|
||||
* the last stream identifier of the GO_AWAY frame will fail with a {@link Http2GoAwayException}.
|
||||
@ -148,8 +123,8 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid;
|
||||
* <h3>Error Handling</h3>
|
||||
*
|
||||
* Exceptions and errors are propagated via {@link ChannelInboundHandler#exceptionCaught}. Exceptions that apply to
|
||||
* a specific HTTP/2 stream are wrapped in a {@link Http2Stream2Exception} and have the corresponding
|
||||
* {@link Http2Stream2} object attached.
|
||||
* a specific HTTP/2 stream are wrapped in a {@link Http2FrameStreamException} and have the corresponding
|
||||
* {@link Http2FrameStream} object attached.
|
||||
*
|
||||
* <h3>Reference Counting</h3>
|
||||
*
|
||||
@ -165,85 +140,35 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid;
|
||||
* HTTP-to-HTTP/2 conversion is performed automatically.
|
||||
*/
|
||||
@UnstableApi
|
||||
public class Http2FrameCodec extends ChannelDuplexHandler {
|
||||
public class Http2FrameCodec extends Http2ConnectionHandler {
|
||||
|
||||
private static final InternalLogger LOG = InternalLoggerFactory.getInstance(Http2FrameCodec.class);
|
||||
|
||||
private final Http2ConnectionHandler http2Handler;
|
||||
private final boolean server;
|
||||
private final PropertyKey streamKey;
|
||||
|
||||
// Used to adjust flow control window on channel active. Set to null afterwards.
|
||||
private Integer initialLocalConnectionWindow;
|
||||
private final Integer initialFlowControlWindowSize;
|
||||
|
||||
private ChannelHandlerContext ctx;
|
||||
private ChannelHandlerContext http2HandlerCtx;
|
||||
|
||||
private Http2Stream2Impl pendingOutboundStreamsTail;
|
||||
|
||||
/** Lock protecting modifications to idle outbound streams. **/
|
||||
private final Object lock = new Object();
|
||||
|
||||
/** Number of buffered streams if the {@link StreamBufferingEncoder} is used. **/
|
||||
private int numBufferedStreams;
|
||||
private DefaultHttp2FrameStream frameStreamToInitialize;
|
||||
|
||||
Http2FrameCodec(Http2ConnectionEncoder encoder, Http2ConnectionDecoder decoder, Http2Settings initialSettings) {
|
||||
super(decoder, encoder, initialSettings);
|
||||
|
||||
/**
|
||||
* Create a new handler. Use {@link Http2FrameCodecBuilder}.
|
||||
*/
|
||||
Http2FrameCodec(Http2ConnectionEncoder encoder, Http2ConnectionDecoder decoder, Http2Settings initialSettings,
|
||||
long gracefulShutdownTimeoutMillis) {
|
||||
decoder.frameListener(new FrameListener());
|
||||
http2Handler = new InternalHttp2ConnectionHandler(decoder, encoder, initialSettings);
|
||||
http2Handler.connection().addListener(new ConnectionListener());
|
||||
http2Handler.gracefulShutdownTimeoutMillis(gracefulShutdownTimeoutMillis);
|
||||
server = http2Handler.connection().isServer();
|
||||
connection().addListener(new ConnectionListener());
|
||||
connection().remote().flowController().listener(new Http2RemoteFlowControllerListener());
|
||||
streamKey = connection().newKey();
|
||||
initialLocalConnectionWindow = initialSettings.initialWindowSize();
|
||||
}
|
||||
|
||||
Http2ConnectionHandler connectionHandler() {
|
||||
return http2Handler;
|
||||
initialFlowControlWindowSize = initialSettings.initialWindowSize();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new outbound/local stream.
|
||||
*
|
||||
* <p>This method may only be called after the handler has been added to a {@link io.netty.channel.ChannelPipeline}.
|
||||
*
|
||||
* <p>This method is thread-safe.
|
||||
*/
|
||||
public Http2FrameCodec(boolean server, Http2FrameLogger frameLogger) {
|
||||
this(server, new DefaultHttp2FrameWriter(), frameLogger, Http2Settings.defaultSettings());
|
||||
}
|
||||
|
||||
// Visible for testing
|
||||
Http2FrameCodec(boolean server, Http2FrameWriter frameWriter, Http2FrameLogger frameLogger,
|
||||
Http2Settings initialSettings) {
|
||||
// TODO(scott): configure maxReservedStreams when API is more finalized.
|
||||
Http2Connection connection = new DefaultHttp2Connection(server);
|
||||
frameWriter = new Http2OutboundFrameLogger(frameWriter, frameLogger);
|
||||
Http2ConnectionEncoder encoder = new DefaultHttp2ConnectionEncoder(connection, frameWriter);
|
||||
Long maxHeaderListSize = initialSettings.maxHeaderListSize();
|
||||
Http2FrameReader frameReader = new DefaultHttp2FrameReader(maxHeaderListSize == null ?
|
||||
new DefaultHttp2HeadersDecoder(true) :
|
||||
new DefaultHttp2HeadersDecoder(true, maxHeaderListSize));
|
||||
Http2FrameReader reader = new Http2InboundFrameLogger(frameReader, frameLogger);
|
||||
Http2ConnectionDecoder decoder = new DefaultHttp2ConnectionDecoder(connection, encoder, reader);
|
||||
decoder.frameListener(new FrameListener());
|
||||
http2Handler = new InternalHttp2ConnectionHandler(decoder, encoder, initialSettings);
|
||||
http2Handler.connection().addListener(new ConnectionListener());
|
||||
streamKey = connection().newKey();
|
||||
this.server = server;
|
||||
}
|
||||
|
||||
// TODO(buchgr): Discuss: Should this method be thread safe?
|
||||
Http2Stream2 newStream() {
|
||||
ChannelHandlerContext ctx0 = ctx;
|
||||
if (ctx0 == null) {
|
||||
throw new IllegalStateException("Channel handler not added to a channel pipeline.");
|
||||
}
|
||||
|
||||
return new Http2Stream2Impl(ctx0.channel());
|
||||
DefaultHttp2FrameStream newStream() {
|
||||
return new DefaultHttp2FrameStream();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -251,75 +176,53 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
|
||||
*
|
||||
* <p>This method must not be called outside of the event loop.
|
||||
*/
|
||||
void forEachActiveStream(final Http2Stream2Visitor streamVisitor) throws Http2Exception {
|
||||
assert ctx.channel().eventLoop().inEventLoop();
|
||||
final void forEachActiveStream(final Http2FrameStreamVisitor streamVisitor) throws Http2Exception {
|
||||
assert ctx.executor().inEventLoop();
|
||||
|
||||
connection().forEachActiveStream(new Http2StreamVisitor() {
|
||||
@Override
|
||||
public boolean visit(Http2Stream stream) {
|
||||
Http2Stream2 stream2 = stream.getProperty(streamKey);
|
||||
if (stream2 == null) {
|
||||
/**
|
||||
* This code is expected to almost never execute. However, in rare cases it's possible that a
|
||||
* stream is active without a {@link Http2Stream2} object attached, as it's set in a listener of
|
||||
* the HEADERS frame write.
|
||||
*/
|
||||
// TODO(buchgr): Remove once Http2Stream2 and Http2Stream are merged.
|
||||
return true;
|
||||
}
|
||||
try {
|
||||
return streamVisitor.visit(stream2);
|
||||
return streamVisitor.visit((Http2FrameStream) stream.getProperty(streamKey));
|
||||
} catch (Throwable cause) {
|
||||
connectionHandler().onError(http2HandlerCtx, cause);
|
||||
onError(ctx, cause);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Load any dependencies.
|
||||
*/
|
||||
@Override
|
||||
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
public final void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
this.ctx = ctx;
|
||||
ctx.pipeline().addBefore(ctx.executor(), ctx.name(), null, http2Handler);
|
||||
http2HandlerCtx = ctx.pipeline().context(http2Handler);
|
||||
sendInitialConnectionWindow();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelActive(ChannelHandlerContext ctx) throws Exception {
|
||||
sendInitialConnectionWindow();
|
||||
super.channelActive(ctx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean up any dependencies.
|
||||
*/
|
||||
@Override
|
||||
public void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
|
||||
ctx.pipeline().remove(http2Handler);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
|
||||
super.channelInactive(ctx);
|
||||
}
|
||||
|
||||
private void sendInitialConnectionWindow() throws Http2Exception {
|
||||
if (ctx.channel().isActive() && initialLocalConnectionWindow != null) {
|
||||
Http2Stream connectionStream = http2Handler.connection().connectionStream();
|
||||
int currentSize = connection().local().flowController().windowSize(connectionStream);
|
||||
int delta = initialLocalConnectionWindow - currentSize;
|
||||
http2Handler.decoder().flowController().incrementWindowSize(connectionStream, delta);
|
||||
initialLocalConnectionWindow = null;
|
||||
ctx.flush();
|
||||
super.handlerAdded(ctx);
|
||||
handlerAdded0(ctx);
|
||||
// Must be after Http2ConnectionHandler does its initialization in handlerAdded above.
|
||||
// The server will not send a connection preface so we are good to send a window update.
|
||||
Http2Connection connection = connection();
|
||||
if (connection.isServer()) {
|
||||
tryExpandConnectionFlowControlWindow(connection);
|
||||
}
|
||||
}
|
||||
|
||||
private Http2Connection connection() {
|
||||
return http2Handler.connection();
|
||||
private void tryExpandConnectionFlowControlWindow(Http2Connection connection) throws Http2Exception {
|
||||
if (initialFlowControlWindowSize != null) {
|
||||
// The window size in the settings explicitly excludes the connection window. So we manually manipulate the
|
||||
// connection window to accommodate more concurrent data per connection.
|
||||
Http2Stream connectionStream = connection.connectionStream();
|
||||
Http2LocalFlowController localFlowController = connection.local().flowController();
|
||||
final int delta = initialFlowControlWindowSize - localFlowController.initialWindowSize(connectionStream);
|
||||
// Only increase the connection window, don't decrease it.
|
||||
if (delta > 0) {
|
||||
// Double the delta just so a single stream can't exhaust the connection window.
|
||||
localFlowController.incrementWindowSize(connectionStream, Math.max(delta << 1, delta));
|
||||
flush(ctx);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void handlerAdded0(@SuppressWarnings("unsed") ChannelHandlerContext ctx) throws Exception {
|
||||
// sub-class can override this for extra steps that needs to be done when the handler is added.
|
||||
}
|
||||
|
||||
/**
|
||||
@ -327,27 +230,31 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
|
||||
* HTTP/2 on stream 1 (the stream specifically reserved for cleartext HTTP upgrade).
|
||||
*/
|
||||
@Override
|
||||
public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
|
||||
if (!(evt instanceof UpgradeEvent)) {
|
||||
super.userEventTriggered(ctx, evt);
|
||||
public final void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
|
||||
if (evt instanceof Http2ConnectionPrefaceWrittenEvent) {
|
||||
// The user event implies that we are on the client.
|
||||
tryExpandConnectionFlowControlWindow(connection());
|
||||
} else if (evt instanceof UpgradeEvent) {
|
||||
UpgradeEvent upgrade = (UpgradeEvent) evt;
|
||||
try {
|
||||
onUpgradeEvent(ctx, upgrade.retain());
|
||||
Http2Stream stream = connection().stream(HTTP_UPGRADE_STREAM_ID);
|
||||
if (stream.getProperty(streamKey) == null) {
|
||||
// TODO: improve handler/stream lifecycle so that stream isn't active before handler added.
|
||||
// The stream was already made active, but ctx may have been null so it wasn't initialized.
|
||||
// https://github.com/netty/netty/issues/4942
|
||||
onStreamActive0(stream);
|
||||
}
|
||||
upgrade.upgradeRequest().headers().setInt(
|
||||
HttpConversionUtil.ExtensionHeaderNames.STREAM_ID.text(), HTTP_UPGRADE_STREAM_ID);
|
||||
InboundHttpToHttp2Adapter.handle(
|
||||
ctx, connection(), decoder().frameListener(), upgrade.upgradeRequest());
|
||||
} finally {
|
||||
upgrade.release();
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
UpgradeEvent upgrade = (UpgradeEvent) evt;
|
||||
ctx.fireUserEventTriggered(upgrade.retain());
|
||||
try {
|
||||
Http2Stream stream = http2Handler.connection().stream(Http2CodecUtil.HTTP_UPGRADE_STREAM_ID);
|
||||
// TODO: improve handler/stream lifecycle so that stream isn't active before handler added.
|
||||
// The stream was already made active, but ctx may have been null so it wasn't initialized.
|
||||
// https://github.com/netty/netty/issues/4942
|
||||
new ConnectionListener().onStreamActive(stream);
|
||||
upgrade.upgradeRequest().headers().setInt(
|
||||
HttpConversionUtil.ExtensionHeaderNames.STREAM_ID.text(), Http2CodecUtil.HTTP_UPGRADE_STREAM_ID);
|
||||
new InboundHttpToHttp2Adapter(http2Handler.connection(), http2Handler.decoder().frameListener())
|
||||
.channelRead(ctx, upgrade.upgradeRequest().retain());
|
||||
} finally {
|
||||
upgrade.release();
|
||||
}
|
||||
super.userEventTriggered(ctx, evt);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -356,34 +263,33 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
|
||||
*/
|
||||
@Override
|
||||
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) {
|
||||
if (!(msg instanceof Http2Frame)) {
|
||||
if (msg instanceof Http2DataFrame) {
|
||||
Http2DataFrame dataFrame = (Http2DataFrame) msg;
|
||||
encoder().writeData(ctx, dataFrame.stream().id(), dataFrame.content(),
|
||||
dataFrame.padding(), dataFrame.isEndStream(), promise);
|
||||
} else if (msg instanceof Http2HeadersFrame) {
|
||||
writeHeadersFrame(ctx, (Http2HeadersFrame) msg, promise);
|
||||
} else if (msg instanceof Http2WindowUpdateFrame) {
|
||||
Http2WindowUpdateFrame frame = (Http2WindowUpdateFrame) msg;
|
||||
writeWindowUpdate(frame.stream().id(), frame.windowSizeIncrement(), promise);
|
||||
} else if (msg instanceof Http2ResetFrame) {
|
||||
Http2ResetFrame rstFrame = (Http2ResetFrame) msg;
|
||||
encoder().writeRstStream(ctx, rstFrame.stream().id(), rstFrame.errorCode(), promise);
|
||||
} else if (msg instanceof Http2PingFrame) {
|
||||
Http2PingFrame frame = (Http2PingFrame) msg;
|
||||
encoder().writePing(ctx, frame.ack(), frame.content(), promise);
|
||||
} else if (msg instanceof Http2SettingsFrame) {
|
||||
encoder().writeSettings(ctx, ((Http2SettingsFrame) msg).settings(), promise);
|
||||
} else if (msg instanceof Http2GoAwayFrame) {
|
||||
writeGoAwayFrame(ctx, (Http2GoAwayFrame) msg, promise);
|
||||
} else if (!(msg instanceof Http2Frame)) {
|
||||
ctx.write(msg, promise);
|
||||
return;
|
||||
}
|
||||
try {
|
||||
if (msg instanceof Http2WindowUpdateFrame) {
|
||||
Http2WindowUpdateFrame frame = (Http2WindowUpdateFrame) msg;
|
||||
writeWindowUpdate(frame.stream().id(), frame.windowSizeIncrement(), promise);
|
||||
} else if (msg instanceof Http2StreamFrame) {
|
||||
writeStreamFrame((Http2StreamFrame) msg, promise);
|
||||
} else if (msg instanceof Http2PingFrame) {
|
||||
writePingFrame((Http2PingFrame) msg, promise);
|
||||
} else if (msg instanceof Http2SettingsFrame) {
|
||||
writeSettingsFrame((Http2SettingsFrame) msg, promise);
|
||||
} else if (msg instanceof Http2GoAwayFrame) {
|
||||
writeGoAwayFrame((Http2GoAwayFrame) msg, promise);
|
||||
} else {
|
||||
throw new UnsupportedMessageTypeException(msg);
|
||||
}
|
||||
} finally {
|
||||
} else {
|
||||
ReferenceCountUtil.release(msg);
|
||||
throw new UnsupportedMessageTypeException(msg);
|
||||
}
|
||||
}
|
||||
|
||||
private void writePingFrame(Http2PingFrame pingFrame, ChannelPromise promise) {
|
||||
http2Handler.encoder().writePing(http2HandlerCtx, pingFrame.ack(), pingFrame.content().retain(), promise);
|
||||
}
|
||||
|
||||
private void writeWindowUpdate(int streamId, int bytes, ChannelPromise promise) {
|
||||
try {
|
||||
if (streamId == 0) {
|
||||
@ -404,186 +310,192 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
|
||||
localFlow.initialWindowSize(targetConnectionWindow);
|
||||
}
|
||||
|
||||
private void consumeBytes(int streamId, int bytes) throws Http2Exception {
|
||||
Http2Stream stream = http2Handler.connection().stream(streamId);
|
||||
http2Handler.connection().local().flowController()
|
||||
.consumeBytes(stream, bytes);
|
||||
final void consumeBytes(int streamId, int bytes) throws Http2Exception {
|
||||
Http2Stream stream = connection().stream(streamId);
|
||||
connection().local().flowController().consumeBytes(stream, bytes);
|
||||
}
|
||||
|
||||
private void writeSettingsFrame(Http2SettingsFrame frame, ChannelPromise promise) {
|
||||
http2Handler.encoder().writeSettings(http2HandlerCtx, frame.settings(), promise);
|
||||
}
|
||||
|
||||
private void writeGoAwayFrame(Http2GoAwayFrame frame, ChannelPromise promise) {
|
||||
private void writeGoAwayFrame(ChannelHandlerContext ctx, Http2GoAwayFrame frame, ChannelPromise promise) {
|
||||
if (frame.lastStreamId() > -1) {
|
||||
frame.release();
|
||||
throw new IllegalArgumentException("Last stream id must not be set on GOAWAY frame");
|
||||
}
|
||||
|
||||
int lastStreamCreated = http2Handler.connection().remote().lastStreamCreated();
|
||||
int lastStreamId = lastStreamCreated + frame.extraStreamIds() * 2;
|
||||
int lastStreamCreated = connection().remote().lastStreamCreated();
|
||||
long lastStreamId = lastStreamCreated + ((long) frame.extraStreamIds()) * 2;
|
||||
// Check if the computation overflowed.
|
||||
if (lastStreamId < lastStreamCreated) {
|
||||
if (lastStreamId > Integer.MAX_VALUE) {
|
||||
lastStreamId = Integer.MAX_VALUE;
|
||||
}
|
||||
http2Handler.goAway(
|
||||
http2HandlerCtx, lastStreamId, frame.errorCode(), frame.content().retain(), promise);
|
||||
goAway(ctx, (int) lastStreamId, frame.errorCode(), frame.content(), promise);
|
||||
}
|
||||
|
||||
private void writeStreamFrame(Http2StreamFrame frame, ChannelPromise promise) {
|
||||
if (!(frame.stream() instanceof Http2Stream2Impl)) {
|
||||
throw new IllegalArgumentException("A stream object created by the frame codec needs to be set. " + frame);
|
||||
}
|
||||
private void writeHeadersFrame(
|
||||
final ChannelHandlerContext ctx, Http2HeadersFrame headersFrame, final ChannelPromise promise) {
|
||||
|
||||
if (frame instanceof Http2DataFrame) {
|
||||
Http2DataFrame dataFrame = (Http2DataFrame) frame;
|
||||
http2Handler.encoder().writeData(http2HandlerCtx, frame.stream().id(), dataFrame.content().retain(),
|
||||
dataFrame.padding(), dataFrame.endStream(), promise);
|
||||
} else if (frame instanceof Http2HeadersFrame) {
|
||||
writeHeadersFrame((Http2HeadersFrame) frame, promise);
|
||||
} else if (frame instanceof Http2ResetFrame) {
|
||||
Http2ResetFrame rstFrame = (Http2ResetFrame) frame;
|
||||
http2Handler.encoder().writeRstStream(http2HandlerCtx, frame.stream().id(), rstFrame.errorCode(), promise);
|
||||
} else {
|
||||
throw new UnsupportedMessageTypeException(frame);
|
||||
}
|
||||
}
|
||||
|
||||
private void writeHeadersFrame(Http2HeadersFrame headersFrame, ChannelPromise promise) {
|
||||
final int streamId;
|
||||
if (isStreamIdValid(headersFrame.stream().id())) {
|
||||
streamId = headersFrame.stream().id();
|
||||
encoder().writeHeaders(ctx, headersFrame.stream().id(), headersFrame.headers(), headersFrame.padding(),
|
||||
headersFrame.isEndStream(), promise);
|
||||
} else {
|
||||
final Http2Stream2Impl stream = (Http2Stream2Impl) headersFrame.stream();
|
||||
final Http2Connection connection = http2Handler.connection();
|
||||
streamId = connection.local().incrementAndGetNextStreamId();
|
||||
final DefaultHttp2FrameStream stream = (DefaultHttp2FrameStream) headersFrame.stream();
|
||||
final Http2Connection connection = connection();
|
||||
final int streamId = connection.local().incrementAndGetNextStreamId();
|
||||
if (streamId < 0) {
|
||||
promise.setFailure(new Http2NoMoreStreamIdsException());
|
||||
return;
|
||||
}
|
||||
numBufferedStreams++;
|
||||
// Set the stream id before completing the promise, as any listener added by a user will be executed
|
||||
// before the below listener, and so the stream identifier is accessible in a user's listener.
|
||||
stream.id(streamId);
|
||||
// Ensure that the listener gets executed before any listeners a user might have attached.
|
||||
// TODO(buchgr): Once Http2Stream2 and Http2Stream are merged this is no longer necessary.
|
||||
ChannelPromiseNotifier promiseNotifier = new ChannelPromiseNotifier(promise);
|
||||
promise = ctx.newPromise();
|
||||
promise.addListener(new ChannelFutureListener() {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
numBufferedStreams--;
|
||||
stream.id = streamId;
|
||||
|
||||
Http2Stream connectionStream = connection.stream(streamId);
|
||||
if (future.isSuccess() && connectionStream != null) {
|
||||
connectionStream.setProperty(streamKey, stream);
|
||||
stream.legacyStream = connectionStream;
|
||||
} else {
|
||||
stream.setClosed();
|
||||
// TODO: This depends on the fact that the connection based API will create Http2Stream objects
|
||||
// synchronously. We should investigate how to refactor this later on when we consolidate some layers.
|
||||
assert frameStreamToInitialize == null;
|
||||
frameStreamToInitialize = stream;
|
||||
|
||||
// TODO(buchgr): Once Http2Stream2 and Http2Stream are merged this is no longer necessary.
|
||||
final ChannelPromise writePromise = ctx.newPromise();
|
||||
|
||||
encoder().writeHeaders(ctx, streamId, headersFrame.headers(), headersFrame.padding(),
|
||||
headersFrame.isEndStream(), writePromise);
|
||||
if (writePromise.isDone()) {
|
||||
notifyHeaderWritePromise(writePromise, promise);
|
||||
} else {
|
||||
numBufferedStreams++;
|
||||
|
||||
writePromise.addListener(new ChannelFutureListener() {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
numBufferedStreams--;
|
||||
|
||||
notifyHeaderWritePromise(future, promise);
|
||||
}
|
||||
}
|
||||
});
|
||||
promise.addListener(promiseNotifier);
|
||||
});
|
||||
}
|
||||
}
|
||||
http2Handler.encoder().writeHeaders(http2HandlerCtx, streamId, headersFrame.headers(), headersFrame.padding(),
|
||||
headersFrame.endStream(), promise);
|
||||
}
|
||||
|
||||
private static void notifyHeaderWritePromise(ChannelFuture future, ChannelPromise promise) {
|
||||
Throwable cause = future.cause();
|
||||
if (cause == null) {
|
||||
promise.setSuccess();
|
||||
} else {
|
||||
promise.setFailure(cause);
|
||||
}
|
||||
}
|
||||
|
||||
private void onStreamActive0(Http2Stream stream) {
|
||||
if (connection().local().isValidStreamId(stream.id())) {
|
||||
return;
|
||||
}
|
||||
|
||||
DefaultHttp2FrameStream stream2 = newStream().setStreamAndProperty(streamKey, stream);
|
||||
onHttp2StreamStateChanged(ctx, stream2);
|
||||
}
|
||||
|
||||
private final class ConnectionListener extends Http2ConnectionAdapter {
|
||||
|
||||
@Override
|
||||
public void onStreamActive(Http2Stream stream) {
|
||||
if (isOutboundStream(server, stream.id())) {
|
||||
return;
|
||||
}
|
||||
public void onStreamAdded(Http2Stream stream) {
|
||||
if (frameStreamToInitialize != null && stream.id() == frameStreamToInitialize.id()) {
|
||||
frameStreamToInitialize.setStreamAndProperty(streamKey, stream);
|
||||
frameStreamToInitialize = null;
|
||||
}
|
||||
}
|
||||
|
||||
Http2Stream2Impl stream2 = new Http2Stream2Impl(ctx.channel()).id(stream.id());
|
||||
stream2.legacyStream = stream;
|
||||
stream.setProperty(streamKey, stream2);
|
||||
@Override
|
||||
public void onStreamActive(Http2Stream stream) {
|
||||
onStreamActive0(stream);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onStreamClosed(Http2Stream stream) {
|
||||
Http2Stream2Impl stream2 = stream.getProperty(streamKey);
|
||||
DefaultHttp2FrameStream stream2 = stream.getProperty(streamKey);
|
||||
if (stream2 != null) {
|
||||
stream2.setClosed();
|
||||
onHttp2StreamStateChanged(ctx, stream2);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onGoAwayReceived(final int lastStreamId, long errorCode, ByteBuf debugData) {
|
||||
ctx.fireChannelRead(new DefaultHttp2GoAwayFrame(lastStreamId, errorCode, debugData.retain()));
|
||||
public void onStreamHalfClosed(Http2Stream stream) {
|
||||
DefaultHttp2FrameStream stream2 = stream.getProperty(streamKey);
|
||||
if (stream2 != null) {
|
||||
onHttp2StreamStateChanged(ctx, stream2);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private final class InternalHttp2ConnectionHandler extends Http2ConnectionHandler {
|
||||
InternalHttp2ConnectionHandler(Http2ConnectionDecoder decoder, Http2ConnectionEncoder encoder,
|
||||
Http2Settings initialSettings) {
|
||||
super(decoder, encoder, initialSettings);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void onConnectionError(ChannelHandlerContext ctx, Throwable cause, Http2Exception http2Ex) {
|
||||
ctx.fireExceptionCaught(cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* Exceptions for unknown streams, that is streams that have no {@link Http2Stream2} object attached
|
||||
* are simply logged and replied to by sending a RST_STREAM frame. There is not much value in propagating such
|
||||
* exceptions through the pipeline, as a user will not have any additional information / state about this
|
||||
* stream and thus can't do any meaningful error handling.
|
||||
*/
|
||||
@Override
|
||||
protected void onStreamError(ChannelHandlerContext ctx, Throwable cause,
|
||||
Http2Exception.StreamException streamException) {
|
||||
int streamId = streamException.streamId();
|
||||
Http2Stream connectionStream = connection().stream(streamId);
|
||||
if (connectionStream == null) {
|
||||
LOG.warn("Stream exception thrown for unkown stream.", cause);
|
||||
// Write a RST_STREAM
|
||||
super.onStreamError(ctx, cause, streamException);
|
||||
return;
|
||||
}
|
||||
|
||||
Http2Stream2 stream2 = connectionStream.getProperty(streamKey);
|
||||
if (stream2 == null) {
|
||||
LOG.warn("Stream exception thrown without stream object attached.", cause);
|
||||
// Write a RST_STREAM
|
||||
super.onStreamError(ctx, cause, streamException);
|
||||
return;
|
||||
}
|
||||
|
||||
fireHttp2Stream2Exception(stream2, streamException.error(), cause);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean isGracefulShutdownComplete() {
|
||||
return super.isGracefulShutdownComplete() && numBufferedStreams == 0;
|
||||
}
|
||||
|
||||
private void fireHttp2Stream2Exception(Http2Stream2 stream, Http2Error error, Throwable cause) {
|
||||
ctx.fireExceptionCaught(new Http2Stream2Exception(stream, error, cause));
|
||||
}
|
||||
@Override
|
||||
protected void onConnectionError(ChannelHandlerContext ctx, Throwable cause, Http2Exception http2Ex) {
|
||||
// allow the user to handle it first in the pipeline, and then automatically clean up.
|
||||
// If this is not desired behavior the user can override this method.
|
||||
ctx.fireExceptionCaught(cause);
|
||||
super.onConnectionError(ctx, cause, http2Ex);
|
||||
}
|
||||
|
||||
private final class FrameListener extends Http2FrameAdapter {
|
||||
/**
|
||||
* Exceptions for unknown streams, that is streams that have no {@link Http2FrameStream} object attached
|
||||
* are simply logged and replied to by sending a RST_STREAM frame.
|
||||
*/
|
||||
@Override
|
||||
protected final void onStreamError(ChannelHandlerContext ctx, Throwable cause,
|
||||
Http2Exception.StreamException streamException) {
|
||||
int streamId = streamException.streamId();
|
||||
Http2Stream connectionStream = connection().stream(streamId);
|
||||
if (connectionStream == null) {
|
||||
onHttp2UnknownStreamError(ctx, cause, streamException);
|
||||
// Write a RST_STREAM
|
||||
super.onStreamError(ctx, cause, streamException);
|
||||
return;
|
||||
}
|
||||
|
||||
Http2FrameStream stream = connectionStream.getProperty(streamKey);
|
||||
if (stream == null) {
|
||||
LOG.warn("Stream exception thrown without stream object attached.", cause);
|
||||
// Write a RST_STREAM
|
||||
super.onStreamError(ctx, cause, streamException);
|
||||
return;
|
||||
}
|
||||
|
||||
onHttp2FrameStreamException(ctx, new Http2FrameStreamException(stream, streamException.error(), cause));
|
||||
}
|
||||
|
||||
void onHttp2UnknownStreamError(@SuppressWarnings("unused") ChannelHandlerContext ctx, Throwable cause,
|
||||
Http2Exception.StreamException streamException) {
|
||||
// Just log....
|
||||
LOG.warn("Stream exception thrown for unkown stream {}.", streamException.streamId(), cause);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final boolean isGracefulShutdownComplete() {
|
||||
return super.isGracefulShutdownComplete() && numBufferedStreams == 0;
|
||||
}
|
||||
|
||||
private final class FrameListener implements Http2FrameListener {
|
||||
|
||||
@Override
|
||||
public void onUnknownFrame(
|
||||
ChannelHandlerContext ctx, byte frameType, int streamId, Http2Flags flags, ByteBuf payload) {
|
||||
onHttp2Frame(ctx, new DefaultHttp2UnknownFrame(frameType, flags, payload)
|
||||
.stream(requireStream(streamId)).retain());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onSettingsRead(ChannelHandlerContext ctx, Http2Settings settings) {
|
||||
ctx.fireChannelRead(new DefaultHttp2SettingsFrame(settings));
|
||||
onHttp2Frame(ctx, new DefaultHttp2SettingsFrame(settings));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPingRead(ChannelHandlerContext ctx, ByteBuf data) {
|
||||
ctx.fireChannelRead(new DefaultHttp2PingFrame(data.retain(), false));
|
||||
onHttp2Frame(ctx, new DefaultHttp2PingFrame(data, false).retain());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPingAckRead(ChannelHandlerContext ctx, ByteBuf data) {
|
||||
ctx.fireChannelRead(new DefaultHttp2PingFrame(data.retain(), true));
|
||||
onHttp2Frame(ctx, new DefaultHttp2PingFrame(data, true).retain());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onRstStreamRead(ChannelHandlerContext ctx, int streamId, long errorCode) {
|
||||
ctx.fireChannelRead(new DefaultHttp2ResetFrame(errorCode).stream(requireStream(streamId)));
|
||||
onHttp2Frame(ctx, new DefaultHttp2ResetFrame(errorCode).stream(requireStream(streamId)));
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -592,7 +504,7 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
|
||||
// Ignore connection window updates.
|
||||
return;
|
||||
}
|
||||
ctx.fireChannelRead(new DefaultHttp2WindowUpdateFrame(windowSizeIncrement).stream(requireStream(streamId)));
|
||||
onHttp2Frame(ctx, new DefaultHttp2WindowUpdateFrame(windowSizeIncrement).stream(requireStream(streamId)));
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -605,21 +517,43 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
|
||||
@Override
|
||||
public void onHeadersRead(ChannelHandlerContext ctx, int streamId, Http2Headers headers,
|
||||
int padding, boolean endOfStream) {
|
||||
ctx.fireChannelRead(new DefaultHttp2HeadersFrame(headers, endOfStream, padding)
|
||||
onHttp2Frame(ctx, new DefaultHttp2HeadersFrame(headers, endOfStream, padding)
|
||||
.stream(requireStream(streamId)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int onDataRead(ChannelHandlerContext ctx, int streamId, ByteBuf data, int padding,
|
||||
boolean endOfStream) {
|
||||
ctx.fireChannelRead(new DefaultHttp2DataFrame(data.retain(), endOfStream, padding)
|
||||
.stream(requireStream(streamId)));
|
||||
onHttp2Frame(ctx, new DefaultHttp2DataFrame(data, endOfStream, padding)
|
||||
.stream(requireStream(streamId)).retain());
|
||||
// We return the bytes in consumeBytes() once the stream channel consumed the bytes.
|
||||
return 0;
|
||||
}
|
||||
|
||||
private <V> Http2Stream2 requireStream(int streamId) {
|
||||
Http2Stream2 stream = connection().stream(streamId).getProperty(streamKey);
|
||||
@Override
|
||||
public void onGoAwayRead(ChannelHandlerContext ctx, int lastStreamId, long errorCode, ByteBuf debugData) {
|
||||
onHttp2Frame(ctx, new DefaultHttp2GoAwayFrame(lastStreamId, errorCode, debugData).retain());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPriorityRead(
|
||||
ChannelHandlerContext ctx, int streamId, int streamDependency, short weight, boolean exclusive) {
|
||||
// TODO: Maybe handle me
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onSettingsAckRead(ChannelHandlerContext ctx) {
|
||||
// TODO: Maybe handle me
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPushPromiseRead(
|
||||
ChannelHandlerContext ctx, int streamId, int promisedStreamId, Http2Headers headers, int padding) {
|
||||
// TODO: Maybe handle me
|
||||
}
|
||||
|
||||
private Http2FrameStream requireStream(int streamId) {
|
||||
Http2FrameStream stream = connection().stream(streamId).getProperty(streamKey);
|
||||
if (stream == null) {
|
||||
throw new IllegalStateException("Stream object required for identifier: " + streamId);
|
||||
}
|
||||
@ -627,99 +561,70 @@ public class Http2FrameCodec extends ChannelDuplexHandler {
|
||||
}
|
||||
}
|
||||
|
||||
void onUpgradeEvent(ChannelHandlerContext ctx, UpgradeEvent evt) {
|
||||
ctx.fireUserEventTriggered(evt);
|
||||
}
|
||||
|
||||
void onHttp2StreamWritabilityChanged(ChannelHandlerContext ctx, Http2FrameStream stream,
|
||||
@SuppressWarnings("unused") boolean writable) {
|
||||
ctx.fireUserEventTriggered(Http2FrameStreamEvent.writabilityChanged(stream));
|
||||
}
|
||||
|
||||
void onHttp2StreamStateChanged(ChannelHandlerContext ctx, Http2FrameStream stream) {
|
||||
ctx.fireUserEventTriggered(Http2FrameStreamEvent.stateChanged(stream));
|
||||
}
|
||||
|
||||
void onHttp2Frame(ChannelHandlerContext ctx, Http2Frame frame) {
|
||||
ctx.fireChannelRead(frame);
|
||||
}
|
||||
|
||||
void onHttp2FrameStreamException(ChannelHandlerContext ctx, Http2FrameStreamException cause) {
|
||||
ctx.fireExceptionCaught(cause);
|
||||
}
|
||||
|
||||
private final class Http2RemoteFlowControllerListener implements Http2RemoteFlowController.Listener {
|
||||
@Override
|
||||
public void writabilityChanged(Http2Stream stream) {
|
||||
Http2FrameStream frameStream = stream.getProperty(streamKey);
|
||||
if (frameStream == null) {
|
||||
return;
|
||||
}
|
||||
onHttp2StreamWritabilityChanged(
|
||||
ctx, frameStream, connection().remote().flowController().isWritable(stream));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link Http2Stream2} implementation.
|
||||
* {@link Http2FrameStream} implementation.
|
||||
*/
|
||||
// TODO(buchgr): Merge Http2Stream2 and Http2Stream.
|
||||
static final class Http2Stream2Impl extends DefaultChannelPromise implements Http2Stream2 {
|
||||
// TODO(buchgr): Merge Http2FrameStream and Http2Stream.
|
||||
static class DefaultHttp2FrameStream implements Http2FrameStream {
|
||||
|
||||
private volatile int id = -1;
|
||||
private volatile Object managedState;
|
||||
private volatile Http2Stream legacyStream;
|
||||
private volatile Http2Stream stream;
|
||||
|
||||
Http2Stream2Impl(Channel channel) {
|
||||
super(channel);
|
||||
setUncancellable();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream2Impl id(int id) {
|
||||
if (!isStreamIdValid(id)) {
|
||||
throw new IllegalArgumentException("Stream identifier invalid. Was: " + id);
|
||||
}
|
||||
this.id = id;
|
||||
DefaultHttp2FrameStream setStreamAndProperty(PropertyKey streamKey, Http2Stream stream) {
|
||||
assert id == -1 || stream.id() == id;
|
||||
this.stream = stream;
|
||||
stream.setProperty(streamKey, this);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int id() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream2Impl managedState(Object state) {
|
||||
managedState = state;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object managedState() {
|
||||
return managedState;
|
||||
Http2Stream stream = this.stream;
|
||||
return stream == null ? id : stream.id();
|
||||
}
|
||||
|
||||
@Override
|
||||
public State state() {
|
||||
Http2Stream stream0 = legacyStream;
|
||||
return stream0 == null
|
||||
? State.IDLE
|
||||
: stream0.state();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture closeFuture() {
|
||||
if (state() == State.IDLE) {
|
||||
throw new IllegalStateException("This method may not be called on IDLE streams.");
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelPromise setSuccess() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelPromise setSuccess(Void result) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean trySuccess() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelPromise setFailure(Throwable cause) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean tryFailure(Throwable cause) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean cancel(boolean mayInterruptIfRunning) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
void setClosed() {
|
||||
super.trySuccess();
|
||||
Http2Stream stream = this.stream;
|
||||
return stream == null ? State.IDLE : stream.state();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.valueOf(id);
|
||||
return String.valueOf(id());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1,5 +1,5 @@
|
||||
/*
|
||||
* Copyright 2016 The Netty Project
|
||||
* Copyright 2017 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
|
||||
@ -16,38 +16,21 @@
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.handler.codec.http2.StreamBufferingEncoder.Http2ChannelClosedException;
|
||||
import io.netty.handler.codec.http2.StreamBufferingEncoder.Http2GoAwayException;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static io.netty.handler.codec.http2.DefaultHttp2LocalFlowController.DEFAULT_WINDOW_UPDATE_RATIO;
|
||||
import static io.netty.util.internal.ObjectUtil.checkNotNull;
|
||||
import static io.netty.util.internal.ObjectUtil.checkPositiveOrZero;
|
||||
|
||||
/**
|
||||
* Builder for the {@link Http2FrameCodec}.
|
||||
*/
|
||||
@UnstableApi
|
||||
public final class Http2FrameCodecBuilder {
|
||||
public class Http2FrameCodecBuilder extends
|
||||
AbstractHttp2ConnectionHandlerBuilder<Http2FrameCodec, Http2FrameCodecBuilder> {
|
||||
|
||||
private final boolean server;
|
||||
private Http2FrameWriter frameWriter;
|
||||
private Http2FrameReader frameReader;
|
||||
private Http2Settings initialSettings;
|
||||
private long gracefulShutdownTimeoutMillis;
|
||||
private float windowUpdateRatio;
|
||||
private Http2FrameLogger frameLogger;
|
||||
private boolean bufferOutboundStreams;
|
||||
|
||||
private Http2FrameCodecBuilder(boolean server) {
|
||||
this.server = server;
|
||||
frameWriter = new DefaultHttp2FrameWriter();
|
||||
frameReader = new DefaultHttp2FrameReader();
|
||||
initialSettings = new Http2Settings();
|
||||
gracefulShutdownTimeoutMillis = Http2CodecUtil.DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT_MILLIS;
|
||||
windowUpdateRatio = DEFAULT_WINDOW_UPDATE_RATIO;
|
||||
Http2FrameCodecBuilder(boolean server) {
|
||||
server(server);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -64,111 +47,131 @@ public final class Http2FrameCodecBuilder {
|
||||
return new Http2FrameCodecBuilder(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Specify the {@link Http2FrameWriter} to use.
|
||||
*
|
||||
* <p>If not set, the {@link DefaultHttp2FrameWriter} is used.
|
||||
*/
|
||||
public Http2FrameCodecBuilder frameWriter(Http2FrameWriter frameWriter) {
|
||||
// For testing only.
|
||||
Http2FrameCodecBuilder frameWriter(Http2FrameWriter frameWriter) {
|
||||
this.frameWriter = checkNotNull(frameWriter, "frameWriter");
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Specify the {@link Http2FrameWriter} to use.
|
||||
*
|
||||
* <p>If not set, the {@link DefaultHttp2FrameReader} is used.
|
||||
*/
|
||||
public Http2FrameCodecBuilder frameReader(Http2FrameReader frameReader) {
|
||||
this.frameReader = checkNotNull(frameReader, "frameReader");
|
||||
return this;
|
||||
@Override
|
||||
public Http2Settings initialSettings() {
|
||||
return super.initialSettings();
|
||||
}
|
||||
|
||||
/**
|
||||
* Specify the initial {@link Http2Settings} to send to the remote endpoint.
|
||||
*
|
||||
* <p>If not set, the default values of {@link Http2Settings} are used.
|
||||
*/
|
||||
public Http2FrameCodecBuilder initialSettings(Http2Settings initialSettings) {
|
||||
this.initialSettings = checkNotNull(initialSettings, "initialSettings");
|
||||
return this;
|
||||
@Override
|
||||
public Http2FrameCodecBuilder initialSettings(Http2Settings settings) {
|
||||
return super.initialSettings(settings);
|
||||
}
|
||||
|
||||
/**
|
||||
* The amount of time to wait for all active streams to be closed, before the connection is closed.
|
||||
*
|
||||
* <p>The default value is {@link Http2CodecUtil#DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT_MILLIS}.
|
||||
*/
|
||||
public Http2FrameCodecBuilder gracefulShutdownTimeout(long timeout, TimeUnit unit) {
|
||||
gracefulShutdownTimeoutMillis =
|
||||
checkNotNull(unit, "unit").toMillis(checkPositiveOrZero(timeout, "timeout"));
|
||||
return this;
|
||||
@Override
|
||||
public long gracefulShutdownTimeoutMillis() {
|
||||
return super.gracefulShutdownTimeoutMillis();
|
||||
}
|
||||
|
||||
/**
|
||||
* Specify the HTTP/2 flow control window update ratio for both the connection and stream window.
|
||||
*/
|
||||
public Http2FrameCodecBuilder windowUpdateRatio(float windowUpdateRatio) {
|
||||
if (Float.compare(windowUpdateRatio, 0) < 1 || Float.compare(windowUpdateRatio, 1) > -1) {
|
||||
throw new IllegalArgumentException("windowUpdateRatio must be (0,1). Was: " + windowUpdateRatio);
|
||||
}
|
||||
this.windowUpdateRatio = windowUpdateRatio;
|
||||
return this;
|
||||
@Override
|
||||
public Http2FrameCodecBuilder gracefulShutdownTimeoutMillis(long gracefulShutdownTimeoutMillis) {
|
||||
return super.gracefulShutdownTimeoutMillis(gracefulShutdownTimeoutMillis);
|
||||
}
|
||||
|
||||
/**
|
||||
* Specify the {@link Http2FrameLogger} to use.
|
||||
*
|
||||
* <p>By default no frame logger is used.
|
||||
*/
|
||||
@Override
|
||||
public boolean isServer() {
|
||||
return super.isServer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxReservedStreams() {
|
||||
return super.maxReservedStreams();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameCodecBuilder maxReservedStreams(int maxReservedStreams) {
|
||||
return super.maxReservedStreams(maxReservedStreams);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isValidateHeaders() {
|
||||
return super.isValidateHeaders();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameCodecBuilder validateHeaders(boolean validateHeaders) {
|
||||
return super.validateHeaders(validateHeaders);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameLogger frameLogger() {
|
||||
return super.frameLogger();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameCodecBuilder frameLogger(Http2FrameLogger frameLogger) {
|
||||
this.frameLogger = checkNotNull(frameLogger, "frameLogger");
|
||||
return this;
|
||||
return super.frameLogger(frameLogger);
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether to buffer new outbound HTTP/2 streams when the {@code MAX_CONCURRENT_STREAMS} limit is reached.
|
||||
*
|
||||
* <p>When this limit is hit, instead of rejecting any new streams, newly created streams and their corresponding
|
||||
* frames are buffered. Once an active stream gets closed or the maximum number of concurrent streams is increased,
|
||||
* the codec will automatically try to empty its buffer and create as many new streams as possible.
|
||||
*
|
||||
* <p>If a {@code GOAWAY} frame is received from the remote endpoint, all buffered writes for streams with an ID
|
||||
* less than the specified {@code lastStreamId} will immediately fail with a {@link Http2GoAwayException}.
|
||||
*
|
||||
* <p>If the channel gets closed, all new and buffered writes will immediately fail with a
|
||||
* {@link Http2ChannelClosedException}.
|
||||
*
|
||||
* <p>This implementation makes the buffering mostly transparent and does not enforce an upper bound as to how many
|
||||
* streams/frames can be buffered.
|
||||
*/
|
||||
public Http2FrameCodecBuilder bufferOutboundStreams(boolean bufferOutboundStreams) {
|
||||
this.bufferOutboundStreams = bufferOutboundStreams;
|
||||
return this;
|
||||
@Override
|
||||
public boolean encoderEnforceMaxConcurrentStreams() {
|
||||
return super.encoderEnforceMaxConcurrentStreams();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameCodecBuilder encoderEnforceMaxConcurrentStreams(boolean encoderEnforceMaxConcurrentStreams) {
|
||||
return super.encoderEnforceMaxConcurrentStreams(encoderEnforceMaxConcurrentStreams);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2HeadersEncoder.SensitivityDetector headerSensitivityDetector() {
|
||||
return super.headerSensitivityDetector();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameCodecBuilder headerSensitivityDetector(
|
||||
Http2HeadersEncoder.SensitivityDetector headerSensitivityDetector) {
|
||||
return super.headerSensitivityDetector(headerSensitivityDetector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameCodecBuilder encoderIgnoreMaxHeaderListSize(boolean ignoreMaxHeaderListSize) {
|
||||
return super.encoderIgnoreMaxHeaderListSize(ignoreMaxHeaderListSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameCodecBuilder initialHuffmanDecodeCapacity(int initialHuffmanDecodeCapacity) {
|
||||
return super.initialHuffmanDecodeCapacity(initialHuffmanDecodeCapacity);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build a {@link Http2FrameCodec} object.
|
||||
*/
|
||||
@Override
|
||||
public Http2FrameCodec build() {
|
||||
Http2Connection connection = new DefaultHttp2Connection(server);
|
||||
Http2FrameWriter frameWriter = this.frameWriter;
|
||||
if (frameWriter != null) {
|
||||
// This is to support our tests and will never be executed by the user as frameWriter(...)
|
||||
// is package-private.
|
||||
DefaultHttp2Connection connection = new DefaultHttp2Connection(isServer(), maxReservedStreams());
|
||||
Long maxHeaderListSize = initialSettings().maxHeaderListSize();
|
||||
Http2FrameReader frameReader = new DefaultHttp2FrameReader(maxHeaderListSize == null ?
|
||||
new DefaultHttp2HeadersDecoder(true) :
|
||||
new DefaultHttp2HeadersDecoder(true, maxHeaderListSize));
|
||||
|
||||
if (frameLogger != null) {
|
||||
frameWriter = new Http2OutboundFrameLogger(frameWriter, frameLogger);
|
||||
frameReader = new Http2InboundFrameLogger(frameReader, frameLogger);
|
||||
if (frameLogger() != null) {
|
||||
frameWriter = new Http2OutboundFrameLogger(frameWriter, frameLogger());
|
||||
frameReader = new Http2InboundFrameLogger(frameReader, frameLogger());
|
||||
}
|
||||
Http2ConnectionEncoder encoder = new DefaultHttp2ConnectionEncoder(connection, frameWriter);
|
||||
if (encoderEnforceMaxConcurrentStreams()) {
|
||||
encoder = new StreamBufferingEncoder(encoder);
|
||||
}
|
||||
Http2ConnectionDecoder decoder = new DefaultHttp2ConnectionDecoder(connection, encoder, frameReader);
|
||||
|
||||
return build(decoder, encoder, initialSettings());
|
||||
}
|
||||
return super.build();
|
||||
}
|
||||
|
||||
Http2ConnectionEncoder encoder = new DefaultHttp2ConnectionEncoder(connection, frameWriter);
|
||||
|
||||
if (bufferOutboundStreams) {
|
||||
encoder = new StreamBufferingEncoder(encoder);
|
||||
}
|
||||
|
||||
connection.local().flowController(new DefaultHttp2LocalFlowController(connection, windowUpdateRatio,
|
||||
true /* auto refill conn window */));
|
||||
|
||||
Http2ConnectionDecoder decoder = new DefaultHttp2ConnectionDecoder(connection, encoder, frameReader);
|
||||
|
||||
return new Http2FrameCodec(encoder, decoder, initialSettings, gracefulShutdownTimeoutMillis);
|
||||
@Override
|
||||
protected Http2FrameCodec build(
|
||||
Http2ConnectionDecoder decoder, Http2ConnectionEncoder encoder, Http2Settings initialSettings) {
|
||||
return new Http2FrameCodec(encoder, decoder, initialSettings);
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Copyright 2016 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.handler.codec.http2.Http2Stream.State;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
/**
|
||||
* A single stream within a HTTP/2 connection. To be used with the {@link Http2FrameCodec}.
|
||||
*/
|
||||
@UnstableApi
|
||||
public interface Http2FrameStream {
|
||||
|
||||
/**
|
||||
* The stream with identifier 0, representing the HTTP/2 connection.
|
||||
*/
|
||||
Http2FrameStream CONNECTION_STREAM = new Http2FrameStream() {
|
||||
|
||||
@Override
|
||||
public int id() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public State state() {
|
||||
return State.IDLE;
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Returns the stream identifier.
|
||||
*
|
||||
* <p>Use {@link Http2CodecUtil#isStreamIdValid(int)} to check if the stream has already been assigned an
|
||||
* identifier.
|
||||
*/
|
||||
int id();
|
||||
|
||||
/**
|
||||
* Returns the state of this stream.
|
||||
*/
|
||||
State state();
|
||||
}
|
@ -0,0 +1,51 @@
|
||||
/*
|
||||
* Copyright 2017 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
@UnstableApi
|
||||
public final class Http2FrameStreamEvent {
|
||||
|
||||
private final Http2FrameStream stream;
|
||||
private final Type type;
|
||||
|
||||
enum Type {
|
||||
State,
|
||||
Writability
|
||||
}
|
||||
|
||||
private Http2FrameStreamEvent(Http2FrameStream stream, Type type) {
|
||||
this.stream = stream;
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
public Http2FrameStream stream() {
|
||||
return stream;
|
||||
}
|
||||
|
||||
public Type type() {
|
||||
return type;
|
||||
}
|
||||
|
||||
static Http2FrameStreamEvent stateChanged(Http2FrameStream stream) {
|
||||
return new Http2FrameStreamEvent(stream, Type.State);
|
||||
}
|
||||
|
||||
static Http2FrameStreamEvent writabilityChanged(Http2FrameStream stream) {
|
||||
return new Http2FrameStreamEvent(stream, Type.Writability);
|
||||
}
|
||||
}
|
@ -16,27 +16,32 @@
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
import static io.netty.util.internal.ObjectUtil.checkNotNull;
|
||||
|
||||
/**
|
||||
* A HTTP/2 exception for a specific {@link Http2Stream2}.
|
||||
* A HTTP/2 exception for a specific {@link Http2FrameStream}.
|
||||
*/
|
||||
public class Http2Stream2Exception extends Exception {
|
||||
@UnstableApi
|
||||
public final class Http2FrameStreamException extends Exception {
|
||||
|
||||
private static final long serialVersionUID = -4407186173493887044L;
|
||||
|
||||
private final Http2Error error;
|
||||
private final Http2Stream2 stream;
|
||||
private final Http2FrameStream stream;
|
||||
|
||||
public <T> Http2Stream2Exception(Http2Stream2 stream, Http2Error error, Throwable cause) {
|
||||
public Http2FrameStreamException(Http2FrameStream stream, Http2Error error, Throwable cause) {
|
||||
super(cause.getMessage(), cause);
|
||||
this.stream = stream;
|
||||
this.error = error;
|
||||
this.stream = checkNotNull(stream, "stream");
|
||||
this.error = checkNotNull(error, "error");
|
||||
}
|
||||
|
||||
public Http2Error error() {
|
||||
return error;
|
||||
}
|
||||
|
||||
public Http2Stream2 stream() {
|
||||
public Http2FrameStream stream() {
|
||||
return stream;
|
||||
}
|
||||
}
|
@ -19,10 +19,10 @@ package io.netty.handler.codec.http2;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
/**
|
||||
* A visitor that allows to iterate over a collection of {@link Http2Stream2}s.
|
||||
* A visitor that allows to iterate over a collection of {@link Http2FrameStream}s.
|
||||
*/
|
||||
@UnstableApi
|
||||
public interface Http2Stream2Visitor {
|
||||
public interface Http2FrameStreamVisitor {
|
||||
|
||||
/**
|
||||
* This method is called once for each stream of the collection.
|
||||
@ -34,5 +34,5 @@ public interface Http2Stream2Visitor {
|
||||
* <li>{@code false} if the visitor wants to stop handling the stream and abort the loop.</li>
|
||||
* </ul>
|
||||
*/
|
||||
boolean visit(Http2Stream2 stream);
|
||||
boolean visit(Http2FrameStream stream);
|
||||
}
|
@ -36,5 +36,5 @@ public interface Http2HeadersFrame extends Http2StreamFrame {
|
||||
/**
|
||||
* Returns {@code true} if the END_STREAM flag ist set.
|
||||
*/
|
||||
boolean endStream();
|
||||
boolean isEndStream();
|
||||
}
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,162 @@
|
||||
/*
|
||||
* Copyright 2017 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.channel.ChannelHandler;
|
||||
import io.netty.channel.ChannelHandlerAdapter;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
import static io.netty.util.internal.ObjectUtil.checkNotNull;
|
||||
|
||||
/**
|
||||
* A builder for {@link Http2MultiplexCodec}.
|
||||
*/
|
||||
@UnstableApi
|
||||
public class Http2MultiplexCodecBuilder
|
||||
extends AbstractHttp2ConnectionHandlerBuilder<Http2MultiplexCodec, Http2MultiplexCodecBuilder> {
|
||||
|
||||
final ChannelHandler childHandler;
|
||||
|
||||
Http2MultiplexCodecBuilder(boolean server, ChannelHandler childHandler) {
|
||||
server(server);
|
||||
this.childHandler = checkSharable(checkNotNull(childHandler, "childHandler"));
|
||||
}
|
||||
|
||||
private static ChannelHandler checkSharable(ChannelHandler handler) {
|
||||
if ((handler instanceof ChannelHandlerAdapter && !((ChannelHandlerAdapter) handler).isSharable()) ||
|
||||
!handler.getClass().isAnnotationPresent(ChannelHandler.Sharable.class)) {
|
||||
throw new IllegalArgumentException("The handler must be Sharable");
|
||||
}
|
||||
return handler;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a builder for a HTTP/2 client.
|
||||
*
|
||||
* @param childHandler the handler added to channels for remotely-created streams. It must be
|
||||
* {@link ChannelHandler.Sharable}.
|
||||
*/
|
||||
public static Http2MultiplexCodecBuilder forClient(ChannelHandler childHandler) {
|
||||
return new Http2MultiplexCodecBuilder(false, childHandler);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a builder for a HTTP/2 server.
|
||||
*
|
||||
* @param childHandler the handler added to channels for remotely-created streams. It must be
|
||||
* {@link ChannelHandler.Sharable}.
|
||||
*/
|
||||
public static Http2MultiplexCodecBuilder forServer(ChannelHandler childHandler) {
|
||||
return new Http2MultiplexCodecBuilder(true, childHandler);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Settings initialSettings() {
|
||||
return super.initialSettings();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2MultiplexCodecBuilder initialSettings(Http2Settings settings) {
|
||||
return super.initialSettings(settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long gracefulShutdownTimeoutMillis() {
|
||||
return super.gracefulShutdownTimeoutMillis();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2MultiplexCodecBuilder gracefulShutdownTimeoutMillis(long gracefulShutdownTimeoutMillis) {
|
||||
return super.gracefulShutdownTimeoutMillis(gracefulShutdownTimeoutMillis);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isServer() {
|
||||
return super.isServer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxReservedStreams() {
|
||||
return super.maxReservedStreams();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2MultiplexCodecBuilder maxReservedStreams(int maxReservedStreams) {
|
||||
return super.maxReservedStreams(maxReservedStreams);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isValidateHeaders() {
|
||||
return super.isValidateHeaders();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2MultiplexCodecBuilder validateHeaders(boolean validateHeaders) {
|
||||
return super.validateHeaders(validateHeaders);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameLogger frameLogger() {
|
||||
return super.frameLogger();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2MultiplexCodecBuilder frameLogger(Http2FrameLogger frameLogger) {
|
||||
return super.frameLogger(frameLogger);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean encoderEnforceMaxConcurrentStreams() {
|
||||
return super.encoderEnforceMaxConcurrentStreams();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2MultiplexCodecBuilder encoderEnforceMaxConcurrentStreams(boolean encoderEnforceMaxConcurrentStreams) {
|
||||
return super.encoderEnforceMaxConcurrentStreams(encoderEnforceMaxConcurrentStreams);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2HeadersEncoder.SensitivityDetector headerSensitivityDetector() {
|
||||
return super.headerSensitivityDetector();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2MultiplexCodecBuilder headerSensitivityDetector(
|
||||
Http2HeadersEncoder.SensitivityDetector headerSensitivityDetector) {
|
||||
return super.headerSensitivityDetector(headerSensitivityDetector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2MultiplexCodecBuilder encoderIgnoreMaxHeaderListSize(boolean ignoreMaxHeaderListSize) {
|
||||
return super.encoderIgnoreMaxHeaderListSize(ignoreMaxHeaderListSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2MultiplexCodecBuilder initialHuffmanDecodeCapacity(int initialHuffmanDecodeCapacity) {
|
||||
return super.initialHuffmanDecodeCapacity(initialHuffmanDecodeCapacity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2MultiplexCodec build() {
|
||||
return super.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Http2MultiplexCodec build(
|
||||
Http2ConnectionDecoder decoder, Http2ConnectionEncoder encoder, Http2Settings initialSettings) {
|
||||
return new Http2MultiplexCodec(encoder, decoder, initialSettings, childHandler);
|
||||
}
|
||||
}
|
@ -70,7 +70,7 @@ public class Http2ServerDowngrader extends MessageToMessageCodec<Http2StreamFram
|
||||
Http2HeadersFrame headersFrame = (Http2HeadersFrame) frame;
|
||||
Http2Headers headers = headersFrame.headers();
|
||||
|
||||
if (headersFrame.endStream()) {
|
||||
if (headersFrame.isEndStream()) {
|
||||
if (headers.method() == null) {
|
||||
LastHttpContent last = new DefaultLastHttpContent(Unpooled.EMPTY_BUFFER, validateHeaders);
|
||||
HttpConversionUtil.addHttp2ToHttpHeaders(id, headers, last.trailingHeaders(),
|
||||
@ -90,7 +90,7 @@ public class Http2ServerDowngrader extends MessageToMessageCodec<Http2StreamFram
|
||||
}
|
||||
} else if (frame instanceof Http2DataFrame) {
|
||||
Http2DataFrame dataFrame = (Http2DataFrame) frame;
|
||||
if (dataFrame.endStream()) {
|
||||
if (dataFrame.isEndStream()) {
|
||||
out.add(new DefaultLastHttpContent(dataFrame.content(), validateHeaders));
|
||||
} else {
|
||||
out.add(new DefaultHttpContent(dataFrame.content()));
|
||||
|
@ -71,7 +71,7 @@ public class Http2ServerUpgradeCodec implements HttpServerUpgradeHandler.Upgrade
|
||||
*
|
||||
* @param http2Codec the HTTP/2 multiplexing handler.
|
||||
*/
|
||||
public Http2ServerUpgradeCodec(Http2Codec http2Codec) {
|
||||
public Http2ServerUpgradeCodec(Http2MultiplexCodec http2Codec) {
|
||||
this(null, http2Codec);
|
||||
}
|
||||
|
||||
@ -92,8 +92,8 @@ public class Http2ServerUpgradeCodec implements HttpServerUpgradeHandler.Upgrade
|
||||
* @param handlerName the name of the HTTP/2 connection handler to be used in the pipeline.
|
||||
* @param http2Codec the HTTP/2 multiplexing handler.
|
||||
*/
|
||||
public Http2ServerUpgradeCodec(String handlerName, Http2Codec http2Codec) {
|
||||
this(handlerName, http2Codec.frameCodec().connectionHandler(), http2Codec);
|
||||
public Http2ServerUpgradeCodec(String handlerName, Http2MultiplexCodec http2Codec) {
|
||||
this(handlerName, http2Codec, http2Codec);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -104,7 +104,7 @@ public class Http2ServerUpgradeCodec implements HttpServerUpgradeHandler.Upgrade
|
||||
* @param handlers the handlers that will handle the {@link Http2Frame}s.
|
||||
*/
|
||||
public Http2ServerUpgradeCodec(final Http2FrameCodec http2Codec, final ChannelHandler... handlers) {
|
||||
this(null, http2Codec.connectionHandler(), new ChannelHandlerAdapter() {
|
||||
this(null, http2Codec, new ChannelHandlerAdapter() {
|
||||
|
||||
@Override
|
||||
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
|
@ -1,107 +0,0 @@
|
||||
/*
|
||||
* Copyright 2016 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.handler.codec.http2.Http2Stream.State;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
/**
|
||||
* A single stream within a HTTP/2 connection. To be used with the {@link Http2FrameCodec}.
|
||||
*/
|
||||
@UnstableApi
|
||||
public interface Http2Stream2 {
|
||||
|
||||
/**
|
||||
* The stream with identifier 0, representing the HTTP/2 connection.
|
||||
*/
|
||||
Http2Stream2 CONNECTION_STREAM = new Http2Stream2() {
|
||||
|
||||
@Override
|
||||
public Http2Stream2 id(int id) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int id() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream2 managedState(Object state) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object managedState() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public State state() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture closeFuture() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Set the stream identifier to a value greater than zero.
|
||||
*
|
||||
* <p>This method must never be called by user code, except it might be useful in tests. This method may be called
|
||||
* at most once.
|
||||
*/
|
||||
Http2Stream2 id(int id);
|
||||
|
||||
/**
|
||||
* Returns the stream identifier.
|
||||
*
|
||||
* <p>Use {@link Http2CodecUtil#isStreamIdValid(int)} to check if the stream has already been assigned an
|
||||
* identifier.
|
||||
*/
|
||||
int id();
|
||||
|
||||
/**
|
||||
* Attach application specific state to this HTTP/2 stream.
|
||||
*
|
||||
* <p>The state is maintained until the stream or the channel are closed (whatever happens first).
|
||||
*/
|
||||
Http2Stream2 managedState(Object state);
|
||||
|
||||
/**
|
||||
* Returns the application specific state object or {@code null} if no state has been attached yet.
|
||||
*/
|
||||
Object managedState();
|
||||
|
||||
/**
|
||||
* Returns the state of this stream.
|
||||
*/
|
||||
State state();
|
||||
|
||||
/**
|
||||
* A {@link ChannelFuture} that will complete when the stream or the {@link io.netty.channel.Channel} are closed
|
||||
* (whatever happens first).
|
||||
*
|
||||
* <p><strong>NOTE:</strong> It's not safe to call this method on a stream in {@link State#IDLE} state.
|
||||
*
|
||||
* @throws IllegalStateException if this method is called on a stream in {@link State#IDLE} state.
|
||||
*/
|
||||
ChannelFuture closeFuture();
|
||||
}
|
@ -0,0 +1,33 @@
|
||||
/*
|
||||
* Copyright 2017 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
// TODO: Should we have an extra method to "open" the stream and so Channel and take care of sending the
|
||||
// Http2HeadersFrame under the hood ?
|
||||
// TODO: Should we extend SocketChannel and map input and output state to the stream state ?
|
||||
//
|
||||
@UnstableApi
|
||||
public interface Http2StreamChannel extends Channel {
|
||||
|
||||
/**
|
||||
* Returns the {@link Http2FrameStream} that belongs to this channel.
|
||||
*/
|
||||
Http2FrameStream stream();
|
||||
}
|
@ -1,5 +1,5 @@
|
||||
/*
|
||||
* Copyright 2016 The Netty Project
|
||||
* Copyright 2017 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
|
||||
@ -13,227 +13,189 @@
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelFutureListener;
|
||||
import io.netty.channel.ChannelHandler;
|
||||
import io.netty.channel.ChannelHandler.Sharable;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelOption;
|
||||
import io.netty.channel.ChannelPipeline;
|
||||
import io.netty.channel.EventLoop;
|
||||
import io.netty.channel.EventLoopGroup;
|
||||
import io.netty.handler.codec.http2.Http2MultiplexCodec.Http2StreamChannel;
|
||||
import io.netty.util.AttributeKey;
|
||||
import io.netty.util.concurrent.EventExecutor;
|
||||
import io.netty.util.concurrent.Future;
|
||||
import io.netty.util.concurrent.Promise;
|
||||
import io.netty.util.internal.ObjectUtil;
|
||||
import io.netty.util.internal.StringUtil;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
import io.netty.util.internal.logging.InternalLogger;
|
||||
import io.netty.util.internal.logging.InternalLoggerFactory;
|
||||
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static io.netty.util.internal.ObjectUtil.checkNotNull;
|
||||
import static java.util.Collections.synchronizedMap;
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
|
||||
/**
|
||||
* A class that makes it easy to bootstrap a new HTTP/2 stream as a {@link Channel}.
|
||||
*
|
||||
* <p>The bootstrap requires a registered parent {@link Channel} with a {@link ChannelPipeline} that contains the
|
||||
* {@link Http2MultiplexCodec}.
|
||||
*
|
||||
* <h3>Channel Events</h3>
|
||||
*
|
||||
* A child channel becomes active as soon as it is registered to an {@link EventLoop}. Therefore, an active channel
|
||||
* does not map to an active HTTP/2 stream immediately. Only once a {@link Http2HeadersFrame} has been successfully sent
|
||||
* or received, does the channel map to an active HTTP/2 stream. In case it is not possible to open a new HTTP/2 stream
|
||||
* (i.e. due to the maximum number of active streams being exceeded), the child channel receives an exception
|
||||
* indicating the cause and is closed immediately thereafter.
|
||||
*
|
||||
* <h3>Writability and Flow Control</h3>
|
||||
*
|
||||
* A child channel observes outbound/remote flow control via the channel's writability. A channel only becomes writable
|
||||
* when it maps to an active HTTP/2 stream and the stream's flow control window is greater than zero. A child channel
|
||||
* does not know about the connection-level flow control window. {@link ChannelHandler}s are free to ignore the
|
||||
* channel's writability, in which case the excessive writes will be buffered by the parent channel. It's important to
|
||||
* note that only {@link Http2DataFrame}s are subject to HTTP/2 flow control. So it's perfectly legal (and expected)
|
||||
* by a handler that aims to respect the channel's writability to e.g. write a {@link Http2DataFrame} even if the
|
||||
* channel is marked unwritable.
|
||||
*
|
||||
* <p>This class is thread-safe.
|
||||
*/
|
||||
// TODO(buchgr): Should we deliver a user event when the stream becomes active? For all stream states?
|
||||
@UnstableApi
|
||||
public class Http2StreamChannelBootstrap {
|
||||
|
||||
private volatile ParentChannelAndMultiplexCodec channelAndCodec;
|
||||
public final class Http2StreamChannelBootstrap {
|
||||
private static final InternalLogger logger = InternalLoggerFactory.getInstance(Http2StreamChannelBootstrap.class);
|
||||
|
||||
private final Map<ChannelOption<?>, Object> options = new LinkedHashMap<ChannelOption<?>, Object>();
|
||||
private final Map<AttributeKey<?>, Object> attrs = new LinkedHashMap<AttributeKey<?>, Object>();
|
||||
private final Channel channel;
|
||||
private volatile ChannelHandler handler;
|
||||
private volatile EventLoopGroup group;
|
||||
private final Map<ChannelOption<?>, Object> options;
|
||||
private final Map<AttributeKey<?>, Object> attributes;
|
||||
|
||||
public Http2StreamChannelBootstrap() {
|
||||
options = synchronizedMap(new LinkedHashMap<ChannelOption<?>, Object>());
|
||||
attributes = synchronizedMap(new LinkedHashMap<AttributeKey<?>, Object>());
|
||||
}
|
||||
|
||||
// Copy constructor
|
||||
Http2StreamChannelBootstrap(Http2StreamChannelBootstrap bootstrap0) {
|
||||
checkNotNull(bootstrap0, "bootstrap must not be null");
|
||||
channelAndCodec = bootstrap0.channelAndCodec;
|
||||
handler = bootstrap0.handler;
|
||||
group = bootstrap0.group;
|
||||
options = synchronizedMap(new LinkedHashMap<ChannelOption<?>, Object>(bootstrap0.options));
|
||||
attributes = synchronizedMap(new LinkedHashMap<AttributeKey<?>, Object>(bootstrap0.attributes));
|
||||
public Http2StreamChannelBootstrap(Channel channel) {
|
||||
this.channel = ObjectUtil.checkNotNull(channel, "channel");
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new channel that will eventually map to a local/outbound HTTP/2 stream.
|
||||
*/
|
||||
public ChannelFuture connect() {
|
||||
Http2Stream2 newStream = channelAndCodec.multiplexCodec.newStream();
|
||||
return connect(newStream);
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by the {@link Http2MultiplexCodec} to instantiate incoming/remotely-created streams.
|
||||
*/
|
||||
ChannelFuture connect(Http2Stream2 stream) {
|
||||
validateState();
|
||||
|
||||
ParentChannelAndMultiplexCodec channelAndCodec0 = channelAndCodec;
|
||||
Channel parentChannel = channelAndCodec0.parentChannel;
|
||||
Http2MultiplexCodec multiplexCodec = channelAndCodec0.multiplexCodec;
|
||||
|
||||
EventLoopGroup group0 = group;
|
||||
group0 = group0 == null ? parentChannel.eventLoop() : group0;
|
||||
|
||||
return multiplexCodec.createStreamChannel(parentChannel, group0, handler, options, attributes, stream);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the parent channel that must have the {@link Http2MultiplexCodec} in its pipeline.
|
||||
*
|
||||
* @param parent a registered channel with the {@link Http2MultiplexCodec} in its pipeline. This channel will
|
||||
* be the {@link Channel#parent()} of all channels created via {@link #connect()}.
|
||||
* @return {@code this}
|
||||
*/
|
||||
public Http2StreamChannelBootstrap parentChannel(Channel parent) {
|
||||
channelAndCodec = new ParentChannelAndMultiplexCodec(parent);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the channel handler that should be added to the channels's pipeline.
|
||||
*
|
||||
* @param handler the channel handler to add to the channel's pipeline. The handler must be
|
||||
* {@link Sharable}.
|
||||
* @return {@code this}
|
||||
*/
|
||||
public Http2StreamChannelBootstrap handler(ChannelHandler handler) {
|
||||
this.handler = checkSharable(checkNotNull(handler, "handler"));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link EventLoop} to which channels created with this bootstrap are registered.
|
||||
*
|
||||
* @param group the eventloop or {@code null} if the eventloop of the parent channel should be used.
|
||||
* @return {@code this}
|
||||
*/
|
||||
public Http2StreamChannelBootstrap group(EventLoopGroup group) {
|
||||
this.group = group;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Specify {@link ChannelOption}s to be set on newly created channels. An option can be removed by specifying a
|
||||
* value of {@code null}.
|
||||
* Allow to specify a {@link ChannelOption} which is used for the {@link Http2StreamChannel} instances once they got
|
||||
* created. Use a value of {@code null} to remove a previous set {@link ChannelOption}.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public <T> Http2StreamChannelBootstrap option(ChannelOption<T> option, T value) {
|
||||
checkNotNull(option, "option must not be null");
|
||||
if (option == null) {
|
||||
throw new NullPointerException("option");
|
||||
}
|
||||
if (value == null) {
|
||||
options.remove(option);
|
||||
synchronized (options) {
|
||||
options.remove(option);
|
||||
}
|
||||
} else {
|
||||
options.put(option, value);
|
||||
synchronized (options) {
|
||||
options.put(option, value);
|
||||
}
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Specify attributes with an initial value to be set on newly created channels. An attribute can be removed by
|
||||
* specifying a value of {@code null}.
|
||||
* Allow to specify an initial attribute of the newly created {@link Http2StreamChannel}. If the {@code value} is
|
||||
* {@code null}, the attribute of the specified {@code key} is removed.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public <T> Http2StreamChannelBootstrap attr(AttributeKey<T> key, T value) {
|
||||
checkNotNull(key, "key must not be null");
|
||||
if (key == null) {
|
||||
throw new NullPointerException("key");
|
||||
}
|
||||
if (value == null) {
|
||||
attributes.remove(key);
|
||||
synchronized (attrs) {
|
||||
attrs.remove(key);
|
||||
}
|
||||
} else {
|
||||
attributes.put(key, value);
|
||||
synchronized (attrs) {
|
||||
attrs.put(key, value);
|
||||
}
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public Channel parentChannel() {
|
||||
ParentChannelAndMultiplexCodec channelAndCodec0 = channelAndCodec;
|
||||
if (channelAndCodec0 != null) {
|
||||
return channelAndCodec0.parentChannel;
|
||||
}
|
||||
return null;
|
||||
/**
|
||||
* the {@link ChannelHandler} to use for serving the requests.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
public Http2StreamChannelBootstrap handler(ChannelHandler handler) {
|
||||
this.handler = ObjectUtil.checkNotNull(handler, "handler");
|
||||
return this;
|
||||
}
|
||||
|
||||
public ChannelHandler handler() {
|
||||
return handler;
|
||||
public Future<Http2StreamChannel> open() {
|
||||
return open(channel.eventLoop().<Http2StreamChannel>newPromise());
|
||||
}
|
||||
|
||||
public EventLoopGroup group() {
|
||||
return group;
|
||||
}
|
||||
|
||||
public Map<ChannelOption<?>, Object> options() {
|
||||
return unmodifiableMap(new LinkedHashMap<ChannelOption<?>, Object>(options));
|
||||
}
|
||||
|
||||
public Map<AttributeKey<?>, Object> attributes() {
|
||||
return unmodifiableMap(new LinkedHashMap<AttributeKey<?>, Object>(attributes));
|
||||
}
|
||||
|
||||
private void validateState() {
|
||||
checkNotNull(handler, "handler must be set");
|
||||
checkNotNull(channelAndCodec, "parent channel must be set");
|
||||
}
|
||||
|
||||
private static ChannelHandler checkSharable(ChannelHandler handler) {
|
||||
if (!handler.getClass().isAnnotationPresent(Sharable.class)) {
|
||||
throw new IllegalArgumentException("The handler must be Sharable");
|
||||
}
|
||||
return handler;
|
||||
}
|
||||
|
||||
private static class ParentChannelAndMultiplexCodec {
|
||||
final Channel parentChannel;
|
||||
final Http2MultiplexCodec multiplexCodec;
|
||||
|
||||
ParentChannelAndMultiplexCodec(Channel parentChannel) {
|
||||
this.parentChannel = checkRegistered(checkNotNull(parentChannel, "parentChannel"));
|
||||
this.multiplexCodec = requireMultiplexCodec(parentChannel.pipeline());
|
||||
}
|
||||
|
||||
private static Http2MultiplexCodec requireMultiplexCodec(ChannelPipeline pipeline) {
|
||||
ChannelHandlerContext ctx = pipeline.context(Http2MultiplexCodec.class);
|
||||
if (ctx == null) {
|
||||
throw new IllegalArgumentException(Http2MultiplexCodec.class.getSimpleName()
|
||||
+ " was not found in the channel pipeline.");
|
||||
public Future<Http2StreamChannel> open(final Promise<Http2StreamChannel> promise) {
|
||||
final ChannelHandlerContext ctx = channel.pipeline().context(Http2MultiplexCodec.class);
|
||||
if (ctx == null) {
|
||||
if (channel.isActive()) {
|
||||
promise.setFailure(new IllegalStateException(StringUtil.simpleClassName(Http2MultiplexCodec.class) +
|
||||
" must be in the ChannelPipeline of Channel " + channel));
|
||||
} else {
|
||||
promise.setFailure(new ClosedChannelException());
|
||||
}
|
||||
return (Http2MultiplexCodec) ctx.handler();
|
||||
} else {
|
||||
EventExecutor executor = ctx.executor();
|
||||
if (executor.inEventLoop()) {
|
||||
open0(ctx, promise);
|
||||
} else {
|
||||
executor.execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
open0(ctx, promise);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
return promise;
|
||||
}
|
||||
|
||||
public void open0(ChannelHandlerContext ctx, final Promise<Http2StreamChannel> promise) {
|
||||
assert ctx.executor().inEventLoop();
|
||||
final Http2StreamChannel streamChannel = ((Http2MultiplexCodec) ctx.handler()).newOutboundStream();
|
||||
try {
|
||||
init(streamChannel);
|
||||
} catch (Exception e) {
|
||||
streamChannel.unsafe().closeForcibly();
|
||||
promise.setFailure(e);
|
||||
return;
|
||||
}
|
||||
|
||||
private static Channel checkRegistered(Channel channel) {
|
||||
if (!channel.isRegistered()) {
|
||||
throw new IllegalArgumentException("The channel must be registered to an eventloop.");
|
||||
ChannelFuture future = ctx.channel().eventLoop().register(streamChannel);
|
||||
future.addListener(new ChannelFutureListener() {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
if (future.isSuccess()) {
|
||||
promise.setSuccess(streamChannel);
|
||||
} else if (future.isCancelled()) {
|
||||
promise.cancel(false);
|
||||
} else {
|
||||
if (streamChannel.isRegistered()) {
|
||||
streamChannel.close();
|
||||
} else {
|
||||
streamChannel.unsafe().closeForcibly();
|
||||
}
|
||||
|
||||
promise.setFailure(future.cause());
|
||||
}
|
||||
}
|
||||
return channel;
|
||||
});
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private void init(Channel channel) throws Exception {
|
||||
ChannelPipeline p = channel.pipeline();
|
||||
ChannelHandler handler = this.handler;
|
||||
if (handler != null) {
|
||||
p.addLast(handler);
|
||||
}
|
||||
synchronized (options) {
|
||||
setChannelOptions(channel, options, logger);
|
||||
}
|
||||
|
||||
synchronized (attrs) {
|
||||
for (Map.Entry<AttributeKey<?>, Object> e: attrs.entrySet()) {
|
||||
channel.attr((AttributeKey<Object>) e.getKey()).set(e.getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void setChannelOptions(
|
||||
Channel channel, Map<ChannelOption<?>, Object> options, InternalLogger logger) {
|
||||
for (Map.Entry<ChannelOption<?>, Object> e: options.entrySet()) {
|
||||
setChannelOption(channel, e.getKey(), e.getValue(), logger);
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private static void setChannelOption(
|
||||
Channel channel, ChannelOption<?> option, Object value, InternalLogger logger) {
|
||||
try {
|
||||
if (!channel.config().setOption((ChannelOption<Object>) option, value)) {
|
||||
logger.warn("Unknown channel option '{}' for channel '{}'", option, channel);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
logger.warn(
|
||||
"Failed to set channel option '{}' with value '{}' for channel '{}'", option, value, channel, t);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,70 @@
|
||||
/*
|
||||
* Copyright 2017 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.channel.ChannelId;
|
||||
|
||||
/**
|
||||
* ChannelId implementation which is used by our {@link Http2StreamChannel} implementation.
|
||||
*/
|
||||
final class Http2StreamChannelId implements ChannelId {
|
||||
|
||||
private final int id;
|
||||
private final ChannelId parentId;
|
||||
|
||||
Http2StreamChannelId(ChannelId parentId, int id) {
|
||||
this.parentId = parentId;
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String asShortText() {
|
||||
return parentId.asShortText() + '/' + id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String asLongText() {
|
||||
return parentId.asLongText() + '/' + id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ChannelId o) {
|
||||
if (o instanceof Http2StreamChannelId) {
|
||||
Http2StreamChannelId otherId = (Http2StreamChannelId) o;
|
||||
int res = parentId.compareTo(otherId.parentId);
|
||||
if (res == 0) {
|
||||
return id - otherId.id;
|
||||
} else {
|
||||
return res;
|
||||
}
|
||||
}
|
||||
return parentId.compareTo(o);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return parentId.hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (!(obj instanceof Http2StreamChannelId)) {
|
||||
return false;
|
||||
}
|
||||
Http2StreamChannelId otherId = (Http2StreamChannelId) obj;
|
||||
return id == otherId.id && parentId.equals(otherId.parentId);
|
||||
}
|
||||
}
|
@ -20,20 +20,20 @@ import io.netty.util.internal.UnstableApi;
|
||||
/**
|
||||
* A frame whose meaning <em>may</em> apply to a particular stream, instead of the entire connection. It is still
|
||||
* possible for this frame type to apply to the entire connection. In such cases, the {@link #stream()} must return
|
||||
* {@link Http2Stream2#CONNECTION_STREAM}. If the frame applies to a stream, the {@link Http2Stream2#id()} must be
|
||||
* greater than zero.
|
||||
* {@link Http2FrameStream#CONNECTION_STREAM}. If the frame applies to a stream, the {@link Http2FrameStream#id()} must
|
||||
* be greater than zero.
|
||||
*/
|
||||
@UnstableApi
|
||||
public interface Http2StreamFrame extends Http2Frame {
|
||||
|
||||
/**
|
||||
* Set the {@link Http2Stream2} object for this frame.
|
||||
* Set the {@link Http2FrameStream} object for this frame.
|
||||
*/
|
||||
Http2StreamFrame stream(Http2Stream2 stream);
|
||||
Http2StreamFrame stream(Http2FrameStream stream);
|
||||
|
||||
/**
|
||||
* Returns the {@link Http2Stream2} object for this frame, or {@code null} if the frame has yet to be associated
|
||||
* Returns the {@link Http2FrameStream} object for this frame, or {@code null} if the frame has yet to be associated
|
||||
* with a stream.
|
||||
*/
|
||||
Http2Stream2 stream();
|
||||
Http2FrameStream stream();
|
||||
}
|
||||
|
@ -0,0 +1,59 @@
|
||||
/*
|
||||
* Copyright 2017 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufHolder;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
@UnstableApi
|
||||
public interface Http2UnknownFrame extends Http2Frame, ByteBufHolder {
|
||||
|
||||
Http2FrameStream stream();
|
||||
|
||||
/**
|
||||
* Set the {@link Http2FrameStream} object for this frame.
|
||||
*/
|
||||
Http2UnknownFrame stream(Http2FrameStream stream);
|
||||
|
||||
byte frameType();
|
||||
|
||||
Http2Flags flags();
|
||||
|
||||
@Override
|
||||
Http2UnknownFrame copy();
|
||||
|
||||
@Override
|
||||
Http2UnknownFrame duplicate();
|
||||
|
||||
@Override
|
||||
Http2UnknownFrame retainedDuplicate();
|
||||
|
||||
@Override
|
||||
Http2UnknownFrame replace(ByteBuf content);
|
||||
|
||||
@Override
|
||||
Http2UnknownFrame retain();
|
||||
|
||||
@Override
|
||||
Http2UnknownFrame retain(int increment);
|
||||
|
||||
@Override
|
||||
Http2UnknownFrame touch();
|
||||
|
||||
@Override
|
||||
Http2UnknownFrame touch(Object hint);
|
||||
}
|
@ -35,7 +35,7 @@ public class InboundHttpToHttp2Adapter extends ChannelInboundHandlerAdapter {
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
private int getStreamId(HttpHeaders httpHeaders) {
|
||||
private static int getStreamId(Http2Connection connection, HttpHeaders httpHeaders) {
|
||||
return httpHeaders.getInt(HttpConversionUtil.ExtensionHeaderNames.STREAM_ID.text(),
|
||||
connection.remote().incrementAndGetNextStreamId());
|
||||
}
|
||||
@ -43,32 +43,36 @@ public class InboundHttpToHttp2Adapter extends ChannelInboundHandlerAdapter {
|
||||
@Override
|
||||
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
|
||||
if (msg instanceof FullHttpMessage) {
|
||||
FullHttpMessage message = (FullHttpMessage) msg;
|
||||
try {
|
||||
int streamId = getStreamId(message.headers());
|
||||
Http2Stream stream = connection.stream(streamId);
|
||||
if (stream == null) {
|
||||
stream = connection.remote().createStream(streamId, false);
|
||||
}
|
||||
message.headers().set(HttpConversionUtil.ExtensionHeaderNames.SCHEME.text(), HttpScheme.HTTP.name());
|
||||
Http2Headers messageHeaders = HttpConversionUtil.toHttp2Headers(message, true);
|
||||
boolean hasContent = message.content().isReadable();
|
||||
boolean hasTrailers = !message.trailingHeaders().isEmpty();
|
||||
listener.onHeadersRead(
|
||||
ctx, streamId, messageHeaders, 0, !(hasContent || hasTrailers));
|
||||
if (hasContent) {
|
||||
listener.onDataRead(ctx, streamId, message.content(), 0, !hasTrailers);
|
||||
}
|
||||
if (hasTrailers) {
|
||||
Http2Headers headers = HttpConversionUtil.toHttp2Headers(message.trailingHeaders(), true);
|
||||
listener.onHeadersRead(ctx, streamId, headers, 0, true);
|
||||
}
|
||||
stream.closeRemoteSide();
|
||||
} finally {
|
||||
message.release();
|
||||
}
|
||||
handle(ctx, connection, listener, (FullHttpMessage) msg);
|
||||
} else {
|
||||
super.channelRead(ctx, msg);
|
||||
}
|
||||
}
|
||||
|
||||
static void handle(ChannelHandlerContext ctx, Http2Connection connection,
|
||||
Http2FrameListener listener, FullHttpMessage message) throws Http2Exception {
|
||||
try {
|
||||
int streamId = getStreamId(connection, message.headers());
|
||||
Http2Stream stream = connection.stream(streamId);
|
||||
if (stream == null) {
|
||||
stream = connection.remote().createStream(streamId, false);
|
||||
}
|
||||
message.headers().set(HttpConversionUtil.ExtensionHeaderNames.SCHEME.text(), HttpScheme.HTTP.name());
|
||||
Http2Headers messageHeaders = HttpConversionUtil.toHttp2Headers(message, true);
|
||||
boolean hasContent = message.content().isReadable();
|
||||
boolean hasTrailers = !message.trailingHeaders().isEmpty();
|
||||
listener.onHeadersRead(
|
||||
ctx, streamId, messageHeaders, 0, !(hasContent || hasTrailers));
|
||||
if (hasContent) {
|
||||
listener.onDataRead(ctx, streamId, message.content(), 0, !hasTrailers);
|
||||
}
|
||||
if (hasTrailers) {
|
||||
Http2Headers headers = HttpConversionUtil.toHttp2Headers(message.trailingHeaders(), true);
|
||||
listener.onHeadersRead(ctx, streamId, headers, 0, true);
|
||||
}
|
||||
stream.closeRemoteSide();
|
||||
} finally {
|
||||
message.release();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -184,8 +184,8 @@ public class CleartextHttp2ServerUpgradeHandlerTest {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void usedHttp2Codec() throws Exception {
|
||||
final Http2Codec http2Codec = new Http2CodecBuilder(true, new ChannelInitializer<Channel>() {
|
||||
public void usedHttp2MultiplexCodec() throws Exception {
|
||||
final Http2MultiplexCodec http2Codec = new Http2MultiplexCodecBuilder(true, new ChannelInitializer<Channel>() {
|
||||
@Override
|
||||
protected void initChannel(Channel ch) throws Exception {
|
||||
}
|
||||
@ -196,7 +196,7 @@ public class CleartextHttp2ServerUpgradeHandlerTest {
|
||||
return new Http2ServerUpgradeCodec(http2Codec);
|
||||
}
|
||||
};
|
||||
http2ConnectionHandler = http2Codec.frameCodec().connectionHandler();
|
||||
http2ConnectionHandler = http2Codec;
|
||||
|
||||
userEvents = new ArrayList<Object>();
|
||||
|
||||
|
@ -20,7 +20,6 @@ import io.netty.buffer.Unpooled;
|
||||
import io.netty.buffer.UnpooledByteBufAllocator;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelFutureListener;
|
||||
import io.netty.channel.ChannelHandler;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
import io.netty.channel.embedded.EmbeddedChannel;
|
||||
@ -44,14 +43,11 @@ import org.mockito.ArgumentCaptor;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid;
|
||||
import static io.netty.handler.codec.http2.Http2Stream2.CONNECTION_STREAM;
|
||||
import static io.netty.util.ReferenceCountUtil.releaseLater;
|
||||
import static io.netty.handler.codec.http2.Http2FrameStream.CONNECTION_STREAM;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.*;
|
||||
@ -83,15 +79,29 @@ public class Http2FrameCodecTest {
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
inboundHandler.finishAndReleaseAll();
|
||||
channel.close();
|
||||
if (inboundHandler != null) {
|
||||
inboundHandler.finishAndReleaseAll();
|
||||
inboundHandler = null;
|
||||
}
|
||||
if (channel != null) {
|
||||
channel.finishAndReleaseAll();
|
||||
channel.close();
|
||||
channel = null;
|
||||
}
|
||||
}
|
||||
|
||||
private void setUp(Http2FrameCodecBuilder frameCodecBuilder, Http2Settings initialRemoteSettings) throws Exception {
|
||||
/**
|
||||
* Some tests call this method twice. Once with JUnit's @Before and once directly to pass special settings.
|
||||
* This call ensures that in case of two consecutive calls to setUp(), the previous channel is shutdown and
|
||||
* ByteBufs are released correctly.
|
||||
*/
|
||||
tearDown();
|
||||
|
||||
frameWriter = spy(new VerifiableHttp2FrameWriter());
|
||||
frameCodec = frameCodecBuilder.frameWriter(frameWriter).frameLogger(new Http2FrameLogger(LogLevel.TRACE))
|
||||
.initialSettings(initialRemoteSettings).build();
|
||||
frameListener = ((DefaultHttp2ConnectionDecoder) frameCodec.connectionHandler().decoder())
|
||||
frameListener = ((DefaultHttp2ConnectionDecoder) frameCodec.decoder())
|
||||
.internalFrameListener();
|
||||
inboundHandler = new LastInboundHandler();
|
||||
|
||||
@ -101,7 +111,7 @@ public class Http2FrameCodecTest {
|
||||
channel.pipeline().addLast(inboundHandler);
|
||||
channel.pipeline().fireChannelActive();
|
||||
|
||||
http2HandlerCtx = channel.pipeline().context(frameCodec.connectionHandler());
|
||||
http2HandlerCtx = channel.pipeline().context(frameCodec);
|
||||
|
||||
// Handshake
|
||||
verify(frameWriter).writeSettings(eq(http2HandlerCtx),
|
||||
@ -117,28 +127,47 @@ public class Http2FrameCodecTest {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void connectionHandlerShouldBeAddedBeforeFramingHandler() {
|
||||
Iterator<Entry<String, ChannelHandler>> iter = channel.pipeline().iterator();
|
||||
while (iter.hasNext()) {
|
||||
ChannelHandler handler = iter.next().getValue();
|
||||
if (handler instanceof Http2ConnectionHandler) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(iter.hasNext());
|
||||
assertThat(iter.next().getValue(), instanceOf(Http2FrameCodec.class));
|
||||
public void stateChanges() throws Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 1, request, 31, true);
|
||||
|
||||
Http2Stream stream = frameCodec.connection().stream(1);
|
||||
assertNotNull(stream);
|
||||
assertEquals(State.HALF_CLOSED_REMOTE, stream.state());
|
||||
|
||||
Http2FrameStreamEvent event = inboundHandler.readInboundMessageOrUserEvent();
|
||||
assertEquals(State.HALF_CLOSED_REMOTE, event.stream().state());
|
||||
|
||||
Http2StreamFrame inboundFrame = inboundHandler.readInbound();
|
||||
Http2FrameStream stream2 = inboundFrame.stream();
|
||||
assertNotNull(stream2);
|
||||
assertEquals(1, stream2.id());
|
||||
assertEquals(inboundFrame, new DefaultHttp2HeadersFrame(request, true, 31).stream(stream2));
|
||||
assertNull(inboundHandler.readInbound());
|
||||
|
||||
inboundHandler.writeOutbound(new DefaultHttp2HeadersFrame(response, true, 27).stream(stream2));
|
||||
verify(frameWriter).writeHeaders(
|
||||
eq(http2HandlerCtx), eq(1), eq(response), anyInt(), anyShort(), anyBoolean(),
|
||||
eq(27), eq(true), anyChannelPromise());
|
||||
verify(frameWriter, never()).writeRstStream(
|
||||
any(ChannelHandlerContext.class), anyInt(), anyLong(), anyChannelPromise());
|
||||
|
||||
assertEquals(State.CLOSED, stream.state());
|
||||
event = inboundHandler.readInboundMessageOrUserEvent();
|
||||
assertEquals(State.CLOSED, event.stream().state());
|
||||
|
||||
assertTrue(channel.isActive());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void headerRequestHeaderResponse() throws Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 1, request, 31, true);
|
||||
|
||||
Http2Stream stream = frameCodec.connectionHandler().connection().stream(1);
|
||||
Http2Stream stream = frameCodec.connection().stream(1);
|
||||
assertNotNull(stream);
|
||||
assertEquals(State.HALF_CLOSED_REMOTE, stream.state());
|
||||
|
||||
Http2StreamFrame inboundFrame = inboundHandler.readInbound();
|
||||
Http2Stream2 stream2 = inboundFrame.stream();
|
||||
Http2FrameStream stream2 = inboundFrame.stream();
|
||||
assertNotNull(stream2);
|
||||
assertEquals(1, stream2.id());
|
||||
assertEquals(inboundFrame, new DefaultHttp2HeadersFrame(request, true, 31).stream(stream2));
|
||||
@ -159,12 +188,12 @@ public class Http2FrameCodecTest {
|
||||
public void entityRequestEntityResponse() throws Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 1, request, 0, false);
|
||||
|
||||
Http2Stream stream = frameCodec.connectionHandler().connection().stream(1);
|
||||
Http2Stream stream = frameCodec.connection().stream(1);
|
||||
assertNotNull(stream);
|
||||
assertEquals(State.OPEN, stream.state());
|
||||
|
||||
Http2HeadersFrame inboundHeaders = inboundHandler.readInbound();
|
||||
Http2Stream2 stream2 = inboundHeaders.stream();
|
||||
Http2FrameStream stream2 = inboundHeaders.stream();
|
||||
assertNotNull(stream2);
|
||||
assertEquals(1, stream2.id());
|
||||
assertEquals(new DefaultHttp2HeadersFrame(request, false).stream(stream2), inboundHeaders);
|
||||
@ -205,15 +234,15 @@ public class Http2FrameCodecTest {
|
||||
public void sendRstStream() throws Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 3, request, 31, true);
|
||||
|
||||
Http2Stream stream = frameCodec.connectionHandler().connection().stream(3);
|
||||
Http2Stream stream = frameCodec.connection().stream(3);
|
||||
assertNotNull(stream);
|
||||
assertEquals(State.HALF_CLOSED_REMOTE, stream.state());
|
||||
|
||||
Http2HeadersFrame inboundHeaders = inboundHandler.readInbound();
|
||||
assertNotNull(inboundHeaders);
|
||||
assertTrue(inboundHeaders.endStream());
|
||||
assertTrue(inboundHeaders.isEndStream());
|
||||
|
||||
Http2Stream2 stream2 = inboundHeaders.stream();
|
||||
Http2FrameStream stream2 = inboundHeaders.stream();
|
||||
assertNotNull(stream2);
|
||||
assertEquals(3, stream2.id());
|
||||
|
||||
@ -228,7 +257,7 @@ public class Http2FrameCodecTest {
|
||||
public void receiveRstStream() throws Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 3, request, 31, false);
|
||||
|
||||
Http2Stream stream = frameCodec.connectionHandler().connection().stream(3);
|
||||
Http2Stream stream = frameCodec.connection().stream(3);
|
||||
assertNotNull(stream);
|
||||
assertEquals(State.OPEN, stream.state());
|
||||
|
||||
@ -249,7 +278,7 @@ public class Http2FrameCodecTest {
|
||||
public void sendGoAway() throws Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 3, request, 31, false);
|
||||
|
||||
Http2Stream stream = frameCodec.connectionHandler().connection().stream(3);
|
||||
Http2Stream stream = frameCodec.connection().stream(3);
|
||||
assertNotNull(stream);
|
||||
assertEquals(State.OPEN, stream.state());
|
||||
|
||||
@ -317,7 +346,7 @@ public class Http2FrameCodecTest {
|
||||
public void goAwayLastStreamIdOverflowed() throws Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 5, request, 31, false);
|
||||
|
||||
Http2Stream stream = frameCodec.connectionHandler().connection().stream(5);
|
||||
Http2Stream stream = frameCodec.connection().stream(5);
|
||||
assertNotNull(stream);
|
||||
assertEquals(State.OPEN, stream.state());
|
||||
|
||||
@ -338,19 +367,22 @@ public class Http2FrameCodecTest {
|
||||
public void streamErrorShouldFireException() throws Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 3, request, 31, false);
|
||||
|
||||
Http2Stream stream = frameCodec.connectionHandler().connection().stream(3);
|
||||
Http2Stream stream = frameCodec.connection().stream(3);
|
||||
assertNotNull(stream);
|
||||
|
||||
StreamException streamEx = new StreamException(3, Http2Error.INTERNAL_ERROR, "foo");
|
||||
frameCodec.connectionHandler().onError(http2HandlerCtx, streamEx);
|
||||
frameCodec.onError(http2HandlerCtx, streamEx);
|
||||
|
||||
Http2FrameStreamEvent event = inboundHandler.readInboundMessageOrUserEvent();
|
||||
assertEquals(Http2FrameStreamEvent.Type.State, event.type());
|
||||
assertEquals(State.OPEN, event.stream().state());
|
||||
Http2HeadersFrame headersFrame = inboundHandler.readInboundMessageOrUserEvent();
|
||||
assertNotNull(headersFrame);
|
||||
|
||||
try {
|
||||
inboundHandler.checkException();
|
||||
fail("stream exception expected");
|
||||
} catch (Http2Stream2Exception e) {
|
||||
} catch (Http2FrameStreamException e) {
|
||||
assertEquals(streamEx, e.getCause());
|
||||
}
|
||||
|
||||
@ -361,7 +393,7 @@ public class Http2FrameCodecTest {
|
||||
public void windowUpdateFrameDecrementsConsumedBytes() throws Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 3, request, 31, false);
|
||||
|
||||
Http2Connection connection = frameCodec.connectionHandler().connection();
|
||||
Http2Connection connection = frameCodec.connection();
|
||||
Http2Stream stream = connection.stream(3);
|
||||
assertNotNull(stream);
|
||||
|
||||
@ -372,7 +404,7 @@ public class Http2FrameCodecTest {
|
||||
assertNotNull(inboundHeaders);
|
||||
assertNotNull(inboundHeaders.stream());
|
||||
|
||||
Http2Stream2 stream2 = inboundHeaders.stream();
|
||||
Http2FrameStream stream2 = inboundHeaders.stream();
|
||||
|
||||
int before = connection.local().flowController().unconsumedBytes(stream);
|
||||
ChannelFuture f = channel.write(new DefaultHttp2WindowUpdateFrame(100).stream(stream2));
|
||||
@ -385,14 +417,14 @@ public class Http2FrameCodecTest {
|
||||
@Test
|
||||
public void windowUpdateMayFail() throws Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 3, request, 31, false);
|
||||
Http2Connection connection = frameCodec.connectionHandler().connection();
|
||||
Http2Connection connection = frameCodec.connection();
|
||||
Http2Stream stream = connection.stream(3);
|
||||
assertNotNull(stream);
|
||||
|
||||
Http2HeadersFrame inboundHeaders = inboundHandler.readInbound();
|
||||
assertNotNull(inboundHeaders);
|
||||
|
||||
Http2Stream2 stream2 = inboundHeaders.stream();
|
||||
Http2FrameStream stream2 = inboundHeaders.stream();
|
||||
|
||||
// Fails, cause trying to return too many bytes to the flow controller
|
||||
ChannelFuture f = channel.write(new DefaultHttp2WindowUpdateFrame(100).stream(stream2));
|
||||
@ -422,7 +454,7 @@ public class Http2FrameCodecTest {
|
||||
|
||||
@Test
|
||||
public void streamZeroWindowUpdateIncrementsConnectionWindow() throws Exception {
|
||||
Http2Connection connection = frameCodec.connectionHandler().connection();
|
||||
Http2Connection connection = frameCodec.connection();
|
||||
Http2LocalFlowController localFlow = connection.local().flowController();
|
||||
int initialWindowSizeBefore = localFlow.initialWindowSize();
|
||||
|
||||
@ -443,7 +475,7 @@ public class Http2FrameCodecTest {
|
||||
|
||||
@Test(timeout = 1000)
|
||||
public void newOutboundStream() {
|
||||
final Http2Stream2 stream = frameCodec.newStream();
|
||||
final Http2FrameStream stream = frameCodec.newStream();
|
||||
|
||||
assertNotNull(stream);
|
||||
assertFalse(isStreamIdValid(stream.id()));
|
||||
@ -456,8 +488,6 @@ public class Http2FrameCodecTest {
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
assertTrue(future.isSuccess());
|
||||
assertTrue(isStreamIdValid(stream.id()));
|
||||
assertFalse(stream.closeFuture().isDone());
|
||||
|
||||
listenerExecuted.setSuccess(null);
|
||||
}
|
||||
}
|
||||
@ -472,11 +502,11 @@ public class Http2FrameCodecTest {
|
||||
|
||||
@Test
|
||||
public void newOutboundStreamsShouldBeBuffered() throws Exception {
|
||||
setUp(Http2FrameCodecBuilder.forServer().bufferOutboundStreams(true),
|
||||
setUp(Http2FrameCodecBuilder.forServer().encoderEnforceMaxConcurrentStreams(true),
|
||||
new Http2Settings().maxConcurrentStreams(1));
|
||||
|
||||
Http2Stream2 stream1 = frameCodec.newStream();
|
||||
Http2Stream2 stream2 = frameCodec.newStream();
|
||||
Http2FrameStream stream1 = frameCodec.newStream();
|
||||
Http2FrameStream stream2 = frameCodec.newStream();
|
||||
|
||||
ChannelPromise promise1 = channel.newPromise();
|
||||
ChannelPromise promise2 = channel.newPromise();
|
||||
@ -485,6 +515,7 @@ public class Http2FrameCodecTest {
|
||||
channel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers()).stream(stream2), promise2);
|
||||
|
||||
assertTrue(isStreamIdValid(stream1.id()));
|
||||
channel.runPendingTasks();
|
||||
assertTrue(isStreamIdValid(stream2.id()));
|
||||
|
||||
assertTrue(promise1.syncUninterruptibly().isSuccess());
|
||||
@ -502,9 +533,9 @@ public class Http2FrameCodecTest {
|
||||
public void streamIdentifiersExhausted() throws Http2Exception {
|
||||
int maxServerStreamId = Integer.MAX_VALUE - 1;
|
||||
|
||||
assertNotNull(frameCodec.connectionHandler().connection().local().createStream(maxServerStreamId, false));
|
||||
assertNotNull(frameCodec.connection().local().createStream(maxServerStreamId, false));
|
||||
|
||||
Http2Stream2 stream = frameCodec.newStream();
|
||||
Http2FrameStream stream = frameCodec.newStream();
|
||||
assertNotNull(stream);
|
||||
|
||||
ChannelPromise writePromise = channel.newPromise();
|
||||
@ -516,7 +547,7 @@ public class Http2FrameCodecTest {
|
||||
@Test
|
||||
public void receivePing() throws Http2Exception {
|
||||
ByteBuf data = Unpooled.buffer(8).writeLong(12345);
|
||||
frameListener.onPingRead(http2HandlerCtx, releaseLater(data));
|
||||
frameListener.onPingRead(http2HandlerCtx, data);
|
||||
|
||||
Http2PingFrame pingFrame = inboundHandler.readInbound();
|
||||
assertNotNull(pingFrame);
|
||||
@ -524,6 +555,7 @@ public class Http2FrameCodecTest {
|
||||
assertEquals(data, pingFrame.content());
|
||||
assertFalse(pingFrame.ack());
|
||||
pingFrame.release();
|
||||
data.release();
|
||||
}
|
||||
|
||||
@Test
|
||||
@ -552,44 +584,9 @@ public class Http2FrameCodecTest {
|
||||
verify(frameWriter).writeSettings(eq(http2HandlerCtx), eq(settings), anyChannelPromise());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void managedStateShouldPersist() throws Http2Exception {
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 3, request, 0, false);
|
||||
|
||||
Http2HeadersFrame headersFrame = inboundHandler.readInbound();
|
||||
assertNotNull(headersFrame);
|
||||
|
||||
Http2Stream2 inboundStream = headersFrame.stream();
|
||||
assertNotNull(inboundStream);
|
||||
assertNull(inboundStream.managedState());
|
||||
|
||||
Object inboundState = new Object();
|
||||
inboundStream.managedState(inboundState);
|
||||
|
||||
Http2Stream2 outboundStream = frameCodec.newStream();
|
||||
Object outboundState = new Object();
|
||||
outboundStream.managedState(outboundState);
|
||||
channel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers()).stream(outboundStream));
|
||||
|
||||
ByteBuf inboundData = Unpooled.buffer(100).writeZero(100);
|
||||
frameListener.onDataRead(http2HandlerCtx, inboundStream.id(), inboundData, 0, false);
|
||||
|
||||
Http2DataFrame dataFrame = inboundHandler.readInbound();
|
||||
assertNotNull(dataFrame);
|
||||
assertSame(inboundStream, dataFrame.stream());
|
||||
assertSame(inboundState, dataFrame.stream().managedState());
|
||||
dataFrame.release();
|
||||
|
||||
frameListener.onHeadersRead(http2HandlerCtx, outboundStream.id(), new DefaultHttp2Headers(), 0, false);
|
||||
headersFrame = inboundHandler.readInbound();
|
||||
assertNotNull(headersFrame);
|
||||
assertSame(outboundStream, headersFrame.stream());
|
||||
assertSame(outboundState, headersFrame.stream().managedState());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void iterateActiveStreams() throws Exception {
|
||||
setUp(Http2FrameCodecBuilder.forServer().bufferOutboundStreams(true),
|
||||
setUp(Http2FrameCodecBuilder.forServer().encoderEnforceMaxConcurrentStreams(true),
|
||||
new Http2Settings().maxConcurrentStreams(1));
|
||||
|
||||
frameListener.onHeadersRead(http2HandlerCtx, 3, request, 0, false);
|
||||
@ -597,21 +594,21 @@ public class Http2FrameCodecTest {
|
||||
Http2HeadersFrame headersFrame = inboundHandler.readInbound();
|
||||
assertNotNull(headersFrame);
|
||||
|
||||
Http2Stream2 activeInbond = headersFrame.stream();
|
||||
Http2FrameStream activeInbond = headersFrame.stream();
|
||||
|
||||
Http2Stream2 activeOutbound = frameCodec.newStream();
|
||||
Http2FrameStream activeOutbound = frameCodec.newStream();
|
||||
channel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers()).stream(activeOutbound));
|
||||
|
||||
Http2Stream2 bufferedOutbound = frameCodec.newStream();
|
||||
Http2FrameStream bufferedOutbound = frameCodec.newStream();
|
||||
channel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers()).stream(bufferedOutbound));
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
Http2Stream2 idleStream = frameCodec.newStream();
|
||||
Http2FrameStream idleStream = frameCodec.newStream();
|
||||
|
||||
final Set<Http2Stream2> activeStreams = new HashSet<Http2Stream2>();
|
||||
frameCodec.forEachActiveStream(new Http2Stream2Visitor() {
|
||||
final Set<Http2FrameStream> activeStreams = new HashSet<Http2FrameStream>();
|
||||
frameCodec.forEachActiveStream(new Http2FrameStreamVisitor() {
|
||||
@Override
|
||||
public boolean visit(Http2Stream2 stream) {
|
||||
public boolean visit(Http2FrameStream stream) {
|
||||
activeStreams.add(stream);
|
||||
return true;
|
||||
}
|
||||
@ -619,7 +616,7 @@ public class Http2FrameCodecTest {
|
||||
|
||||
assertEquals(2, activeStreams.size());
|
||||
|
||||
Set<Http2Stream2> expectedStreams = new HashSet<Http2Stream2>();
|
||||
Set<Http2FrameStream> expectedStreams = new HashSet<Http2FrameStream>();
|
||||
expectedStreams.add(activeInbond);
|
||||
expectedStreams.add(activeOutbound);
|
||||
assertEquals(expectedStreams, activeStreams);
|
||||
@ -627,7 +624,7 @@ public class Http2FrameCodecTest {
|
||||
|
||||
@Test
|
||||
public void streamShouldBeOpenInListener() {
|
||||
final Http2Stream2 stream2 = frameCodec.newStream();
|
||||
final Http2FrameStream stream2 = frameCodec.newStream();
|
||||
assertEquals(State.IDLE, stream2.state());
|
||||
|
||||
final AtomicBoolean listenerExecuted = new AtomicBoolean();
|
||||
|
@ -21,6 +21,7 @@ import io.netty.bootstrap.ServerBootstrap;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelHandler;
|
||||
import io.netty.channel.ChannelHandler.Sharable;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelInitializer;
|
||||
@ -45,9 +46,9 @@ import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* Unit tests for {@link Http2Codec}.
|
||||
* Unit tests for {@link Http2MultiplexCodec}.
|
||||
*/
|
||||
public class Http2CodecTest {
|
||||
public class Http2MultiplexCodecBuilderTest {
|
||||
|
||||
private static EventLoopGroup group;
|
||||
private Channel serverChannel;
|
||||
@ -72,7 +73,7 @@ public class Http2CodecTest {
|
||||
@Override
|
||||
protected void initChannel(Channel ch) throws Exception {
|
||||
serverConnectedChannel = ch;
|
||||
ch.pipeline().addLast(new Http2CodecBuilder(true, serverLastInboundHandler).build());
|
||||
ch.pipeline().addLast(new Http2MultiplexCodecBuilder(true, serverLastInboundHandler).build());
|
||||
serverChannelLatch.countDown();
|
||||
}
|
||||
});
|
||||
@ -81,7 +82,7 @@ public class Http2CodecTest {
|
||||
Bootstrap cb = new Bootstrap()
|
||||
.channel(LocalChannel.class)
|
||||
.group(group)
|
||||
.handler(new Http2CodecBuilder(false, new TestChannelInitializer()).build());
|
||||
.handler(new Http2MultiplexCodecBuilder(false, new TestChannelInitializer()).build());
|
||||
clientChannel = cb.connect(serverAddress).sync().channel();
|
||||
assertTrue(serverChannelLatch.await(5, SECONDS));
|
||||
}
|
||||
@ -108,17 +109,18 @@ public class Http2CodecTest {
|
||||
}
|
||||
}
|
||||
|
||||
private Http2StreamChannel newOutboundStream(ChannelHandler handler) {
|
||||
return new Http2StreamChannelBootstrap(clientChannel).handler(handler).open().syncUninterruptibly().getNow();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void multipleOutboundStreams() {
|
||||
Http2StreamChannelBootstrap b = new Http2StreamChannelBootstrap();
|
||||
b.parentChannel(clientChannel).handler(new TestChannelInitializer());
|
||||
|
||||
Channel childChannel1 = b.connect().syncUninterruptibly().channel();
|
||||
Http2StreamChannel childChannel1 = newOutboundStream(new TestChannelInitializer());
|
||||
assertTrue(childChannel1.isActive());
|
||||
assertFalse(isStreamIdValid(((AbstractHttp2StreamChannel) childChannel1).stream().id()));
|
||||
Channel childChannel2 = b.connect().channel();
|
||||
assertFalse(isStreamIdValid(childChannel1.stream().id()));
|
||||
Http2StreamChannel childChannel2 = newOutboundStream(new TestChannelInitializer());
|
||||
assertTrue(childChannel2.isActive());
|
||||
assertFalse(isStreamIdValid(((AbstractHttp2StreamChannel) childChannel2).stream().id()));
|
||||
assertFalse(isStreamIdValid(childChannel2.stream().id()));
|
||||
|
||||
Http2Headers headers1 = new DefaultHttp2Headers();
|
||||
Http2Headers headers2 = new DefaultHttp2Headers();
|
||||
@ -135,8 +137,8 @@ public class Http2CodecTest {
|
||||
assertNotNull(headersFrame1);
|
||||
assertEquals(5, headersFrame1.stream().id());
|
||||
|
||||
assertEquals(3, ((AbstractHttp2StreamChannel) childChannel2).stream().id());
|
||||
assertEquals(5, ((AbstractHttp2StreamChannel) childChannel1).stream().id());
|
||||
assertEquals(3, childChannel2.stream().id());
|
||||
assertEquals(5, childChannel1.stream().id());
|
||||
|
||||
childChannel1.close();
|
||||
childChannel2.close();
|
||||
@ -144,9 +146,7 @@ public class Http2CodecTest {
|
||||
|
||||
@Test
|
||||
public void createOutboundStream() {
|
||||
Http2StreamChannelBootstrap b = new Http2StreamChannelBootstrap();
|
||||
Channel childChannel = b.parentChannel(clientChannel).handler(new TestChannelInitializer())
|
||||
.connect().syncUninterruptibly().channel();
|
||||
Channel childChannel = newOutboundStream(new TestChannelInitializer());
|
||||
assertTrue(childChannel.isRegistered());
|
||||
assertTrue(childChannel.isActive());
|
||||
|
||||
@ -164,7 +164,7 @@ public class Http2CodecTest {
|
||||
assertNotNull(dataFrame);
|
||||
assertEquals(3, dataFrame.stream().id());
|
||||
assertEquals(data.resetReaderIndex(), dataFrame.content());
|
||||
assertTrue(dataFrame.endStream());
|
||||
assertTrue(dataFrame.isEndStream());
|
||||
dataFrame.release();
|
||||
|
||||
childChannel.close();
|
@ -16,28 +16,25 @@ package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufUtil;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.buffer.UnpooledByteBufAllocator;
|
||||
import io.netty.channel.Channel;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelHandler;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelInboundHandlerAdapter;
|
||||
import io.netty.channel.ChannelOption;
|
||||
import io.netty.channel.ChannelOutboundHandlerAdapter;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
import io.netty.channel.DefaultChannelPromise;
|
||||
import io.netty.channel.embedded.EmbeddedChannel;
|
||||
import io.netty.handler.codec.http.HttpMethod;
|
||||
import io.netty.handler.codec.http.HttpScheme;
|
||||
import io.netty.handler.codec.http2.Http2Exception.StreamException;
|
||||
import io.netty.handler.codec.http2.Http2Stream.State;
|
||||
import io.netty.util.AsciiString;
|
||||
import io.netty.util.AttributeKey;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import org.hamcrest.Matchers;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
import static io.netty.util.ReferenceCountUtil.release;
|
||||
@ -46,15 +43,15 @@ import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertSame;
|
||||
import static org.junit.Assert.assertThat;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* Unit tests for {@link Http2MultiplexCodec} and {@link Http2StreamChannelBootstrap}.
|
||||
* Unit tests for {@link Http2MultiplexCodec}.
|
||||
*/
|
||||
public class Http2MultiplexCodecTest {
|
||||
|
||||
private EmbeddedChannel parentChannel;
|
||||
private Writer writer;
|
||||
|
||||
private TestChannelInitializer childChannelInitializer;
|
||||
|
||||
@ -62,26 +59,30 @@ public class Http2MultiplexCodecTest {
|
||||
.method(HttpMethod.GET.asciiName()).scheme(HttpScheme.HTTPS.name())
|
||||
.authority(new AsciiString("example.org")).path(new AsciiString("/foo"));
|
||||
|
||||
private Http2Stream2 inboundStream;
|
||||
|
||||
private Http2Stream2 outboundStream;
|
||||
private TestableHttp2MultiplexCodec codec;
|
||||
private TestableHttp2MultiplexCodec.Stream inboundStream;
|
||||
private TestableHttp2MultiplexCodec.Stream outboundStream;
|
||||
|
||||
private static final int initialRemoteStreamWindow = 1024;
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
childChannelInitializer = new TestChannelInitializer();
|
||||
Http2StreamChannelBootstrap bootstrap = new Http2StreamChannelBootstrap().handler(childChannelInitializer);
|
||||
parentChannel = new EmbeddedChannel();
|
||||
writer = new Writer();
|
||||
|
||||
parentChannel.connect(new InetSocketAddress(0));
|
||||
parentChannel.pipeline().addLast(new TestableHttp2MultiplexCodec(true, bootstrap));
|
||||
codec = new TestableHttp2MultiplexCodecBuilder(true, childChannelInitializer).build();
|
||||
parentChannel.pipeline().addLast(codec);
|
||||
parentChannel.runPendingTasks();
|
||||
|
||||
Http2Settings settings = new Http2Settings().initialWindowSize(initialRemoteStreamWindow);
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2SettingsFrame(settings));
|
||||
codec.onHttp2Frame(new DefaultHttp2SettingsFrame(settings));
|
||||
|
||||
inboundStream = new Http2Stream2Impl(parentChannel).id(3);
|
||||
outboundStream = new Http2Stream2Impl(parentChannel).id(2);
|
||||
inboundStream = codec.newStream();
|
||||
inboundStream.id = 3;
|
||||
outboundStream = codec.newStream();
|
||||
outboundStream.id = 2;
|
||||
}
|
||||
|
||||
@After
|
||||
@ -90,12 +91,9 @@ public class Http2MultiplexCodecTest {
|
||||
((LastInboundHandler) childChannelInitializer.handler).finishAndReleaseAll();
|
||||
}
|
||||
parentChannel.finishAndReleaseAll();
|
||||
|
||||
((ChannelPromise) inboundStream.closeFuture()).trySuccess();
|
||||
((ChannelPromise) outboundStream.closeFuture()).trySuccess();
|
||||
codec = null;
|
||||
}
|
||||
|
||||
// TODO(buchgr): Thread model of child channel
|
||||
// TODO(buchgr): Flush from child channel
|
||||
// TODO(buchgr): ChildChannel.childReadComplete()
|
||||
// TODO(buchgr): GOAWAY Logic
|
||||
@ -111,10 +109,12 @@ public class Http2MultiplexCodecTest {
|
||||
Http2DataFrame dataFrame2 = new DefaultHttp2DataFrame(bb("world")).stream(inboundStream);
|
||||
|
||||
assertFalse(inboundHandler.isChannelActive());
|
||||
parentChannel.pipeline().fireChannelRead(headersFrame);
|
||||
inboundStream.state = Http2Stream.State.OPEN;
|
||||
codec.onHttp2StreamStateChanged(inboundStream);
|
||||
codec.onHttp2Frame(headersFrame);
|
||||
assertTrue(inboundHandler.isChannelActive());
|
||||
parentChannel.pipeline().fireChannelRead(dataFrame1);
|
||||
parentChannel.pipeline().fireChannelRead(dataFrame2);
|
||||
codec.onHttp2Frame(dataFrame1);
|
||||
codec.onHttp2Frame(dataFrame2);
|
||||
|
||||
assertEquals(headersFrame, inboundHandler.readInbound());
|
||||
assertEquals(dataFrame1, inboundHandler.readInbound());
|
||||
@ -128,9 +128,13 @@ public class Http2MultiplexCodecTest {
|
||||
@Test
|
||||
public void framesShouldBeMultiplexed() {
|
||||
|
||||
Http2Stream2 stream3 = new Http2Stream2Impl(parentChannel).id(3);
|
||||
Http2Stream2 stream5 = new Http2Stream2Impl(parentChannel).id(5);
|
||||
Http2Stream2 stream11 = new Http2Stream2Impl(parentChannel).id(11);
|
||||
TestableHttp2MultiplexCodec.Stream stream3 = codec.newStream();
|
||||
stream3.id = 3;
|
||||
TestableHttp2MultiplexCodec.Stream stream5 = codec.newStream();
|
||||
stream5.id = 5;
|
||||
|
||||
TestableHttp2MultiplexCodec.Stream stream11 = codec.newStream();
|
||||
stream11.id = 11;
|
||||
|
||||
LastInboundHandler inboundHandler3 = streamActiveAndWriteHeaders(stream3);
|
||||
LastInboundHandler inboundHandler5 = streamActiveAndWriteHeaders(stream5);
|
||||
@ -140,29 +144,22 @@ public class Http2MultiplexCodecTest {
|
||||
verifyFramesMultiplexedToCorrectChannel(stream5, inboundHandler5, 1);
|
||||
verifyFramesMultiplexedToCorrectChannel(stream11, inboundHandler11, 1);
|
||||
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2DataFrame(bb("hello"), false).stream(stream5));
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2DataFrame(bb("foo"), true).stream(stream3));
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2DataFrame(bb("world"), true).stream(stream5));
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2DataFrame(bb("bar"), true).stream(stream11));
|
||||
codec.onHttp2Frame(new DefaultHttp2DataFrame(bb("hello"), false).stream(stream5));
|
||||
codec.onHttp2Frame(new DefaultHttp2DataFrame(bb("foo"), true).stream(stream3));
|
||||
codec.onHttp2Frame(new DefaultHttp2DataFrame(bb("world"), true).stream(stream5));
|
||||
codec.onHttp2Frame(new DefaultHttp2DataFrame(bb("bar"), true).stream(stream11));
|
||||
verifyFramesMultiplexedToCorrectChannel(stream5, inboundHandler5, 2);
|
||||
verifyFramesMultiplexedToCorrectChannel(stream3, inboundHandler3, 1);
|
||||
verifyFramesMultiplexedToCorrectChannel(stream11, inboundHandler11, 1);
|
||||
|
||||
((ChannelPromise) stream3.closeFuture()).setSuccess();
|
||||
((ChannelPromise) stream5.closeFuture()).setSuccess();
|
||||
((ChannelPromise) stream11.closeFuture()).setSuccess();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void inboundDataFrameShouldEmitWindowUpdateFrame() {
|
||||
LastInboundHandler inboundHandler = streamActiveAndWriteHeaders(inboundStream);
|
||||
ByteBuf tenBytes = bb("0123456789");
|
||||
parentChannel.pipeline().fireChannelRead(
|
||||
new DefaultHttp2DataFrame(tenBytes, true).stream(inboundStream));
|
||||
parentChannel.pipeline().fireChannelReadComplete();
|
||||
codec.onHttp2Frame(new DefaultHttp2DataFrame(tenBytes, true).stream(inboundStream));
|
||||
codec.onChannelReadComplete();
|
||||
|
||||
// Flush is only necessary cause of EmbeddedChannel
|
||||
parentChannel.flush();
|
||||
Http2WindowUpdateFrame windowUpdate = parentChannel.readOutbound();
|
||||
assertNotNull(windowUpdate);
|
||||
|
||||
@ -182,16 +179,16 @@ public class Http2MultiplexCodecTest {
|
||||
assertNotNull(headersFrame);
|
||||
|
||||
childChannel.config().setAutoRead(false);
|
||||
parentChannel.pipeline().fireChannelRead(
|
||||
codec.onHttp2Frame(
|
||||
new DefaultHttp2DataFrame(bb("hello world"), false).stream(inboundStream));
|
||||
parentChannel.pipeline().fireChannelReadComplete();
|
||||
codec.onChannelReadComplete();
|
||||
Http2DataFrame dataFrame0 = inboundHandler.readInbound();
|
||||
assertNotNull(dataFrame0);
|
||||
release(dataFrame0);
|
||||
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2DataFrame(bb("foo"), false).stream(inboundStream));
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2DataFrame(bb("bar"), true).stream(inboundStream));
|
||||
parentChannel.pipeline().fireChannelReadComplete();
|
||||
codec.onHttp2Frame(new DefaultHttp2DataFrame(bb("foo"), false).stream(inboundStream));
|
||||
codec.onHttp2Frame(new DefaultHttp2DataFrame(bb("bar"), true).stream(inboundStream));
|
||||
codec.onChannelReadComplete();
|
||||
|
||||
dataFrame0 = inboundHandler.readInbound();
|
||||
assertNull(dataFrame0);
|
||||
@ -200,17 +197,21 @@ public class Http2MultiplexCodecTest {
|
||||
verifyFramesMultiplexedToCorrectChannel(inboundStream, inboundHandler, 2);
|
||||
}
|
||||
|
||||
private Http2StreamChannel newOutboundStream() {
|
||||
return new Http2StreamChannelBootstrap(parentChannel).handler(childChannelInitializer)
|
||||
.open().syncUninterruptibly().getNow();
|
||||
}
|
||||
|
||||
/**
|
||||
* A child channel for a HTTP/2 stream in IDLE state (that is no headers sent or received),
|
||||
* should not emit a RST_STREAM frame on close, as this is a connection error of type protocol error.
|
||||
*/
|
||||
|
||||
@Test
|
||||
public void idleOutboundStreamShouldNotWriteResetFrameOnClose() {
|
||||
childChannelInitializer.handler = new LastInboundHandler();
|
||||
|
||||
Http2StreamChannelBootstrap b = new Http2StreamChannelBootstrap();
|
||||
b.parentChannel(parentChannel).handler(childChannelInitializer);
|
||||
Channel childChannel = b.connect().channel();
|
||||
Channel childChannel = newOutboundStream();
|
||||
assertTrue(childChannel.isActive());
|
||||
|
||||
childChannel.close();
|
||||
@ -231,14 +232,10 @@ public class Http2MultiplexCodecTest {
|
||||
}
|
||||
};
|
||||
|
||||
Http2StreamChannelBootstrap b = new Http2StreamChannelBootstrap();
|
||||
b.parentChannel(parentChannel).handler(childChannelInitializer);
|
||||
Channel childChannel = b.connect().channel();
|
||||
Channel childChannel = newOutboundStream();
|
||||
assertTrue(childChannel.isActive());
|
||||
|
||||
parentChannel.flush();
|
||||
|
||||
Http2Stream2 stream2 = readOutboundHeadersAndAssignId();
|
||||
Http2FrameStream stream2 = readOutboundHeadersAndAssignId();
|
||||
|
||||
childChannel.close();
|
||||
parentChannel.runPendingTasks();
|
||||
@ -252,13 +249,13 @@ public class Http2MultiplexCodecTest {
|
||||
public void inboundRstStreamFireChannelInactive() {
|
||||
LastInboundHandler inboundHandler = streamActiveAndWriteHeaders(inboundStream);
|
||||
assertTrue(inboundHandler.isChannelActive());
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2ResetFrame(Http2Error.INTERNAL_ERROR)
|
||||
codec.onHttp2Frame(new DefaultHttp2ResetFrame(Http2Error.INTERNAL_ERROR)
|
||||
.stream(inboundStream));
|
||||
parentChannel.pipeline().fireChannelReadComplete();
|
||||
codec.onChannelReadComplete();
|
||||
|
||||
// This will be called by the frame codec.
|
||||
((ChannelPromise) inboundStream.closeFuture()).setSuccess();
|
||||
|
||||
inboundStream.state = Http2Stream.State.CLOSED;
|
||||
codec.onHttp2StreamStateChanged(inboundStream);
|
||||
parentChannel.runPendingTasks();
|
||||
|
||||
assertFalse(inboundHandler.isChannelActive());
|
||||
@ -271,8 +268,9 @@ public class Http2MultiplexCodecTest {
|
||||
LastInboundHandler inboundHandler = streamActiveAndWriteHeaders(inboundStream);
|
||||
|
||||
StreamException cause = new StreamException(inboundStream.id(), Http2Error.PROTOCOL_ERROR, "baaam!");
|
||||
Exception http2Ex = new Http2Stream2Exception(inboundStream, Http2Error.PROTOCOL_ERROR, cause);
|
||||
parentChannel.pipeline().fireExceptionCaught(http2Ex);
|
||||
Http2FrameStreamException http2Ex = new Http2FrameStreamException(
|
||||
inboundStream, Http2Error.PROTOCOL_ERROR, cause);
|
||||
codec.onHttp2FrameStreamException(http2Ex);
|
||||
|
||||
inboundHandler.checkException();
|
||||
}
|
||||
@ -283,8 +281,9 @@ public class Http2MultiplexCodecTest {
|
||||
|
||||
assertTrue(inboundHandler.isChannelActive());
|
||||
StreamException cause = new StreamException(inboundStream.id(), Http2Error.PROTOCOL_ERROR, "baaam!");
|
||||
Exception http2Ex = new Http2Stream2Exception(inboundStream, Http2Error.PROTOCOL_ERROR, cause);
|
||||
parentChannel.pipeline().fireExceptionCaught(http2Ex);
|
||||
Http2FrameStreamException http2Ex = new Http2FrameStreamException(
|
||||
inboundStream, Http2Error.PROTOCOL_ERROR, cause);
|
||||
codec.onHttp2FrameStreamException(http2Ex);
|
||||
parentChannel.runPendingTasks();
|
||||
|
||||
assertFalse(inboundHandler.isChannelActive());
|
||||
@ -296,10 +295,7 @@ public class Http2MultiplexCodecTest {
|
||||
LastInboundHandler inboundHandler = new LastInboundHandler();
|
||||
childChannelInitializer.handler = inboundHandler;
|
||||
|
||||
Http2StreamChannelBootstrap b = new Http2StreamChannelBootstrap();
|
||||
b.parentChannel(parentChannel).handler(childChannelInitializer);
|
||||
AbstractHttp2StreamChannel childChannel = (AbstractHttp2StreamChannel) b.connect().channel();
|
||||
assertThat(childChannel, Matchers.instanceOf(Http2MultiplexCodec.Http2StreamChannel.class));
|
||||
Http2StreamChannel childChannel = newOutboundStream();
|
||||
assertTrue(childChannel.isActive());
|
||||
assertTrue(inboundHandler.isChannelActive());
|
||||
|
||||
@ -311,9 +307,8 @@ public class Http2MultiplexCodecTest {
|
||||
|
||||
// Read from the child channel
|
||||
headers = new DefaultHttp2Headers().scheme("https").status("200");
|
||||
parentChannel.pipeline().fireChannelRead(
|
||||
new DefaultHttp2HeadersFrame(headers).stream(childChannel.stream()));
|
||||
parentChannel.pipeline().fireChannelReadComplete();
|
||||
codec.onHttp2Frame(new DefaultHttp2HeadersFrame(headers).stream(childChannel.stream()));
|
||||
codec.onChannelReadComplete();
|
||||
|
||||
Http2HeadersFrame headersFrame = inboundHandler.readInbound();
|
||||
assertNotNull(headersFrame);
|
||||
@ -332,182 +327,89 @@ public class Http2MultiplexCodecTest {
|
||||
assertFalse(inboundHandler.isChannelActive());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test failing the promise of the first headers frame of an outbound stream. In practice this error case would most
|
||||
* likely happen due to the max concurrent streams limit being hit or the channel running out of stream identifiers.
|
||||
*/
|
||||
// Test failing the promise of the first headers frame of an outbound stream. In practice this error case would most
|
||||
// likely happen due to the max concurrent streams limit being hit or the channel running out of stream identifiers.
|
||||
//
|
||||
@Test(expected = Http2NoMoreStreamIdsException.class)
|
||||
public void failedOutboundStreamCreationThrowsAndClosesChannel() throws Exception {
|
||||
parentChannel.pipeline().addFirst(new ChannelOutboundHandlerAdapter() {
|
||||
writer = new Writer() {
|
||||
@Override
|
||||
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) {
|
||||
void write(Object msg, ChannelPromise promise) {
|
||||
promise.tryFailure(new Http2NoMoreStreamIdsException());
|
||||
}
|
||||
});
|
||||
|
||||
};
|
||||
LastInboundHandler inboundHandler = new LastInboundHandler();
|
||||
childChannelInitializer.handler = inboundHandler;
|
||||
|
||||
Http2StreamChannelBootstrap b = new Http2StreamChannelBootstrap();
|
||||
Channel childChannel = b.parentChannel(parentChannel).handler(childChannelInitializer).connect().channel();
|
||||
Channel childChannel = newOutboundStream();
|
||||
assertTrue(childChannel.isActive());
|
||||
|
||||
childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers()));
|
||||
ChannelFuture future = childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers()));
|
||||
parentChannel.flush();
|
||||
|
||||
assertFalse(childChannel.isActive());
|
||||
assertFalse(childChannel.isOpen());
|
||||
|
||||
inboundHandler.checkException();
|
||||
|
||||
future.syncUninterruptibly();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void settingChannelOptsAndAttrsOnBootstrap() {
|
||||
public void settingChannelOptsAndAttrs() {
|
||||
AttributeKey<String> key = AttributeKey.newInstance("foo");
|
||||
Http2StreamChannelBootstrap b = new Http2StreamChannelBootstrap();
|
||||
b.parentChannel(parentChannel).handler(childChannelInitializer)
|
||||
.option(ChannelOption.AUTO_READ, false).option(ChannelOption.WRITE_SPIN_COUNT, 1000)
|
||||
.attr(key, "bar");
|
||||
|
||||
Channel channel = b.connect().channel();
|
||||
|
||||
assertFalse(channel.config().isAutoRead());
|
||||
assertEquals(1000, channel.config().getWriteSpinCount());
|
||||
assertEquals("bar", channel.attr(key).get());
|
||||
Channel childChannel = newOutboundStream();
|
||||
childChannel.config().setAutoRead(false).setWriteSpinCount(1000);
|
||||
childChannel.attr(key).set("bar");
|
||||
assertFalse(childChannel.config().isAutoRead());
|
||||
assertEquals(1000, childChannel.config().getWriteSpinCount());
|
||||
assertEquals("bar", childChannel.attr(key).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void outboundStreamShouldWriteGoAwayWithoutReset() {
|
||||
childChannelInitializer.handler = new ChannelInboundHandlerAdapter() {
|
||||
@Override
|
||||
public void channelActive(ChannelHandlerContext ctx) throws Exception {
|
||||
ctx.writeAndFlush(new DefaultHttp2GoAwayFrame(Http2Error.NO_ERROR));
|
||||
ctx.fireChannelActive();
|
||||
}
|
||||
};
|
||||
|
||||
Http2StreamChannelBootstrap b = new Http2StreamChannelBootstrap();
|
||||
b.parentChannel(parentChannel).handler(childChannelInitializer);
|
||||
Channel childChannel = b.connect().channel();
|
||||
public void outboundFlowControlWritability() {
|
||||
Http2StreamChannel childChannel = newOutboundStream();
|
||||
assertTrue(childChannel.isActive());
|
||||
|
||||
Http2GoAwayFrame goAwayFrame = parentChannel.readOutbound();
|
||||
assertNotNull(goAwayFrame);
|
||||
goAwayFrame.release();
|
||||
|
||||
childChannel.close();
|
||||
parentChannel.runPendingTasks();
|
||||
|
||||
Http2ResetFrame reset = parentChannel.readOutbound();
|
||||
assertNull(reset);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void outboundFlowControlWindowShouldBeSetAndUpdated() {
|
||||
Http2StreamChannelBootstrap b = new Http2StreamChannelBootstrap();
|
||||
b.parentChannel(parentChannel).handler(childChannelInitializer);
|
||||
AbstractHttp2StreamChannel childChannel = (AbstractHttp2StreamChannel) b.connect().channel();
|
||||
assertTrue(childChannel.isActive());
|
||||
|
||||
assertEquals(0, childChannel.getOutboundFlowControlWindow());
|
||||
assertFalse(childChannel.isWritable());
|
||||
childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers()));
|
||||
parentChannel.flush();
|
||||
|
||||
Http2Stream2 stream2 = readOutboundHeadersAndAssignId();
|
||||
Http2FrameStream stream = readOutboundHeadersAndAssignId();
|
||||
|
||||
// Test for initial window size
|
||||
assertEquals(initialRemoteStreamWindow, childChannel.getOutboundFlowControlWindow());
|
||||
assertEquals(initialRemoteStreamWindow, childChannel.config().getWriteBufferHighWaterMark());
|
||||
|
||||
// Test for increment via WINDOW_UPDATE
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2WindowUpdateFrame(1).stream(stream2));
|
||||
parentChannel.pipeline().fireChannelReadComplete();
|
||||
|
||||
assertEquals(initialRemoteStreamWindow + 1, childChannel.getOutboundFlowControlWindow());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void onlyDataFramesShouldBeFlowControlled() {
|
||||
LastInboundHandler inboundHandler = streamActiveAndWriteHeaders(inboundStream);
|
||||
AbstractHttp2StreamChannel childChannel = (AbstractHttp2StreamChannel) inboundHandler.channel();
|
||||
codec.onHttp2StreamWritabilityChanged(stream, true);
|
||||
assertTrue(childChannel.isWritable());
|
||||
assertEquals(initialRemoteStreamWindow, childChannel.getOutboundFlowControlWindow());
|
||||
|
||||
childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers()));
|
||||
assertTrue(childChannel.isWritable());
|
||||
assertEquals(initialRemoteStreamWindow, childChannel.getOutboundFlowControlWindow());
|
||||
|
||||
ByteBuf data = Unpooled.buffer(100).writeZero(100);
|
||||
childChannel.writeAndFlush(new DefaultHttp2DataFrame(data));
|
||||
assertTrue(childChannel.isWritable());
|
||||
assertEquals(initialRemoteStreamWindow - 100, childChannel.getOutboundFlowControlWindow());
|
||||
codec.onHttp2StreamWritabilityChanged(stream, false);
|
||||
assertFalse(childChannel.isWritable());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void writabilityAndFlowControl() {
|
||||
LastInboundHandler inboundHandler = streamActiveAndWriteHeaders(inboundStream);
|
||||
AbstractHttp2StreamChannel childChannel = (AbstractHttp2StreamChannel) inboundHandler.channel();
|
||||
verifyFlowControlWindowAndWritability(childChannel, initialRemoteStreamWindow);
|
||||
assertEquals("true", inboundHandler.writabilityStates());
|
||||
Http2StreamChannel childChannel = (Http2StreamChannel) inboundHandler.channel();
|
||||
assertEquals("", inboundHandler.writabilityStates());
|
||||
|
||||
// HEADERS frames are not flow controlled, so they should not affect the flow control window.
|
||||
childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers()));
|
||||
verifyFlowControlWindowAndWritability(childChannel, initialRemoteStreamWindow);
|
||||
codec.onHttp2StreamWritabilityChanged(childChannel.stream(), true);
|
||||
|
||||
assertEquals("true", inboundHandler.writabilityStates());
|
||||
|
||||
ByteBuf data = Unpooled.buffer(initialRemoteStreamWindow - 1).writeZero(initialRemoteStreamWindow - 1);
|
||||
childChannel.writeAndFlush(new DefaultHttp2DataFrame(data));
|
||||
verifyFlowControlWindowAndWritability(childChannel, 1);
|
||||
assertEquals("true,false,true", inboundHandler.writabilityStates());
|
||||
|
||||
ByteBuf data1 = Unpooled.buffer(100).writeZero(100);
|
||||
childChannel.writeAndFlush(new DefaultHttp2DataFrame(data1));
|
||||
verifyFlowControlWindowAndWritability(childChannel, -99);
|
||||
assertEquals("true,false,true,false", inboundHandler.writabilityStates());
|
||||
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2WindowUpdateFrame(99).stream(inboundStream));
|
||||
parentChannel.pipeline().fireChannelReadComplete();
|
||||
// the flow control window should be updated, but the channel should still not be writable.
|
||||
verifyFlowControlWindowAndWritability(childChannel, 0);
|
||||
assertEquals("true,false,true,false", inboundHandler.writabilityStates());
|
||||
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2WindowUpdateFrame(1).stream(inboundStream));
|
||||
parentChannel.pipeline().fireChannelReadComplete();
|
||||
verifyFlowControlWindowAndWritability(childChannel, 1);
|
||||
assertEquals("true,false,true,false,true", inboundHandler.writabilityStates());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void failedWriteShouldReturnFlowControlWindow() {
|
||||
ByteBuf data = Unpooled.buffer().writeZero(initialRemoteStreamWindow);
|
||||
final Http2DataFrame frameToCancel = new DefaultHttp2DataFrame(data);
|
||||
parentChannel.pipeline().addFirst(new ChannelOutboundHandlerAdapter() {
|
||||
@Override
|
||||
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception {
|
||||
if (msg == frameToCancel) {
|
||||
promise.tryFailure(new Throwable());
|
||||
} else {
|
||||
super.write(ctx, msg, promise);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
LastInboundHandler inboundHandler = streamActiveAndWriteHeaders(inboundStream);
|
||||
Channel childChannel = inboundHandler.channel();
|
||||
|
||||
childChannel.write(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers()));
|
||||
data = Unpooled.buffer().writeZero(initialRemoteStreamWindow / 2);
|
||||
childChannel.write(new DefaultHttp2DataFrame(data));
|
||||
codec.onHttp2StreamWritabilityChanged(childChannel.stream(), true);
|
||||
assertEquals("true", inboundHandler.writabilityStates());
|
||||
|
||||
childChannel.write(frameToCancel);
|
||||
codec.onHttp2StreamWritabilityChanged(childChannel.stream(), false);
|
||||
assertEquals("true,false", inboundHandler.writabilityStates());
|
||||
assertFalse(childChannel.isWritable());
|
||||
childChannel.flush();
|
||||
|
||||
assertTrue(childChannel.isWritable());
|
||||
assertEquals("true,false,true", inboundHandler.writabilityStates());
|
||||
codec.onHttp2StreamWritabilityChanged(childChannel.stream(), false);
|
||||
assertEquals("true,false", inboundHandler.writabilityStates());
|
||||
}
|
||||
|
||||
@Ignore("not supported anymore atm")
|
||||
@Test
|
||||
public void cancellingWritesBeforeFlush() {
|
||||
LastInboundHandler inboundHandler = streamActiveAndWriteHeaders(inboundStream);
|
||||
@ -525,27 +427,20 @@ public class Http2MultiplexCodecTest {
|
||||
assertSame(headers, headers2);
|
||||
}
|
||||
|
||||
private static void verifyFlowControlWindowAndWritability(AbstractHttp2StreamChannel channel,
|
||||
int expectedWindowSize) {
|
||||
assertEquals(expectedWindowSize, channel.getOutboundFlowControlWindow());
|
||||
assertEquals(Math.max(0, expectedWindowSize), channel.config().getWriteBufferHighWaterMark());
|
||||
assertEquals(channel.config().getWriteBufferHighWaterMark(), channel.config().getWriteBufferLowWaterMark());
|
||||
assertEquals(expectedWindowSize > 0, channel.isWritable());
|
||||
}
|
||||
|
||||
private LastInboundHandler streamActiveAndWriteHeaders(Http2Stream2 stream) {
|
||||
private LastInboundHandler streamActiveAndWriteHeaders(Http2FrameStream stream) {
|
||||
LastInboundHandler inboundHandler = new LastInboundHandler();
|
||||
childChannelInitializer.handler = inboundHandler;
|
||||
assertFalse(inboundHandler.isChannelActive());
|
||||
|
||||
parentChannel.pipeline().fireChannelRead(new DefaultHttp2HeadersFrame(request).stream(stream));
|
||||
parentChannel.pipeline().fireChannelReadComplete();
|
||||
((TestableHttp2MultiplexCodec.Stream) stream).state = Http2Stream.State.OPEN;
|
||||
codec.onHttp2StreamStateChanged(stream);
|
||||
codec.onHttp2Frame(new DefaultHttp2HeadersFrame(request).stream(stream));
|
||||
codec.onChannelReadComplete();
|
||||
assertTrue(inboundHandler.isChannelActive());
|
||||
|
||||
return inboundHandler;
|
||||
}
|
||||
|
||||
private static void verifyFramesMultiplexedToCorrectChannel(Http2Stream2 stream,
|
||||
private static void verifyFramesMultiplexedToCorrectChannel(Http2FrameStream stream,
|
||||
LastInboundHandler inboundHandler,
|
||||
int numFrames) {
|
||||
for (int i = 0; i < numFrames; i++) {
|
||||
@ -564,14 +459,14 @@ public class Http2MultiplexCodecTest {
|
||||
/**
|
||||
* Simulates the frame codec, in first assigning an identifier and the completing the write promise.
|
||||
*/
|
||||
Http2Stream2 readOutboundHeadersAndAssignId() {
|
||||
private Http2FrameStream readOutboundHeadersAndAssignId() {
|
||||
// Only peek at the frame, so to not complete the promise of the write. We need to first
|
||||
// assign a stream identifier, as the frame codec would do.
|
||||
Http2HeadersFrame headersFrame = (Http2HeadersFrame) parentChannel.outboundMessages().peek();
|
||||
assertNotNull(headersFrame);
|
||||
assertNotNull(headersFrame.stream());
|
||||
assertFalse(Http2CodecUtil.isStreamIdValid(headersFrame.stream().id()));
|
||||
headersFrame.stream().id(outboundStream.id());
|
||||
((TestableHttp2MultiplexCodec.Stream) headersFrame.stream()).id = outboundStream.id();
|
||||
|
||||
// Now read it and complete the write promise.
|
||||
assertSame(headersFrame, parentChannel.readOutbound());
|
||||
@ -580,71 +475,95 @@ public class Http2MultiplexCodecTest {
|
||||
}
|
||||
|
||||
/**
|
||||
* This class removes the bits that would require the frame codec, so that the class becomes testable.
|
||||
* This class removes the bits that would transform the frames to bytes and so make it easier to test the actual
|
||||
* special handling of the codec.
|
||||
*/
|
||||
static final class TestableHttp2MultiplexCodec extends Http2MultiplexCodec {
|
||||
private final class TestableHttp2MultiplexCodec extends Http2MultiplexCodec {
|
||||
|
||||
TestableHttp2MultiplexCodec(boolean server, Http2StreamChannelBootstrap bootstrap) {
|
||||
super(server, bootstrap);
|
||||
public TestableHttp2MultiplexCodec(Http2ConnectionEncoder encoder, Http2ConnectionDecoder decoder,
|
||||
Http2Settings initialSettings, ChannelHandler inboundStreamHandler) {
|
||||
super(encoder, decoder, initialSettings, inboundStreamHandler);
|
||||
}
|
||||
|
||||
void onHttp2Frame(Http2Frame frame) {
|
||||
onHttp2Frame(ctx, frame);
|
||||
}
|
||||
|
||||
void onChannelReadComplete() {
|
||||
onChannelReadComplete(ctx);
|
||||
}
|
||||
|
||||
void onHttp2StreamStateChanged(Http2FrameStream stream) {
|
||||
onHttp2StreamStateChanged(ctx, stream);
|
||||
}
|
||||
|
||||
void onHttp2FrameStreamException(Http2FrameStreamException cause) {
|
||||
onHttp2FrameStreamException(ctx, cause);
|
||||
}
|
||||
|
||||
void onHttp2StreamWritabilityChanged(Http2FrameStream stream, boolean writable) {
|
||||
onHttp2StreamWritabilityChanged(ctx, stream, writable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
this.ctx = ctx;
|
||||
bootstrap.parentChannel(ctx.channel());
|
||||
void onBytesConsumed(ChannelHandlerContext ctx, Http2FrameStream stream, int bytes) {
|
||||
writer.write(new DefaultHttp2WindowUpdateFrame(bytes).stream(stream), ctx.newPromise());
|
||||
}
|
||||
|
||||
@Override
|
||||
void forEachActiveStream0(Http2Stream2Visitor streamVisitor) {
|
||||
throw new UnsupportedOperationException();
|
||||
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) {
|
||||
writer.write(msg, promise);
|
||||
}
|
||||
|
||||
@Override
|
||||
Http2Stream2 newStream0() {
|
||||
return new Http2Stream2Impl(ctx.channel());
|
||||
void flush0(ChannelHandlerContext ctx) {
|
||||
// Do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
Stream newStream() {
|
||||
return new Stream();
|
||||
}
|
||||
|
||||
final class Stream extends Http2MultiplexCodecStream {
|
||||
Http2Stream.State state = Http2Stream.State.IDLE;
|
||||
int id = -1;
|
||||
|
||||
@Override
|
||||
public int id() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream.State state() {
|
||||
return state;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static final class Http2Stream2Impl implements Http2Stream2 {
|
||||
|
||||
private int id = -1;
|
||||
private Object managedState;
|
||||
private final ChannelPromise closeFuture;
|
||||
|
||||
Http2Stream2Impl(Channel ch) {
|
||||
closeFuture = new DefaultChannelPromise(ch);
|
||||
private final class TestableHttp2MultiplexCodecBuilder extends Http2MultiplexCodecBuilder {
|
||||
TestableHttp2MultiplexCodecBuilder(boolean server, ChannelHandler childHandler) {
|
||||
super(server, childHandler);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream2 id(int id) {
|
||||
this.id = id;
|
||||
return this;
|
||||
public TestableHttp2MultiplexCodec build() {
|
||||
return (TestableHttp2MultiplexCodec) super.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int id() {
|
||||
return id;
|
||||
protected Http2MultiplexCodec build(
|
||||
Http2ConnectionDecoder decoder, Http2ConnectionEncoder encoder, Http2Settings initialSettings) {
|
||||
return new TestableHttp2MultiplexCodec(
|
||||
encoder, decoder, initialSettings, childHandler);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream2 managedState(Object state) {
|
||||
managedState = state;
|
||||
return this;
|
||||
}
|
||||
class Writer {
|
||||
|
||||
@Override
|
||||
public Object managedState() {
|
||||
return managedState;
|
||||
}
|
||||
|
||||
@Override
|
||||
public State state() {
|
||||
return State.OPEN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture closeFuture() {
|
||||
return closeFuture;
|
||||
void write(Object msg, ChannelPromise promise) {
|
||||
parentChannel.outboundMessages().add(msg);
|
||||
promise.setSuccess();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ public class Http2ServerDowngraderTest {
|
||||
|
||||
Http2HeadersFrame headersFrame = ch.readOutbound();
|
||||
assertThat(headersFrame.headers().status().toString(), is("200"));
|
||||
assertTrue(headersFrame.endStream());
|
||||
assertTrue(headersFrame.isEndStream());
|
||||
|
||||
assertThat(ch.readOutbound(), is(nullValue()));
|
||||
assertFalse(ch.finish());
|
||||
@ -68,12 +68,12 @@ public class Http2ServerDowngraderTest {
|
||||
|
||||
Http2HeadersFrame headersFrame = ch.readOutbound();
|
||||
assertThat(headersFrame.headers().status().toString(), is("200"));
|
||||
assertFalse(headersFrame.endStream());
|
||||
assertFalse(headersFrame.isEndStream());
|
||||
|
||||
Http2DataFrame dataFrame = ch.readOutbound();
|
||||
try {
|
||||
assertThat(dataFrame.content().toString(CharsetUtil.UTF_8), is("hello world"));
|
||||
assertTrue(dataFrame.endStream());
|
||||
assertTrue(dataFrame.isEndStream());
|
||||
} finally {
|
||||
dataFrame.release();
|
||||
}
|
||||
@ -92,11 +92,11 @@ public class Http2ServerDowngraderTest {
|
||||
|
||||
Http2HeadersFrame headersFrame = ch.readOutbound();
|
||||
assertThat(headersFrame.headers().status().toString(), is("200"));
|
||||
assertFalse(headersFrame.endStream());
|
||||
assertFalse(headersFrame.isEndStream());
|
||||
|
||||
Http2HeadersFrame trailersFrame = ch.readOutbound();
|
||||
assertThat(trailersFrame.headers().get("key").toString(), is("value"));
|
||||
assertTrue(trailersFrame.endStream());
|
||||
assertTrue(trailersFrame.isEndStream());
|
||||
|
||||
assertThat(ch.readOutbound(), is(nullValue()));
|
||||
assertFalse(ch.finish());
|
||||
@ -113,19 +113,19 @@ public class Http2ServerDowngraderTest {
|
||||
|
||||
Http2HeadersFrame headersFrame = ch.readOutbound();
|
||||
assertThat(headersFrame.headers().status().toString(), is("200"));
|
||||
assertFalse(headersFrame.endStream());
|
||||
assertFalse(headersFrame.isEndStream());
|
||||
|
||||
Http2DataFrame dataFrame = ch.readOutbound();
|
||||
try {
|
||||
assertThat(dataFrame.content().toString(CharsetUtil.UTF_8), is("hello world"));
|
||||
assertFalse(dataFrame.endStream());
|
||||
assertFalse(dataFrame.isEndStream());
|
||||
} finally {
|
||||
dataFrame.release();
|
||||
}
|
||||
|
||||
Http2HeadersFrame trailersFrame = ch.readOutbound();
|
||||
assertThat(trailersFrame.headers().get("key").toString(), is("value"));
|
||||
assertTrue(trailersFrame.endStream());
|
||||
assertTrue(trailersFrame.isEndStream());
|
||||
|
||||
assertThat(ch.readOutbound(), is(nullValue()));
|
||||
assertFalse(ch.finish());
|
||||
@ -139,7 +139,7 @@ public class Http2ServerDowngraderTest {
|
||||
|
||||
Http2HeadersFrame headersFrame = ch.readOutbound();
|
||||
assertThat(headersFrame.headers().status().toString(), is("200"));
|
||||
assertFalse(headersFrame.endStream());
|
||||
assertFalse(headersFrame.isEndStream());
|
||||
|
||||
assertThat(ch.readOutbound(), is(nullValue()));
|
||||
assertFalse(ch.finish());
|
||||
@ -155,7 +155,7 @@ public class Http2ServerDowngraderTest {
|
||||
Http2DataFrame dataFrame = ch.readOutbound();
|
||||
try {
|
||||
assertThat(dataFrame.content().toString(CharsetUtil.UTF_8), is("hello world"));
|
||||
assertFalse(dataFrame.endStream());
|
||||
assertFalse(dataFrame.isEndStream());
|
||||
} finally {
|
||||
dataFrame.release();
|
||||
}
|
||||
@ -173,7 +173,7 @@ public class Http2ServerDowngraderTest {
|
||||
Http2DataFrame emptyFrame = ch.readOutbound();
|
||||
try {
|
||||
assertThat(emptyFrame.content().readableBytes(), is(0));
|
||||
assertTrue(emptyFrame.endStream());
|
||||
assertTrue(emptyFrame.isEndStream());
|
||||
} finally {
|
||||
emptyFrame.release();
|
||||
}
|
||||
@ -192,7 +192,7 @@ public class Http2ServerDowngraderTest {
|
||||
Http2DataFrame dataFrame = ch.readOutbound();
|
||||
try {
|
||||
assertThat(dataFrame.content().toString(CharsetUtil.UTF_8), is("hello world"));
|
||||
assertTrue(dataFrame.endStream());
|
||||
assertTrue(dataFrame.isEndStream());
|
||||
} finally {
|
||||
dataFrame.release();
|
||||
}
|
||||
@ -211,7 +211,7 @@ public class Http2ServerDowngraderTest {
|
||||
|
||||
Http2HeadersFrame headerFrame = ch.readOutbound();
|
||||
assertThat(headerFrame.headers().get("key").toString(), is("value"));
|
||||
assertTrue(headerFrame.endStream());
|
||||
assertTrue(headerFrame.isEndStream());
|
||||
|
||||
assertThat(ch.readOutbound(), is(nullValue()));
|
||||
assertFalse(ch.finish());
|
||||
@ -229,14 +229,14 @@ public class Http2ServerDowngraderTest {
|
||||
Http2DataFrame dataFrame = ch.readOutbound();
|
||||
try {
|
||||
assertThat(dataFrame.content().toString(CharsetUtil.UTF_8), is("hello world"));
|
||||
assertFalse(dataFrame.endStream());
|
||||
assertFalse(dataFrame.isEndStream());
|
||||
} finally {
|
||||
dataFrame.release();
|
||||
}
|
||||
|
||||
Http2HeadersFrame headerFrame = ch.readOutbound();
|
||||
assertThat(headerFrame.headers().get("key").toString(), is("value"));
|
||||
assertTrue(headerFrame.endStream());
|
||||
assertTrue(headerFrame.isEndStream());
|
||||
|
||||
assertThat(ch.readOutbound(), is(nullValue()));
|
||||
assertFalse(ch.finish());
|
||||
|
@ -25,6 +25,7 @@ import io.netty.handler.codec.http2.DefaultHttp2Headers;
|
||||
import io.netty.handler.codec.http2.DefaultHttp2HeadersFrame;
|
||||
import io.netty.handler.codec.http2.DefaultHttp2WindowUpdateFrame;
|
||||
import io.netty.handler.codec.http2.Http2DataFrame;
|
||||
import io.netty.handler.codec.http2.Http2FrameStream;
|
||||
import io.netty.handler.codec.http2.Http2Headers;
|
||||
import io.netty.handler.codec.http2.Http2HeadersFrame;
|
||||
import io.netty.util.CharsetUtil;
|
||||
@ -70,18 +71,17 @@ public class HelloWorldHttp2Handler extends ChannelDuplexHandler {
|
||||
* If receive a frame with end-of-stream set, send a pre-canned response.
|
||||
*/
|
||||
private static void onDataRead(ChannelHandlerContext ctx, Http2DataFrame data) throws Exception {
|
||||
int consumed = data.padding() + data.content().readableBytes();
|
||||
int streamId = data.streamId();
|
||||
Http2FrameStream stream = data.stream();
|
||||
|
||||
if (data.isEndStream()) {
|
||||
sendResponse(ctx, streamId, data.content());
|
||||
sendResponse(ctx, stream, data.content());
|
||||
} else {
|
||||
// We do not send back the response to the remote-peer, so we need to release it.
|
||||
data.release();
|
||||
}
|
||||
|
||||
// Update the flowcontroller
|
||||
ctx.write(new DefaultHttp2WindowUpdateFrame(consumed).streamId(streamId));
|
||||
ctx.write(new DefaultHttp2WindowUpdateFrame(data.initialFlowControlledBytes()).stream(stream));
|
||||
}
|
||||
|
||||
/**
|
||||
@ -93,17 +93,17 @@ public class HelloWorldHttp2Handler extends ChannelDuplexHandler {
|
||||
ByteBuf content = ctx.alloc().buffer();
|
||||
content.writeBytes(RESPONSE_BYTES.duplicate());
|
||||
ByteBufUtil.writeAscii(content, " - via HTTP/2");
|
||||
sendResponse(ctx, headers.streamId(), content);
|
||||
sendResponse(ctx, headers.stream(), content);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends a "Hello World" DATA frame to the client.
|
||||
*/
|
||||
private static void sendResponse(ChannelHandlerContext ctx, int streamId, ByteBuf payload) {
|
||||
private static void sendResponse(ChannelHandlerContext ctx, Http2FrameStream stream, ByteBuf payload) {
|
||||
// Send a frame for the response status
|
||||
Http2Headers headers = new DefaultHttp2Headers().status(OK.codeAsText());
|
||||
ctx.write(new DefaultHttp2HeadersFrame(headers).streamId(streamId));
|
||||
ctx.write(new DefaultHttp2DataFrame(payload, true).streamId(streamId));
|
||||
ctx.write(new DefaultHttp2HeadersFrame(headers).stream(stream));
|
||||
ctx.write(new DefaultHttp2DataFrame(payload, true).stream(stream));
|
||||
}
|
||||
}
|
||||
|
@ -19,6 +19,7 @@ import io.netty.example.http2.helloworld.server.HelloWorldHttp1Handler;
|
||||
import io.netty.handler.codec.http.HttpObjectAggregator;
|
||||
import io.netty.handler.codec.http.HttpServerCodec;
|
||||
import io.netty.handler.codec.http2.Http2FrameCodec;
|
||||
import io.netty.handler.codec.http2.Http2FrameCodecBuilder;
|
||||
import io.netty.handler.ssl.ApplicationProtocolNames;
|
||||
import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler;
|
||||
|
||||
@ -37,7 +38,7 @@ public class Http2OrHttpHandler extends ApplicationProtocolNegotiationHandler {
|
||||
@Override
|
||||
protected void configurePipeline(ChannelHandlerContext ctx, String protocol) throws Exception {
|
||||
if (ApplicationProtocolNames.HTTP_2.equals(protocol)) {
|
||||
ctx.pipeline().addLast(new Http2FrameCodec(true), new HelloWorldHttp2Handler());
|
||||
ctx.pipeline().addLast(Http2FrameCodecBuilder.forServer().build(), new HelloWorldHttp2Handler());
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -30,7 +30,7 @@ import io.netty.handler.codec.http.HttpServerUpgradeHandler;
|
||||
import io.netty.handler.codec.http.HttpServerUpgradeHandler.UpgradeCodec;
|
||||
import io.netty.handler.codec.http.HttpServerUpgradeHandler.UpgradeCodecFactory;
|
||||
import io.netty.handler.codec.http2.Http2CodecUtil;
|
||||
import io.netty.handler.codec.http2.Http2FrameCodec;
|
||||
import io.netty.handler.codec.http2.Http2FrameCodecBuilder;
|
||||
import io.netty.handler.codec.http2.Http2ServerUpgradeCodec;
|
||||
import io.netty.handler.ssl.SslContext;
|
||||
import io.netty.util.AsciiString;
|
||||
@ -46,7 +46,8 @@ public class Http2ServerInitializer extends ChannelInitializer<SocketChannel> {
|
||||
@Override
|
||||
public UpgradeCodec newUpgradeCodec(CharSequence protocol) {
|
||||
if (AsciiString.contentEquals(Http2CodecUtil.HTTP_UPGRADE_PROTOCOL_NAME, protocol)) {
|
||||
return new Http2ServerUpgradeCodec(new Http2FrameCodec(true), new HelloWorldHttp2Handler());
|
||||
return new Http2ServerUpgradeCodec(
|
||||
Http2FrameCodecBuilder.forServer().build(), new HelloWorldHttp2Handler());
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
@ -69,7 +69,7 @@ public class HelloWorldHttp2Handler extends ChannelDuplexHandler {
|
||||
* If receive a frame with end-of-stream set, send a pre-canned response.
|
||||
*/
|
||||
private static void onDataRead(ChannelHandlerContext ctx, Http2DataFrame data) throws Exception {
|
||||
if (data.endStream()) {
|
||||
if (data.isEndStream()) {
|
||||
sendResponse(ctx, data.content());
|
||||
} else {
|
||||
// We do not send back the response to the remote-peer, so we need to release it.
|
||||
@ -82,7 +82,7 @@ public class HelloWorldHttp2Handler extends ChannelDuplexHandler {
|
||||
*/
|
||||
private static void onHeadersRead(ChannelHandlerContext ctx, Http2HeadersFrame headers)
|
||||
throws Exception {
|
||||
if (headers.endStream()) {
|
||||
if (headers.isEndStream()) {
|
||||
ByteBuf content = ctx.alloc().buffer();
|
||||
content.writeBytes(RESPONSE_BYTES.duplicate());
|
||||
ByteBufUtil.writeAscii(content, " - via HTTP/2");
|
||||
|
@ -18,7 +18,7 @@ import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.example.http2.helloworld.server.HelloWorldHttp1Handler;
|
||||
import io.netty.handler.codec.http.HttpObjectAggregator;
|
||||
import io.netty.handler.codec.http.HttpServerCodec;
|
||||
import io.netty.handler.codec.http2.Http2CodecBuilder;
|
||||
import io.netty.handler.codec.http2.Http2MultiplexCodecBuilder;
|
||||
import io.netty.handler.ssl.ApplicationProtocolNames;
|
||||
import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler;
|
||||
|
||||
@ -37,7 +37,7 @@ public class Http2OrHttpHandler extends ApplicationProtocolNegotiationHandler {
|
||||
@Override
|
||||
protected void configurePipeline(ChannelHandlerContext ctx, String protocol) throws Exception {
|
||||
if (ApplicationProtocolNames.HTTP_2.equals(protocol)) {
|
||||
ctx.pipeline().addLast(new Http2CodecBuilder(true, new HelloWorldHttp2Handler()).build());
|
||||
ctx.pipeline().addLast(Http2MultiplexCodecBuilder.forServer(new HelloWorldHttp2Handler()).build());
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -29,7 +29,7 @@ import io.netty.handler.codec.http.HttpServerCodec;
|
||||
import io.netty.handler.codec.http.HttpServerUpgradeHandler;
|
||||
import io.netty.handler.codec.http.HttpServerUpgradeHandler.UpgradeCodec;
|
||||
import io.netty.handler.codec.http.HttpServerUpgradeHandler.UpgradeCodecFactory;
|
||||
import io.netty.handler.codec.http2.Http2CodecBuilder;
|
||||
import io.netty.handler.codec.http2.Http2MultiplexCodecBuilder;
|
||||
import io.netty.handler.codec.http2.Http2CodecUtil;
|
||||
import io.netty.handler.codec.http2.Http2ServerUpgradeCodec;
|
||||
import io.netty.handler.ssl.SslContext;
|
||||
@ -46,7 +46,8 @@ public class Http2ServerInitializer extends ChannelInitializer<SocketChannel> {
|
||||
@Override
|
||||
public UpgradeCodec newUpgradeCodec(CharSequence protocol) {
|
||||
if (AsciiString.contentEquals(Http2CodecUtil.HTTP_UPGRADE_PROTOCOL_NAME, protocol)) {
|
||||
return new Http2ServerUpgradeCodec(new Http2CodecBuilder(true, new HelloWorldHttp2Handler()).build());
|
||||
return new Http2ServerUpgradeCodec(
|
||||
Http2MultiplexCodecBuilder.forServer(new HelloWorldHttp2Handler()).build());
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
@ -1053,15 +1053,8 @@ abstract class AbstractChannelHandlerContext extends DefaultAttributeMap
|
||||
task.promise = promise;
|
||||
|
||||
if (ESTIMATE_TASK_SIZE_ON_SUBMIT) {
|
||||
ChannelOutboundBuffer buffer = ctx.channel().unsafe().outboundBuffer();
|
||||
|
||||
// Check for null as it may be set to null if the channel is closed already
|
||||
if (buffer != null) {
|
||||
task.size = ctx.pipeline.estimatorHandle().size(msg) + WRITE_TASK_OVERHEAD;
|
||||
buffer.incrementPendingOutboundBytes(task.size);
|
||||
} else {
|
||||
task.size = 0;
|
||||
}
|
||||
task.size = ctx.pipeline.estimatorHandle().size(msg) + WRITE_TASK_OVERHEAD;
|
||||
ctx.pipeline.incrementPendingOutboundBytes(task.size);
|
||||
} else {
|
||||
task.size = 0;
|
||||
}
|
||||
@ -1070,10 +1063,9 @@ abstract class AbstractChannelHandlerContext extends DefaultAttributeMap
|
||||
@Override
|
||||
public final void run() {
|
||||
try {
|
||||
ChannelOutboundBuffer buffer = ctx.channel().unsafe().outboundBuffer();
|
||||
// Check for null as it may be set to null if the channel is closed already
|
||||
if (ESTIMATE_TASK_SIZE_ON_SUBMIT && buffer != null) {
|
||||
buffer.decrementPendingOutboundBytes(size);
|
||||
if (ESTIMATE_TASK_SIZE_ON_SUBMIT) {
|
||||
ctx.pipeline.decrementPendingOutboundBytes(size);
|
||||
}
|
||||
write(ctx, msg, promise);
|
||||
} finally {
|
||||
|
@ -264,33 +264,6 @@ public final class ChannelOutboundBuffer {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the current flushed message and returns its {@link ChannelPromise}. Unlike {@link #remove()} this method
|
||||
* does not release the message or complete the promise. If no flushed message exist, this method returns
|
||||
* {@code null}.
|
||||
*/
|
||||
public ChannelPromise steal() {
|
||||
Entry e = flushedEntry;
|
||||
if (e == null) {
|
||||
clearNioBuffers();
|
||||
return null;
|
||||
}
|
||||
|
||||
ChannelPromise promise = e.promise;
|
||||
final int size = e.pendingSize;
|
||||
|
||||
removeEntry(e);
|
||||
|
||||
if (!e.cancelled && size > 0) {
|
||||
decrementPendingOutboundBytes(size, false, true);
|
||||
}
|
||||
|
||||
// recycle the entry
|
||||
e.recycle();
|
||||
|
||||
return promise;
|
||||
}
|
||||
|
||||
/**
|
||||
* Will remove the current message, mark its {@link ChannelPromise} as failure using the given {@link Throwable}
|
||||
* and return {@code true}. If no flushed message exists at the time this method is called it will return
|
||||
|
@ -23,6 +23,7 @@ import io.netty.util.concurrent.EventExecutorGroup;
|
||||
import io.netty.util.concurrent.FastThreadLocal;
|
||||
import io.netty.util.internal.ObjectUtil;
|
||||
import io.netty.util.internal.StringUtil;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
import io.netty.util.internal.logging.InternalLogger;
|
||||
import io.netty.util.internal.logging.InternalLoggerFactory;
|
||||
|
||||
@ -1178,6 +1179,22 @@ public class DefaultChannelPipeline implements ChannelPipeline {
|
||||
}
|
||||
}
|
||||
|
||||
@UnstableApi
|
||||
protected void incrementPendingOutboundBytes(long size) {
|
||||
ChannelOutboundBuffer buffer = channel.unsafe().outboundBuffer();
|
||||
if (buffer != null) {
|
||||
buffer.incrementPendingOutboundBytes(size);
|
||||
}
|
||||
}
|
||||
|
||||
@UnstableApi
|
||||
protected void decrementPendingOutboundBytes(long size) {
|
||||
ChannelOutboundBuffer buffer = channel.unsafe().outboundBuffer();
|
||||
if (buffer != null) {
|
||||
buffer.decrementPendingOutboundBytes(size);
|
||||
}
|
||||
}
|
||||
|
||||
// A special catch-all handler that handles both bytes and messages.
|
||||
final class TailContext extends AbstractChannelHandlerContext implements ChannelInboundHandler {
|
||||
|
||||
|
@ -18,6 +18,7 @@ package io.netty.channel;
|
||||
import io.netty.util.Recycler;
|
||||
import io.netty.util.ReferenceCountUtil;
|
||||
import io.netty.util.concurrent.PromiseCombiner;
|
||||
import io.netty.util.internal.ObjectUtil;
|
||||
import io.netty.util.internal.SystemPropertyUtil;
|
||||
import io.netty.util.internal.logging.InternalLogger;
|
||||
import io.netty.util.internal.logging.InternalLoggerFactory;
|
||||
@ -37,8 +38,7 @@ public final class PendingWriteQueue {
|
||||
SystemPropertyUtil.getInt("io.netty.transport.pendingWriteSizeOverhead", 64);
|
||||
|
||||
private final ChannelHandlerContext ctx;
|
||||
private final ChannelOutboundBuffer buffer;
|
||||
private final MessageSizeEstimator.Handle estimatorHandle;
|
||||
private final PendingTracker tracker;
|
||||
|
||||
// head and tail pointers for the linked-list structure. If empty head and tail are null.
|
||||
private PendingWrite head;
|
||||
@ -46,13 +46,80 @@ public final class PendingWriteQueue {
|
||||
private int size;
|
||||
private long bytes;
|
||||
|
||||
private interface PendingTracker extends MessageSizeEstimator.Handle {
|
||||
void incrementPendingOutboundBytes(long bytes);
|
||||
void decrementPendingOutboundBytes(long bytes);
|
||||
}
|
||||
|
||||
public PendingWriteQueue(ChannelHandlerContext ctx) {
|
||||
if (ctx == null) {
|
||||
throw new NullPointerException("ctx");
|
||||
this.ctx = ObjectUtil.checkNotNull(ctx, "ctx");
|
||||
if (ctx.pipeline() instanceof DefaultChannelPipeline) {
|
||||
final DefaultChannelPipeline pipeline = (DefaultChannelPipeline) ctx.pipeline();
|
||||
tracker = new PendingTracker() {
|
||||
@Override
|
||||
public void incrementPendingOutboundBytes(long bytes) {
|
||||
pipeline.incrementPendingOutboundBytes(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decrementPendingOutboundBytes(long bytes) {
|
||||
pipeline.decrementPendingOutboundBytes(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size(Object msg) {
|
||||
return pipeline.estimatorHandle().size(msg);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
final MessageSizeEstimator.Handle estimator = ctx.channel().config().getMessageSizeEstimator().newHandle();
|
||||
final ChannelOutboundBuffer buffer = ctx.channel().unsafe().outboundBuffer();
|
||||
if (buffer == null) {
|
||||
tracker = new PendingTracker() {
|
||||
@Override
|
||||
public void incrementPendingOutboundBytes(long bytes) {
|
||||
// noop
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decrementPendingOutboundBytes(long bytes) {
|
||||
// noop
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size(Object msg) {
|
||||
return estimator.size(msg);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
tracker = new PendingTracker() {
|
||||
@Override
|
||||
public void incrementPendingOutboundBytes(long bytes) {
|
||||
// We need to guard against null as channel.unsafe().outboundBuffer() may returned null
|
||||
// if the channel was already closed when constructing the PendingWriteQueue.
|
||||
// See https://github.com/netty/netty/issues/3967
|
||||
if (buffer != null) {
|
||||
buffer.incrementPendingOutboundBytes(bytes);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decrementPendingOutboundBytes(long bytes) {
|
||||
// We need to guard against null as channel.unsafe().outboundBuffer() may returned null
|
||||
// if the channel was already closed when constructing the PendingWriteQueue.
|
||||
// See https://github.com/netty/netty/issues/3967
|
||||
if (buffer != null) {
|
||||
buffer.decrementPendingOutboundBytes(bytes);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size(Object msg) {
|
||||
return estimator.size(msg);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
this.ctx = ctx;
|
||||
buffer = ctx.channel().unsafe().outboundBuffer();
|
||||
estimatorHandle = ctx.channel().config().getMessageSizeEstimator().newHandle();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -83,7 +150,7 @@ public final class PendingWriteQueue {
|
||||
private int size(Object msg) {
|
||||
// It is possible for writes to be triggered from removeAndFailAll(). To preserve ordering,
|
||||
// we should add them to the queue and let removeAndFailAll() fail them later.
|
||||
int messageSize = estimatorHandle.size(msg);
|
||||
int messageSize = tracker.size(msg);
|
||||
if (messageSize < 0) {
|
||||
// Size may be unknown so just use 0
|
||||
messageSize = 0;
|
||||
@ -116,12 +183,7 @@ public final class PendingWriteQueue {
|
||||
}
|
||||
size ++;
|
||||
bytes += messageSize;
|
||||
// We need to guard against null as channel.unsafe().outboundBuffer() may returned null
|
||||
// if the channel was already closed when constructing the PendingWriteQueue.
|
||||
// See https://github.com/netty/netty/issues/3967
|
||||
if (buffer != null) {
|
||||
buffer.incrementPendingOutboundBytes(write.size);
|
||||
}
|
||||
tracker.incrementPendingOutboundBytes(write.size);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -286,12 +348,7 @@ public final class PendingWriteQueue {
|
||||
}
|
||||
|
||||
write.recycle();
|
||||
// We need to guard against null as channel.unsafe().outboundBuffer() may returned null
|
||||
// if the channel was already closed when constructing the PendingWriteQueue.
|
||||
// See https://github.com/netty/netty/issues/3967
|
||||
if (buffer != null) {
|
||||
buffer.decrementPendingOutboundBytes(writeSize);
|
||||
}
|
||||
tracker.decrementPendingOutboundBytes(writeSize);
|
||||
}
|
||||
|
||||
private static void safeFail(ChannelPromise promise, Throwable cause) {
|
||||
|
@ -18,10 +18,12 @@ package io.netty.channel;
|
||||
import io.netty.util.concurrent.AbstractFuture;
|
||||
import io.netty.util.concurrent.Future;
|
||||
import io.netty.util.concurrent.GenericFutureListener;
|
||||
import io.netty.util.internal.UnstableApi;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
final class VoidChannelPromise extends AbstractFuture<Void> implements ChannelPromise {
|
||||
@UnstableApi
|
||||
public final class VoidChannelPromise extends AbstractFuture<Void> implements ChannelPromise {
|
||||
|
||||
private final Channel channel;
|
||||
private final boolean fireException;
|
||||
@ -31,7 +33,7 @@ final class VoidChannelPromise extends AbstractFuture<Void> implements ChannelPr
|
||||
*
|
||||
* @param channel the {@link Channel} associated with this future
|
||||
*/
|
||||
VoidChannelPromise(Channel channel, boolean fireException) {
|
||||
public VoidChannelPromise(Channel channel, boolean fireException) {
|
||||
if (channel == null) {
|
||||
throw new NullPointerException("channel");
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user