From 307efbe49c6c71664250e06200123c0ae3e1ec4a Mon Sep 17 00:00:00 2001 From: Norman Maurer Date: Mon, 24 Jun 2019 09:17:15 +0200 Subject: [PATCH] Split multiplexing from frame decoding to allow easier customization of frame processing and better seperation of responsibilities (#9239) Motivation: In the past we had the following class hierarchy: Http2ConnectionHandler --- Http2FrameCodec -- Http2MultiplexCodec This hierarchy makes it impossible to plug in any code that would like to act on Http2Frame and Http2StreamFrame which can be quite useful for various situations (like metrics, logging etc). Beside this it also made the implementtion very hacky. To allow easier maintainance and also allow more flexible costumizations we should split Http2MultiplexCodec and Http2FrameCode. Modifications: - Introduce Http2MultiplexHandler (which is a replacement for Http2MultiplexCodec when used together with Http2FrameCodec) - Mark Http2MultiplexCodecBuilder and Http2MultiplexCodec as deprecated. People should use Http2FrameCodecBuilder / Http2FrameCodec together with Http2MultiplexHandlder in the future - Adjust / Add tests - Adjust examples Result: More flexible usage possible and less hacky / coupled implementation for http2 multiplexing --- .../http2/AbstractHttp2StreamChannel.java | 1050 ++++++++++++++++ .../http2/Http2ChannelDuplexHandler.java | 12 + .../handler/codec/http2/Http2FrameCodec.java | 34 +- .../codec/http2/Http2MultiplexCodec.java | 1105 +---------------- .../http2/Http2MultiplexCodecBuilder.java | 3 + .../codec/http2/Http2MultiplexHandler.java | 395 ++++++ .../http2/Http2StreamChannelBootstrap.java | 15 +- .../Http2MultiplexClientUpgradeTest.java | 79 ++ .../Http2MultiplexCodecClientUpgradeTest.java | 67 +- .../codec/http2/Http2MultiplexCodecTest.java | 1021 +-------------- ...ttp2MultiplexHandlerClientUpgradeTest.java | 30 + .../http2/Http2MultiplexHandlerTest.java | 34 + .../codec/http2/Http2MultiplexTest.java | 1041 ++++++++++++++++ ....java => Http2MultiplexTransportTest.java} | 20 +- .../http2/Http2ServerUpgradeCodecTest.java | 23 +- .../multiplex/server/Http2OrHttpHandler.java | 6 +- .../server/Http2ServerInitializer.java | 6 +- 17 files changed, 2798 insertions(+), 2143 deletions(-) create mode 100644 codec-http2/src/main/java/io/netty/handler/codec/http2/AbstractHttp2StreamChannel.java create mode 100644 codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexHandler.java create mode 100644 codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexClientUpgradeTest.java create mode 100644 codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexHandlerClientUpgradeTest.java create mode 100644 codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexHandlerTest.java create mode 100644 codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexTest.java rename codec-http2/src/test/java/io/netty/handler/codec/http2/{Http2MultiplexCodecTransportTest.java => Http2MultiplexTransportTest.java} (87%) diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/AbstractHttp2StreamChannel.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/AbstractHttp2StreamChannel.java new file mode 100644 index 0000000000..cb67e277f2 --- /dev/null +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/AbstractHttp2StreamChannel.java @@ -0,0 +1,1050 @@ +/* + * Copyright 2019 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.ByteBufAllocator; +import io.netty.channel.Channel; +import io.netty.channel.ChannelConfig; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelId; +import io.netty.channel.ChannelMetadata; +import io.netty.channel.ChannelOutboundBuffer; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.ChannelProgressivePromise; +import io.netty.channel.ChannelPromise; +import io.netty.channel.DefaultChannelConfig; +import io.netty.channel.DefaultChannelPipeline; +import io.netty.channel.EventLoop; +import io.netty.channel.MessageSizeEstimator; +import io.netty.channel.RecvByteBufAllocator; +import io.netty.channel.VoidChannelPromise; +import io.netty.channel.WriteBufferWaterMark; +import io.netty.handler.codec.http2.Http2FrameCodec.DefaultHttp2FrameStream; +import io.netty.util.DefaultAttributeMap; +import io.netty.util.ReferenceCountUtil; +import io.netty.util.internal.StringUtil; +import io.netty.util.internal.logging.InternalLogger; +import io.netty.util.internal.logging.InternalLoggerFactory; + +import java.io.IOException; +import java.net.SocketAddress; +import java.nio.channels.ClosedChannelException; +import java.util.ArrayDeque; +import java.util.Queue; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; + +import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid; +import static java.lang.Math.min; + +abstract class AbstractHttp2StreamChannel extends DefaultAttributeMap implements Http2StreamChannel { + + static final Http2FrameStreamVisitor WRITABLE_VISITOR = new Http2FrameStreamVisitor() { + @Override + public boolean visit(Http2FrameStream stream) { + final AbstractHttp2StreamChannel childChannel = (AbstractHttp2StreamChannel) + ((DefaultHttp2FrameStream) stream).attachment; + childChannel.trySetWritable(); + return true; + } + }; + + private static final InternalLogger logger = InternalLoggerFactory.getInstance(AbstractHttp2StreamChannel.class); + + private static final ChannelMetadata METADATA = new ChannelMetadata(false, 16); + + /** + * Number of bytes to consider non-payload messages. 9 is arbitrary, but also the minimum size of an HTTP/2 frame. + * Primarily is non-zero. + */ + private static final int MIN_HTTP2_FRAME_SIZE = 9; + + /** + * Returns the flow-control size for DATA frames, and {@value MIN_HTTP2_FRAME_SIZE} for all other frames. + */ + private static final class FlowControlledFrameSizeEstimator implements MessageSizeEstimator { + + static final FlowControlledFrameSizeEstimator INSTANCE = new FlowControlledFrameSizeEstimator(); + + private static final Handle HANDLE_INSTANCE = new Handle() { + @Override + public int size(Object msg) { + return msg instanceof Http2DataFrame ? + // Guard against overflow. + (int) min(Integer.MAX_VALUE, ((Http2DataFrame) msg).initialFlowControlledBytes() + + (long) MIN_HTTP2_FRAME_SIZE) : MIN_HTTP2_FRAME_SIZE; + } + }; + + @Override + public Handle newHandle() { + return HANDLE_INSTANCE; + } + } + + private static final AtomicLongFieldUpdater TOTAL_PENDING_SIZE_UPDATER = + AtomicLongFieldUpdater.newUpdater(AbstractHttp2StreamChannel.class, "totalPendingSize"); + + private static final AtomicIntegerFieldUpdater UNWRITABLE_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(AbstractHttp2StreamChannel.class, "unwritable"); + + /** + * The current status of the read-processing for a {@link AbstractHttp2StreamChannel}. + */ + private enum ReadStatus { + /** + * No read in progress and no read was requested (yet) + */ + IDLE, + + /** + * Reading in progress + */ + IN_PROGRESS, + + /** + * A read operation was requested. + */ + REQUESTED + } + + private final AbstractHttp2StreamChannel.Http2StreamChannelConfig config = new Http2StreamChannelConfig(this); + private final AbstractHttp2StreamChannel.Http2ChannelUnsafe unsafe = new Http2ChannelUnsafe(); + private final ChannelId channelId; + private final ChannelPipeline pipeline; + private final DefaultHttp2FrameStream stream; + private final ChannelPromise closePromise; + + private volatile boolean registered; + + private volatile long totalPendingSize; + private volatile int unwritable; + + // Cached to reduce GC + private Runnable fireChannelWritabilityChangedTask; + + private boolean outboundClosed; + + /** + * This variable represents if a read is in progress for the current channel or was requested. + * Note that depending upon the {@link RecvByteBufAllocator} behavior a read may extend beyond the + * {@link Http2ChannelUnsafe#beginRead()} method scope. The {@link Http2ChannelUnsafe#beginRead()} loop may + * drain all pending data, and then if the parent channel is reading this channel may still accept frames. + */ + private ReadStatus readStatus = ReadStatus.IDLE; + + private Queue inboundBuffer; + + /** {@code true} after the first HEADERS frame has been written **/ + private boolean firstFrameWritten; + + // Currently the child channel and parent channel are always on the same EventLoop thread. This allows us to + // extend the read loop of a child channel if the child channel drains its queued data during read, and the + // parent channel is still in its read loop. The next/previous links build a doubly linked list that the parent + // channel will iterate in its channelReadComplete to end the read cycle for each child channel in the list. + AbstractHttp2StreamChannel next; + AbstractHttp2StreamChannel previous; + + AbstractHttp2StreamChannel(DefaultHttp2FrameStream stream, int id, ChannelHandler inboundHandler) { + this.stream = stream; + stream.attachment = this; + pipeline = new DefaultChannelPipeline(this) { + @Override + protected void incrementPendingOutboundBytes(long size) { + AbstractHttp2StreamChannel.this.incrementPendingOutboundBytes(size, true); + } + + @Override + protected void decrementPendingOutboundBytes(long size) { + AbstractHttp2StreamChannel.this.decrementPendingOutboundBytes(size, true); + } + }; + + closePromise = pipeline.newPromise(); + channelId = new Http2StreamChannelId(parent().id(), id); + + if (inboundHandler != null) { + // Add the handler to the pipeline now that we are registered. + pipeline.addLast(inboundHandler); + } + } + + private void incrementPendingOutboundBytes(long size, boolean invokeLater) { + if (size == 0) { + return; + } + + long newWriteBufferSize = TOTAL_PENDING_SIZE_UPDATER.addAndGet(this, size); + if (newWriteBufferSize > config().getWriteBufferHighWaterMark()) { + setUnwritable(invokeLater); + } + } + + private void decrementPendingOutboundBytes(long size, boolean invokeLater) { + if (size == 0) { + return; + } + + long newWriteBufferSize = TOTAL_PENDING_SIZE_UPDATER.addAndGet(this, -size); + // Once the totalPendingSize dropped below the low water-mark we can mark the child channel + // as writable again. Before doing so we also need to ensure the parent channel is writable to + // prevent excessive buffering in the parent outbound buffer. If the parent is not writable + // we will mark the child channel as writable once the parent becomes writable by calling + // trySetWritable() later. + if (newWriteBufferSize < config().getWriteBufferLowWaterMark() && parent().isWritable()) { + setWritable(invokeLater); + } + } + + final void trySetWritable() { + // The parent is writable again but the child channel itself may still not be writable. + // Lets try to set the child channel writable to match the state of the parent channel + // if (and only if) the totalPendingSize is smaller then the low water-mark. + // If this is not the case we will try again later once we drop under it. + if (totalPendingSize < config().getWriteBufferLowWaterMark()) { + setWritable(false); + } + } + + private void setWritable(boolean invokeLater) { + for (;;) { + final int oldValue = unwritable; + final int newValue = oldValue & ~1; + if (UNWRITABLE_UPDATER.compareAndSet(this, oldValue, newValue)) { + if (oldValue != 0 && newValue == 0) { + fireChannelWritabilityChanged(invokeLater); + } + break; + } + } + } + + private void setUnwritable(boolean invokeLater) { + for (;;) { + final int oldValue = unwritable; + final int newValue = oldValue | 1; + if (UNWRITABLE_UPDATER.compareAndSet(this, oldValue, newValue)) { + if (oldValue == 0 && newValue != 0) { + fireChannelWritabilityChanged(invokeLater); + } + break; + } + } + } + + private void fireChannelWritabilityChanged(boolean invokeLater) { + final ChannelPipeline pipeline = pipeline(); + if (invokeLater) { + Runnable task = fireChannelWritabilityChangedTask; + if (task == null) { + fireChannelWritabilityChangedTask = task = new Runnable() { + @Override + public void run() { + pipeline.fireChannelWritabilityChanged(); + } + }; + } + eventLoop().execute(task); + } else { + pipeline.fireChannelWritabilityChanged(); + } + } + @Override + public Http2FrameStream stream() { + return stream; + } + + void closeOutbound() { + outboundClosed = true; + } + + void streamClosed() { + unsafe.readEOS(); + // Attempt to drain any queued data from the queue and deliver it to the application before closing this + // channel. + unsafe.doBeginRead(); + } + + @Override + public ChannelMetadata metadata() { + return METADATA; + } + + @Override + public ChannelConfig config() { + return config; + } + + @Override + public boolean isOpen() { + return !closePromise.isDone(); + } + + @Override + public boolean isActive() { + return isOpen(); + } + + @Override + public boolean isWritable() { + return unwritable == 0; + } + + @Override + public ChannelId id() { + return channelId; + } + + @Override + public EventLoop eventLoop() { + return parent().eventLoop(); + } + + @Override + public Channel parent() { + return parentContext().channel(); + } + + @Override + public boolean isRegistered() { + return registered; + } + + @Override + public SocketAddress localAddress() { + return parent().localAddress(); + } + + @Override + public SocketAddress remoteAddress() { + return parent().remoteAddress(); + } + + @Override + public ChannelFuture closeFuture() { + return closePromise; + } + + @Override + public long bytesBeforeUnwritable() { + long bytes = config().getWriteBufferHighWaterMark() - totalPendingSize; + // If bytes is negative we know we are not writable, but if bytes is non-negative we have to check + // writability. Note that totalPendingSize and isWritable() use different volatile variables that are not + // synchronized together. totalPendingSize will be updated before isWritable(). + if (bytes > 0) { + return isWritable() ? bytes : 0; + } + return 0; + } + + @Override + public long bytesBeforeWritable() { + long bytes = totalPendingSize - config().getWriteBufferLowWaterMark(); + // If bytes is negative we know we are writable, but if bytes is non-negative we have to check writability. + // Note that totalPendingSize and isWritable() use different volatile variables that are not synchronized + // together. totalPendingSize will be updated before isWritable(). + if (bytes > 0) { + return isWritable() ? 0 : bytes; + } + return 0; + } + + @Override + public Unsafe unsafe() { + return unsafe; + } + + @Override + public ChannelPipeline pipeline() { + return pipeline; + } + + @Override + public ByteBufAllocator alloc() { + return config().getAllocator(); + } + + @Override + public Channel read() { + pipeline().read(); + return this; + } + + @Override + public Channel flush() { + pipeline().flush(); + return this; + } + + @Override + public ChannelFuture bind(SocketAddress localAddress) { + return pipeline().bind(localAddress); + } + + @Override + public ChannelFuture connect(SocketAddress remoteAddress) { + return pipeline().connect(remoteAddress); + } + + @Override + public ChannelFuture connect(SocketAddress remoteAddress, SocketAddress localAddress) { + return pipeline().connect(remoteAddress, localAddress); + } + + @Override + public ChannelFuture disconnect() { + return pipeline().disconnect(); + } + + @Override + public ChannelFuture close() { + return pipeline().close(); + } + + @Override + public ChannelFuture deregister() { + return pipeline().deregister(); + } + + @Override + public ChannelFuture bind(SocketAddress localAddress, ChannelPromise promise) { + return pipeline().bind(localAddress, promise); + } + + @Override + public ChannelFuture connect(SocketAddress remoteAddress, ChannelPromise promise) { + return pipeline().connect(remoteAddress, promise); + } + + @Override + public ChannelFuture connect(SocketAddress remoteAddress, SocketAddress localAddress, ChannelPromise promise) { + return pipeline().connect(remoteAddress, localAddress, promise); + } + + @Override + public ChannelFuture disconnect(ChannelPromise promise) { + return pipeline().disconnect(promise); + } + + @Override + public ChannelFuture close(ChannelPromise promise) { + return pipeline().close(promise); + } + + @Override + public ChannelFuture deregister(ChannelPromise promise) { + return pipeline().deregister(promise); + } + + @Override + public ChannelFuture write(Object msg) { + return pipeline().write(msg); + } + + @Override + public ChannelFuture write(Object msg, ChannelPromise promise) { + return pipeline().write(msg, promise); + } + + @Override + public ChannelFuture writeAndFlush(Object msg, ChannelPromise promise) { + return pipeline().writeAndFlush(msg, promise); + } + + @Override + public ChannelFuture writeAndFlush(Object msg) { + return pipeline().writeAndFlush(msg); + } + + @Override + public ChannelPromise newPromise() { + return pipeline().newPromise(); + } + + @Override + public ChannelProgressivePromise newProgressivePromise() { + return pipeline().newProgressivePromise(); + } + + @Override + public ChannelFuture newSucceededFuture() { + return pipeline().newSucceededFuture(); + } + + @Override + public ChannelFuture newFailedFuture(Throwable cause) { + return pipeline().newFailedFuture(cause); + } + + @Override + public ChannelPromise voidPromise() { + return pipeline().voidPromise(); + } + + @Override + public int hashCode() { + return id().hashCode(); + } + + @Override + public boolean equals(Object o) { + return this == o; + } + + @Override + public int compareTo(Channel o) { + if (this == o) { + return 0; + } + + return id().compareTo(o.id()); + } + + @Override + public String toString() { + return parent().toString() + "(H2 - " + stream + ')'; + } + + /** + * Receive a read message. This does not notify handlers unless a read is in progress on the + * channel. + */ + void fireChildRead(Http2Frame frame) { + assert eventLoop().inEventLoop(); + if (!isActive()) { + ReferenceCountUtil.release(frame); + } else if (readStatus != ReadStatus.IDLE) { + // If a read is in progress or has been requested, there cannot be anything in the queue, + // otherwise we would have drained it from the queue and processed it during the read cycle. + assert inboundBuffer == null || inboundBuffer.isEmpty(); + final RecvByteBufAllocator.Handle allocHandle = unsafe.recvBufAllocHandle(); + unsafe.doRead0(frame, allocHandle); + // We currently don't need to check for readEOS because the parent channel and child channel are limited + // to the same EventLoop thread. There are a limited number of frame types that may come after EOS is + // read (unknown, reset) and the trade off is less conditionals for the hot path (headers/data) at the + // cost of additional readComplete notifications on the rare path. + if (allocHandle.continueReading()) { + tryAddChildChannelToReadPendingQueue(); + } else { + tryRemoveChildChannelFromReadPendingQueue(); + unsafe.notifyReadComplete(allocHandle); + } + } else { + if (inboundBuffer == null) { + inboundBuffer = new ArrayDeque(4); + } + inboundBuffer.add(frame); + } + } + + void fireChildReadComplete() { + assert eventLoop().inEventLoop(); + assert readStatus != ReadStatus.IDLE; + unsafe.notifyReadComplete(unsafe.recvBufAllocHandle()); + } + + private final class Http2ChannelUnsafe implements Unsafe { + private final VoidChannelPromise unsafeVoidPromise = + new VoidChannelPromise(AbstractHttp2StreamChannel.this, false); + @SuppressWarnings("deprecation") + private RecvByteBufAllocator.Handle recvHandle; + private boolean writeDoneAndNoFlush; + private boolean closeInitiated; + private boolean readEOS; + + @Override + public void connect(final SocketAddress remoteAddress, + SocketAddress localAddress, final ChannelPromise promise) { + if (!promise.setUncancellable()) { + return; + } + promise.setFailure(new UnsupportedOperationException()); + } + + @Override + public RecvByteBufAllocator.Handle recvBufAllocHandle() { + if (recvHandle == null) { + recvHandle = config().getRecvByteBufAllocator().newHandle(); + recvHandle.reset(config()); + } + return recvHandle; + } + + @Override + public SocketAddress localAddress() { + return parent().unsafe().localAddress(); + } + + @Override + public SocketAddress remoteAddress() { + return parent().unsafe().remoteAddress(); + } + + @Override + public void register(EventLoop eventLoop, ChannelPromise promise) { + if (!promise.setUncancellable()) { + return; + } + if (registered) { + promise.setFailure(new UnsupportedOperationException("Re-register is not supported")); + return; + } + + registered = true; + + promise.setSuccess(); + + pipeline().fireChannelRegistered(); + if (isActive()) { + pipeline().fireChannelActive(); + } + } + + @Override + public void bind(SocketAddress localAddress, ChannelPromise promise) { + if (!promise.setUncancellable()) { + return; + } + promise.setFailure(new UnsupportedOperationException()); + } + + @Override + public void disconnect(ChannelPromise promise) { + close(promise); + } + + @Override + public void close(final ChannelPromise promise) { + if (!promise.setUncancellable()) { + return; + } + if (closeInitiated) { + if (closePromise.isDone()) { + // Closed already. + promise.setSuccess(); + } else if (!(promise instanceof VoidChannelPromise)) { // Only needed if no VoidChannelPromise. + // This means close() was called before so we just register a listener and return + closePromise.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) { + promise.setSuccess(); + } + }); + } + return; + } + closeInitiated = true; + + tryRemoveChildChannelFromReadPendingQueue(); + + final boolean wasActive = isActive(); + + // Only ever send a reset frame if the connection is still alive and if the stream may have existed + // as otherwise we may send a RST on a stream in an invalid state and cause a connection error. + if (parent().isActive() && !readEOS && streamMayHaveExisted(stream())) { + Http2StreamFrame resetFrame = new DefaultHttp2ResetFrame(Http2Error.CANCEL).stream(stream()); + write(resetFrame, unsafe().voidPromise()); + flush(); + } + + if (inboundBuffer != null) { + for (;;) { + Object msg = inboundBuffer.poll(); + if (msg == null) { + break; + } + ReferenceCountUtil.release(msg); + } + inboundBuffer = null; + } + + // The promise should be notified before we call fireChannelInactive(). + outboundClosed = true; + closePromise.setSuccess(); + promise.setSuccess(); + + fireChannelInactiveAndDeregister(voidPromise(), wasActive); + } + + @Override + public void closeForcibly() { + close(unsafe().voidPromise()); + } + + @Override + public void deregister(ChannelPromise promise) { + fireChannelInactiveAndDeregister(promise, false); + } + + private void fireChannelInactiveAndDeregister(final ChannelPromise promise, + final boolean fireChannelInactive) { + if (!promise.setUncancellable()) { + return; + } + + if (!registered) { + promise.setSuccess(); + return; + } + + // As a user may call deregister() from within any method while doing processing in the ChannelPipeline, + // we need to ensure we do the actual deregister operation later. This is necessary to preserve the + // behavior of the AbstractChannel, which always invokes channelUnregistered and channelInactive + // events 'later' to ensure the current events in the handler are completed before these events. + // + // See: + // https://github.com/netty/netty/issues/4435 + invokeLater(new Runnable() { + @Override + public void run() { + if (fireChannelInactive) { + pipeline.fireChannelInactive(); + } + // The user can fire `deregister` events multiple times but we only want to fire the pipeline + // event if the channel was actually registered. + if (registered) { + registered = false; + pipeline.fireChannelUnregistered(); + } + safeSetSuccess(promise); + } + }); + } + + private void safeSetSuccess(ChannelPromise promise) { + if (!(promise instanceof VoidChannelPromise) && !promise.trySuccess()) { + logger.warn("Failed to mark a promise as success because it is done already: {}", promise); + } + } + + private void invokeLater(Runnable task) { + try { + // This method is used by outbound operation implementations to trigger an inbound event later. + // They do not trigger an inbound event immediately because an outbound operation might have been + // triggered by another inbound event handler method. If fired immediately, the call stack + // will look like this for example: + // + // handlerA.inboundBufferUpdated() - (1) an inbound handler method closes a connection. + // -> handlerA.ctx.close() + // -> channel.unsafe.close() + // -> handlerA.channelInactive() - (2) another inbound handler method called while in (1) yet + // + // which means the execution of two inbound handler methods of the same handler overlap undesirably. + eventLoop().execute(task); + } catch (RejectedExecutionException e) { + logger.warn("Can't invoke task later as EventLoop rejected it", e); + } + } + + @Override + public void beginRead() { + if (!isActive()) { + return; + } + switch (readStatus) { + case IDLE: + readStatus = ReadStatus.IN_PROGRESS; + doBeginRead(); + break; + case IN_PROGRESS: + readStatus = ReadStatus.REQUESTED; + break; + default: + break; + } + } + + void doBeginRead() { + Object message; + if (inboundBuffer == null || (message = inboundBuffer.poll()) == null) { + if (readEOS) { + unsafe.closeForcibly(); + } + } else { + final RecvByteBufAllocator.Handle allocHandle = recvBufAllocHandle(); + allocHandle.reset(config()); + boolean continueReading = false; + do { + doRead0((Http2Frame) message, allocHandle); + } while ((readEOS || (continueReading = allocHandle.continueReading())) && + (message = inboundBuffer.poll()) != null); + + if (continueReading && isParentReadInProgress() && !readEOS) { + // Currently the parent and child channel are on the same EventLoop thread. If the parent is + // currently reading it is possile that more frames will be delivered to this child channel. In + // the case that this child channel still wants to read we delay the channelReadComplete on this + // child channel until the parent is done reading. + boolean added = tryAddChildChannelToReadPendingQueue(); + assert added; + } else { + notifyReadComplete(allocHandle); + } + } + } + + void readEOS() { + readEOS = true; + } + + void notifyReadComplete(RecvByteBufAllocator.Handle allocHandle) { + assert next == null && previous == null; + if (readStatus == ReadStatus.REQUESTED) { + readStatus = ReadStatus.IN_PROGRESS; + } else { + readStatus = ReadStatus.IDLE; + } + allocHandle.readComplete(); + pipeline().fireChannelReadComplete(); + // Reading data may result in frames being written (e.g. WINDOW_UPDATE, RST, etc..). If the parent + // channel is not currently reading we need to force a flush at the child channel, because we cannot + // rely upon flush occurring in channelReadComplete on the parent channel. + flush(); + if (readEOS) { + unsafe.closeForcibly(); + } + } + + @SuppressWarnings("deprecation") + void doRead0(Http2Frame frame, RecvByteBufAllocator.Handle allocHandle) { + pipeline().fireChannelRead(frame); + allocHandle.incMessagesRead(1); + + if (frame instanceof Http2DataFrame) { + final int numBytesToBeConsumed = ((Http2DataFrame) frame).initialFlowControlledBytes(); + allocHandle.attemptedBytesRead(numBytesToBeConsumed); + allocHandle.lastBytesRead(numBytesToBeConsumed); + if (numBytesToBeConsumed != 0) { + try { + if (consumeBytes(stream, numBytesToBeConsumed)) { + // We wrote some WINDOW_UPDATE frame, so we may need to do a flush. + writeDoneAndNoFlush = true; + flush(); + } + } catch (Http2Exception e) { + pipeline().fireExceptionCaught(e); + } + } + } else { + allocHandle.attemptedBytesRead(MIN_HTTP2_FRAME_SIZE); + allocHandle.lastBytesRead(MIN_HTTP2_FRAME_SIZE); + } + } + + @Override + public void write(Object msg, final ChannelPromise promise) { + // After this point its not possible to cancel a write anymore. + if (!promise.setUncancellable()) { + ReferenceCountUtil.release(msg); + return; + } + + if (!isActive() || + // Once the outbound side was closed we should not allow header / data frames + outboundClosed && (msg instanceof Http2HeadersFrame || msg instanceof Http2DataFrame)) { + ReferenceCountUtil.release(msg); + promise.setFailure(new ClosedChannelException()); + return; + } + + try { + if (msg instanceof Http2StreamFrame) { + Http2StreamFrame frame = validateStreamFrame((Http2StreamFrame) msg).stream(stream()); + if (!firstFrameWritten && !isStreamIdValid(stream().id())) { + if (!(frame instanceof Http2HeadersFrame)) { + ReferenceCountUtil.release(frame); + promise.setFailure( + new IllegalArgumentException("The first frame must be a headers frame. Was: " + + frame.name())); + return; + } + firstFrameWritten = true; + ChannelFuture f = write0(parentContext(), frame); + if (f.isDone()) { + firstWriteComplete(f, promise); + } else { + final long bytes = FlowControlledFrameSizeEstimator.HANDLE_INSTANCE.size(msg); + incrementPendingOutboundBytes(bytes, false); + f.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) { + firstWriteComplete(future, promise); + decrementPendingOutboundBytes(bytes, false); + } + }); + writeDoneAndNoFlush = true; + } + return; + } + } else { + String msgStr = msg.toString(); + ReferenceCountUtil.release(msg); + promise.setFailure(new IllegalArgumentException( + "Message must be an " + StringUtil.simpleClassName(Http2StreamFrame.class) + + ": " + msgStr)); + return; + } + + ChannelFuture f = write0(parentContext(), msg); + if (f.isDone()) { + writeComplete(f, promise); + } else { + final long bytes = FlowControlledFrameSizeEstimator.HANDLE_INSTANCE.size(msg); + incrementPendingOutboundBytes(bytes, false); + f.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) { + writeComplete(future, promise); + decrementPendingOutboundBytes(bytes, false); + } + }); + writeDoneAndNoFlush = true; + } + } catch (Throwable t) { + promise.tryFailure(t); + } + } + + private void firstWriteComplete(ChannelFuture future, ChannelPromise promise) { + Throwable cause = future.cause(); + if (cause == null) { + promise.setSuccess(); + } else { + // If the first write fails there is not much we can do, just close + closeForcibly(); + promise.setFailure(wrapStreamClosedError(cause)); + } + } + + private void writeComplete(ChannelFuture future, ChannelPromise promise) { + Throwable cause = future.cause(); + if (cause == null) { + promise.setSuccess(); + } else { + Throwable error = wrapStreamClosedError(cause); + // To make it more consistent with AbstractChannel we handle all IOExceptions here. + if (error instanceof IOException) { + if (config.isAutoClose()) { + // Close channel if needed. + closeForcibly(); + } else { + // TODO: Once Http2StreamChannel extends DuplexChannel we should call shutdownOutput(...) + outboundClosed = true; + } + } + promise.setFailure(error); + } + } + + private Throwable wrapStreamClosedError(Throwable cause) { + // If the error was caused by STREAM_CLOSED we should use a ClosedChannelException to better + // mimic other transports and make it easier to reason about what exceptions to expect. + if (cause instanceof Http2Exception && ((Http2Exception) cause).error() == Http2Error.STREAM_CLOSED) { + return new ClosedChannelException().initCause(cause); + } + return cause; + } + + private Http2StreamFrame validateStreamFrame(Http2StreamFrame frame) { + if (frame.stream() != null && frame.stream() != stream) { + String msgString = frame.toString(); + ReferenceCountUtil.release(frame); + throw new IllegalArgumentException( + "Stream " + frame.stream() + " must not be set on the frame: " + msgString); + } + return frame; + } + + @Override + public void flush() { + // If we are currently in the parent channel's read loop we should just ignore the flush. + // We will ensure we trigger ctx.flush() after we processed all Channels later on and + // so aggregate the flushes. This is done as ctx.flush() is expensive when as it may trigger an + // write(...) or writev(...) operation on the socket. + if (!writeDoneAndNoFlush || isParentReadInProgress()) { + // There is nothing to flush so this is a NOOP. + return; + } + try { + flush0(parentContext()); + } finally { + writeDoneAndNoFlush = false; + } + } + + @Override + public ChannelPromise voidPromise() { + return unsafeVoidPromise; + } + + @Override + public ChannelOutboundBuffer outboundBuffer() { + // Always return null as we not use the ChannelOutboundBuffer and not even support it. + return null; + } + } + + /** + * {@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 { + Http2StreamChannelConfig(Channel channel) { + super(channel); + } + + @Override + public MessageSizeEstimator getMessageSizeEstimator() { + return FlowControlledFrameSizeEstimator.INSTANCE; + } + + @Override + public ChannelConfig setMessageSizeEstimator(MessageSizeEstimator estimator) { + throw new UnsupportedOperationException(); + } + + @Override + public ChannelConfig setRecvByteBufAllocator(RecvByteBufAllocator allocator) { + if (!(allocator.newHandle() instanceof RecvByteBufAllocator.ExtendedHandle)) { + throw new IllegalArgumentException("allocator.newHandle() must return an object of type: " + + RecvByteBufAllocator.ExtendedHandle.class); + } + super.setRecvByteBufAllocator(allocator); + return this; + } + } + + protected void flush0(ChannelHandlerContext ctx) { + ctx.flush(); + } + + protected ChannelFuture write0(ChannelHandlerContext ctx, Object msg) { + ChannelPromise promise = ctx.newPromise(); + ctx.write(msg, promise); + return promise; + } + + protected abstract boolean consumeBytes(Http2FrameStream stream, int bytes) throws Http2Exception; + protected abstract boolean isParentReadInProgress(); + protected abstract boolean streamMayHaveExisted(Http2FrameStream stream); + protected abstract void tryRemoveChildChannelFromReadPendingQueue(); + protected abstract boolean tryAddChildChannelToReadPendingQueue(); + protected abstract ChannelHandlerContext parentContext(); +} diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2ChannelDuplexHandler.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2ChannelDuplexHandler.java index b595696f4e..6c09014aaa 100644 --- a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2ChannelDuplexHandler.java +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2ChannelDuplexHandler.java @@ -91,4 +91,16 @@ public abstract class Http2ChannelDuplexHandler extends ChannelDuplexHandler { } return (Http2FrameCodec) frameCodecCtx.handler(); } + + boolean isValidLocalStreamId(Http2FrameStream stream) { + return frameCodec.connection().local().isValidStreamId(stream.id()); + } + + boolean streamMayHaveExisted(Http2FrameStream stream) { + return frameCodec.connection().streamMayHaveExisted(stream.id()); + } + + boolean consumeBytes(Http2FrameStream stream, int bytes) throws Http2Exception { + return frameCodec.consumeBytes(stream.id(), bytes); + } } diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2FrameCodec.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2FrameCodec.java index 2ef75c4456..c11c5e31ed 100644 --- a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2FrameCodec.java +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2FrameCodec.java @@ -16,6 +16,7 @@ package io.netty.handler.codec.http2; import io.netty.buffer.ByteBuf; +import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; @@ -185,18 +186,19 @@ public class Http2FrameCodec extends Http2ConnectionHandler { */ final void forEachActiveStream(final Http2FrameStreamVisitor streamVisitor) throws Http2Exception { assert ctx.executor().inEventLoop(); - - connection().forEachActiveStream(new Http2StreamVisitor() { - @Override - public boolean visit(Http2Stream stream) { - try { - return streamVisitor.visit((Http2FrameStream) stream.getProperty(streamKey)); - } catch (Throwable cause) { - onError(ctx, false, cause); - return false; + if (connection().numActiveStreams() > 0) { + connection().forEachActiveStream(new Http2StreamVisitor() { + @Override + public boolean visit(Http2Stream stream) { + try { + return streamVisitor.visit((Http2FrameStream) stream.getProperty(streamKey)); + } catch (Throwable cause) { + onError(ctx, false, cause); + return false; + } } - } - }); + }); + } } @Override @@ -416,7 +418,8 @@ public class Http2FrameCodec extends Http2ConnectionHandler { } private void onStreamActive0(Http2Stream stream) { - if (connection().local().isValidStreamId(stream.id())) { + if (stream.id() != Http2CodecUtil.HTTP_UPGRADE_STREAM_ID && + connection().local().isValidStreamId(stream.id())) { return; } @@ -624,11 +627,6 @@ public class Http2FrameCodec extends Http2ConnectionHandler { ctx.fireExceptionCaught(cause); } - final boolean isWritable(DefaultHttp2FrameStream stream) { - Http2Stream s = stream.stream; - return s != null && connection().remote().flowController().isWritable(s); - } - private final class Http2RemoteFlowControllerListener implements Http2RemoteFlowController.Listener { @Override public void writabilityChanged(Http2Stream stream) { @@ -650,6 +648,8 @@ public class Http2FrameCodec extends Http2ConnectionHandler { private volatile int id = -1; volatile Http2Stream stream; + Channel attachment; + DefaultHttp2FrameStream setStreamAndProperty(PropertyKey streamKey, Http2Stream stream) { assert id == -1 || stream.id() == id; this.stream = stream; diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexCodec.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexCodec.java index 109857f5c6..a9d49d6e44 100644 --- a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexCodec.java +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexCodec.java @@ -16,49 +16,20 @@ package io.netty.handler.codec.http2; import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufAllocator; import io.netty.channel.Channel; import io.netty.channel.ChannelConfig; import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelId; -import io.netty.channel.ChannelMetadata; -import io.netty.channel.ChannelOutboundBuffer; -import io.netty.channel.ChannelPipeline; -import io.netty.channel.ChannelProgressivePromise; import io.netty.channel.ChannelPromise; -import io.netty.channel.DefaultChannelConfig; -import io.netty.channel.DefaultChannelPipeline; import io.netty.channel.EventLoop; -import io.netty.channel.MessageSizeEstimator; -import io.netty.channel.RecvByteBufAllocator; -import io.netty.channel.RecvByteBufAllocator.Handle; -import io.netty.channel.VoidChannelPromise; -import io.netty.channel.WriteBufferWaterMark; -import io.netty.util.DefaultAttributeMap; -import io.netty.util.ReferenceCountUtil; import io.netty.util.ReferenceCounted; -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.io.IOException; -import java.net.SocketAddress; -import java.nio.channels.ClosedChannelException; -import java.util.ArrayDeque; -import java.util.Queue; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; -import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import io.netty.util.internal.UnstableApi; import static io.netty.handler.codec.http2.Http2CodecUtil.HTTP_UPGRADE_STREAM_ID; -import static io.netty.handler.codec.http2.Http2CodecUtil.isStreamIdValid; import static io.netty.handler.codec.http2.Http2Error.INTERNAL_ERROR; import static io.netty.handler.codec.http2.Http2Exception.connectionError; -import static java.lang.Math.min; /** * An HTTP/2 handler that creates child channels for each stream. @@ -104,75 +75,24 @@ import static java.lang.Math.min; * 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. + * + * @deprecated use {@link Http2FrameCodecBuilder} together with {@link Http2MultiplexHandler}. */ +@Deprecated @UnstableApi public class Http2MultiplexCodec extends Http2FrameCodec { - private static final InternalLogger logger = InternalLoggerFactory.getInstance(DefaultHttp2StreamChannel.class); - - private static final ChannelFutureListener CHILD_CHANNEL_REGISTRATION_LISTENER = new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) { - registerDone(future); - } - }; - - private static final ChannelMetadata METADATA = new ChannelMetadata(false, 16); - /** - * Number of bytes to consider non-payload messages. 9 is arbitrary, but also the minimum size of an HTTP/2 frame. - * Primarily is non-zero. - */ - private static final int MIN_HTTP2_FRAME_SIZE = 9; - - /** - * Returns the flow-control size for DATA frames, and 0 for all other frames. - */ - private static final class FlowControlledFrameSizeEstimator implements MessageSizeEstimator { - - static final FlowControlledFrameSizeEstimator INSTANCE = new FlowControlledFrameSizeEstimator(); - - static final MessageSizeEstimator.Handle HANDLE_INSTANCE = new MessageSizeEstimator.Handle() { - @Override - public int size(Object msg) { - return msg instanceof Http2DataFrame ? - // Guard against overflow. - (int) min(Integer.MAX_VALUE, ((Http2DataFrame) msg).initialFlowControlledBytes() + - (long) MIN_HTTP2_FRAME_SIZE) : MIN_HTTP2_FRAME_SIZE; - } - }; - - @Override - public Handle newHandle() { - return HANDLE_INSTANCE; - } - } - - private static final AtomicLongFieldUpdater TOTAL_PENDING_SIZE_UPDATER = - AtomicLongFieldUpdater.newUpdater(DefaultHttp2StreamChannel.class, "totalPendingSize"); - - private static final AtomicIntegerFieldUpdater UNWRITABLE_UPDATER = - AtomicIntegerFieldUpdater.newUpdater(DefaultHttp2StreamChannel.class, "unwritable"); - private final ChannelHandler inboundStreamHandler; private final ChannelHandler upgradeStreamHandler; - private final Http2FrameStreamVisitor writableVisitor = new Http2FrameStreamVisitor() { - @Override - public boolean visit(Http2FrameStream stream) { - final DefaultHttp2StreamChannel childChannel = ((Http2MultiplexCodecStream) stream).channel; - childChannel.trySetWritable(); - return true; - } - }; - private boolean parentReadInProgress; private int idCount; - // Linked-List for DefaultHttp2StreamChannel instances that need to be processed by channelReadComplete(...) - private DefaultHttp2StreamChannel head; - private DefaultHttp2StreamChannel tail; + // Linked-List for Http2MultiplexCodecStreamChannel instances that need to be processed by channelReadComplete(...) + private AbstractHttp2StreamChannel head; + private AbstractHttp2StreamChannel tail; - // Need to be volatile as accessed from within the DefaultHttp2StreamChannel in a multi-threaded fashion. + // Need to be volatile as accessed from within the Http2MultiplexCodecStreamChannel in a multi-threaded fashion. volatile ChannelHandlerContext ctx; Http2MultiplexCodec(Http2ConnectionEncoder encoder, @@ -194,25 +114,11 @@ public class Http2MultiplexCodec extends Http2FrameCodec { // Creates the Http2Stream in the Connection. super.onHttpClientUpgrade(); // Now make a new FrameStream, set it's underlying Http2Stream, and initialize it. - Http2MultiplexCodecStream codecStream = newStream(); + DefaultHttp2FrameStream codecStream = newStream(); codecStream.setStreamAndProperty(streamKey, connection().stream(HTTP_UPGRADE_STREAM_ID)); onHttp2UpgradeStreamInitialized(ctx, codecStream); } - private static void registerDone(ChannelFuture future) { - // Handle any errors that occurred on the local thread while registering. Even though - // failures can happen after this point, they will be handled by the channel by closing the - // childChannel. - if (!future.isSuccess()) { - Channel childChannel = future.channel(); - if (childChannel.isRegistered()) { - childChannel.close(); - } else { - childChannel.unsafe().closeForcibly(); - } - } - } - @Override public final void handlerAdded0(ChannelHandlerContext ctx) throws Exception { if (ctx.executor() != ctx.channel().eventLoop()) { @@ -226,25 +132,22 @@ public class Http2MultiplexCodec extends Http2FrameCodec { super.handlerRemoved0(ctx); // Unlink the linked list to guard against GC nepotism. - DefaultHttp2StreamChannel ch = head; + AbstractHttp2StreamChannel ch = head; while (ch != null) { - DefaultHttp2StreamChannel curr = ch; + AbstractHttp2StreamChannel curr = ch; ch = curr.next; curr.next = curr.previous = null; } head = tail = null; } - @Override - Http2MultiplexCodecStream newStream() { - return new Http2MultiplexCodecStream(); - } - @Override final void onHttp2Frame(ChannelHandlerContext ctx, Http2Frame frame) { if (frame instanceof Http2StreamFrame) { Http2StreamFrame streamFrame = (Http2StreamFrame) frame; - ((Http2MultiplexCodecStream) streamFrame.stream()).channel.fireChildRead(streamFrame); + AbstractHttp2StreamChannel channel = (AbstractHttp2StreamChannel) + ((DefaultHttp2FrameStream) streamFrame.stream()).attachment; + channel.fireChildRead(streamFrame); return; } if (frame instanceof Http2GoAwayFrame) { @@ -254,43 +157,44 @@ public class Http2MultiplexCodec extends Http2FrameCodec { ctx.fireChannelRead(frame); } - private void onHttp2UpgradeStreamInitialized(ChannelHandlerContext ctx, Http2MultiplexCodecStream stream) { + private void onHttp2UpgradeStreamInitialized(ChannelHandlerContext ctx, DefaultHttp2FrameStream stream) { assert stream.state() == Http2Stream.State.HALF_CLOSED_LOCAL; - DefaultHttp2StreamChannel ch = new DefaultHttp2StreamChannel(stream, true); - ch.outboundClosed = true; + AbstractHttp2StreamChannel ch = new Http2MultiplexCodecStreamChannel(stream, null); + ch.closeOutbound(); // Add our upgrade handler to the channel and then register the channel. // The register call fires the channelActive, etc. ch.pipeline().addLast(upgradeStreamHandler); ChannelFuture future = ctx.channel().eventLoop().register(ch); if (future.isDone()) { - registerDone(future); + Http2MultiplexHandler.registerDone(future); } else { - future.addListener(CHILD_CHANNEL_REGISTRATION_LISTENER); + future.addListener(Http2MultiplexHandler.CHILD_CHANNEL_REGISTRATION_LISTENER); } } @Override final void onHttp2StreamStateChanged(ChannelHandlerContext ctx, Http2FrameStream stream) { - Http2MultiplexCodecStream s = (Http2MultiplexCodecStream) stream; + DefaultHttp2FrameStream s = (DefaultHttp2FrameStream) stream; switch (stream.state()) { case HALF_CLOSED_REMOTE: case OPEN: - if (s.channel != null) { + if (s.attachment != null) { // ignore if child channel was already created. break; } // fall-trough - ChannelFuture future = ctx.channel().eventLoop().register(new DefaultHttp2StreamChannel(s, false)); + ChannelFuture future = ctx.channel().eventLoop().register( + new Http2MultiplexCodecStreamChannel(s, inboundStreamHandler)); if (future.isDone()) { - registerDone(future); + Http2MultiplexHandler.registerDone(future); } else { - future.addListener(CHILD_CHANNEL_REGISTRATION_LISTENER); + future.addListener(Http2MultiplexHandler.CHILD_CHANNEL_REGISTRATION_LISTENER); } break; case CLOSED: - DefaultHttp2StreamChannel channel = s.channel; + AbstractHttp2StreamChannel channel = (AbstractHttp2StreamChannel) s.attachment; if (channel != null) { channel.streamClosed(); } @@ -303,32 +207,34 @@ public class Http2MultiplexCodec extends Http2FrameCodec { // TODO: This is most likely not the best way to expose this, need to think more about it. final Http2StreamChannel newOutboundStream() { - return new DefaultHttp2StreamChannel(newStream(), true); + return new Http2MultiplexCodecStreamChannel(newStream(), null); } @Override final void onHttp2FrameStreamException(ChannelHandlerContext ctx, Http2FrameStreamException cause) { Http2FrameStream stream = cause.stream(); - DefaultHttp2StreamChannel childChannel = ((Http2MultiplexCodecStream) stream).channel; + AbstractHttp2StreamChannel channel = (AbstractHttp2StreamChannel) ((DefaultHttp2FrameStream) stream).attachment; try { - childChannel.pipeline().fireExceptionCaught(cause.getCause()); + channel.pipeline().fireExceptionCaught(cause.getCause()); } finally { - childChannel.unsafe().closeForcibly(); + channel.unsafe().closeForcibly(); } } - private boolean isChildChannelInReadPendingQueue(DefaultHttp2StreamChannel childChannel) { + private boolean isChildChannelInReadPendingQueue(AbstractHttp2StreamChannel childChannel) { return childChannel.previous != null || childChannel.next != null || head == childChannel; } - final void tryAddChildChannelToReadPendingQueue(DefaultHttp2StreamChannel childChannel) { + private boolean tryAddChildChannelToReadPendingQueue(AbstractHttp2StreamChannel childChannel) { if (!isChildChannelInReadPendingQueue(childChannel)) { addChildChannelToReadPendingQueue(childChannel); + return true; } + return false; } - final void addChildChannelToReadPendingQueue(DefaultHttp2StreamChannel childChannel) { + private void addChildChannelToReadPendingQueue(AbstractHttp2StreamChannel childChannel) { if (tail == null) { assert head == null; tail = head = childChannel; @@ -339,14 +245,14 @@ public class Http2MultiplexCodec extends Http2FrameCodec { } } - private void tryRemoveChildChannelFromReadPendingQueue(DefaultHttp2StreamChannel childChannel) { + private void tryRemoveChildChannelFromReadPendingQueue(AbstractHttp2StreamChannel childChannel) { if (isChildChannelInReadPendingQueue(childChannel)) { removeChildChannelFromReadPendingQueue(childChannel); } } - private void removeChildChannelFromReadPendingQueue(DefaultHttp2StreamChannel childChannel) { - DefaultHttp2StreamChannel previous = childChannel.previous; + private void removeChildChannelFromReadPendingQueue(AbstractHttp2StreamChannel childChannel) { + AbstractHttp2StreamChannel previous = childChannel.previous; if (childChannel.next != null) { childChannel.next.previous = previous; } else { @@ -366,9 +272,10 @@ public class Http2MultiplexCodec extends Http2FrameCodec { @Override public boolean visit(Http2FrameStream stream) { final int streamId = stream.id(); - final DefaultHttp2StreamChannel childChannel = ((Http2MultiplexCodecStream) stream).channel; + AbstractHttp2StreamChannel channel = (AbstractHttp2StreamChannel) + ((DefaultHttp2FrameStream) stream).attachment; if (streamId > goAwayFrame.lastStreamId() && connection().local().isValidStreamId(streamId)) { - childChannel.pipeline().fireUserEventTriggered(goAwayFrame.retainedDuplicate()); + channel.pipeline().fireUserEventTriggered(goAwayFrame.retainedDuplicate()); } return true; } @@ -406,7 +313,7 @@ public class Http2MultiplexCodec extends Http2FrameCodec { if (ctx.channel().isWritable()) { // While the writability state may change during iterating of the streams we just set all of the streams // to writable to not affect fairness. These will be "limited" by their own watermarks in any case. - forEachActiveStream(writableVisitor); + forEachActiveStream(AbstractHttp2StreamChannel.WRITABLE_VISITOR); } ctx.fireChannelWritabilityChanged(); @@ -416,9 +323,9 @@ public class Http2MultiplexCodec extends Http2FrameCodec { // If we have many child channel we can optimize for the case when multiple call flush() in // channelReadComplete(...) callbacks and only do it once as otherwise we will end-up with multiple // write calls on the socket which is expensive. - DefaultHttp2StreamChannel current = head; + AbstractHttp2StreamChannel current = head; while (current != null) { - DefaultHttp2StreamChannel childChannel = current; + AbstractHttp2StreamChannel childChannel = current; // Clear early in case fireChildReadComplete() causes it to need to be re-processed current = current.next; childChannel.next = childChannel.previous = null; @@ -430,934 +337,52 @@ public class Http2MultiplexCodec extends Http2FrameCodec { flush(ctx); } - static final class Http2MultiplexCodecStream extends DefaultHttp2FrameStream { - DefaultHttp2StreamChannel channel; - } + private final class Http2MultiplexCodecStreamChannel extends AbstractHttp2StreamChannel { - /** - * The current status of the read-processing for a {@link Http2StreamChannel}. - */ - private enum ReadStatus { - /** - * No read in progress and no read was requested (yet) - */ - IDLE, - - /** - * Reading in progress - */ - IN_PROGRESS, - - /** - * A read operation was requested. - */ - REQUESTED - } - - private final class DefaultHttp2StreamChannel extends DefaultAttributeMap implements Http2StreamChannel { - private final Http2StreamChannelConfig config = new Http2StreamChannelConfig(this); - private final Http2ChannelUnsafe unsafe = new Http2ChannelUnsafe(); - private final ChannelId channelId; - private final ChannelPipeline pipeline; - private final DefaultHttp2FrameStream stream; - private final ChannelPromise closePromise; - private final boolean outbound; - - private volatile boolean registered; - - // Needs to be package-private to be able to access it from the outer-class AtomicLongFieldUpdater. - volatile long totalPendingSize; - volatile int unwritable; - - // Cached to reduce GC - private Runnable fireChannelWritabilityChangedTask; - - private boolean outboundClosed; - - /** - * This variable represents if a read is in progress for the current channel or was requested. - * Note that depending upon the {@link RecvByteBufAllocator} behavior a read may extend beyond the - * {@link Http2ChannelUnsafe#beginRead()} method scope. The {@link Http2ChannelUnsafe#beginRead()} loop may - * drain all pending data, and then if the parent channel is reading this channel may still accept frames. - */ - private ReadStatus readStatus = ReadStatus.IDLE; - - private Queue inboundBuffer; - - /** {@code true} after the first HEADERS frame has been written **/ - private boolean firstFrameWritten; - - // Currently the child channel and parent channel are always on the same EventLoop thread. This allows us to - // extend the read loop of a child channel if the child channel drains its queued data during read, and the - // parent channel is still in its read loop. The next/previous links build a doubly linked list that the parent - // channel will iterate in its channelReadComplete to end the read cycle for each child channel in the list. - DefaultHttp2StreamChannel next; - DefaultHttp2StreamChannel previous; - - DefaultHttp2StreamChannel(DefaultHttp2FrameStream stream, boolean outbound) { - this.stream = stream; - this.outbound = outbound; - ((Http2MultiplexCodecStream) stream).channel = this; - pipeline = new DefaultChannelPipeline(this) { - @Override - protected void incrementPendingOutboundBytes(long size) { - DefaultHttp2StreamChannel.this.incrementPendingOutboundBytes(size, true); - } - - @Override - protected void decrementPendingOutboundBytes(long size) { - DefaultHttp2StreamChannel.this.decrementPendingOutboundBytes(size, true); - } - }; - closePromise = pipeline.newPromise(); - channelId = new Http2StreamChannelId(parent().id(), ++idCount); - } - - private void incrementPendingOutboundBytes(long size, boolean invokeLater) { - if (size == 0) { - return; - } - - long newWriteBufferSize = TOTAL_PENDING_SIZE_UPDATER.addAndGet(this, size); - if (newWriteBufferSize > config().getWriteBufferHighWaterMark()) { - setUnwritable(invokeLater); - } - } - - private void decrementPendingOutboundBytes(long size, boolean invokeLater) { - if (size == 0) { - return; - } - - long newWriteBufferSize = TOTAL_PENDING_SIZE_UPDATER.addAndGet(this, -size); - // Once the totalPendingSize dropped below the low water-mark we can mark the child channel - // as writable again. Before doing so we also need to ensure the parent channel is writable to - // prevent excessive buffering in the parent outbound buffer. If the parent is not writable - // we will mark the child channel as writable once the parent becomes writable by calling - // trySetWritable() later. - if (newWriteBufferSize < config().getWriteBufferLowWaterMark() && parent().isWritable()) { - setWritable(invokeLater); - } - } - - void trySetWritable() { - // The parent is writable again but the child channel itself may still not be writable. - // Lets try to set the child channel writable to match the state of the parent channel - // if (and only if) the totalPendingSize is smaller then the low water-mark. - // If this is not the case we will try again later once we drop under it. - if (totalPendingSize < config().getWriteBufferLowWaterMark()) { - setWritable(false); - } - } - - private void setWritable(boolean invokeLater) { - for (;;) { - final int oldValue = unwritable; - final int newValue = oldValue & ~1; - if (UNWRITABLE_UPDATER.compareAndSet(this, oldValue, newValue)) { - if (oldValue != 0 && newValue == 0) { - fireChannelWritabilityChanged(invokeLater); - } - break; - } - } - } - - private void setUnwritable(boolean invokeLater) { - for (;;) { - final int oldValue = unwritable; - final int newValue = oldValue | 1; - if (UNWRITABLE_UPDATER.compareAndSet(this, oldValue, newValue)) { - if (oldValue == 0 && newValue != 0) { - fireChannelWritabilityChanged(invokeLater); - } - break; - } - } - } - - private void fireChannelWritabilityChanged(boolean invokeLater) { - final ChannelPipeline pipeline = pipeline(); - if (invokeLater) { - Runnable task = fireChannelWritabilityChangedTask; - if (task == null) { - fireChannelWritabilityChangedTask = task = new Runnable() { - @Override - public void run() { - pipeline.fireChannelWritabilityChanged(); - } - }; - } - eventLoop().execute(task); - } else { - pipeline.fireChannelWritabilityChanged(); - } + Http2MultiplexCodecStreamChannel(DefaultHttp2FrameStream stream, ChannelHandler inboundHandler) { + super(stream, ++idCount, inboundHandler); } @Override - public Http2FrameStream stream() { - return stream; - } - - void streamClosed() { - unsafe.readEOS(); - // Attempt to drain any queued data from the queue and deliver it to the application before closing this - // channel. - unsafe.doBeginRead(); + protected boolean consumeBytes(Http2FrameStream stream, int bytes) throws Http2Exception { + return Http2MultiplexCodec.this.consumeBytes(stream.id(), bytes); } @Override - public ChannelMetadata metadata() { - return METADATA; + protected boolean isParentReadInProgress() { + return parentReadInProgress; } @Override - public ChannelConfig config() { - return config; + protected boolean streamMayHaveExisted(Http2FrameStream stream) { + return Http2MultiplexCodec.this.connection().streamMayHaveExisted(stream.id()); } @Override - public boolean isOpen() { - return !closePromise.isDone(); + protected void tryRemoveChildChannelFromReadPendingQueue() { + Http2MultiplexCodec.this.tryRemoveChildChannelFromReadPendingQueue(this); } @Override - public boolean isActive() { - return isOpen(); + protected boolean tryAddChildChannelToReadPendingQueue() { + return Http2MultiplexCodec.this.tryAddChildChannelToReadPendingQueue(this); } @Override - public boolean isWritable() { - return unwritable == 0; + protected ChannelHandlerContext parentContext() { + return ctx; } @Override - public ChannelId id() { - return channelId; + protected ChannelFuture write0(ChannelHandlerContext ctx, Object msg) { + ChannelPromise promise = ctx.newPromise(); + Http2MultiplexCodec.this.write(ctx, msg, promise); + return promise; } @Override - public EventLoop eventLoop() { - return parent().eventLoop(); - } - - @Override - public Channel parent() { - return ctx.channel(); - } - - @Override - public boolean isRegistered() { - return registered; - } - - @Override - public SocketAddress localAddress() { - return parent().localAddress(); - } - - @Override - public SocketAddress remoteAddress() { - return parent().remoteAddress(); - } - - @Override - public ChannelFuture closeFuture() { - return closePromise; - } - - @Override - public long bytesBeforeUnwritable() { - long bytes = config().getWriteBufferHighWaterMark() - totalPendingSize; - // If bytes is negative we know we are not writable, but if bytes is non-negative we have to check - // writability. Note that totalPendingSize and isWritable() use different volatile variables that are not - // synchronized together. totalPendingSize will be updated before isWritable(). - if (bytes > 0) { - return isWritable() ? bytes : 0; - } - return 0; - } - - @Override - public long bytesBeforeWritable() { - long bytes = totalPendingSize - config().getWriteBufferLowWaterMark(); - // If bytes is negative we know we are writable, but if bytes is non-negative we have to check writability. - // Note that totalPendingSize and isWritable() use different volatile variables that are not synchronized - // together. totalPendingSize will be updated before isWritable(). - if (bytes > 0) { - return isWritable() ? 0 : bytes; - } - return 0; - } - - @Override - public Unsafe unsafe() { - return unsafe; - } - - @Override - public ChannelPipeline pipeline() { - return pipeline; - } - - @Override - public ByteBufAllocator alloc() { - return config().getAllocator(); - } - - @Override - public Channel read() { - pipeline().read(); - return this; - } - - @Override - public Channel flush() { - pipeline().flush(); - return this; - } - - @Override - public ChannelFuture bind(SocketAddress localAddress) { - return pipeline().bind(localAddress); - } - - @Override - public ChannelFuture connect(SocketAddress remoteAddress) { - return pipeline().connect(remoteAddress); - } - - @Override - public ChannelFuture connect(SocketAddress remoteAddress, SocketAddress localAddress) { - return pipeline().connect(remoteAddress, localAddress); - } - - @Override - public ChannelFuture disconnect() { - return pipeline().disconnect(); - } - - @Override - public ChannelFuture close() { - return pipeline().close(); - } - - @Override - public ChannelFuture deregister() { - return pipeline().deregister(); - } - - @Override - public ChannelFuture bind(SocketAddress localAddress, ChannelPromise promise) { - return pipeline().bind(localAddress, promise); - } - - @Override - public ChannelFuture connect(SocketAddress remoteAddress, ChannelPromise promise) { - return pipeline().connect(remoteAddress, promise); - } - - @Override - public ChannelFuture connect(SocketAddress remoteAddress, SocketAddress localAddress, ChannelPromise promise) { - return pipeline().connect(remoteAddress, localAddress, promise); - } - - @Override - public ChannelFuture disconnect(ChannelPromise promise) { - return pipeline().disconnect(promise); - } - - @Override - public ChannelFuture close(ChannelPromise promise) { - return pipeline().close(promise); - } - - @Override - public ChannelFuture deregister(ChannelPromise promise) { - return pipeline().deregister(promise); - } - - @Override - public ChannelFuture write(Object msg) { - return pipeline().write(msg); - } - - @Override - public ChannelFuture write(Object msg, ChannelPromise promise) { - return pipeline().write(msg, promise); - } - - @Override - public ChannelFuture writeAndFlush(Object msg, ChannelPromise promise) { - return pipeline().writeAndFlush(msg, promise); - } - - @Override - public ChannelFuture writeAndFlush(Object msg) { - return pipeline().writeAndFlush(msg); - } - - @Override - public ChannelPromise newPromise() { - return pipeline().newPromise(); - } - - @Override - public ChannelProgressivePromise newProgressivePromise() { - return pipeline().newProgressivePromise(); - } - - @Override - public ChannelFuture newSucceededFuture() { - return pipeline().newSucceededFuture(); - } - - @Override - public ChannelFuture newFailedFuture(Throwable cause) { - return pipeline().newFailedFuture(cause); - } - - @Override - public ChannelPromise voidPromise() { - return pipeline().voidPromise(); - } - - @Override - public int hashCode() { - return id().hashCode(); - } - - @Override - public boolean equals(Object o) { - return this == o; - } - - @Override - public int compareTo(Channel o) { - if (this == o) { - return 0; - } - - return id().compareTo(o.id()); - } - - @Override - public String toString() { - return parent().toString() + "(H2 - " + stream + ')'; - } - - /** - * Receive a read message. This does not notify handlers unless a read is in progress on the - * channel. - */ - void fireChildRead(Http2Frame frame) { - assert eventLoop().inEventLoop(); - if (!isActive()) { - ReferenceCountUtil.release(frame); - } else if (readStatus != ReadStatus.IDLE) { - // If a read is in progress or has been requested, there cannot be anything in the queue, - // otherwise we would have drained it from the queue and processed it during the read cycle. - assert inboundBuffer == null || inboundBuffer.isEmpty(); - final Handle allocHandle = unsafe.recvBufAllocHandle(); - unsafe.doRead0(frame, allocHandle); - // We currently don't need to check for readEOS because the parent channel and child channel are limited - // to the same EventLoop thread. There are a limited number of frame types that may come after EOS is - // read (unknown, reset) and the trade off is less conditionals for the hot path (headers/data) at the - // cost of additional readComplete notifications on the rare path. - if (allocHandle.continueReading()) { - tryAddChildChannelToReadPendingQueue(this); - } else { - tryRemoveChildChannelFromReadPendingQueue(this); - unsafe.notifyReadComplete(allocHandle); - } - } else { - if (inboundBuffer == null) { - inboundBuffer = new ArrayDeque(4); - } - inboundBuffer.add(frame); - } - } - - void fireChildReadComplete() { - assert eventLoop().inEventLoop(); - assert readStatus != ReadStatus.IDLE; - unsafe.notifyReadComplete(unsafe.recvBufAllocHandle()); - } - - private final class Http2ChannelUnsafe implements Unsafe { - private final VoidChannelPromise unsafeVoidPromise = - new VoidChannelPromise(DefaultHttp2StreamChannel.this, false); - @SuppressWarnings("deprecation") - private Handle recvHandle; - private boolean writeDoneAndNoFlush; - private boolean closeInitiated; - private boolean readEOS; - - @Override - public void connect(final SocketAddress remoteAddress, - SocketAddress localAddress, final ChannelPromise promise) { - if (!promise.setUncancellable()) { - return; - } - promise.setFailure(new UnsupportedOperationException()); - } - - @Override - public Handle recvBufAllocHandle() { - if (recvHandle == null) { - recvHandle = config().getRecvByteBufAllocator().newHandle(); - recvHandle.reset(config()); - } - return recvHandle; - } - - @Override - public SocketAddress localAddress() { - return parent().unsafe().localAddress(); - } - - @Override - public SocketAddress remoteAddress() { - return parent().unsafe().remoteAddress(); - } - - @Override - public void register(EventLoop eventLoop, ChannelPromise promise) { - if (!promise.setUncancellable()) { - return; - } - if (registered) { - throw new UnsupportedOperationException("Re-register is not supported"); - } - - registered = true; - - if (!outbound) { - // Add the handler to the pipeline now that we are registered. - pipeline().addLast(inboundStreamHandler); - } - - promise.setSuccess(); - - pipeline().fireChannelRegistered(); - if (isActive()) { - pipeline().fireChannelActive(); - } - } - - @Override - public void bind(SocketAddress localAddress, ChannelPromise promise) { - if (!promise.setUncancellable()) { - return; - } - promise.setFailure(new UnsupportedOperationException()); - } - - @Override - public void disconnect(ChannelPromise promise) { - close(promise); - } - - @Override - public void close(final ChannelPromise promise) { - if (!promise.setUncancellable()) { - return; - } - if (closeInitiated) { - if (closePromise.isDone()) { - // Closed already. - promise.setSuccess(); - } else if (!(promise instanceof VoidChannelPromise)) { // Only needed if no VoidChannelPromise. - // This means close() was called before so we just register a listener and return - closePromise.addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) { - promise.setSuccess(); - } - }); - } - return; - } - closeInitiated = true; - - tryRemoveChildChannelFromReadPendingQueue(DefaultHttp2StreamChannel.this); - - final boolean wasActive = isActive(); - - // Only ever send a reset frame if the connection is still alive and if the stream may have existed - // as otherwise we may send a RST on a stream in an invalid state and cause a connection error. - if (parent().isActive() && !readEOS && connection().streamMayHaveExisted(stream().id())) { - Http2StreamFrame resetFrame = new DefaultHttp2ResetFrame(Http2Error.CANCEL).stream(stream()); - write(resetFrame, unsafe().voidPromise()); - flush(); - } - - if (inboundBuffer != null) { - for (;;) { - Object msg = inboundBuffer.poll(); - if (msg == null) { - break; - } - ReferenceCountUtil.release(msg); - } - } - - // The promise should be notified before we call fireChannelInactive(). - outboundClosed = true; - closePromise.setSuccess(); - promise.setSuccess(); - - fireChannelInactiveAndDeregister(voidPromise(), wasActive); - } - - @Override - public void closeForcibly() { - close(unsafe().voidPromise()); - } - - @Override - public void deregister(ChannelPromise promise) { - fireChannelInactiveAndDeregister(promise, false); - } - - private void fireChannelInactiveAndDeregister(final ChannelPromise promise, - final boolean fireChannelInactive) { - if (!promise.setUncancellable()) { - return; - } - - if (!registered) { - promise.setSuccess(); - return; - } - - // As a user may call deregister() from within any method while doing processing in the ChannelPipeline, - // we need to ensure we do the actual deregister operation later. This is necessary to preserve the - // behavior of the AbstractChannel, which always invokes channelUnregistered and channelInactive - // events 'later' to ensure the current events in the handler are completed before these events. - // - // See: - // https://github.com/netty/netty/issues/4435 - invokeLater(new Runnable() { - @Override - public void run() { - if (fireChannelInactive) { - pipeline.fireChannelInactive(); - } - // The user can fire `deregister` events multiple times but we only want to fire the pipeline - // event if the channel was actually registered. - if (registered) { - registered = false; - pipeline.fireChannelUnregistered(); - } - safeSetSuccess(promise); - } - }); - } - - private void safeSetSuccess(ChannelPromise promise) { - if (!(promise instanceof VoidChannelPromise) && !promise.trySuccess()) { - logger.warn("Failed to mark a promise as success because it is done already: {}", promise); - } - } - - private void invokeLater(Runnable task) { - try { - // This method is used by outbound operation implementations to trigger an inbound event later. - // They do not trigger an inbound event immediately because an outbound operation might have been - // triggered by another inbound event handler method. If fired immediately, the call stack - // will look like this for example: - // - // handlerA.inboundBufferUpdated() - (1) an inbound handler method closes a connection. - // -> handlerA.ctx.close() - // -> channel.unsafe.close() - // -> handlerA.channelInactive() - (2) another inbound handler method called while in (1) yet - // - // which means the execution of two inbound handler methods of the same handler overlap undesirably. - eventLoop().execute(task); - } catch (RejectedExecutionException e) { - logger.warn("Can't invoke task later as EventLoop rejected it", e); - } - } - - @Override - public void beginRead() { - if (!isActive()) { - return; - } - switch (readStatus) { - case IDLE: - readStatus = ReadStatus.IN_PROGRESS; - doBeginRead(); - break; - case IN_PROGRESS: - readStatus = ReadStatus.REQUESTED; - break; - default: - break; - } - } - - void doBeginRead() { - Object message; - if (inboundBuffer == null || (message = inboundBuffer.poll()) == null) { - if (readEOS) { - unsafe.closeForcibly(); - } - } else { - final Handle allocHandle = recvBufAllocHandle(); - allocHandle.reset(config()); - boolean continueReading = false; - do { - doRead0((Http2Frame) message, allocHandle); - } while ((readEOS || (continueReading = allocHandle.continueReading())) && - (message = inboundBuffer.poll()) != null); - - if (continueReading && parentReadInProgress && !readEOS) { - // Currently the parent and child channel are on the same EventLoop thread. If the parent is - // currently reading it is possile that more frames will be delivered to this child channel. In - // the case that this child channel still wants to read we delay the channelReadComplete on this - // child channel until the parent is done reading. - assert !isChildChannelInReadPendingQueue(DefaultHttp2StreamChannel.this); - addChildChannelToReadPendingQueue(DefaultHttp2StreamChannel.this); - } else { - notifyReadComplete(allocHandle); - } - } - } - - void readEOS() { - readEOS = true; - } - - void notifyReadComplete(Handle allocHandle) { - assert next == null && previous == null; - if (readStatus == ReadStatus.REQUESTED) { - readStatus = ReadStatus.IN_PROGRESS; - } else { - readStatus = ReadStatus.IDLE; - } - allocHandle.readComplete(); - pipeline().fireChannelReadComplete(); - // Reading data may result in frames being written (e.g. WINDOW_UPDATE, RST, etc..). If the parent - // channel is not currently reading we need to force a flush at the child channel, because we cannot - // rely upon flush occurring in channelReadComplete on the parent channel. - flush(); - if (readEOS) { - unsafe.closeForcibly(); - } - } - - @SuppressWarnings("deprecation") - void doRead0(Http2Frame frame, Handle allocHandle) { - pipeline().fireChannelRead(frame); - allocHandle.incMessagesRead(1); - - if (frame instanceof Http2DataFrame) { - final int numBytesToBeConsumed = ((Http2DataFrame) frame).initialFlowControlledBytes(); - allocHandle.attemptedBytesRead(numBytesToBeConsumed); - allocHandle.lastBytesRead(numBytesToBeConsumed); - if (numBytesToBeConsumed != 0) { - try { - writeDoneAndNoFlush |= consumeBytes(stream.id(), numBytesToBeConsumed); - } catch (Http2Exception e) { - pipeline().fireExceptionCaught(e); - } - } - } else { - allocHandle.attemptedBytesRead(MIN_HTTP2_FRAME_SIZE); - allocHandle.lastBytesRead(MIN_HTTP2_FRAME_SIZE); - } - } - - @Override - public void write(Object msg, final ChannelPromise promise) { - // After this point its not possible to cancel a write anymore. - if (!promise.setUncancellable()) { - ReferenceCountUtil.release(msg); - return; - } - - if (!isActive() || - // Once the outbound side was closed we should not allow header / data frames - outboundClosed && (msg instanceof Http2HeadersFrame || msg instanceof Http2DataFrame)) { - ReferenceCountUtil.release(msg); - promise.setFailure(new ClosedChannelException()); - return; - } - - try { - if (msg instanceof Http2StreamFrame) { - Http2StreamFrame frame = validateStreamFrame((Http2StreamFrame) msg).stream(stream()); - if (!firstFrameWritten && !isStreamIdValid(stream().id())) { - if (!(frame instanceof Http2HeadersFrame)) { - ReferenceCountUtil.release(frame); - promise.setFailure( - new IllegalArgumentException("The first frame must be a headers frame. Was: " - + frame.name())); - return; - } - firstFrameWritten = true; - ChannelFuture f = write0(frame); - if (f.isDone()) { - firstWriteComplete(f, promise); - } else { - final long bytes = FlowControlledFrameSizeEstimator.HANDLE_INSTANCE.size(msg); - incrementPendingOutboundBytes(bytes, false); - f.addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) { - firstWriteComplete(future, promise); - decrementPendingOutboundBytes(bytes, false); - } - }); - } - return; - } - } else { - String msgStr = msg.toString(); - ReferenceCountUtil.release(msg); - promise.setFailure(new IllegalArgumentException( - "Message must be an " + StringUtil.simpleClassName(Http2StreamFrame.class) + - ": " + msgStr)); - return; - } - - ChannelFuture f = write0(msg); - if (f.isDone()) { - writeComplete(f, promise); - } else { - final long bytes = FlowControlledFrameSizeEstimator.HANDLE_INSTANCE.size(msg); - incrementPendingOutboundBytes(bytes, false); - f.addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) { - writeComplete(future, promise); - decrementPendingOutboundBytes(bytes, false); - } - }); - } - } catch (Throwable t) { - promise.tryFailure(t); - } finally { - writeDoneAndNoFlush = true; - } - } - - private void firstWriteComplete(ChannelFuture future, ChannelPromise promise) { - Throwable cause = future.cause(); - if (cause == null) { - promise.setSuccess(); - } else { - // If the first write fails there is not much we can do, just close - closeForcibly(); - promise.setFailure(wrapStreamClosedError(cause)); - } - } - - private void writeComplete(ChannelFuture future, ChannelPromise promise) { - Throwable cause = future.cause(); - if (cause == null) { - promise.setSuccess(); - } else { - Throwable error = wrapStreamClosedError(cause); - // To make it more consistent with AbstractChannel we handle all IOExceptions here. - if (error instanceof IOException) { - if (config.isAutoClose()) { - // Close channel if needed. - closeForcibly(); - } else { - // TODO: Once Http2StreamChannel extends DuplexChannel we should call shutdownOutput(...) - outboundClosed = true; - } - } - promise.setFailure(error); - } - } - - private Throwable wrapStreamClosedError(Throwable cause) { - // If the error was caused by STREAM_CLOSED we should use a ClosedChannelException to better - // mimic other transports and make it easier to reason about what exceptions to expect. - if (cause instanceof Http2Exception && ((Http2Exception) cause).error() == Http2Error.STREAM_CLOSED) { - return new ClosedChannelException().initCause(cause); - } - return cause; - } - - private Http2StreamFrame validateStreamFrame(Http2StreamFrame frame) { - if (frame.stream() != null && frame.stream() != stream) { - String msgString = frame.toString(); - ReferenceCountUtil.release(frame); - throw new IllegalArgumentException( - "Stream " + frame.stream() + " must not be set on the frame: " + msgString); - } - return frame; - } - - private ChannelFuture write0(Object msg) { - ChannelPromise promise = ctx.newPromise(); - Http2MultiplexCodec.this.write(ctx, msg, promise); - return promise; - } - - @Override - public void flush() { - // If we are currently in the parent channel's read loop we should just ignore the flush. - // We will ensure we trigger ctx.flush() after we processed all Channels later on and - // so aggregate the flushes. This is done as ctx.flush() is expensive when as it may trigger an - // write(...) or writev(...) operation on the socket. - if (!writeDoneAndNoFlush || parentReadInProgress) { - // There is nothing to flush so this is a NOOP. - return; - } - try { - flush0(ctx); - } finally { - writeDoneAndNoFlush = false; - } - } - - @Override - public ChannelPromise voidPromise() { - return unsafeVoidPromise; - } - - @Override - public ChannelOutboundBuffer outboundBuffer() { - // Always return null as we not use the ChannelOutboundBuffer and not even support it. - return null; - } - } - - /** - * {@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 { - Http2StreamChannelConfig(Channel channel) { - super(channel); - } - - @Override - public MessageSizeEstimator getMessageSizeEstimator() { - return FlowControlledFrameSizeEstimator.INSTANCE; - } - - @Override - public ChannelConfig setMessageSizeEstimator(MessageSizeEstimator estimator) { - throw new UnsupportedOperationException(); - } - - @Override - public ChannelConfig setRecvByteBufAllocator(RecvByteBufAllocator allocator) { - if (!(allocator.newHandle() instanceof RecvByteBufAllocator.ExtendedHandle)) { - throw new IllegalArgumentException("allocator.newHandle() must return an object of type: " + - RecvByteBufAllocator.ExtendedHandle.class); - } - super.setRecvByteBufAllocator(allocator); - return this; - } + protected void flush0(ChannelHandlerContext ctx) { + Http2MultiplexCodec.this.flush0(ctx); } } } diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexCodecBuilder.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexCodecBuilder.java index ca0e22b189..b9fc170207 100644 --- a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexCodecBuilder.java +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexCodecBuilder.java @@ -23,7 +23,10 @@ import static io.netty.util.internal.ObjectUtil.checkNotNull; /** * A builder for {@link Http2MultiplexCodec}. + * + * @deprecated use {@link Http2FrameCodecBuilder} together with {@link Http2MultiplexHandler}. */ +@Deprecated @UnstableApi public class Http2MultiplexCodecBuilder extends AbstractHttp2ConnectionHandlerBuilder { diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexHandler.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexHandler.java new file mode 100644 index 0000000000..4a1a3db4a3 --- /dev/null +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2MultiplexHandler.java @@ -0,0 +1,395 @@ +/* + * Copyright 2019 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.channel.Channel; +import io.netty.channel.ChannelConfig; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.EventLoop; +import io.netty.handler.codec.http2.Http2FrameCodec.DefaultHttp2FrameStream; +import io.netty.util.ReferenceCounted; +import io.netty.util.internal.ObjectUtil; +import io.netty.util.internal.UnstableApi; + +import static io.netty.handler.codec.http2.Http2Error.INTERNAL_ERROR; +import static io.netty.handler.codec.http2.Http2Exception.connectionError; + +/** + * An HTTP/2 handler that creates child channels for each stream. This handler must be used in combination + * with {@link Http2FrameCodec}. + * + *

When a new stream is created, a new {@link Channel} is created for it. Applications send and + * receive {@link Http2StreamFrame}s on the created channel. {@link ByteBuf}s cannot be processed by the channel; + * all writes that reach the head of the pipeline must be an instance of {@link Http2StreamFrame}. Writes that reach + * the head of the pipeline are processed directly by this handler and cannot be intercepted. + * + *

The child channel will be notified of user events that impact the stream, such as {@link + * Http2GoAwayFrame} and {@link Http2ResetFrame}, as soon as they occur. Although {@code + * Http2GoAwayFrame} and {@code Http2ResetFrame} signify that the remote is ignoring further + * communication, closing of the channel is delayed until any inbound queue is drained with {@link + * Channel#read()}, which follows the default behavior of channels in Netty. Applications are + * free to close the channel in response to such events if they don't have use for any queued + * messages. Any connection level events like {@link Http2SettingsFrame} and {@link Http2GoAwayFrame} + * will be processed internally and also propagated down the pipeline for other handlers to act on. + * + *

Outbound streams are supported via the {@link Http2StreamChannelBootstrap}. + * + *

{@link ChannelConfig#setMaxMessagesPerRead(int)} and {@link ChannelConfig#setAutoRead(boolean)} are supported. + * + *

Reference Counting

+ * + * Some {@link Http2StreamFrame}s implement the {@link ReferenceCounted} interface, as they carry + * reference counted objects (e.g. {@link ByteBuf}s). The multiplex codec will call {@link ReferenceCounted#retain()} + * before propagating a reference counted object through the pipeline, and thus an application handler needs to release + * such an object after having consumed it. For more information on reference counting take a look at + * http://netty.io/wiki/reference-counted-objects.html + * + *

Channel Events

+ * + * 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. + * + *

Writability and Flow Control

+ * + * 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. + */ +@UnstableApi +public final class Http2MultiplexHandler extends Http2ChannelDuplexHandler { + + static final ChannelFutureListener CHILD_CHANNEL_REGISTRATION_LISTENER = new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) { + registerDone(future); + } + }; + + private final ChannelHandler inboundStreamHandler; + private final ChannelHandler upgradeStreamHandler; + + private boolean parentReadInProgress; + private int idCount; + + // Linked-List for Http2MultiplexHandlerStreamChannel instances that need to be processed by + // channelReadComplete(...) + private AbstractHttp2StreamChannel head; + private AbstractHttp2StreamChannel tail; + + // Need to be volatile as accessed from within the Http2MultiplexHandlerStreamChannel in a multi-threaded fashion. + private volatile ChannelHandlerContext ctx; + + /** + * Creates a new instance + * + * @param inboundStreamHandler the {@link ChannelHandler} that will be added to the {@link ChannelPipeline} of + * the {@link Channel}s created for new inbound streams. + */ + public Http2MultiplexHandler(ChannelHandler inboundStreamHandler) { + this(inboundStreamHandler, null); + } + + /** + * Creates a new instance + * + * @param inboundStreamHandler the {@link ChannelHandler} that will be added to the {@link ChannelPipeline} of + * the {@link Channel}s created for new inbound streams. + * @param upgradeStreamHandler the {@link ChannelHandler} that will be added to the {@link ChannelPipeline} of the + * upgraded {@link Channel}. + */ + public Http2MultiplexHandler(ChannelHandler inboundStreamHandler, ChannelHandler upgradeStreamHandler) { + this.inboundStreamHandler = ObjectUtil.checkNotNull(inboundStreamHandler, "inboundStreamHandler"); + this.upgradeStreamHandler = upgradeStreamHandler; + } + + static void registerDone(ChannelFuture future) { + // Handle any errors that occurred on the local thread while registering. Even though + // failures can happen after this point, they will be handled by the channel by closing the + // childChannel. + if (!future.isSuccess()) { + Channel childChannel = future.channel(); + if (childChannel.isRegistered()) { + childChannel.close(); + } else { + childChannel.unsafe().closeForcibly(); + } + } + } + + @Override + protected void handlerAdded0(ChannelHandlerContext ctx) { + if (ctx.executor() != ctx.channel().eventLoop()) { + throw new IllegalStateException("EventExecutor must be EventLoop of Channel"); + } + this.ctx = ctx; + } + + @Override + protected void handlerRemoved0(ChannelHandlerContext ctx) { + // Unlink the linked list to guard against GC nepotism. + AbstractHttp2StreamChannel ch = head; + while (ch != null) { + AbstractHttp2StreamChannel curr = ch; + ch = curr.next; + curr.next = curr.previous = null; + } + head = tail = null; + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + parentReadInProgress = true; + if (msg instanceof Http2StreamFrame) { + Http2StreamFrame streamFrame = (Http2StreamFrame) msg; + DefaultHttp2FrameStream s = + (DefaultHttp2FrameStream) streamFrame.stream(); + ((AbstractHttp2StreamChannel) s.attachment).fireChildRead(streamFrame); + return; + } + + if (msg instanceof Http2GoAwayFrame) { + onHttp2GoAwayFrame(ctx, (Http2GoAwayFrame) msg); + } + + // Send everything down the pipeline + ctx.fireChannelRead(msg); + } + + @Override + public void channelWritabilityChanged(final ChannelHandlerContext ctx) throws Exception { + if (ctx.channel().isWritable()) { + // While the writability state may change during iterating of the streams we just set all of the streams + // to writable to not affect fairness. These will be "limited" by their own watermarks in any case. + forEachActiveStream(AbstractHttp2StreamChannel.WRITABLE_VISITOR); + } + + ctx.fireChannelWritabilityChanged(); + } + + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof Http2FrameStreamEvent) { + Http2FrameStreamEvent event = (Http2FrameStreamEvent) evt; + DefaultHttp2FrameStream stream = (DefaultHttp2FrameStream) event.stream(); + if (event.type() == Http2FrameStreamEvent.Type.State) { + switch (stream.state()) { + case HALF_CLOSED_LOCAL: + if (stream.id() != Http2CodecUtil.HTTP_UPGRADE_STREAM_ID) { + // Ignore everything which was not caused by an upgrade + break; + } + // We must have an upgrade handler or else we can't handle the stream + if (upgradeStreamHandler == null) { + throw connectionError(INTERNAL_ERROR, + "Client is misconfigured for upgrade requests"); + } + // fall-trough + case HALF_CLOSED_REMOTE: + // fall-trough + case OPEN: + if (stream.attachment != null) { + // ignore if child channel was already created. + break; + } + final AbstractHttp2StreamChannel ch; + if (stream.state() == Http2Stream.State.HALF_CLOSED_LOCAL) { + ch = new Http2MultiplexHandlerStreamChannel(stream, null); + ch.closeOutbound(); + // Add our upgrade handler to the channel and then register the channel. + // The register call fires the channelActive, etc. + ch.pipeline().addLast(upgradeStreamHandler); + } else { + ch = new Http2MultiplexHandlerStreamChannel(stream, inboundStreamHandler); + } + ChannelFuture future = ctx.channel().eventLoop().register(ch); + if (future.isDone()) { + registerDone(future); + } else { + future.addListener(CHILD_CHANNEL_REGISTRATION_LISTENER); + } + break; + case CLOSED: + AbstractHttp2StreamChannel channel = (AbstractHttp2StreamChannel) stream.attachment; + if (channel != null) { + channel.streamClosed(); + } + break; + default: + // ignore for now + break; + } + } + return; + } + ctx.fireUserEventTriggered(evt); + } + + // TODO: This is most likely not the best way to expose this, need to think more about it. + Http2StreamChannel newOutboundStream() { + return new Http2MultiplexHandlerStreamChannel((DefaultHttp2FrameStream) newStream(), null); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + if (cause instanceof Http2FrameStreamException) { + Http2FrameStreamException exception = (Http2FrameStreamException) cause; + Http2FrameStream stream = exception.stream(); + AbstractHttp2StreamChannel childChannel = (AbstractHttp2StreamChannel) + ((DefaultHttp2FrameStream) stream).attachment; + try { + childChannel.pipeline().fireExceptionCaught(cause.getCause()); + } finally { + childChannel.unsafe().closeForcibly(); + } + return; + } + ctx.fireExceptionCaught(cause); + } + + private boolean isChildChannelInReadPendingQueue(AbstractHttp2StreamChannel childChannel) { + return childChannel.previous != null || childChannel.next != null || head == childChannel; + } + + private boolean tryAddChildChannelToReadPendingQueue(AbstractHttp2StreamChannel childChannel) { + if (!isChildChannelInReadPendingQueue(childChannel)) { + if (tail == null) { + assert head == null; + tail = head = childChannel; + } else { + childChannel.previous = tail; + tail.next = childChannel; + tail = childChannel; + } + return true; + } + return false; + } + + private void tryRemoveChildChannelFromReadPendingQueue(AbstractHttp2StreamChannel childChannel) { + if (isChildChannelInReadPendingQueue(childChannel)) { + AbstractHttp2StreamChannel previous = childChannel.previous; + if (childChannel.next != null) { + childChannel.next.previous = previous; + } else { + tail = tail.previous; // If there is no next, this childChannel is the tail, so move the tail back. + } + if (previous != null) { + previous.next = childChannel.next; + } else { + head = head.next; // If there is no previous, this childChannel is the head, so move the tail forward. + } + childChannel.next = childChannel.previous = null; + } + } + + private void onHttp2GoAwayFrame(ChannelHandlerContext ctx, final Http2GoAwayFrame goAwayFrame) { + try { + forEachActiveStream(new Http2FrameStreamVisitor() { + @Override + public boolean visit(Http2FrameStream stream) { + final int streamId = stream.id(); + if (streamId > goAwayFrame.lastStreamId() && isValidLocalStreamId(stream)) { + final AbstractHttp2StreamChannel childChannel = (AbstractHttp2StreamChannel) + ((DefaultHttp2FrameStream) stream).attachment; + childChannel.pipeline().fireUserEventTriggered(goAwayFrame.retainedDuplicate()); + } + return true; + } + }); + } catch (Http2Exception e) { + ctx.fireExceptionCaught(e); + ctx.close(); + } + } + + /** + * Notifies any child streams of the read completion. + */ + @Override + public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { + parentReadInProgress = true; + // If we have many child channel we can optimize for the case when multiple call flush() in + // channelReadComplete(...) callbacks and only do it once as otherwise we will end-up with multiple + // write calls on the socket which is expensive. + AbstractHttp2StreamChannel current = head; + if (current != null) { + try { + do { + AbstractHttp2StreamChannel childChannel = current; + // Clear early in case fireChildReadComplete() causes it to need to be re-processed + current = current.next; + childChannel.next = childChannel.previous = null; + childChannel.fireChildReadComplete(); + } while (current != null); + } finally { + parentReadInProgress = false; + tail = head = null; + ctx.flush(); + } + } else { + parentReadInProgress = false; + } + ctx.fireChannelReadComplete(); + } + + private final class Http2MultiplexHandlerStreamChannel extends AbstractHttp2StreamChannel { + + Http2MultiplexHandlerStreamChannel(DefaultHttp2FrameStream stream, ChannelHandler inboundHandler) { + super(stream, ++idCount, inboundHandler); + } + + @Override + protected boolean consumeBytes(Http2FrameStream stream, int bytes) throws Http2Exception { + return Http2MultiplexHandler.this.consumeBytes(stream, bytes); + } + + @Override + protected boolean isParentReadInProgress() { + return parentReadInProgress; + } + + @Override + protected boolean streamMayHaveExisted(Http2FrameStream stream) { + return Http2MultiplexHandler.this.streamMayHaveExisted(stream); + } + + @Override + protected void tryRemoveChildChannelFromReadPendingQueue() { + Http2MultiplexHandler.this.tryRemoveChildChannelFromReadPendingQueue(this); + } + + @Override + protected boolean tryAddChildChannelToReadPendingQueue() { + return Http2MultiplexHandler.this.tryAddChildChannelToReadPendingQueue(this); + } + + @Override + protected ChannelHandlerContext parentContext() { + return ctx; + } + } +} + diff --git a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2StreamChannelBootstrap.java b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2StreamChannelBootstrap.java index 6e9b797224..0166d4ead5 100644 --- a/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2StreamChannelBootstrap.java +++ b/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2StreamChannelBootstrap.java @@ -105,7 +105,10 @@ public final class Http2StreamChannelBootstrap { } public Future open(final Promise promise) { - final ChannelHandlerContext ctx = channel.pipeline().context(Http2MultiplexCodec.class); + ChannelHandlerContext ctx = channel.pipeline().context(Http2MultiplexCodec.class); + if (ctx == null) { + ctx = channel.pipeline().context(Http2MultiplexHandler.class); + } if (ctx == null) { if (channel.isActive()) { promise.setFailure(new IllegalStateException(StringUtil.simpleClassName(Http2MultiplexCodec.class) + @@ -118,10 +121,11 @@ public final class Http2StreamChannelBootstrap { if (executor.inEventLoop()) { open0(ctx, promise); } else { + final ChannelHandlerContext finalCtx = ctx; executor.execute(new Runnable() { @Override public void run() { - open0(ctx, promise); + open0(finalCtx, promise); } }); } @@ -131,7 +135,12 @@ public final class Http2StreamChannelBootstrap { public void open0(ChannelHandlerContext ctx, final Promise promise) { assert ctx.executor().inEventLoop(); - final Http2StreamChannel streamChannel = ((Http2MultiplexCodec) ctx.handler()).newOutboundStream(); + final Http2StreamChannel streamChannel; + if (ctx.handler() instanceof Http2MultiplexCodec) { + streamChannel = ((Http2MultiplexCodec) ctx.handler()).newOutboundStream(); + } else { + streamChannel = ((Http2MultiplexHandler) ctx.handler()).newOutboundStream(); + } try { init(streamChannel); } catch (Exception e) { diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexClientUpgradeTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexClientUpgradeTest.java new file mode 100644 index 0000000000..e2bb88ff59 --- /dev/null +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexClientUpgradeTest.java @@ -0,0 +1,79 @@ +/* + * Copyright 2018 The Netty Project + * + * The Netty Project licenses this file to you under the Apache License, version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at: + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package io.netty.handler.codec.http2; + +import org.junit.Test; + +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.embedded.EmbeddedChannel; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public abstract class Http2MultiplexClientUpgradeTest { + + @ChannelHandler.Sharable + final class NoopHandler extends ChannelInboundHandlerAdapter { + @Override + public void channelActive(ChannelHandlerContext ctx) { + ctx.channel().close(); + } + } + + private final class UpgradeHandler extends ChannelInboundHandlerAdapter { + Http2Stream.State stateOnActive; + int streamId; + + @Override + public void channelActive(ChannelHandlerContext ctx) throws Exception { + Http2StreamChannel ch = (Http2StreamChannel) ctx.channel(); + stateOnActive = ch.stream().state(); + streamId = ch.stream().id(); + super.channelActive(ctx); + } + } + + protected abstract C newCodec(ChannelHandler upgradeHandler); + + protected abstract ChannelHandler newMultiplexer(ChannelHandler upgradeHandler); + + @Test + public void upgradeHandlerGetsActivated() throws Exception { + UpgradeHandler upgradeHandler = new UpgradeHandler(); + C codec = newCodec(upgradeHandler); + EmbeddedChannel ch = new EmbeddedChannel(codec, newMultiplexer(upgradeHandler)); + + codec.onHttpClientUpgrade(); + + assertFalse(upgradeHandler.stateOnActive.localSideOpen()); + assertTrue(upgradeHandler.stateOnActive.remoteSideOpen()); + assertEquals(1, upgradeHandler.streamId); + assertTrue(ch.finishAndReleaseAll()); + } + + @Test(expected = Http2Exception.class) + public void clientUpgradeWithoutUpgradeHandlerThrowsHttp2Exception() throws Http2Exception { + C codec = newCodec(null); + EmbeddedChannel ch = new EmbeddedChannel(codec, newMultiplexer(null)); + try { + codec.onHttpClientUpgrade(); + } finally { + assertTrue(ch.finishAndReleaseAll()); + } + } +} diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecClientUpgradeTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecClientUpgradeTest.java index 26b63ed7f9..b917feb345 100644 --- a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecClientUpgradeTest.java +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecClientUpgradeTest.java @@ -1,5 +1,5 @@ /* - * Copyright 2018 The Netty Project + * Copyright 2019 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 @@ -14,68 +14,21 @@ */ package io.netty.handler.codec.http2; -import org.junit.Test; - import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundHandlerAdapter; -import io.netty.channel.embedded.EmbeddedChannel; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +public class Http2MultiplexCodecClientUpgradeTest extends Http2MultiplexClientUpgradeTest { -public class Http2MultiplexCodecClientUpgradeTest { - - @ChannelHandler.Sharable - private final class NoopHandler extends ChannelInboundHandlerAdapter { - @Override - public void channelActive(ChannelHandlerContext ctx) { - ctx.channel().close(); - } - } - - private final class UpgradeHandler extends ChannelInboundHandlerAdapter { - Http2Stream.State stateOnActive; - int streamId; - - @Override - public void channelActive(ChannelHandlerContext ctx) throws Exception { - Http2StreamChannel ch = (Http2StreamChannel) ctx.channel(); - stateOnActive = ch.stream().state(); - streamId = ch.stream().id(); - super.channelActive(ctx); - } - } - - private Http2MultiplexCodec newCodec(ChannelHandler upgradeHandler) { + @Override + protected Http2MultiplexCodec newCodec(ChannelHandler upgradeHandler) { Http2MultiplexCodecBuilder builder = Http2MultiplexCodecBuilder.forClient(new NoopHandler()); - builder.withUpgradeStreamHandler(upgradeHandler); + if (upgradeHandler != null) { + builder.withUpgradeStreamHandler(upgradeHandler); + } return builder.build(); } - @Test - public void upgradeHandlerGetsActivated() throws Exception { - UpgradeHandler upgradeHandler = new UpgradeHandler(); - Http2MultiplexCodec codec = newCodec(upgradeHandler); - EmbeddedChannel ch = new EmbeddedChannel(codec); - - codec.onHttpClientUpgrade(); - - assertFalse(upgradeHandler.stateOnActive.localSideOpen()); - assertTrue(upgradeHandler.stateOnActive.remoteSideOpen()); - assertEquals(1, upgradeHandler.streamId); - assertTrue(ch.finishAndReleaseAll()); - } - - @Test(expected = Http2Exception.class) - public void clientUpgradeWithoutUpgradeHandlerThrowsHttp2Exception() throws Http2Exception { - Http2MultiplexCodec codec = Http2MultiplexCodecBuilder.forClient(new NoopHandler()).build(); - EmbeddedChannel ch = new EmbeddedChannel(codec); - try { - codec.onHttpClientUpgrade(); - } finally { - assertTrue(ch.finishAndReleaseAll()); - } + @Override + protected ChannelHandler newMultiplexer(ChannelHandler upgradeHandler) { + return null; } } diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecTest.java index 262d7dbd7c..3f18918dc4 100644 --- a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecTest.java +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecTest.java @@ -1,5 +1,5 @@ /* - * Copyright 2016 The Netty Project + * Copyright 2019 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 @@ -14,1022 +14,17 @@ */ package io.netty.handler.codec.http2; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundHandlerAdapter; -import io.netty.channel.ChannelPromise; -import io.netty.channel.WriteBufferWaterMark; -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.LastInboundHandler.Consumer; -import io.netty.util.AsciiString; -import io.netty.util.AttributeKey; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.mockito.ArgumentMatcher; -import org.mockito.Mockito; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import java.net.InetSocketAddress; -import java.nio.channels.ClosedChannelException; -import java.util.ArrayDeque; -import java.util.Queue; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; +public class Http2MultiplexCodecTest extends Http2MultiplexTest { -import static io.netty.handler.codec.http2.Http2TestUtil.anyChannelPromise; -import static io.netty.handler.codec.http2.Http2TestUtil.anyHttp2Settings; -import static io.netty.handler.codec.http2.Http2TestUtil.assertEqualsAndRelease; -import static io.netty.handler.codec.http2.Http2TestUtil.bb; -import static io.netty.util.ReferenceCountUtil.release; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyShort; -import static org.mockito.ArgumentMatchers.argThat; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -/** - * Unit tests for {@link Http2MultiplexCodec}. - */ -public class Http2MultiplexCodecTest { - private final Http2Headers request = new DefaultHttp2Headers() - .method(HttpMethod.GET.asciiName()).scheme(HttpScheme.HTTPS.name()) - .authority(new AsciiString("example.org")).path(new AsciiString("/foo")); - - private EmbeddedChannel parentChannel; - private Http2FrameWriter frameWriter; - private Http2FrameInboundWriter frameInboundWriter; - private TestChannelInitializer childChannelInitializer; - private Http2MultiplexCodec codec; - - private static final int initialRemoteStreamWindow = 1024; - - @Before - public void setUp() { - childChannelInitializer = new TestChannelInitializer(); - parentChannel = new EmbeddedChannel(); - frameInboundWriter = new Http2FrameInboundWriter(parentChannel); - parentChannel.connect(new InetSocketAddress(0)); - frameWriter = Http2TestUtil.mockedFrameWriter(); - codec = new Http2MultiplexCodecBuilder(true, childChannelInitializer).frameWriter(frameWriter).build(); - parentChannel.pipeline().addLast(codec); - parentChannel.runPendingTasks(); - parentChannel.pipeline().fireChannelActive(); - - parentChannel.writeInbound(Http2CodecUtil.connectionPrefaceBuf()); - - Http2Settings settings = new Http2Settings().initialWindowSize(initialRemoteStreamWindow); - frameInboundWriter.writeInboundSettings(settings); - - verify(frameWriter).writeSettingsAck(eqMultiplexCodecCtx(), anyChannelPromise()); - - frameInboundWriter.writeInboundSettingsAck(); - - Http2SettingsFrame settingsFrame = parentChannel.readInbound(); - assertNotNull(settingsFrame); - Http2SettingsAckFrame settingsAckFrame = parentChannel.readInbound(); - assertNotNull(settingsAckFrame); - - // Handshake - verify(frameWriter).writeSettings(eqMultiplexCodecCtx(), - anyHttp2Settings(), anyChannelPromise()); + @Override + protected Http2FrameCodec newCodec(TestChannelInitializer childChannelInitializer, Http2FrameWriter frameWriter) { + return new Http2MultiplexCodecBuilder(true, childChannelInitializer).frameWriter(frameWriter).build(); } - private ChannelHandlerContext eqMultiplexCodecCtx() { - return eq(codec.ctx); - } - - @After - public void tearDown() throws Exception { - if (childChannelInitializer.handler instanceof LastInboundHandler) { - ((LastInboundHandler) childChannelInitializer.handler).finishAndReleaseAll(); - } - parentChannel.finishAndReleaseAll(); - codec = null; - } - - // TODO(buchgr): Flush from child channel - // TODO(buchgr): ChildChannel.childReadComplete() - // TODO(buchgr): GOAWAY Logic - // TODO(buchgr): Test ChannelConfig.setMaxMessagesPerRead - - @Test - public void writeUnknownFrame() { - Http2StreamChannel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter() { - @Override - public void channelActive(ChannelHandlerContext ctx) { - ctx.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); - ctx.writeAndFlush(new DefaultHttp2UnknownFrame((byte) 99, new Http2Flags())); - ctx.fireChannelActive(); - } - }); - assertTrue(childChannel.isActive()); - - parentChannel.runPendingTasks(); - - verify(frameWriter).writeFrame(eq(codec.ctx), eq((byte) 99), eqStreamId(childChannel), any(Http2Flags.class), - any(ByteBuf.class), any(ChannelPromise.class)); - } - - private Http2StreamChannel newInboundStream(int streamId, boolean endStream, final ChannelHandler childHandler) { - return newInboundStream(streamId, endStream, null, childHandler); - } - - private Http2StreamChannel newInboundStream(int streamId, boolean endStream, - AtomicInteger maxReads, final ChannelHandler childHandler) { - final AtomicReference streamChannelRef = new AtomicReference(); - childChannelInitializer.maxReads = maxReads; - childChannelInitializer.handler = new ChannelInboundHandlerAdapter() { - @Override - public void channelRegistered(ChannelHandlerContext ctx) { - assertNull(streamChannelRef.get()); - streamChannelRef.set((Http2StreamChannel) ctx.channel()); - ctx.pipeline().addLast(childHandler); - ctx.fireChannelRegistered(); - } - }; - - frameInboundWriter.writeInboundHeaders(streamId, request, 0, endStream); - parentChannel.runPendingTasks(); - Http2StreamChannel channel = streamChannelRef.get(); - assertEquals(streamId, channel.stream().id()); - return channel; - } - - @Test - public void readUnkownFrame() { - LastInboundHandler handler = new LastInboundHandler(); - - Http2StreamChannel channel = newInboundStream(3, true, handler); - frameInboundWriter.writeInboundFrame((byte) 99, channel.stream().id(), new Http2Flags(), Unpooled.EMPTY_BUFFER); - - // header frame and unknown frame - verifyFramesMultiplexedToCorrectChannel(channel, handler, 2); - - Channel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter()); - assertTrue(childChannel.isActive()); - } - - @Test - public void headerAndDataFramesShouldBeDelivered() { - LastInboundHandler inboundHandler = new LastInboundHandler(); - - Http2StreamChannel channel = newInboundStream(3, false, inboundHandler); - Http2HeadersFrame headersFrame = new DefaultHttp2HeadersFrame(request).stream(channel.stream()); - Http2DataFrame dataFrame1 = new DefaultHttp2DataFrame(bb("hello")).stream(channel.stream()); - Http2DataFrame dataFrame2 = new DefaultHttp2DataFrame(bb("world")).stream(channel.stream()); - - assertTrue(inboundHandler.isChannelActive()); - frameInboundWriter.writeInboundData(channel.stream().id(), bb("hello"), 0, false); - frameInboundWriter.writeInboundData(channel.stream().id(), bb("world"), 0, false); - - assertEquals(headersFrame, inboundHandler.readInbound()); - - assertEqualsAndRelease(dataFrame1, inboundHandler.readInbound()); - assertEqualsAndRelease(dataFrame2, inboundHandler.readInbound()); - - assertNull(inboundHandler.readInbound()); - } - - @Test - public void framesShouldBeMultiplexed() { - LastInboundHandler handler1 = new LastInboundHandler(); - Http2StreamChannel channel1 = newInboundStream(3, false, handler1); - LastInboundHandler handler2 = new LastInboundHandler(); - Http2StreamChannel channel2 = newInboundStream(5, false, handler2); - LastInboundHandler handler3 = new LastInboundHandler(); - Http2StreamChannel channel3 = newInboundStream(11, false, handler3); - - verifyFramesMultiplexedToCorrectChannel(channel1, handler1, 1); - verifyFramesMultiplexedToCorrectChannel(channel2, handler2, 1); - verifyFramesMultiplexedToCorrectChannel(channel3, handler3, 1); - - frameInboundWriter.writeInboundData(channel2.stream().id(), bb("hello"), 0, false); - frameInboundWriter.writeInboundData(channel1.stream().id(), bb("foo"), 0, true); - frameInboundWriter.writeInboundData(channel2.stream().id(), bb("world"), 0, true); - frameInboundWriter.writeInboundData(channel3.stream().id(), bb("bar"), 0, true); - - verifyFramesMultiplexedToCorrectChannel(channel1, handler1, 1); - verifyFramesMultiplexedToCorrectChannel(channel2, handler2, 2); - verifyFramesMultiplexedToCorrectChannel(channel3, handler3, 1); - } - - @Test - public void inboundDataFrameShouldUpdateLocalFlowController() throws Http2Exception { - Http2LocalFlowController flowController = Mockito.mock(Http2LocalFlowController.class); - codec.connection().local().flowController(flowController); - - LastInboundHandler handler = new LastInboundHandler(); - final Http2StreamChannel channel = newInboundStream(3, false, handler); - - ByteBuf tenBytes = bb("0123456789"); - - frameInboundWriter.writeInboundData(channel.stream().id(), tenBytes, 0, true); - - // Verify we marked the bytes as consumed - verify(flowController).consumeBytes(argThat(new ArgumentMatcher() { - @Override - public boolean matches(Http2Stream http2Stream) { - return http2Stream.id() == channel.stream().id(); - } - }), eq(10)); - - // headers and data frame - verifyFramesMultiplexedToCorrectChannel(channel, handler, 2); - } - - @Test - public void unhandledHttp2FramesShouldBePropagated() { - Http2PingFrame pingFrame = new DefaultHttp2PingFrame(0); - frameInboundWriter.writeInboundPing(false, 0); - assertEquals(parentChannel.readInbound(), pingFrame); - - DefaultHttp2GoAwayFrame goAwayFrame = new DefaultHttp2GoAwayFrame(1, - parentChannel.alloc().buffer().writeLong(8)); - frameInboundWriter.writeInboundGoAway(0, goAwayFrame.errorCode(), goAwayFrame.content().retainedDuplicate()); - - Http2GoAwayFrame frame = parentChannel.readInbound(); - assertEqualsAndRelease(frame, goAwayFrame); - } - - @Test - public void channelReadShouldRespectAutoRead() { - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); - assertTrue(childChannel.config().isAutoRead()); - Http2HeadersFrame headersFrame = inboundHandler.readInbound(); - assertNotNull(headersFrame); - - childChannel.config().setAutoRead(false); - - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("hello world"), 0, false); - Http2DataFrame dataFrame0 = inboundHandler.readInbound(); - assertNotNull(dataFrame0); - release(dataFrame0); - - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("foo"), 0, false); - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("bar"), 0, false); - - assertNull(inboundHandler.readInbound()); - - childChannel.config().setAutoRead(true); - verifyFramesMultiplexedToCorrectChannel(childChannel, inboundHandler, 2); - } - - @Test - public void readInChannelReadWithoutAutoRead() { - useReadWithoutAutoRead(false); - } - - @Test - public void readInChannelReadCompleteWithoutAutoRead() { - useReadWithoutAutoRead(true); - } - - private void useReadWithoutAutoRead(final boolean readComplete) { - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); - assertTrue(childChannel.config().isAutoRead()); - childChannel.config().setAutoRead(false); - assertFalse(childChannel.config().isAutoRead()); - - Http2HeadersFrame headersFrame = inboundHandler.readInbound(); - assertNotNull(headersFrame); - - // Add a handler which will request reads. - childChannel.pipeline().addFirst(new ChannelInboundHandlerAdapter() { - @Override - public void channelRead(ChannelHandlerContext ctx, Object msg) { - ctx.fireChannelRead(msg); - if (!readComplete) { - ctx.read(); - } - } - - @Override - public void channelReadComplete(ChannelHandlerContext ctx) { - ctx.fireChannelReadComplete(); - if (readComplete) { - ctx.read(); - } - } - }); - - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("hello world"), 0, false); - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("foo"), 0, false); - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("bar"), 0, false); - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("hello world"), 0, false); - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("foo"), 0, false); - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("bar"), 0, true); - - verifyFramesMultiplexedToCorrectChannel(childChannel, inboundHandler, 6); - } - - private Http2StreamChannel newOutboundStream(ChannelHandler handler) { - return new Http2StreamChannelBootstrap(parentChannel).handler(handler) - .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() { - LastInboundHandler handler = new LastInboundHandler(); - - Channel childChannel = newOutboundStream(handler); - assertTrue(childChannel.isActive()); - - childChannel.close(); - parentChannel.runPendingTasks(); - - assertFalse(childChannel.isOpen()); - assertFalse(childChannel.isActive()); - assertNull(parentChannel.readOutbound()); - } - - @Test - public void outboundStreamShouldWriteResetFrameOnClose_headersSent() { - ChannelHandler handler = new ChannelInboundHandlerAdapter() { - @Override - public void channelActive(ChannelHandlerContext ctx) { - ctx.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); - ctx.fireChannelActive(); - } - }; - - Http2StreamChannel childChannel = newOutboundStream(handler); - assertTrue(childChannel.isActive()); - - childChannel.close(); - verify(frameWriter).writeRstStream(eqMultiplexCodecCtx(), - eqStreamId(childChannel), eq(Http2Error.CANCEL.code()), anyChannelPromise()); - } - - @Test - public void outboundStreamShouldNotWriteResetFrameOnClose_IfStreamDidntExist() { - when(frameWriter.writeHeaders(eqMultiplexCodecCtx(), anyInt(), - any(Http2Headers.class), anyInt(), anyShort(), anyBoolean(), anyInt(), anyBoolean(), - any(ChannelPromise.class))).thenAnswer(new Answer() { - - private boolean headersWritten; - @Override - public ChannelFuture answer(InvocationOnMock invocationOnMock) { - // We want to fail to write the first headers frame. This is what happens if the connection - // refuses to allocate a new stream due to having received a GOAWAY. - if (!headersWritten) { - headersWritten = true; - return ((ChannelPromise) invocationOnMock.getArgument(8)).setFailure(new Exception("boom")); - } - return ((ChannelPromise) invocationOnMock.getArgument(8)).setSuccess(); - } - }); - - Http2StreamChannel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter() { - @Override - public void channelActive(ChannelHandlerContext ctx) { - ctx.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); - ctx.fireChannelActive(); - } - }); - - assertFalse(childChannel.isActive()); - - childChannel.close(); - parentChannel.runPendingTasks(); - // The channel was never active so we should not generate a RST frame. - verify(frameWriter, never()).writeRstStream(eqMultiplexCodecCtx(), eqStreamId(childChannel), anyLong(), - anyChannelPromise()); - - assertTrue(parentChannel.outboundMessages().isEmpty()); - } - - @Test - public void inboundRstStreamFireChannelInactive() { - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel channel = newInboundStream(3, false, inboundHandler); - assertTrue(inboundHandler.isChannelActive()); - frameInboundWriter.writeInboundRstStream(channel.stream().id(), Http2Error.INTERNAL_ERROR.code()); - - assertFalse(inboundHandler.isChannelActive()); - - // A RST_STREAM frame should NOT be emitted, as we received a RST_STREAM. - verify(frameWriter, Mockito.never()).writeRstStream(eqMultiplexCodecCtx(), eqStreamId(channel), - anyLong(), anyChannelPromise()); - } - - @Test(expected = StreamException.class) - public void streamExceptionTriggersChildChannelExceptionAndClose() throws Exception { - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel channel = newInboundStream(3, false, inboundHandler); - assertTrue(channel.isActive()); - StreamException cause = new StreamException(channel.stream().id(), Http2Error.PROTOCOL_ERROR, "baaam!"); - parentChannel.pipeline().fireExceptionCaught(cause); - - assertFalse(channel.isActive()); - inboundHandler.checkException(); - } - - @Test(expected = ClosedChannelException.class) - public void streamClosedErrorTranslatedToClosedChannelExceptionOnWrites() throws Exception { - LastInboundHandler inboundHandler = new LastInboundHandler(); - - final Http2StreamChannel childChannel = newOutboundStream(inboundHandler); - assertTrue(childChannel.isActive()); - - Http2Headers headers = new DefaultHttp2Headers(); - when(frameWriter.writeHeaders(eqMultiplexCodecCtx(), anyInt(), - eq(headers), anyInt(), anyShort(), anyBoolean(), anyInt(), anyBoolean(), - any(ChannelPromise.class))).thenAnswer(new Answer() { - @Override - public ChannelFuture answer(InvocationOnMock invocationOnMock) { - return ((ChannelPromise) invocationOnMock.getArgument(8)).setFailure( - new StreamException(childChannel.stream().id(), Http2Error.STREAM_CLOSED, "Stream Closed")); - } - }); - ChannelFuture future = childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); - - parentChannel.flush(); - - assertFalse(childChannel.isActive()); - assertFalse(childChannel.isOpen()); - - inboundHandler.checkException(); - - future.syncUninterruptibly(); - } - - @Test - public void creatingWritingReadingAndClosingOutboundStreamShouldWork() { - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel childChannel = newOutboundStream(inboundHandler); - assertTrue(childChannel.isActive()); - assertTrue(inboundHandler.isChannelActive()); - - // Write to the child channel - Http2Headers headers = new DefaultHttp2Headers().scheme("https").method("GET").path("/foo.txt"); - childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(headers)); - - // Read from the child channel - frameInboundWriter.writeInboundHeaders(childChannel.stream().id(), headers, 0, false); - - Http2HeadersFrame headersFrame = inboundHandler.readInbound(); - assertNotNull(headersFrame); - assertEquals(headers, headersFrame.headers()); - - // Close the child channel. - childChannel.close(); - - parentChannel.runPendingTasks(); - // An active outbound stream should emit a RST_STREAM frame. - verify(frameWriter).writeRstStream(eqMultiplexCodecCtx(), eqStreamId(childChannel), - anyLong(), anyChannelPromise()); - - assertFalse(childChannel.isOpen()); - assertFalse(childChannel.isActive()); - 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(expected = Http2NoMoreStreamIdsException.class) - public void failedOutboundStreamCreationThrowsAndClosesChannel() throws Exception { - LastInboundHandler handler = new LastInboundHandler(); - Http2StreamChannel childChannel = newOutboundStream(handler); - assertTrue(childChannel.isActive()); - - Http2Headers headers = new DefaultHttp2Headers(); - when(frameWriter.writeHeaders(eqMultiplexCodecCtx(), anyInt(), - eq(headers), anyInt(), anyShort(), anyBoolean(), anyInt(), anyBoolean(), - any(ChannelPromise.class))).thenAnswer(new Answer() { - @Override - public ChannelFuture answer(InvocationOnMock invocationOnMock) { - return ((ChannelPromise) invocationOnMock.getArgument(8)).setFailure( - new Http2NoMoreStreamIdsException()); - } - }); - - ChannelFuture future = childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(headers)); - parentChannel.flush(); - - assertFalse(childChannel.isActive()); - assertFalse(childChannel.isOpen()); - - handler.checkException(); - - future.syncUninterruptibly(); - } - - @Test - public void channelClosedWhenCloseListenerCompletes() { - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); - - assertTrue(childChannel.isOpen()); - assertTrue(childChannel.isActive()); - - final AtomicBoolean channelOpen = new AtomicBoolean(true); - final AtomicBoolean channelActive = new AtomicBoolean(true); - - // Create a promise before actually doing the close, because otherwise we would be adding a listener to a future - // that is already completed because we are using EmbeddedChannel which executes code in the JUnit thread. - ChannelPromise p = childChannel.newPromise(); - p.addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) { - channelOpen.set(future.channel().isOpen()); - channelActive.set(future.channel().isActive()); - } - }); - childChannel.close(p).syncUninterruptibly(); - - assertFalse(channelOpen.get()); - assertFalse(channelActive.get()); - assertFalse(childChannel.isActive()); - } - - @Test - public void channelClosedWhenChannelClosePromiseCompletes() { - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); - - assertTrue(childChannel.isOpen()); - assertTrue(childChannel.isActive()); - - final AtomicBoolean channelOpen = new AtomicBoolean(true); - final AtomicBoolean channelActive = new AtomicBoolean(true); - - childChannel.closeFuture().addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) { - channelOpen.set(future.channel().isOpen()); - channelActive.set(future.channel().isActive()); - } - }); - childChannel.close().syncUninterruptibly(); - - assertFalse(channelOpen.get()); - assertFalse(channelActive.get()); - assertFalse(childChannel.isActive()); - } - - @Test - public void channelClosedWhenWriteFutureFails() { - final Queue writePromises = new ArrayDeque(); - - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); - - assertTrue(childChannel.isOpen()); - assertTrue(childChannel.isActive()); - - final AtomicBoolean channelOpen = new AtomicBoolean(true); - final AtomicBoolean channelActive = new AtomicBoolean(true); - - Http2Headers headers = new DefaultHttp2Headers(); - when(frameWriter.writeHeaders(eqMultiplexCodecCtx(), anyInt(), - eq(headers), anyInt(), anyShort(), anyBoolean(), anyInt(), anyBoolean(), - any(ChannelPromise.class))).thenAnswer(new Answer() { - @Override - public ChannelFuture answer(InvocationOnMock invocationOnMock) { - ChannelPromise promise = invocationOnMock.getArgument(8); - writePromises.offer(promise); - return promise; - } - }); - - ChannelFuture f = childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(headers)); - assertFalse(f.isDone()); - f.addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - channelOpen.set(future.channel().isOpen()); - channelActive.set(future.channel().isActive()); - } - }); - - ChannelPromise first = writePromises.poll(); - first.setFailure(new ClosedChannelException()); - f.awaitUninterruptibly(); - - assertFalse(channelOpen.get()); - assertFalse(channelActive.get()); - assertFalse(childChannel.isActive()); - } - - @Test - public void channelClosedTwiceMarksPromiseAsSuccessful() { - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); - - assertTrue(childChannel.isOpen()); - assertTrue(childChannel.isActive()); - childChannel.close().syncUninterruptibly(); - childChannel.close().syncUninterruptibly(); - - assertFalse(childChannel.isOpen()); - assertFalse(childChannel.isActive()); - } - - @Test - public void settingChannelOptsAndAttrs() { - AttributeKey key = AttributeKey.newInstance("foo"); - - Channel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter()); - 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 outboundFlowControlWritability() { - Http2StreamChannel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter()); - assertTrue(childChannel.isActive()); - - assertTrue(childChannel.isWritable()); - childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); - parentChannel.flush(); - - // Test for initial window size - assertTrue(initialRemoteStreamWindow < childChannel.config().getWriteBufferHighWaterMark()); - - assertTrue(childChannel.isWritable()); - childChannel.write(new DefaultHttp2DataFrame(Unpooled.buffer().writeZero(16 * 1024 * 1024))); - assertEquals(0, childChannel.bytesBeforeUnwritable()); - assertFalse(childChannel.isWritable()); - } - - @Test - public void writabilityOfParentIsRespected() { - Http2StreamChannel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter()); - childChannel.config().setWriteBufferWaterMark(new WriteBufferWaterMark(2048, 4096)); - parentChannel.config().setWriteBufferWaterMark(new WriteBufferWaterMark(256, 512)); - assertTrue(childChannel.isWritable()); - assertTrue(parentChannel.isActive()); - - childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); - parentChannel.flush(); - - assertTrue(childChannel.isWritable()); - childChannel.write(new DefaultHttp2DataFrame(Unpooled.buffer().writeZero(256))); - assertTrue(childChannel.isWritable()); - childChannel.writeAndFlush(new DefaultHttp2DataFrame(Unpooled.buffer().writeZero(512))); - - long bytesBeforeUnwritable = childChannel.bytesBeforeUnwritable(); - assertNotEquals(0, bytesBeforeUnwritable); - // Add something to the ChannelOutboundBuffer of the parent to simulate queuing in the parents channel buffer - // and verify that this only affect the writability of the parent channel while the child stays writable - // until it used all of its credits. - parentChannel.unsafe().outboundBuffer().addMessage( - Unpooled.buffer().writeZero(800), 800, parentChannel.voidPromise()); - assertFalse(parentChannel.isWritable()); - - assertTrue(childChannel.isWritable()); - assertEquals(4096, childChannel.bytesBeforeUnwritable()); - - // Flush everything which simulate writing everything to the socket. - parentChannel.flush(); - assertTrue(parentChannel.isWritable()); - assertTrue(childChannel.isWritable()); - assertEquals(bytesBeforeUnwritable, childChannel.bytesBeforeUnwritable()); - - ChannelFuture future = childChannel.writeAndFlush(new DefaultHttp2DataFrame( - Unpooled.buffer().writeZero((int) bytesBeforeUnwritable))); - assertFalse(childChannel.isWritable()); - assertTrue(parentChannel.isWritable()); - - parentChannel.flush(); - assertFalse(future.isDone()); - assertTrue(parentChannel.isWritable()); - assertFalse(childChannel.isWritable()); - - // Now write an window update frame for the stream which then should ensure we will flush the bytes that were - // queued in the RemoteFlowController before for the stream. - frameInboundWriter.writeInboundWindowUpdate(childChannel.stream().id(), (int) bytesBeforeUnwritable); - assertTrue(childChannel.isWritable()); - assertTrue(future.isDone()); - } - - @Test - public void channelClosedWhenInactiveFired() { - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); - - final AtomicBoolean channelOpen = new AtomicBoolean(false); - final AtomicBoolean channelActive = new AtomicBoolean(false); - assertTrue(childChannel.isOpen()); - assertTrue(childChannel.isActive()); - - childChannel.pipeline().addLast(new ChannelInboundHandlerAdapter() { - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - channelOpen.set(ctx.channel().isOpen()); - channelActive.set(ctx.channel().isActive()); - - super.channelInactive(ctx); - } - }); - - childChannel.close().syncUninterruptibly(); - assertFalse(channelOpen.get()); - assertFalse(channelActive.get()); - } - - @Test - public void channelInactiveHappensAfterExceptionCaughtEvents() throws Exception { - final AtomicInteger count = new AtomicInteger(0); - final AtomicInteger exceptionCaught = new AtomicInteger(-1); - final AtomicInteger channelInactive = new AtomicInteger(-1); - final AtomicInteger channelUnregistered = new AtomicInteger(-1); - Http2StreamChannel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter() { - - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - ctx.close(); - throw new Exception("exception"); - } - }); - - childChannel.pipeline().addLast(new ChannelInboundHandlerAdapter() { - - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - channelInactive.set(count.getAndIncrement()); - super.channelInactive(ctx); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - exceptionCaught.set(count.getAndIncrement()); - super.exceptionCaught(ctx, cause); - } - - @Override - public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { - channelUnregistered.set(count.getAndIncrement()); - super.channelUnregistered(ctx); - } - }); - - childChannel.pipeline().fireUserEventTriggered(new Object()); - parentChannel.runPendingTasks(); - - // The events should have happened in this order because the inactive and deregistration events - // get deferred as they do in the AbstractChannel. - assertEquals(0, exceptionCaught.get()); - assertEquals(1, channelInactive.get()); - assertEquals(2, channelUnregistered.get()); - } - - @Test - public void callUnsafeCloseMultipleTimes() { - LastInboundHandler inboundHandler = new LastInboundHandler(); - Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); - childChannel.unsafe().close(childChannel.voidPromise()); - - ChannelPromise promise = childChannel.newPromise(); - childChannel.unsafe().close(promise); - promise.syncUninterruptibly(); - childChannel.closeFuture().syncUninterruptibly(); - } - - @Test - public void endOfStreamDoesNotDiscardData() { - AtomicInteger numReads = new AtomicInteger(1); - final AtomicBoolean shouldDisableAutoRead = new AtomicBoolean(); - Consumer ctxConsumer = new Consumer() { - @Override - public void accept(ChannelHandlerContext obj) { - if (shouldDisableAutoRead.get()) { - obj.channel().config().setAutoRead(false); - } - } - }; - LastInboundHandler inboundHandler = new LastInboundHandler(ctxConsumer); - Http2StreamChannel childChannel = newInboundStream(3, false, numReads, inboundHandler); - childChannel.config().setAutoRead(false); - - Http2DataFrame dataFrame1 = new DefaultHttp2DataFrame(bb("1")).stream(childChannel.stream()); - Http2DataFrame dataFrame2 = new DefaultHttp2DataFrame(bb("2")).stream(childChannel.stream()); - Http2DataFrame dataFrame3 = new DefaultHttp2DataFrame(bb("3")).stream(childChannel.stream()); - Http2DataFrame dataFrame4 = new DefaultHttp2DataFrame(bb("4")).stream(childChannel.stream()); - - assertEquals(new DefaultHttp2HeadersFrame(request).stream(childChannel.stream()), inboundHandler.readInbound()); - - ChannelHandler readCompleteSupressHandler = new ChannelInboundHandlerAdapter() { - @Override - public void channelReadComplete(ChannelHandlerContext ctx) { - // We want to simulate the parent channel calling channelRead and delay calling channelReadComplete. - } - }; - - parentChannel.pipeline().addFirst(readCompleteSupressHandler); - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("1"), 0, false); - - assertEqualsAndRelease(dataFrame1, inboundHandler.readInbound()); - - // Deliver frames, and then a stream closed while read is inactive. - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("2"), 0, false); - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("3"), 0, false); - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("4"), 0, false); - - shouldDisableAutoRead.set(true); - childChannel.config().setAutoRead(true); - numReads.set(1); - - frameInboundWriter.writeInboundRstStream(childChannel.stream().id(), Http2Error.NO_ERROR.code()); - - // Detecting EOS should flush all pending data regardless of read calls. - assertEqualsAndRelease(dataFrame2, inboundHandler.readInbound()); - assertEqualsAndRelease(dataFrame3, inboundHandler.readInbound()); - assertEqualsAndRelease(dataFrame4, inboundHandler.readInbound()); - - Http2ResetFrame resetFrame = inboundHandler.readInbound(); - assertEquals(childChannel.stream(), resetFrame.stream()); - assertEquals(Http2Error.NO_ERROR.code(), resetFrame.errorCode()); - - assertNull(inboundHandler.readInbound()); - - // Now we want to call channelReadComplete and simulate the end of the read loop. - parentChannel.pipeline().remove(readCompleteSupressHandler); - parentChannel.flushInbound(); - - childChannel.closeFuture().syncUninterruptibly(); - } - - @Test - public void childQueueIsDrainedAndNewDataIsDispatchedInParentReadLoopAutoRead() { - AtomicInteger numReads = new AtomicInteger(1); - final AtomicInteger channelReadCompleteCount = new AtomicInteger(0); - final AtomicBoolean shouldDisableAutoRead = new AtomicBoolean(); - Consumer ctxConsumer = new Consumer() { - @Override - public void accept(ChannelHandlerContext obj) { - channelReadCompleteCount.incrementAndGet(); - if (shouldDisableAutoRead.get()) { - obj.channel().config().setAutoRead(false); - } - } - }; - LastInboundHandler inboundHandler = new LastInboundHandler(ctxConsumer); - Http2StreamChannel childChannel = newInboundStream(3, false, numReads, inboundHandler); - childChannel.config().setAutoRead(false); - - Http2DataFrame dataFrame1 = new DefaultHttp2DataFrame(bb("1")).stream(childChannel.stream()); - Http2DataFrame dataFrame2 = new DefaultHttp2DataFrame(bb("2")).stream(childChannel.stream()); - Http2DataFrame dataFrame3 = new DefaultHttp2DataFrame(bb("3")).stream(childChannel.stream()); - Http2DataFrame dataFrame4 = new DefaultHttp2DataFrame(bb("4")).stream(childChannel.stream()); - - assertEquals(new DefaultHttp2HeadersFrame(request).stream(childChannel.stream()), inboundHandler.readInbound()); - - ChannelHandler readCompleteSupressHandler = new ChannelInboundHandlerAdapter() { - @Override - public void channelReadComplete(ChannelHandlerContext ctx) { - // We want to simulate the parent channel calling channelRead and delay calling channelReadComplete. - } - }; - parentChannel.pipeline().addFirst(readCompleteSupressHandler); - - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("1"), 0, false); - - assertEqualsAndRelease(dataFrame1, inboundHandler.readInbound()); - - // We want one item to be in the queue, and allow the numReads to be larger than 1. This will ensure that - // when beginRead() is called the child channel is added to the readPending queue of the parent channel. - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("2"), 0, false); - - numReads.set(10); - shouldDisableAutoRead.set(true); - childChannel.config().setAutoRead(true); - - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("3"), 0, false); - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("4"), 0, false); - - // Detecting EOS should flush all pending data regardless of read calls. - assertEqualsAndRelease(dataFrame2, inboundHandler.readInbound()); - assertEqualsAndRelease(dataFrame3, inboundHandler.readInbound()); - assertEqualsAndRelease(dataFrame4, inboundHandler.readInbound()); - - assertNull(inboundHandler.readInbound()); - - // Now we want to call channelReadComplete and simulate the end of the read loop. - parentChannel.pipeline().remove(readCompleteSupressHandler); - parentChannel.flushInbound(); - - // 3 = 1 for initialization + 1 for read when auto read was off + 1 for when auto read was back on - assertEquals(3, channelReadCompleteCount.get()); - } - - @Test - public void childQueueIsDrainedAndNewDataIsDispatchedInParentReadLoopNoAutoRead() { - final AtomicInteger numReads = new AtomicInteger(1); - final AtomicInteger channelReadCompleteCount = new AtomicInteger(0); - final AtomicBoolean shouldDisableAutoRead = new AtomicBoolean(); - Consumer ctxConsumer = new Consumer() { - @Override - public void accept(ChannelHandlerContext obj) { - channelReadCompleteCount.incrementAndGet(); - if (shouldDisableAutoRead.get()) { - obj.channel().config().setAutoRead(false); - } - } - }; - final LastInboundHandler inboundHandler = new LastInboundHandler(ctxConsumer); - Http2StreamChannel childChannel = newInboundStream(3, false, numReads, inboundHandler); - childChannel.config().setAutoRead(false); - - Http2DataFrame dataFrame1 = new DefaultHttp2DataFrame(bb("1")).stream(childChannel.stream()); - Http2DataFrame dataFrame2 = new DefaultHttp2DataFrame(bb("2")).stream(childChannel.stream()); - Http2DataFrame dataFrame3 = new DefaultHttp2DataFrame(bb("3")).stream(childChannel.stream()); - Http2DataFrame dataFrame4 = new DefaultHttp2DataFrame(bb("4")).stream(childChannel.stream()); - - assertEquals(new DefaultHttp2HeadersFrame(request).stream(childChannel.stream()), inboundHandler.readInbound()); - - ChannelHandler readCompleteSupressHandler = new ChannelInboundHandlerAdapter() { - @Override - public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { - // We want to simulate the parent channel calling channelRead and delay calling channelReadComplete. - } - }; - parentChannel.pipeline().addFirst(readCompleteSupressHandler); - - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("1"), 0, false); - - assertEqualsAndRelease(dataFrame1, inboundHandler.readInbound()); - - // We want one item to be in the queue, and allow the numReads to be larger than 1. This will ensure that - // when beginRead() is called the child channel is added to the readPending queue of the parent channel. - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("2"), 0, false); - - numReads.set(2); - childChannel.read(); - - assertEqualsAndRelease(dataFrame2, inboundHandler.readInbound()); - - assertNull(inboundHandler.readInbound()); - - // This is the second item that was read, this should be the last until we call read() again. This should also - // notify of readComplete(). - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("3"), 0, false); - - assertEqualsAndRelease(dataFrame3, inboundHandler.readInbound()); - - frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("4"), 0, false); - assertNull(inboundHandler.readInbound()); - - childChannel.read(); - - assertEqualsAndRelease(dataFrame4, inboundHandler.readInbound()); - - assertNull(inboundHandler.readInbound()); - - // Now we want to call channelReadComplete and simulate the end of the read loop. - parentChannel.pipeline().remove(readCompleteSupressHandler); - parentChannel.flushInbound(); - - // 3 = 1 for initialization + 1 for first read of 2 items + 1 for second read of 2 items + - // 1 for parent channel readComplete - assertEquals(4, channelReadCompleteCount.get()); - } - - private static void verifyFramesMultiplexedToCorrectChannel(Http2StreamChannel streamChannel, - LastInboundHandler inboundHandler, - int numFrames) { - for (int i = 0; i < numFrames; i++) { - Http2StreamFrame frame = inboundHandler.readInbound(); - assertNotNull(frame); - assertEquals(streamChannel.stream(), frame.stream()); - release(frame); - } - assertNull(inboundHandler.readInbound()); - } - - private static int eqStreamId(Http2StreamChannel channel) { - return eq(channel.stream().id()); + @Override + protected ChannelHandler newMultiplexer(TestChannelInitializer childChannelInitializer) { + return null; } } diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexHandlerClientUpgradeTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexHandlerClientUpgradeTest.java new file mode 100644 index 0000000000..281c8da70c --- /dev/null +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexHandlerClientUpgradeTest.java @@ -0,0 +1,30 @@ +/* + * Copyright 2019 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; + +public class Http2MultiplexHandlerClientUpgradeTest extends Http2MultiplexClientUpgradeTest { + + @Override + protected Http2FrameCodec newCodec(ChannelHandler upgradeHandler) { + return Http2FrameCodecBuilder.forClient().build(); + } + + @Override + protected ChannelHandler newMultiplexer(ChannelHandler upgradeHandler) { + return new Http2MultiplexHandler(new NoopHandler(), upgradeHandler); + } +} diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexHandlerTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexHandlerTest.java new file mode 100644 index 0000000000..f99aac0df7 --- /dev/null +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexHandlerTest.java @@ -0,0 +1,34 @@ +/* + * Copyright 2019 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 org.junit.Ignore; + +/** + * Unit tests for {@link Http2MultiplexHandler}. + */ +public class Http2MultiplexHandlerTest extends Http2MultiplexTest { + + @Override + protected Http2FrameCodec newCodec(TestChannelInitializer childChannelInitializer, Http2FrameWriter frameWriter) { + return new Http2FrameCodecBuilder(true).frameWriter(frameWriter).build(); + } + + @Override + protected ChannelHandler newMultiplexer(TestChannelInitializer childChannelInitializer) { + return new Http2MultiplexHandler(childChannelInitializer, null); + } +} diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexTest.java new file mode 100644 index 0000000000..cbaadbe0f1 --- /dev/null +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexTest.java @@ -0,0 +1,1041 @@ +/* + * 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.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import io.netty.channel.WriteBufferWaterMark; +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.LastInboundHandler.Consumer; +import io.netty.util.AsciiString; +import io.netty.util.AttributeKey; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentMatcher; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.net.InetSocketAddress; +import java.nio.channels.ClosedChannelException; +import java.util.ArrayDeque; +import java.util.Queue; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import static io.netty.handler.codec.http2.Http2TestUtil.anyChannelPromise; +import static io.netty.handler.codec.http2.Http2TestUtil.anyHttp2Settings; +import static io.netty.handler.codec.http2.Http2TestUtil.assertEqualsAndRelease; +import static io.netty.handler.codec.http2.Http2TestUtil.bb; +import static io.netty.util.ReferenceCountUtil.release; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyShort; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public abstract class Http2MultiplexTest { + private final Http2Headers request = new DefaultHttp2Headers() + .method(HttpMethod.GET.asciiName()).scheme(HttpScheme.HTTPS.name()) + .authority(new AsciiString("example.org")).path(new AsciiString("/foo")); + + private EmbeddedChannel parentChannel; + private Http2FrameWriter frameWriter; + private Http2FrameInboundWriter frameInboundWriter; + private TestChannelInitializer childChannelInitializer; + private C codec; + + private static final int initialRemoteStreamWindow = 1024; + + protected abstract C newCodec(TestChannelInitializer childChannelInitializer, Http2FrameWriter frameWriter); + protected abstract ChannelHandler newMultiplexer(TestChannelInitializer childChannelInitializer); + + @Before + public void setUp() { + childChannelInitializer = new TestChannelInitializer(); + parentChannel = new EmbeddedChannel(); + frameInboundWriter = new Http2FrameInboundWriter(parentChannel); + parentChannel.connect(new InetSocketAddress(0)); + frameWriter = Http2TestUtil.mockedFrameWriter(); + codec = newCodec(childChannelInitializer, frameWriter); + parentChannel.pipeline().addLast(codec); + ChannelHandler multiplexer = newMultiplexer(childChannelInitializer); + if (multiplexer != null) { + parentChannel.pipeline().addLast(multiplexer); + } + + parentChannel.runPendingTasks(); + parentChannel.pipeline().fireChannelActive(); + + parentChannel.writeInbound(Http2CodecUtil.connectionPrefaceBuf()); + + Http2Settings settings = new Http2Settings().initialWindowSize(initialRemoteStreamWindow); + frameInboundWriter.writeInboundSettings(settings); + + verify(frameWriter).writeSettingsAck(eqCodecCtx(), anyChannelPromise()); + + frameInboundWriter.writeInboundSettingsAck(); + + Http2SettingsFrame settingsFrame = parentChannel.readInbound(); + assertNotNull(settingsFrame); + Http2SettingsAckFrame settingsAckFrame = parentChannel.readInbound(); + assertNotNull(settingsAckFrame); + + // Handshake + verify(frameWriter).writeSettings(eqCodecCtx(), + anyHttp2Settings(), anyChannelPromise()); + } + + private ChannelHandlerContext eqCodecCtx() { + return eq(codec.ctx); + } + + @After + public void tearDown() throws Exception { + if (childChannelInitializer.handler instanceof LastInboundHandler) { + ((LastInboundHandler) childChannelInitializer.handler).finishAndReleaseAll(); + } + parentChannel.finishAndReleaseAll(); + codec = null; + } + + // TODO(buchgr): Flush from child channel + // TODO(buchgr): ChildChannel.childReadComplete() + // TODO(buchgr): GOAWAY Logic + // TODO(buchgr): Test ChannelConfig.setMaxMessagesPerRead + + @Test + public void writeUnknownFrame() { + Http2StreamChannel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter() { + @Override + public void channelActive(ChannelHandlerContext ctx) { + ctx.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); + ctx.writeAndFlush(new DefaultHttp2UnknownFrame((byte) 99, new Http2Flags())); + ctx.fireChannelActive(); + } + }); + assertTrue(childChannel.isActive()); + + parentChannel.runPendingTasks(); + + verify(frameWriter).writeFrame(eq(codec.ctx), eq((byte) 99), eqStreamId(childChannel), any(Http2Flags.class), + any(ByteBuf.class), any(ChannelPromise.class)); + } + + private Http2StreamChannel newInboundStream(int streamId, boolean endStream, final ChannelHandler childHandler) { + return newInboundStream(streamId, endStream, null, childHandler); + } + + private Http2StreamChannel newInboundStream(int streamId, boolean endStream, + AtomicInteger maxReads, final ChannelHandler childHandler) { + final AtomicReference streamChannelRef = new AtomicReference(); + childChannelInitializer.maxReads = maxReads; + childChannelInitializer.handler = new ChannelInboundHandlerAdapter() { + @Override + public void channelRegistered(ChannelHandlerContext ctx) { + assertNull(streamChannelRef.get()); + streamChannelRef.set((Http2StreamChannel) ctx.channel()); + ctx.pipeline().addLast(childHandler); + ctx.fireChannelRegistered(); + } + }; + + frameInboundWriter.writeInboundHeaders(streamId, request, 0, endStream); + parentChannel.runPendingTasks(); + Http2StreamChannel channel = streamChannelRef.get(); + assertEquals(streamId, channel.stream().id()); + return channel; + } + + @Test + public void readUnkownFrame() { + LastInboundHandler handler = new LastInboundHandler(); + + Http2StreamChannel channel = newInboundStream(3, true, handler); + frameInboundWriter.writeInboundFrame((byte) 99, channel.stream().id(), new Http2Flags(), Unpooled.EMPTY_BUFFER); + + // header frame and unknown frame + verifyFramesMultiplexedToCorrectChannel(channel, handler, 2); + + Channel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter()); + assertTrue(childChannel.isActive()); + } + + @Test + public void headerAndDataFramesShouldBeDelivered() { + LastInboundHandler inboundHandler = new LastInboundHandler(); + + Http2StreamChannel channel = newInboundStream(3, false, inboundHandler); + Http2HeadersFrame headersFrame = new DefaultHttp2HeadersFrame(request).stream(channel.stream()); + Http2DataFrame dataFrame1 = new DefaultHttp2DataFrame(bb("hello")).stream(channel.stream()); + Http2DataFrame dataFrame2 = new DefaultHttp2DataFrame(bb("world")).stream(channel.stream()); + + assertTrue(inboundHandler.isChannelActive()); + frameInboundWriter.writeInboundData(channel.stream().id(), bb("hello"), 0, false); + frameInboundWriter.writeInboundData(channel.stream().id(), bb("world"), 0, false); + + assertEquals(headersFrame, inboundHandler.readInbound()); + + assertEqualsAndRelease(dataFrame1, inboundHandler.readInbound()); + assertEqualsAndRelease(dataFrame2, inboundHandler.readInbound()); + + assertNull(inboundHandler.readInbound()); + } + + @Test + public void framesShouldBeMultiplexed() { + LastInboundHandler handler1 = new LastInboundHandler(); + Http2StreamChannel channel1 = newInboundStream(3, false, handler1); + LastInboundHandler handler2 = new LastInboundHandler(); + Http2StreamChannel channel2 = newInboundStream(5, false, handler2); + LastInboundHandler handler3 = new LastInboundHandler(); + Http2StreamChannel channel3 = newInboundStream(11, false, handler3); + + verifyFramesMultiplexedToCorrectChannel(channel1, handler1, 1); + verifyFramesMultiplexedToCorrectChannel(channel2, handler2, 1); + verifyFramesMultiplexedToCorrectChannel(channel3, handler3, 1); + + frameInboundWriter.writeInboundData(channel2.stream().id(), bb("hello"), 0, false); + frameInboundWriter.writeInboundData(channel1.stream().id(), bb("foo"), 0, true); + frameInboundWriter.writeInboundData(channel2.stream().id(), bb("world"), 0, true); + frameInboundWriter.writeInboundData(channel3.stream().id(), bb("bar"), 0, true); + + verifyFramesMultiplexedToCorrectChannel(channel1, handler1, 1); + verifyFramesMultiplexedToCorrectChannel(channel2, handler2, 2); + verifyFramesMultiplexedToCorrectChannel(channel3, handler3, 1); + } + + @Test + public void inboundDataFrameShouldUpdateLocalFlowController() throws Http2Exception { + Http2LocalFlowController flowController = Mockito.mock(Http2LocalFlowController.class); + codec.connection().local().flowController(flowController); + + LastInboundHandler handler = new LastInboundHandler(); + final Http2StreamChannel channel = newInboundStream(3, false, handler); + + ByteBuf tenBytes = bb("0123456789"); + + frameInboundWriter.writeInboundData(channel.stream().id(), tenBytes, 0, true); + + // Verify we marked the bytes as consumed + verify(flowController).consumeBytes(argThat(new ArgumentMatcher() { + @Override + public boolean matches(Http2Stream http2Stream) { + return http2Stream.id() == channel.stream().id(); + } + }), eq(10)); + + // headers and data frame + verifyFramesMultiplexedToCorrectChannel(channel, handler, 2); + } + + @Test + public void unhandledHttp2FramesShouldBePropagated() { + Http2PingFrame pingFrame = new DefaultHttp2PingFrame(0); + frameInboundWriter.writeInboundPing(false, 0); + assertEquals(parentChannel.readInbound(), pingFrame); + + DefaultHttp2GoAwayFrame goAwayFrame = new DefaultHttp2GoAwayFrame(1, + parentChannel.alloc().buffer().writeLong(8)); + frameInboundWriter.writeInboundGoAway(0, goAwayFrame.errorCode(), goAwayFrame.content().retainedDuplicate()); + + Http2GoAwayFrame frame = parentChannel.readInbound(); + assertEqualsAndRelease(frame, goAwayFrame); + } + + @Test + public void channelReadShouldRespectAutoRead() { + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); + assertTrue(childChannel.config().isAutoRead()); + Http2HeadersFrame headersFrame = inboundHandler.readInbound(); + assertNotNull(headersFrame); + + childChannel.config().setAutoRead(false); + + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("hello world"), 0, false); + Http2DataFrame dataFrame0 = inboundHandler.readInbound(); + assertNotNull(dataFrame0); + release(dataFrame0); + + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("foo"), 0, false); + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("bar"), 0, false); + + assertNull(inboundHandler.readInbound()); + + childChannel.config().setAutoRead(true); + verifyFramesMultiplexedToCorrectChannel(childChannel, inboundHandler, 2); + } + + @Test + public void readInChannelReadWithoutAutoRead() { + useReadWithoutAutoRead(false); + } + + @Test + public void readInChannelReadCompleteWithoutAutoRead() { + useReadWithoutAutoRead(true); + } + + private void useReadWithoutAutoRead(final boolean readComplete) { + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); + assertTrue(childChannel.config().isAutoRead()); + childChannel.config().setAutoRead(false); + assertFalse(childChannel.config().isAutoRead()); + + Http2HeadersFrame headersFrame = inboundHandler.readInbound(); + assertNotNull(headersFrame); + + // Add a handler which will request reads. + childChannel.pipeline().addFirst(new ChannelInboundHandlerAdapter() { + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) { + ctx.fireChannelRead(msg); + if (!readComplete) { + ctx.read(); + } + } + + @Override + public void channelReadComplete(ChannelHandlerContext ctx) { + ctx.fireChannelReadComplete(); + if (readComplete) { + ctx.read(); + } + } + }); + + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("hello world"), 0, false); + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("foo"), 0, false); + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("bar"), 0, false); + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("hello world"), 0, false); + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("foo"), 0, false); + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("bar"), 0, true); + + verifyFramesMultiplexedToCorrectChannel(childChannel, inboundHandler, 6); + } + + private Http2StreamChannel newOutboundStream(ChannelHandler handler) { + return new Http2StreamChannelBootstrap(parentChannel).handler(handler) + .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() { + LastInboundHandler handler = new LastInboundHandler(); + + Channel childChannel = newOutboundStream(handler); + assertTrue(childChannel.isActive()); + + childChannel.close(); + parentChannel.runPendingTasks(); + + assertFalse(childChannel.isOpen()); + assertFalse(childChannel.isActive()); + assertNull(parentChannel.readOutbound()); + } + + @Test + public void outboundStreamShouldWriteResetFrameOnClose_headersSent() { + ChannelHandler handler = new ChannelInboundHandlerAdapter() { + @Override + public void channelActive(ChannelHandlerContext ctx) { + ctx.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); + ctx.fireChannelActive(); + } + }; + + Http2StreamChannel childChannel = newOutboundStream(handler); + assertTrue(childChannel.isActive()); + + childChannel.close(); + verify(frameWriter).writeRstStream(eqCodecCtx(), + eqStreamId(childChannel), eq(Http2Error.CANCEL.code()), anyChannelPromise()); + } + + @Test + public void outboundStreamShouldNotWriteResetFrameOnClose_IfStreamDidntExist() { + when(frameWriter.writeHeaders(eqCodecCtx(), anyInt(), + any(Http2Headers.class), anyInt(), anyShort(), anyBoolean(), anyInt(), anyBoolean(), + any(ChannelPromise.class))).thenAnswer(new Answer() { + + private boolean headersWritten; + @Override + public ChannelFuture answer(InvocationOnMock invocationOnMock) { + // We want to fail to write the first headers frame. This is what happens if the connection + // refuses to allocate a new stream due to having received a GOAWAY. + if (!headersWritten) { + headersWritten = true; + return ((ChannelPromise) invocationOnMock.getArgument(8)).setFailure(new Exception("boom")); + } + return ((ChannelPromise) invocationOnMock.getArgument(8)).setSuccess(); + } + }); + + Http2StreamChannel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter() { + @Override + public void channelActive(ChannelHandlerContext ctx) { + ctx.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); + ctx.fireChannelActive(); + } + }); + + assertFalse(childChannel.isActive()); + + childChannel.close(); + parentChannel.runPendingTasks(); + // The channel was never active so we should not generate a RST frame. + verify(frameWriter, never()).writeRstStream(eqCodecCtx(), eqStreamId(childChannel), anyLong(), + anyChannelPromise()); + + assertTrue(parentChannel.outboundMessages().isEmpty()); + } + + @Test + public void inboundRstStreamFireChannelInactive() { + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel channel = newInboundStream(3, false, inboundHandler); + assertTrue(inboundHandler.isChannelActive()); + frameInboundWriter.writeInboundRstStream(channel.stream().id(), Http2Error.INTERNAL_ERROR.code()); + + assertFalse(inboundHandler.isChannelActive()); + + // A RST_STREAM frame should NOT be emitted, as we received a RST_STREAM. + verify(frameWriter, Mockito.never()).writeRstStream(eqCodecCtx(), eqStreamId(channel), + anyLong(), anyChannelPromise()); + } + + @Test(expected = StreamException.class) + public void streamExceptionTriggersChildChannelExceptionAndClose() throws Exception { + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel channel = newInboundStream(3, false, inboundHandler); + assertTrue(channel.isActive()); + StreamException cause = new StreamException(channel.stream().id(), Http2Error.PROTOCOL_ERROR, "baaam!"); + parentChannel.pipeline().fireExceptionCaught(cause); + + assertFalse(channel.isActive()); + inboundHandler.checkException(); + } + + @Test(expected = ClosedChannelException.class) + public void streamClosedErrorTranslatedToClosedChannelExceptionOnWrites() throws Exception { + LastInboundHandler inboundHandler = new LastInboundHandler(); + + final Http2StreamChannel childChannel = newOutboundStream(inboundHandler); + assertTrue(childChannel.isActive()); + + Http2Headers headers = new DefaultHttp2Headers(); + when(frameWriter.writeHeaders(eqCodecCtx(), anyInt(), + eq(headers), anyInt(), anyShort(), anyBoolean(), anyInt(), anyBoolean(), + any(ChannelPromise.class))).thenAnswer(new Answer() { + @Override + public ChannelFuture answer(InvocationOnMock invocationOnMock) { + return ((ChannelPromise) invocationOnMock.getArgument(8)).setFailure( + new StreamException(childChannel.stream().id(), Http2Error.STREAM_CLOSED, "Stream Closed")); + } + }); + ChannelFuture future = childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); + + parentChannel.flush(); + + assertFalse(childChannel.isActive()); + assertFalse(childChannel.isOpen()); + + inboundHandler.checkException(); + + future.syncUninterruptibly(); + } + + @Test + public void creatingWritingReadingAndClosingOutboundStreamShouldWork() { + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel childChannel = newOutboundStream(inboundHandler); + assertTrue(childChannel.isActive()); + assertTrue(inboundHandler.isChannelActive()); + + // Write to the child channel + Http2Headers headers = new DefaultHttp2Headers().scheme("https").method("GET").path("/foo.txt"); + childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(headers)); + + // Read from the child channel + frameInboundWriter.writeInboundHeaders(childChannel.stream().id(), headers, 0, false); + + Http2HeadersFrame headersFrame = inboundHandler.readInbound(); + assertNotNull(headersFrame); + assertEquals(headers, headersFrame.headers()); + + // Close the child channel. + childChannel.close(); + + parentChannel.runPendingTasks(); + // An active outbound stream should emit a RST_STREAM frame. + verify(frameWriter).writeRstStream(eqCodecCtx(), eqStreamId(childChannel), + anyLong(), anyChannelPromise()); + + assertFalse(childChannel.isOpen()); + assertFalse(childChannel.isActive()); + 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(expected = Http2NoMoreStreamIdsException.class) + public void failedOutboundStreamCreationThrowsAndClosesChannel() throws Exception { + LastInboundHandler handler = new LastInboundHandler(); + Http2StreamChannel childChannel = newOutboundStream(handler); + assertTrue(childChannel.isActive()); + + Http2Headers headers = new DefaultHttp2Headers(); + when(frameWriter.writeHeaders(eqCodecCtx(), anyInt(), + eq(headers), anyInt(), anyShort(), anyBoolean(), anyInt(), anyBoolean(), + any(ChannelPromise.class))).thenAnswer(new Answer() { + @Override + public ChannelFuture answer(InvocationOnMock invocationOnMock) { + return ((ChannelPromise) invocationOnMock.getArgument(8)).setFailure( + new Http2NoMoreStreamIdsException()); + } + }); + + ChannelFuture future = childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(headers)); + parentChannel.flush(); + + assertFalse(childChannel.isActive()); + assertFalse(childChannel.isOpen()); + + handler.checkException(); + + future.syncUninterruptibly(); + } + + @Test + public void channelClosedWhenCloseListenerCompletes() { + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); + + assertTrue(childChannel.isOpen()); + assertTrue(childChannel.isActive()); + + final AtomicBoolean channelOpen = new AtomicBoolean(true); + final AtomicBoolean channelActive = new AtomicBoolean(true); + + // Create a promise before actually doing the close, because otherwise we would be adding a listener to a future + // that is already completed because we are using EmbeddedChannel which executes code in the JUnit thread. + ChannelPromise p = childChannel.newPromise(); + p.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) { + channelOpen.set(future.channel().isOpen()); + channelActive.set(future.channel().isActive()); + } + }); + childChannel.close(p).syncUninterruptibly(); + + assertFalse(channelOpen.get()); + assertFalse(channelActive.get()); + assertFalse(childChannel.isActive()); + } + + @Test + public void channelClosedWhenChannelClosePromiseCompletes() { + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); + + assertTrue(childChannel.isOpen()); + assertTrue(childChannel.isActive()); + + final AtomicBoolean channelOpen = new AtomicBoolean(true); + final AtomicBoolean channelActive = new AtomicBoolean(true); + + childChannel.closeFuture().addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) { + channelOpen.set(future.channel().isOpen()); + channelActive.set(future.channel().isActive()); + } + }); + childChannel.close().syncUninterruptibly(); + + assertFalse(channelOpen.get()); + assertFalse(channelActive.get()); + assertFalse(childChannel.isActive()); + } + + @Test + public void channelClosedWhenWriteFutureFails() { + final Queue writePromises = new ArrayDeque(); + + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); + + assertTrue(childChannel.isOpen()); + assertTrue(childChannel.isActive()); + + final AtomicBoolean channelOpen = new AtomicBoolean(true); + final AtomicBoolean channelActive = new AtomicBoolean(true); + + Http2Headers headers = new DefaultHttp2Headers(); + when(frameWriter.writeHeaders(eqCodecCtx(), anyInt(), + eq(headers), anyInt(), anyShort(), anyBoolean(), anyInt(), anyBoolean(), + any(ChannelPromise.class))).thenAnswer(new Answer() { + @Override + public ChannelFuture answer(InvocationOnMock invocationOnMock) { + ChannelPromise promise = invocationOnMock.getArgument(8); + writePromises.offer(promise); + return promise; + } + }); + + ChannelFuture f = childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(headers)); + assertFalse(f.isDone()); + f.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + channelOpen.set(future.channel().isOpen()); + channelActive.set(future.channel().isActive()); + } + }); + + ChannelPromise first = writePromises.poll(); + first.setFailure(new ClosedChannelException()); + f.awaitUninterruptibly(); + + assertFalse(channelOpen.get()); + assertFalse(channelActive.get()); + assertFalse(childChannel.isActive()); + } + + @Test + public void channelClosedTwiceMarksPromiseAsSuccessful() { + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); + + assertTrue(childChannel.isOpen()); + assertTrue(childChannel.isActive()); + childChannel.close().syncUninterruptibly(); + childChannel.close().syncUninterruptibly(); + + assertFalse(childChannel.isOpen()); + assertFalse(childChannel.isActive()); + } + + @Test + public void settingChannelOptsAndAttrs() { + AttributeKey key = AttributeKey.newInstance(UUID.randomUUID().toString()); + + Channel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter()); + 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 outboundFlowControlWritability() { + Http2StreamChannel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter()); + assertTrue(childChannel.isActive()); + + assertTrue(childChannel.isWritable()); + childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); + parentChannel.flush(); + + // Test for initial window size + assertTrue(initialRemoteStreamWindow < childChannel.config().getWriteBufferHighWaterMark()); + + assertTrue(childChannel.isWritable()); + childChannel.write(new DefaultHttp2DataFrame(Unpooled.buffer().writeZero(16 * 1024 * 1024))); + assertEquals(0, childChannel.bytesBeforeUnwritable()); + assertFalse(childChannel.isWritable()); + } + + @Test + public void writabilityOfParentIsRespected() { + Http2StreamChannel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter()); + childChannel.config().setWriteBufferWaterMark(new WriteBufferWaterMark(2048, 4096)); + parentChannel.config().setWriteBufferWaterMark(new WriteBufferWaterMark(256, 512)); + assertTrue(childChannel.isWritable()); + assertTrue(parentChannel.isActive()); + + childChannel.writeAndFlush(new DefaultHttp2HeadersFrame(new DefaultHttp2Headers())); + parentChannel.flush(); + + assertTrue(childChannel.isWritable()); + childChannel.write(new DefaultHttp2DataFrame(Unpooled.buffer().writeZero(256))); + assertTrue(childChannel.isWritable()); + childChannel.writeAndFlush(new DefaultHttp2DataFrame(Unpooled.buffer().writeZero(512))); + + long bytesBeforeUnwritable = childChannel.bytesBeforeUnwritable(); + assertNotEquals(0, bytesBeforeUnwritable); + // Add something to the ChannelOutboundBuffer of the parent to simulate queuing in the parents channel buffer + // and verify that this only affect the writability of the parent channel while the child stays writable + // until it used all of its credits. + parentChannel.unsafe().outboundBuffer().addMessage( + Unpooled.buffer().writeZero(800), 800, parentChannel.voidPromise()); + assertFalse(parentChannel.isWritable()); + + assertTrue(childChannel.isWritable()); + assertEquals(4096, childChannel.bytesBeforeUnwritable()); + + // Flush everything which simulate writing everything to the socket. + parentChannel.flush(); + assertTrue(parentChannel.isWritable()); + assertTrue(childChannel.isWritable()); + assertEquals(bytesBeforeUnwritable, childChannel.bytesBeforeUnwritable()); + + ChannelFuture future = childChannel.writeAndFlush(new DefaultHttp2DataFrame( + Unpooled.buffer().writeZero((int) bytesBeforeUnwritable))); + assertFalse(childChannel.isWritable()); + assertTrue(parentChannel.isWritable()); + + parentChannel.flush(); + assertFalse(future.isDone()); + assertTrue(parentChannel.isWritable()); + assertFalse(childChannel.isWritable()); + + // Now write an window update frame for the stream which then should ensure we will flush the bytes that were + // queued in the RemoteFlowController before for the stream. + frameInboundWriter.writeInboundWindowUpdate(childChannel.stream().id(), (int) bytesBeforeUnwritable); + assertTrue(childChannel.isWritable()); + assertTrue(future.isDone()); + } + + @Test + public void channelClosedWhenInactiveFired() { + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); + + final AtomicBoolean channelOpen = new AtomicBoolean(false); + final AtomicBoolean channelActive = new AtomicBoolean(false); + assertTrue(childChannel.isOpen()); + assertTrue(childChannel.isActive()); + + childChannel.pipeline().addLast(new ChannelInboundHandlerAdapter() { + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + channelOpen.set(ctx.channel().isOpen()); + channelActive.set(ctx.channel().isActive()); + + super.channelInactive(ctx); + } + }); + + childChannel.close().syncUninterruptibly(); + assertFalse(channelOpen.get()); + assertFalse(channelActive.get()); + } + + @Test + public void channelInactiveHappensAfterExceptionCaughtEvents() throws Exception { + final AtomicInteger count = new AtomicInteger(0); + final AtomicInteger exceptionCaught = new AtomicInteger(-1); + final AtomicInteger channelInactive = new AtomicInteger(-1); + final AtomicInteger channelUnregistered = new AtomicInteger(-1); + Http2StreamChannel childChannel = newOutboundStream(new ChannelInboundHandlerAdapter() { + + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + ctx.close(); + throw new Exception("exception"); + } + }); + + childChannel.pipeline().addLast(new ChannelInboundHandlerAdapter() { + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + channelInactive.set(count.getAndIncrement()); + super.channelInactive(ctx); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + exceptionCaught.set(count.getAndIncrement()); + super.exceptionCaught(ctx, cause); + } + + @Override + public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { + channelUnregistered.set(count.getAndIncrement()); + super.channelUnregistered(ctx); + } + }); + + childChannel.pipeline().fireUserEventTriggered(new Object()); + parentChannel.runPendingTasks(); + + // The events should have happened in this order because the inactive and deregistration events + // get deferred as they do in the AbstractChannel. + assertEquals(0, exceptionCaught.get()); + assertEquals(1, channelInactive.get()); + assertEquals(2, channelUnregistered.get()); + } + + @Test + public void callUnsafeCloseMultipleTimes() { + LastInboundHandler inboundHandler = new LastInboundHandler(); + Http2StreamChannel childChannel = newInboundStream(3, false, inboundHandler); + childChannel.unsafe().close(childChannel.voidPromise()); + + ChannelPromise promise = childChannel.newPromise(); + childChannel.unsafe().close(promise); + promise.syncUninterruptibly(); + childChannel.closeFuture().syncUninterruptibly(); + } + + @Test + public void endOfStreamDoesNotDiscardData() { + AtomicInteger numReads = new AtomicInteger(1); + final AtomicBoolean shouldDisableAutoRead = new AtomicBoolean(); + Consumer ctxConsumer = new Consumer() { + @Override + public void accept(ChannelHandlerContext obj) { + if (shouldDisableAutoRead.get()) { + obj.channel().config().setAutoRead(false); + } + } + }; + LastInboundHandler inboundHandler = new LastInboundHandler(ctxConsumer); + Http2StreamChannel childChannel = newInboundStream(3, false, numReads, inboundHandler); + childChannel.config().setAutoRead(false); + + Http2DataFrame dataFrame1 = new DefaultHttp2DataFrame(bb("1")).stream(childChannel.stream()); + Http2DataFrame dataFrame2 = new DefaultHttp2DataFrame(bb("2")).stream(childChannel.stream()); + Http2DataFrame dataFrame3 = new DefaultHttp2DataFrame(bb("3")).stream(childChannel.stream()); + Http2DataFrame dataFrame4 = new DefaultHttp2DataFrame(bb("4")).stream(childChannel.stream()); + + assertEquals(new DefaultHttp2HeadersFrame(request).stream(childChannel.stream()), inboundHandler.readInbound()); + + ChannelHandler readCompleteSupressHandler = new ChannelInboundHandlerAdapter() { + @Override + public void channelReadComplete(ChannelHandlerContext ctx) { + // We want to simulate the parent channel calling channelRead and delay calling channelReadComplete. + } + }; + + parentChannel.pipeline().addFirst(readCompleteSupressHandler); + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("1"), 0, false); + + assertEqualsAndRelease(dataFrame1, inboundHandler.readInbound()); + + // Deliver frames, and then a stream closed while read is inactive. + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("2"), 0, false); + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("3"), 0, false); + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("4"), 0, false); + + shouldDisableAutoRead.set(true); + childChannel.config().setAutoRead(true); + numReads.set(1); + + frameInboundWriter.writeInboundRstStream(childChannel.stream().id(), Http2Error.NO_ERROR.code()); + + // Detecting EOS should flush all pending data regardless of read calls. + assertEqualsAndRelease(dataFrame2, inboundHandler.readInbound()); + assertEqualsAndRelease(dataFrame3, inboundHandler.readInbound()); + assertEqualsAndRelease(dataFrame4, inboundHandler.readInbound()); + + Http2ResetFrame resetFrame = inboundHandler.readInbound(); + assertEquals(childChannel.stream(), resetFrame.stream()); + assertEquals(Http2Error.NO_ERROR.code(), resetFrame.errorCode()); + + assertNull(inboundHandler.readInbound()); + + // Now we want to call channelReadComplete and simulate the end of the read loop. + parentChannel.pipeline().remove(readCompleteSupressHandler); + parentChannel.flushInbound(); + + childChannel.closeFuture().syncUninterruptibly(); + } + + @Test + public void childQueueIsDrainedAndNewDataIsDispatchedInParentReadLoopAutoRead() { + AtomicInteger numReads = new AtomicInteger(1); + final AtomicInteger channelReadCompleteCount = new AtomicInteger(0); + final AtomicBoolean shouldDisableAutoRead = new AtomicBoolean(); + Consumer ctxConsumer = new Consumer() { + @Override + public void accept(ChannelHandlerContext obj) { + channelReadCompleteCount.incrementAndGet(); + if (shouldDisableAutoRead.get()) { + obj.channel().config().setAutoRead(false); + } + } + }; + LastInboundHandler inboundHandler = new LastInboundHandler(ctxConsumer); + Http2StreamChannel childChannel = newInboundStream(3, false, numReads, inboundHandler); + childChannel.config().setAutoRead(false); + + Http2DataFrame dataFrame1 = new DefaultHttp2DataFrame(bb("1")).stream(childChannel.stream()); + Http2DataFrame dataFrame2 = new DefaultHttp2DataFrame(bb("2")).stream(childChannel.stream()); + Http2DataFrame dataFrame3 = new DefaultHttp2DataFrame(bb("3")).stream(childChannel.stream()); + Http2DataFrame dataFrame4 = new DefaultHttp2DataFrame(bb("4")).stream(childChannel.stream()); + + assertEquals(new DefaultHttp2HeadersFrame(request).stream(childChannel.stream()), inboundHandler.readInbound()); + + ChannelHandler readCompleteSupressHandler = new ChannelInboundHandlerAdapter() { + @Override + public void channelReadComplete(ChannelHandlerContext ctx) { + // We want to simulate the parent channel calling channelRead and delay calling channelReadComplete. + } + }; + parentChannel.pipeline().addFirst(readCompleteSupressHandler); + + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("1"), 0, false); + + assertEqualsAndRelease(dataFrame1, inboundHandler.readInbound()); + + // We want one item to be in the queue, and allow the numReads to be larger than 1. This will ensure that + // when beginRead() is called the child channel is added to the readPending queue of the parent channel. + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("2"), 0, false); + + numReads.set(10); + shouldDisableAutoRead.set(true); + childChannel.config().setAutoRead(true); + + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("3"), 0, false); + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("4"), 0, false); + + // Detecting EOS should flush all pending data regardless of read calls. + assertEqualsAndRelease(dataFrame2, inboundHandler.readInbound()); + assertEqualsAndRelease(dataFrame3, inboundHandler.readInbound()); + assertEqualsAndRelease(dataFrame4, inboundHandler.readInbound()); + + assertNull(inboundHandler.readInbound()); + + // Now we want to call channelReadComplete and simulate the end of the read loop. + parentChannel.pipeline().remove(readCompleteSupressHandler); + parentChannel.flushInbound(); + + // 3 = 1 for initialization + 1 for read when auto read was off + 1 for when auto read was back on + assertEquals(3, channelReadCompleteCount.get()); + } + + @Test + public void childQueueIsDrainedAndNewDataIsDispatchedInParentReadLoopNoAutoRead() { + final AtomicInteger numReads = new AtomicInteger(1); + final AtomicInteger channelReadCompleteCount = new AtomicInteger(0); + final AtomicBoolean shouldDisableAutoRead = new AtomicBoolean(); + Consumer ctxConsumer = new Consumer() { + @Override + public void accept(ChannelHandlerContext obj) { + channelReadCompleteCount.incrementAndGet(); + if (shouldDisableAutoRead.get()) { + obj.channel().config().setAutoRead(false); + } + } + }; + final LastInboundHandler inboundHandler = new LastInboundHandler(ctxConsumer); + Http2StreamChannel childChannel = newInboundStream(3, false, numReads, inboundHandler); + childChannel.config().setAutoRead(false); + + Http2DataFrame dataFrame1 = new DefaultHttp2DataFrame(bb("1")).stream(childChannel.stream()); + Http2DataFrame dataFrame2 = new DefaultHttp2DataFrame(bb("2")).stream(childChannel.stream()); + Http2DataFrame dataFrame3 = new DefaultHttp2DataFrame(bb("3")).stream(childChannel.stream()); + Http2DataFrame dataFrame4 = new DefaultHttp2DataFrame(bb("4")).stream(childChannel.stream()); + + assertEquals(new DefaultHttp2HeadersFrame(request).stream(childChannel.stream()), inboundHandler.readInbound()); + + ChannelHandler readCompleteSupressHandler = new ChannelInboundHandlerAdapter() { + @Override + public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { + // We want to simulate the parent channel calling channelRead and delay calling channelReadComplete. + } + }; + parentChannel.pipeline().addFirst(readCompleteSupressHandler); + + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("1"), 0, false); + + assertEqualsAndRelease(dataFrame1, inboundHandler.readInbound()); + + // We want one item to be in the queue, and allow the numReads to be larger than 1. This will ensure that + // when beginRead() is called the child channel is added to the readPending queue of the parent channel. + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("2"), 0, false); + + numReads.set(2); + childChannel.read(); + + assertEqualsAndRelease(dataFrame2, inboundHandler.readInbound()); + + assertNull(inboundHandler.readInbound()); + + // This is the second item that was read, this should be the last until we call read() again. This should also + // notify of readComplete(). + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("3"), 0, false); + + assertEqualsAndRelease(dataFrame3, inboundHandler.readInbound()); + + frameInboundWriter.writeInboundData(childChannel.stream().id(), bb("4"), 0, false); + assertNull(inboundHandler.readInbound()); + + childChannel.read(); + + assertEqualsAndRelease(dataFrame4, inboundHandler.readInbound()); + + assertNull(inboundHandler.readInbound()); + + // Now we want to call channelReadComplete and simulate the end of the read loop. + parentChannel.pipeline().remove(readCompleteSupressHandler); + parentChannel.flushInbound(); + + // 3 = 1 for initialization + 1 for first read of 2 items + 1 for second read of 2 items + + // 1 for parent channel readComplete + assertEquals(4, channelReadCompleteCount.get()); + } + + private static void verifyFramesMultiplexedToCorrectChannel(Http2StreamChannel streamChannel, + LastInboundHandler inboundHandler, + int numFrames) { + for (int i = 0; i < numFrames; i++) { + Http2StreamFrame frame = inboundHandler.readInbound(); + assertNotNull(frame); + assertEquals(streamChannel.stream(), frame.stream()); + release(frame); + } + assertNull(inboundHandler.readInbound()); + } + + private static int eqStreamId(Http2StreamChannel channel) { + return eq(channel.stream().id()); + } +} diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecTransportTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexTransportTest.java similarity index 87% rename from codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecTransportTest.java rename to codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexTransportTest.java index 1a925a8369..e9b686c40c 100644 --- a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexCodecTransportTest.java +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2MultiplexTransportTest.java @@ -39,7 +39,7 @@ import java.util.concurrent.atomic.AtomicReference; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.junit.Assert.assertFalse; -public class Http2MultiplexCodecTransportTest { +public class Http2MultiplexTransportTest { private EventLoopGroup eventLoopGroup; private Channel clientChannel; private Channel serverChannel; @@ -65,7 +65,18 @@ public class Http2MultiplexCodecTransportTest { } @Test(timeout = 10000) - public void asyncSettingsAck() throws InterruptedException { + public void asyncSettingsAckWithMultiplexCodec() throws InterruptedException { + asyncSettingsAck0(new Http2MultiplexCodecBuilder(true, new HttpInboundHandler()).build(), null); + } + + @Test(timeout = 10000) + public void asyncSettingsAckWithMultiplexHandler() throws InterruptedException { + asyncSettingsAck0(new Http2FrameCodecBuilder(true).build(), + new Http2MultiplexHandler(new HttpInboundHandler())); + } + + private void asyncSettingsAck0(final Http2FrameCodec codec, final ChannelHandler multiplexer) + throws InterruptedException { // The client expects 2 settings frames. One from the connection setup and one from this test. final CountDownLatch serverAckOneLatch = new CountDownLatch(1); final CountDownLatch serverAckAllLatch = new CountDownLatch(2); @@ -78,7 +89,10 @@ public class Http2MultiplexCodecTransportTest { sb.childHandler(new ChannelInitializer() { @Override protected void initChannel(Channel ch) { - ch.pipeline().addLast(Http2MultiplexCodecBuilder.forServer(new HttpInboundHandler()).build()); + ch.pipeline().addLast(codec); + if (multiplexer != null) { + ch.pipeline().addLast(multiplexer); + } ch.pipeline().addLast(new ChannelInboundHandlerAdapter() { @Override public void channelActive(ChannelHandlerContext ctx) { diff --git a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ServerUpgradeCodecTest.java b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ServerUpgradeCodecTest.java index fe93b6b791..02fa243629 100644 --- a/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ServerUpgradeCodecTest.java +++ b/codec-http2/src/test/java/io/netty/handler/codec/http2/Http2ServerUpgradeCodecTest.java @@ -37,20 +37,26 @@ public class Http2ServerUpgradeCodecTest { @Test public void testUpgradeToHttp2ConnectionHandler() { - testUpgrade(new Http2ConnectionHandlerBuilder().frameListener(new Http2FrameAdapter()).build()); + testUpgrade(new Http2ConnectionHandlerBuilder().frameListener(new Http2FrameAdapter()).build(), null); } @Test public void testUpgradeToHttp2FrameCodec() { - testUpgrade(new Http2FrameCodecBuilder(true).build()); + testUpgrade(new Http2FrameCodecBuilder(true).build(), null); } @Test public void testUpgradeToHttp2MultiplexCodec() { - testUpgrade(new Http2MultiplexCodecBuilder(true, new HttpInboundHandler()).build()); + testUpgrade(new Http2MultiplexCodecBuilder(true, new HttpInboundHandler()).build(), null); } - private static void testUpgrade(Http2ConnectionHandler handler) { + @Test + public void testUpgradeToHttp2FrameCodecWithMultiplexer() { + testUpgrade(new Http2FrameCodecBuilder(true).build(), + new Http2MultiplexHandler(new HttpInboundHandler())); + } + + private static void testUpgrade(Http2ConnectionHandler handler, ChannelHandler multiplexer) { FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.OPTIONS, "*"); request.headers().set(HttpHeaderNames.HOST, "netty.io"); request.headers().set(HttpHeaderNames.CONNECTION, "Upgrade, HTTP2-Settings"); @@ -59,13 +65,18 @@ public class Http2ServerUpgradeCodecTest { EmbeddedChannel channel = new EmbeddedChannel(new ChannelInboundHandlerAdapter()); ChannelHandlerContext ctx = channel.pipeline().firstContext(); - Http2ServerUpgradeCodec codec = new Http2ServerUpgradeCodec("connectionHandler", handler); + Http2ServerUpgradeCodec codec; + if (multiplexer == null) { + codec = new Http2ServerUpgradeCodec(handler); + } else { + codec = new Http2ServerUpgradeCodec((Http2FrameCodec) handler, multiplexer); + } assertTrue(codec.prepareUpgradeResponse(ctx, request, new DefaultHttpHeaders())); codec.upgradeTo(ctx, request); // Flush the channel to ensure we write out all buffered data channel.flush(); - assertSame(handler, channel.pipeline().remove("connectionHandler")); + assertSame(handler, channel.pipeline().remove(handler.getClass())); assertNull(channel.pipeline().get(handler.getClass())); assertTrue(channel.finish()); diff --git a/example/src/main/java/io/netty/example/http2/helloworld/multiplex/server/Http2OrHttpHandler.java b/example/src/main/java/io/netty/example/http2/helloworld/multiplex/server/Http2OrHttpHandler.java index 4637e79259..4099b04440 100644 --- a/example/src/main/java/io/netty/example/http2/helloworld/multiplex/server/Http2OrHttpHandler.java +++ b/example/src/main/java/io/netty/example/http2/helloworld/multiplex/server/Http2OrHttpHandler.java @@ -18,7 +18,8 @@ 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.Http2MultiplexCodecBuilder; +import io.netty.handler.codec.http2.Http2FrameCodecBuilder; +import io.netty.handler.codec.http2.Http2MultiplexHandler; import io.netty.handler.ssl.ApplicationProtocolNames; import io.netty.handler.ssl.ApplicationProtocolNegotiationHandler; @@ -37,7 +38,8 @@ public class Http2OrHttpHandler extends ApplicationProtocolNegotiationHandler { @Override protected void configurePipeline(ChannelHandlerContext ctx, String protocol) throws Exception { if (ApplicationProtocolNames.HTTP_2.equals(protocol)) { - ctx.pipeline().addLast(Http2MultiplexCodecBuilder.forServer(new HelloWorldHttp2Handler()).build()); + ctx.pipeline().addLast(Http2FrameCodecBuilder.forServer().build()); + ctx.pipeline().addLast(new Http2MultiplexHandler(new HelloWorldHttp2Handler())); return; } diff --git a/example/src/main/java/io/netty/example/http2/helloworld/multiplex/server/Http2ServerInitializer.java b/example/src/main/java/io/netty/example/http2/helloworld/multiplex/server/Http2ServerInitializer.java index 891becc432..00f3dc3d4f 100644 --- a/example/src/main/java/io/netty/example/http2/helloworld/multiplex/server/Http2ServerInitializer.java +++ b/example/src/main/java/io/netty/example/http2/helloworld/multiplex/server/Http2ServerInitializer.java @@ -29,8 +29,9 @@ 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.Http2MultiplexCodecBuilder; +import io.netty.handler.codec.http2.Http2FrameCodecBuilder; import io.netty.handler.codec.http2.Http2CodecUtil; +import io.netty.handler.codec.http2.Http2MultiplexHandler; import io.netty.handler.codec.http2.Http2ServerUpgradeCodec; import io.netty.handler.ssl.SslContext; import io.netty.util.AsciiString; @@ -47,7 +48,8 @@ public class Http2ServerInitializer extends ChannelInitializer { public UpgradeCodec newUpgradeCodec(CharSequence protocol) { if (AsciiString.contentEquals(Http2CodecUtil.HTTP_UPGRADE_PROTOCOL_NAME, protocol)) { return new Http2ServerUpgradeCodec( - Http2MultiplexCodecBuilder.forServer(new HelloWorldHttp2Handler()).build()); + Http2FrameCodecBuilder.forServer().build(), + new Http2MultiplexHandler(new HelloWorldHttp2Handler())); } else { return null; }