Add support for HTTP/2 draft 12.
Motivation: Draft 12 has just arrived and has quite a few changes. Need to update in order to keep current with the spec. Modifications: This is a rewrite of the original (draft 10) code. There are only 2 handlers now: preface and connection. The connection handler is now callback based rather than frame based (there are no frame classes anymore). AbstractHttp2ConnectionHandler is the base class for any HTTP/2 handlers. All of the stream priority logic now resides in the outbound flow controller, and its interface exposes methods for adding/updating priority for streams. Upgraded to hpack 0.7.0, which is used by draft12. Also removed draft10 code and moved draft12 code to the ../http2 package (no draft subpackage). Result: Addition of a HTTP/2 draft 12 support.
This commit is contained in:
parent
b1a51f0669
commit
ca7c53d45f
@ -43,7 +43,7 @@
|
||||
<dependency>
|
||||
<groupId>com.twitter</groupId>
|
||||
<artifactId>hpack</artifactId>
|
||||
<version>0.6.0</version>
|
||||
<version>0.7.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
|
@ -0,0 +1,898 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.failAndThrow;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.toByteBuf;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.toHttp2Exception;
|
||||
import static io.netty.handler.codec.http2.Http2Error.NO_ERROR;
|
||||
import static io.netty.handler.codec.http2.Http2Error.PROTOCOL_ERROR;
|
||||
import static io.netty.handler.codec.http2.Http2Error.STREAM_CLOSED;
|
||||
import static io.netty.handler.codec.http2.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.Http2Stream.State.HALF_CLOSED_LOCAL;
|
||||
import static io.netty.handler.codec.http2.Http2Stream.State.HALF_CLOSED_REMOTE;
|
||||
import static io.netty.handler.codec.http2.Http2Stream.State.OPEN;
|
||||
import static io.netty.handler.codec.http2.Http2Stream.State.RESERVED_LOCAL;
|
||||
import static io.netty.handler.codec.http2.Http2Stream.State.RESERVED_REMOTE;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelFutureListener;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
import io.netty.handler.codec.ByteToMessageDecoder;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Abstract base class for a handler of HTTP/2 frames. Handles reading and writing of HTTP/2
|
||||
* frames as well as management of connection state and flow control for both inbound and outbound
|
||||
* data frames.
|
||||
* <p>
|
||||
* Subclasses need to implement the methods defined by the {@link Http2FrameObserver} interface for
|
||||
* receiving inbound frames. Outbound frames are sent via one of the {@code writeXXX} methods.
|
||||
* <p>
|
||||
* It should be noted that the initial SETTINGS frame is sent upon either activation or addition of
|
||||
* this handler to the pipeline. Subclasses overriding {@link #channelActive} or
|
||||
* {@link #handlerAdded} must call this class to write the initial SETTINGS frame to the remote
|
||||
* endpoint.
|
||||
*/
|
||||
public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecoder implements
|
||||
Http2FrameObserver {
|
||||
|
||||
private final Http2FrameObserver internalFrameObserver = new FrameReadObserver();
|
||||
private final Http2FrameReader frameReader;
|
||||
private final Http2FrameWriter frameWriter;
|
||||
private final Http2Connection connection;
|
||||
private final Http2InboundFlowController inboundFlow;
|
||||
private final Http2OutboundFlowController outboundFlow;
|
||||
private boolean initialSettingsSent;
|
||||
private boolean initialSettingsReceived;
|
||||
private ChannelHandlerContext ctx;
|
||||
private ChannelFutureListener closeListener;
|
||||
|
||||
protected AbstractHttp2ConnectionHandler(boolean server) {
|
||||
this(server, false);
|
||||
}
|
||||
|
||||
protected AbstractHttp2ConnectionHandler(boolean server, boolean allowCompression) {
|
||||
this(new DefaultHttp2Connection(server, allowCompression));
|
||||
}
|
||||
|
||||
protected AbstractHttp2ConnectionHandler(Http2Connection connection) {
|
||||
this(connection, new DefaultHttp2FrameReader(connection.isServer()),
|
||||
new DefaultHttp2FrameWriter(connection.isServer()),
|
||||
new DefaultHttp2InboundFlowController(), new DefaultHttp2OutboundFlowController());
|
||||
}
|
||||
|
||||
protected AbstractHttp2ConnectionHandler(Http2Connection connection,
|
||||
Http2FrameReader frameReader, Http2FrameWriter frameWriter,
|
||||
Http2InboundFlowController inboundFlow, Http2OutboundFlowController outboundFlow) {
|
||||
if (connection == null) {
|
||||
throw new NullPointerException("connection");
|
||||
}
|
||||
if (frameReader == null) {
|
||||
throw new NullPointerException("frameReader");
|
||||
}
|
||||
if (frameWriter == null) {
|
||||
throw new NullPointerException("frameWriter");
|
||||
}
|
||||
if (inboundFlow == null) {
|
||||
throw new NullPointerException("inboundFlow");
|
||||
}
|
||||
if (outboundFlow == null) {
|
||||
throw new NullPointerException("outboundFlow");
|
||||
}
|
||||
this.connection = connection;
|
||||
this.frameReader = frameReader;
|
||||
this.frameWriter = frameWriter;
|
||||
this.inboundFlow = inboundFlow;
|
||||
this.outboundFlow = outboundFlow;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelActive(ChannelHandlerContext ctx) throws Exception {
|
||||
// The channel just became active - send the initial settings frame to the remote
|
||||
// endpoint.
|
||||
sendInitialSettings(ctx);
|
||||
super.channelActive(ctx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
// This handler was just added to the context. In case it was handled after
|
||||
// the connection became active, send the initial settings frame now.
|
||||
this.ctx = ctx;
|
||||
sendInitialSettings(ctx);
|
||||
}
|
||||
|
||||
protected final ChannelHandlerContext ctx() {
|
||||
return ctx;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
|
||||
// Avoid NotYetConnectedException
|
||||
if (!ctx.channel().isActive()) {
|
||||
ctx.close(promise);
|
||||
return;
|
||||
}
|
||||
|
||||
sendGoAway(ctx, promise, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
|
||||
ChannelFuture future = ctx.newSucceededFuture();
|
||||
for (Http2Stream stream : connection.activeStreams().toArray(new Http2Stream[0])) {
|
||||
close(stream, ctx, future);
|
||||
}
|
||||
super.channelInactive(ctx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Handles {@link Http2Exception} objects that were thrown from other handlers. Ignores all
|
||||
* other exceptions.
|
||||
*/
|
||||
@Override
|
||||
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
|
||||
if (cause instanceof Http2Exception) {
|
||||
processHttp2Exception(ctx, (Http2Exception) cause);
|
||||
}
|
||||
|
||||
super.exceptionCaught(ctx, cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the local settings for this endpoint of the HTTP/2 connection.
|
||||
*/
|
||||
public final Http2Settings settings() {
|
||||
Http2Settings settings = new Http2Settings();
|
||||
settings.allowCompressedData(connection.local().allowCompressedData());
|
||||
settings.initialWindowSize(inboundFlow.initialInboundWindowSize());
|
||||
settings.pushEnabled(connection.local().allowPushTo());
|
||||
settings.maxConcurrentStreams(connection.remote().maxStreams());
|
||||
settings.maxHeaderTableSize(frameReader.maxHeaderTableSize());
|
||||
return settings;
|
||||
}
|
||||
|
||||
protected ChannelFuture writeData(final ChannelHandlerContext ctx,
|
||||
final ChannelPromise promise, int streamId, final ByteBuf data, int padding,
|
||||
boolean endStream, boolean endSegment, boolean compressed) throws Http2Exception {
|
||||
try {
|
||||
if (connection.isGoAway()) {
|
||||
throw protocolError("Sending data after connection going away.");
|
||||
}
|
||||
|
||||
if (!connection.remote().allowCompressedData() && compressed) {
|
||||
throw protocolError("compression is disallowed for remote endpoint.");
|
||||
}
|
||||
|
||||
Http2Stream stream = connection.requireStream(streamId);
|
||||
stream.verifyState(PROTOCOL_ERROR, OPEN, HALF_CLOSED_REMOTE);
|
||||
|
||||
// Hand control of the frame to the flow controller.
|
||||
outboundFlow.sendFlowControlled(streamId, data, padding, endStream, endSegment,
|
||||
compressed, new FlowControlWriter(ctx, data, promise));
|
||||
|
||||
return promise;
|
||||
} catch (Http2Exception e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception {
|
||||
return writeHeaders(ctx, promise, streamId, headers, 0, DEFAULT_PRIORITY_WEIGHT, false,
|
||||
padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
protected ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, Http2Headers headers, int streamDependency, short weight,
|
||||
boolean exclusive, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception {
|
||||
try {
|
||||
if (connection.isGoAway()) {
|
||||
throw protocolError("Sending headers after connection going away.");
|
||||
}
|
||||
|
||||
Http2Stream stream = connection.stream(streamId);
|
||||
if (stream == null) {
|
||||
// Creates a new locally-initiated stream.
|
||||
stream = connection.local().createStream(streamId, endStream);
|
||||
|
||||
// Allow bi-directional traffic.
|
||||
inboundFlow.addStream(streamId);
|
||||
if (!endStream) {
|
||||
outboundFlow.addStream(streamId, streamDependency, weight, exclusive);
|
||||
}
|
||||
} else {
|
||||
// An existing stream...
|
||||
if (stream.state() == RESERVED_LOCAL) {
|
||||
// Sending headers on a reserved push stream ... open it for push to the remote
|
||||
// endpoint.
|
||||
stream.openForPush();
|
||||
|
||||
// Allow outbound traffic only.
|
||||
if (!endStream) {
|
||||
outboundFlow.addStream(streamId, streamDependency, weight, exclusive);
|
||||
}
|
||||
} else {
|
||||
// The stream already exists, make sure it's in an allowed state.
|
||||
stream.verifyState(PROTOCOL_ERROR, OPEN, HALF_CLOSED_REMOTE);
|
||||
|
||||
// Update the priority for this stream only if we'll be sending more data.
|
||||
if (!endStream) {
|
||||
outboundFlow.updateStream(stream.id(), streamDependency, weight, exclusive);
|
||||
}
|
||||
}
|
||||
|
||||
// If the headers are the end of the stream, close it now.
|
||||
if (endStream) {
|
||||
closeLocalSide(stream, ctx, promise);
|
||||
}
|
||||
}
|
||||
|
||||
return frameWriter.writeHeaders(ctx, promise, streamId, headers, streamDependency,
|
||||
weight, exclusive, padding, endStream, endSegment);
|
||||
} catch (Http2Exception e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected ChannelFuture writePriority(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int streamDependency, short weight, boolean exclusive)
|
||||
throws Http2Exception {
|
||||
try {
|
||||
if (connection.isGoAway()) {
|
||||
throw protocolError("Sending priority after connection going away.");
|
||||
}
|
||||
|
||||
// Update the priority on this stream.
|
||||
outboundFlow.updateStream(streamId, streamDependency, weight, exclusive);
|
||||
|
||||
return frameWriter.writePriority(ctx, promise, streamId, streamDependency, weight,
|
||||
exclusive);
|
||||
} catch (Http2Exception e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected ChannelFuture writeRstStream(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, long errorCode) {
|
||||
Http2Stream stream = connection.stream(streamId);
|
||||
if (stream == null) {
|
||||
// The stream may already have been closed ... ignore.
|
||||
promise.setSuccess();
|
||||
return promise;
|
||||
}
|
||||
|
||||
close(stream, ctx, promise);
|
||||
|
||||
return frameWriter.writeRstStream(ctx, promise, streamId, errorCode);
|
||||
}
|
||||
|
||||
protected ChannelFuture writeSettings(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
Http2Settings settings) throws Http2Exception {
|
||||
try {
|
||||
if (connection.isGoAway()) {
|
||||
throw protocolError("Sending settings after connection going away.");
|
||||
}
|
||||
|
||||
if (settings.hasAllowCompressedData()) {
|
||||
connection.local().allowCompressedData(settings.allowCompressedData());
|
||||
}
|
||||
|
||||
if (settings.hasPushEnabled()) {
|
||||
connection.local().allowPushTo(settings.pushEnabled());
|
||||
}
|
||||
|
||||
if (settings.hasMaxConcurrentStreams()) {
|
||||
connection.remote().maxStreams(settings.maxConcurrentStreams());
|
||||
}
|
||||
|
||||
if (settings.hasMaxHeaderTableSize()) {
|
||||
frameReader.maxHeaderTableSize(settings.maxHeaderTableSize());
|
||||
}
|
||||
|
||||
if (settings.hasInitialWindowSize()) {
|
||||
inboundFlow.initialInboundWindowSize(settings.initialWindowSize());
|
||||
}
|
||||
|
||||
return frameWriter.writeSettings(ctx, promise, settings);
|
||||
} catch (Http2Exception e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected ChannelFuture writePing(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
ByteBuf data) throws Http2Exception {
|
||||
try {
|
||||
if (connection.isGoAway()) {
|
||||
throw protocolError("Sending ping after connection going away.");
|
||||
}
|
||||
|
||||
// Just pass the frame through.
|
||||
return frameWriter.writePing(ctx, promise, false, data);
|
||||
} catch (Http2Exception e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected ChannelFuture writePushPromise(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int promisedStreamId, Http2Headers headers, int padding)
|
||||
throws Http2Exception {
|
||||
try {
|
||||
if (connection.isGoAway()) {
|
||||
throw protocolError("Sending push promise after connection going away.");
|
||||
}
|
||||
|
||||
// Reserve the promised stream.
|
||||
Http2Stream stream = connection.requireStream(streamId);
|
||||
connection.local().reservePushStream(promisedStreamId, stream);
|
||||
|
||||
// Write the frame.
|
||||
return frameWriter.writePushPromise(ctx, promise, streamId, promisedStreamId, headers,
|
||||
padding);
|
||||
} catch (Http2Exception e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected ChannelFuture writeAltSvc(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, long maxAge, int port, ByteBuf protocolId, String host, String origin)
|
||||
throws Http2Exception {
|
||||
return frameWriter.writeAltSvc(ctx, promise, streamId, maxAge, port, protocolId, host,
|
||||
origin);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out)
|
||||
throws Exception {
|
||||
try {
|
||||
frameReader.readFrame(ctx.alloc(), in, internalFrameObserver);
|
||||
} catch (Http2Exception e) {
|
||||
processHttp2Exception(ctx, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Processes the given exception. Depending on the type of exception, delegates to either
|
||||
* {@link #processConnectionError} or {@link #processStreamError}.
|
||||
*/
|
||||
private void processHttp2Exception(ChannelHandlerContext ctx, Http2Exception e) {
|
||||
if (e instanceof Http2StreamException) {
|
||||
processStreamError(ctx, (Http2StreamException) e);
|
||||
} else {
|
||||
processConnectionError(ctx, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void processConnectionError(ChannelHandlerContext ctx, Http2Exception cause) {
|
||||
sendGoAway(ctx, ctx.newPromise(), cause);
|
||||
}
|
||||
|
||||
private void processStreamError(ChannelHandlerContext ctx, Http2StreamException cause) {
|
||||
// Close the stream if it was open.
|
||||
int streamId = cause.streamId();
|
||||
Http2Stream stream = connection.stream(streamId);
|
||||
if (stream != null) {
|
||||
close(stream, ctx, null);
|
||||
}
|
||||
|
||||
// Send the Rst frame to the remote endpoint.
|
||||
frameWriter.writeRstStream(ctx, ctx.newPromise(), streamId, cause.error().code());
|
||||
}
|
||||
|
||||
private void sendGoAway(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
Http2Exception cause) {
|
||||
ChannelFuture future = null;
|
||||
ChannelPromise closePromise = promise;
|
||||
if (!connection.isGoAway()) {
|
||||
connection.goAwaySent();
|
||||
|
||||
int errorCode = cause != null ? cause.error().code() : NO_ERROR.code();
|
||||
ByteBuf debugData = toByteBuf(ctx, cause);
|
||||
|
||||
future = frameWriter.writeGoAway(ctx, promise, connection.remote().lastStreamCreated(),
|
||||
errorCode, debugData);
|
||||
closePromise = null;
|
||||
}
|
||||
|
||||
closeListener = getOrCreateCloseListener(ctx, closePromise);
|
||||
|
||||
// If there are no active streams, close immediately after the send is complete.
|
||||
// Otherwise wait until all streams are inactive.
|
||||
if (cause != null || connection.numActiveStreams() == 0) {
|
||||
if (future == null) {
|
||||
future = ctx.newSucceededFuture();
|
||||
}
|
||||
future.addListener(closeListener);
|
||||
}
|
||||
}
|
||||
|
||||
private ChannelFutureListener getOrCreateCloseListener(final ChannelHandlerContext ctx,
|
||||
ChannelPromise promise) {
|
||||
final ChannelPromise closePromise = promise == null? ctx.newPromise() : promise;
|
||||
if (closeListener == null) {
|
||||
// If no promise was provided, create a new one.
|
||||
closeListener = new ChannelFutureListener() {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
ctx.close(closePromise);
|
||||
freeResources();
|
||||
}
|
||||
};
|
||||
} else {
|
||||
closePromise.setSuccess();
|
||||
}
|
||||
|
||||
return closeListener;
|
||||
}
|
||||
|
||||
private void freeResources() {
|
||||
frameReader.close();
|
||||
frameWriter.close();
|
||||
}
|
||||
|
||||
private void closeLocalSide(Http2Stream stream, ChannelHandlerContext ctx, ChannelFuture future) {
|
||||
switch (stream.state()) {
|
||||
case HALF_CLOSED_LOCAL:
|
||||
case OPEN:
|
||||
stream.closeLocalSide();
|
||||
outboundFlow.removeStream(stream.id());
|
||||
break;
|
||||
default:
|
||||
close(stream, ctx, future);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
private void closeRemoteSide(Http2Stream stream, ChannelHandlerContext ctx, ChannelFuture future) {
|
||||
switch (stream.state()) {
|
||||
case HALF_CLOSED_REMOTE:
|
||||
case OPEN:
|
||||
stream.closeRemoteSide();
|
||||
inboundFlow.removeStream(stream.id());
|
||||
break;
|
||||
default:
|
||||
close(stream, ctx, future);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
private void close(Http2Stream stream, ChannelHandlerContext ctx, ChannelFuture future) {
|
||||
stream.close();
|
||||
|
||||
// Notify the flow controllers.
|
||||
inboundFlow.removeStream(stream.id());
|
||||
outboundFlow.removeStream(stream.id());
|
||||
|
||||
// If this connection is closing and there are no longer any
|
||||
// active streams, close after the current operation completes.
|
||||
if (closeListener != null && connection.numActiveStreams() == 0) {
|
||||
future.addListener(closeListener);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyInitialSettingsReceived() throws Http2Exception {
|
||||
if (!initialSettingsReceived) {
|
||||
throw protocolError("Received non-SETTINGS as first frame.");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends the initial settings frame upon establishment of the connection, if not already sent.
|
||||
*/
|
||||
private void sendInitialSettings(final ChannelHandlerContext ctx) throws Http2Exception {
|
||||
if (!initialSettingsSent && ctx.channel().isActive()) {
|
||||
initialSettingsSent = true;
|
||||
frameWriter.writeSettings(ctx, ctx.newPromise(), settings()).addListener(
|
||||
ChannelFutureListener.CLOSE_ON_FAILURE);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Handles all inbound frames from the network.
|
||||
*/
|
||||
private final class FrameReadObserver implements Http2FrameObserver {
|
||||
|
||||
@Override
|
||||
public void onDataRead(int streamId, ByteBuf data, int padding, boolean endOfStream,
|
||||
boolean endOfSegment, boolean compressed) throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (!connection.local().allowCompressedData() && compressed) {
|
||||
throw protocolError("compression is disallowed.");
|
||||
}
|
||||
|
||||
// Check if we received a data frame for a stream which is half-closed
|
||||
Http2Stream stream = connection.requireStream(streamId);
|
||||
stream.verifyState(STREAM_CLOSED, OPEN, HALF_CLOSED_LOCAL);
|
||||
|
||||
// Apply flow control.
|
||||
inboundFlow.applyInboundFlowControl(streamId, data, padding, endOfStream, endOfSegment,
|
||||
compressed, new Http2InboundFlowController.FrameWriter() {
|
||||
|
||||
@Override
|
||||
public void writeFrame(int streamId, int windowSizeIncrement)
|
||||
throws Http2Exception {
|
||||
frameWriter.writeWindowUpdate(ctx, ctx.newPromise(), streamId,
|
||||
windowSizeIncrement);
|
||||
}
|
||||
});
|
||||
|
||||
if (isInboundStreamAfterGoAway(streamId)) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (endOfStream) {
|
||||
closeRemoteSide(stream, ctx, ctx.newSucceededFuture());
|
||||
}
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onDataRead(streamId, data, padding, endOfStream,
|
||||
endOfSegment, compressed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onHeadersRead(int streamId, Http2Headers headers, int padding,
|
||||
boolean endStream, boolean endSegment) throws Http2Exception {
|
||||
onHeadersRead(streamId, headers, 0, DEFAULT_PRIORITY_WEIGHT, false, padding, endStream,
|
||||
endSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void
|
||||
onHeadersRead(int streamId, Http2Headers headers, int streamDependency,
|
||||
short weight, boolean exclusive, int padding, boolean endStream,
|
||||
boolean endSegment) throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(streamId)) {
|
||||
return;
|
||||
}
|
||||
|
||||
Http2Stream stream = connection.stream(streamId);
|
||||
if (stream == null) {
|
||||
// Create the new stream.
|
||||
connection.remote().createStream(streamId, endStream);
|
||||
|
||||
// Allow bi-directional traffic.
|
||||
outboundFlow.addStream(streamId, streamDependency, weight, exclusive);
|
||||
if (!endStream) {
|
||||
inboundFlow.addStream(streamId);
|
||||
}
|
||||
} else {
|
||||
if (stream.state() == RESERVED_REMOTE) {
|
||||
// Received headers for a reserved push stream ... open it for push to the
|
||||
// local endpoint.
|
||||
stream.verifyState(PROTOCOL_ERROR, RESERVED_REMOTE);
|
||||
stream.openForPush();
|
||||
|
||||
// Allow inbound traffic only.
|
||||
if (!endStream) {
|
||||
inboundFlow.addStream(streamId);
|
||||
}
|
||||
} else {
|
||||
// Receiving headers on an existing stream. Make sure the stream is in an
|
||||
// allowed
|
||||
// state.
|
||||
stream.verifyState(PROTOCOL_ERROR, OPEN, HALF_CLOSED_LOCAL);
|
||||
|
||||
// Update the outbound priority if outbound traffic is allowed.
|
||||
if (stream.state() == OPEN) {
|
||||
outboundFlow.updateStream(streamId, streamDependency, weight, exclusive);
|
||||
}
|
||||
}
|
||||
|
||||
// If the headers completes this stream, close it.
|
||||
if (endStream) {
|
||||
closeRemoteSide(stream, ctx, ctx.newSucceededFuture());
|
||||
}
|
||||
}
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onHeadersRead(streamId, headers, streamDependency,
|
||||
weight, exclusive, padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPriorityRead(int streamId, int streamDependency, short weight,
|
||||
boolean exclusive) throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(streamId)) {
|
||||
// Ignore frames for any stream created after we sent a go-away.
|
||||
return;
|
||||
}
|
||||
|
||||
// Set the priority for this stream on the flow controller.
|
||||
outboundFlow.updateStream(streamId, streamDependency, weight, exclusive);
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onPriorityRead(streamId, streamDependency, weight,
|
||||
exclusive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onRstStreamRead(int streamId, long errorCode) throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(streamId)) {
|
||||
// Ignore frames for any stream created after we sent a go-away.
|
||||
return;
|
||||
}
|
||||
|
||||
Http2Stream stream = connection.stream(streamId);
|
||||
if (stream == null) {
|
||||
// RstStream frames must be ignored for closed streams.
|
||||
return;
|
||||
}
|
||||
|
||||
close(stream, ctx, ctx.newSucceededFuture());
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onRstStreamRead(streamId, errorCode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onSettingsAckRead() throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onSettingsAckRead();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onSettingsRead(Http2Settings settings) throws Http2Exception {
|
||||
if (settings.hasAllowCompressedData()) {
|
||||
connection.remote().allowCompressedData(settings.allowCompressedData());
|
||||
}
|
||||
|
||||
if (settings.hasMaxConcurrentStreams()) {
|
||||
connection.local().maxStreams(settings.maxConcurrentStreams());
|
||||
}
|
||||
|
||||
if (settings.hasPushEnabled()) {
|
||||
connection.remote().allowPushTo(settings.pushEnabled());
|
||||
}
|
||||
|
||||
if (settings.hasMaxHeaderTableSize()) {
|
||||
frameWriter.maxHeaderTableSize(settings.maxHeaderTableSize());
|
||||
}
|
||||
|
||||
if (settings.hasInitialWindowSize()) {
|
||||
outboundFlow.initialOutboundWindowSize(settings.initialWindowSize());
|
||||
}
|
||||
|
||||
// Acknowledge receipt of the settings.
|
||||
frameWriter.writeSettingsAck(ctx, ctx.newPromise());
|
||||
|
||||
// We've received at least one non-ack settings frame from the remote endpoint.
|
||||
initialSettingsReceived = true;
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onSettingsRead(settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPingRead(ByteBuf data) throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
// Send an ack back to the remote client.
|
||||
frameWriter.writePing(ctx, ctx.newPromise(), true, data);
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onPingRead(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPingAckRead(ByteBuf data) throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onPingAckRead(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPushPromiseRead(int streamId, int promisedStreamId, Http2Headers headers,
|
||||
int padding) throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(streamId)) {
|
||||
// Ignore frames for any stream created after we sent a go-away.
|
||||
return;
|
||||
}
|
||||
|
||||
// Reserve the push stream based with a priority based on the current stream's priority.
|
||||
Http2Stream parentStream = connection.requireStream(streamId);
|
||||
connection.remote().reservePushStream(promisedStreamId, parentStream);
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onPushPromiseRead(streamId, promisedStreamId,
|
||||
headers, padding);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onGoAwayRead(int lastStreamId, long errorCode, ByteBuf debugData)
|
||||
throws Http2Exception {
|
||||
// Don't allow any more connections to be created.
|
||||
connection.goAwayReceived();
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onGoAwayRead(lastStreamId, errorCode, debugData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onWindowUpdateRead(int streamId, int windowSizeIncrement) throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(streamId)) {
|
||||
// Ignore frames for any stream created after we sent a go-away.
|
||||
return;
|
||||
}
|
||||
|
||||
if (streamId > 0) {
|
||||
Http2Stream stream = connection.stream(streamId);
|
||||
if (stream == null) {
|
||||
// Window Update frames must be ignored for closed streams.
|
||||
return;
|
||||
}
|
||||
stream.verifyState(PROTOCOL_ERROR, OPEN, HALF_CLOSED_REMOTE);
|
||||
}
|
||||
|
||||
// Update the outbound flow controller.
|
||||
outboundFlow.updateOutboundWindowSize(streamId, windowSizeIncrement);
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onWindowUpdateRead(streamId, windowSizeIncrement);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onAltSvcRead(int streamId, long maxAge, int port, ByteBuf protocolId,
|
||||
String host, String origin) throws Http2Exception {
|
||||
AbstractHttp2ConnectionHandler.this.onAltSvcRead(streamId, maxAge, port, protocolId,
|
||||
host, origin);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onBlockedRead(int streamId) throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(streamId)) {
|
||||
// Ignore frames for any stream created after we sent a go-away.
|
||||
return;
|
||||
}
|
||||
|
||||
if (streamId > 0) {
|
||||
Http2Stream stream = connection.stream(streamId);
|
||||
if (stream == null) {
|
||||
// Window Update frames must be ignored for closed streams.
|
||||
return;
|
||||
}
|
||||
stream.verifyState(PROTOCOL_ERROR, OPEN, HALF_CLOSED_REMOTE);
|
||||
}
|
||||
|
||||
// Update the outbound flow controller.
|
||||
outboundFlow.setBlocked(streamId);
|
||||
|
||||
AbstractHttp2ConnectionHandler.this.onBlockedRead(streamId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether or not the stream was created after we sent a go-away frame. Frames
|
||||
* from streams created after we sent a go-away should be ignored. Frames for the connection
|
||||
* stream ID (i.e. 0) will always be allowed.
|
||||
*/
|
||||
private boolean isInboundStreamAfterGoAway(int streamId) {
|
||||
return connection.isGoAwaySent() && connection.remote().lastStreamCreated() <= streamId;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Controls the write for a single outbound DATA frame. This writer is passed to the outbound flow
|
||||
* controller, which may break the frame into chunks as dictated by the flow control window. If
|
||||
* the write of any chunk fails, the original promise fails as well. Success occurs after the last
|
||||
* chunk is written successfully.
|
||||
*/
|
||||
private final class FlowControlWriter implements Http2OutboundFlowController.FrameWriter {
|
||||
private final ChannelHandlerContext ctx;
|
||||
private final ChannelPromise promise;
|
||||
private final List<ChannelPromise> promises;
|
||||
private int remaining;
|
||||
|
||||
FlowControlWriter(ChannelHandlerContext ctx, ByteBuf data, ChannelPromise promise) {
|
||||
this.ctx = ctx;
|
||||
this.promise = promise;
|
||||
promises = new ArrayList<ChannelPromise>(
|
||||
Arrays.asList(promise));
|
||||
remaining = data.readableBytes();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeFrame(int streamId, ByteBuf data, int padding,
|
||||
boolean endStream, boolean endSegment, boolean compressed) {
|
||||
try {
|
||||
if (promise.isDone()) {
|
||||
// Most likely the write already failed. Just release the
|
||||
// buffer.
|
||||
data.release();
|
||||
return;
|
||||
}
|
||||
|
||||
remaining -= data.readableBytes();
|
||||
|
||||
// The flow controller may split the write into chunks. Use a new
|
||||
// promise for intermediate writes.
|
||||
final ChannelPromise chunkPromise =
|
||||
remaining == 0 ? promise : ctx.newPromise();
|
||||
|
||||
// The original promise is already in the list, so don't add again.
|
||||
if (chunkPromise != promise) {
|
||||
promises.add(chunkPromise);
|
||||
}
|
||||
|
||||
// TODO: consider adding a flush() method to this interface. The
|
||||
// frameWriter flushes on each write which isn't optimal
|
||||
// for the case of the outbound flow controller, which sends a batch
|
||||
// of frames when the flow control window changes. We should let
|
||||
// the flow controller manually flush after all writes are.
|
||||
// complete.
|
||||
|
||||
// Write the frame.
|
||||
ChannelFuture future =
|
||||
frameWriter.writeData(ctx, chunkPromise, streamId, data,
|
||||
padding, endStream, endSegment, compressed);
|
||||
|
||||
// Close the connection on write failures that leave the outbound
|
||||
// flow
|
||||
// control window in a corrupt state.
|
||||
future.addListener(new ChannelFutureListener() {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future)
|
||||
throws Exception {
|
||||
if (!future.isSuccess()) {
|
||||
// If any of the chunk writes fail, also fail the
|
||||
// original
|
||||
// future that was returned to the caller.
|
||||
failAllPromises(future.cause());
|
||||
processHttp2Exception(ctx,
|
||||
toHttp2Exception(future.cause()));
|
||||
}
|
||||
}
|
||||
});
|
||||
} catch (Http2Exception e) {
|
||||
processHttp2Exception(ctx, e);
|
||||
}
|
||||
|
||||
// Close the local side of the stream if this is the last frame
|
||||
if (endStream) {
|
||||
Http2Stream stream = connection.stream(streamId);
|
||||
closeLocalSide(stream, ctx, ctx.newPromise());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setFailure(Throwable cause) {
|
||||
failAllPromises(cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when the write for any chunk fails. Fails all promises including
|
||||
* the one returned to the caller.
|
||||
*/
|
||||
private void failAllPromises(Throwable cause) {
|
||||
for (ChannelPromise chunkPromise : promises) {
|
||||
if (!chunkPromise.isDone()) {
|
||||
chunkPromise.setFailure(cause);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,341 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2Exception.format;
|
||||
import static io.netty.handler.codec.http2.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.Http2Stream.State.HALF_CLOSED_LOCAL;
|
||||
import static io.netty.handler.codec.http2.Http2Stream.State.HALF_CLOSED_REMOTE;
|
||||
import static io.netty.handler.codec.http2.Http2Stream.State.OPEN;
|
||||
import static io.netty.handler.codec.http2.Http2Stream.State.RESERVED_LOCAL;
|
||||
import static io.netty.handler.codec.http2.Http2Stream.State.RESERVED_REMOTE;
|
||||
import io.netty.handler.codec.http2.Http2Stream.State;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Simple implementation of {@link Http2Connection}.
|
||||
*/
|
||||
public class DefaultHttp2Connection implements Http2Connection {
|
||||
|
||||
private final Map<Integer, Http2Stream> streamMap = new HashMap<Integer, Http2Stream>();
|
||||
private final Set<Http2Stream> activeStreams = new LinkedHashSet<Http2Stream>();
|
||||
private final DefaultEndpoint localEndpoint;
|
||||
private final DefaultEndpoint remoteEndpoint;
|
||||
private boolean goAwaySent;
|
||||
private boolean goAwayReceived;
|
||||
private boolean server;
|
||||
|
||||
public DefaultHttp2Connection(boolean server, boolean allowCompressedData) {
|
||||
this.server = server;
|
||||
localEndpoint = new DefaultEndpoint(server, allowCompressedData);
|
||||
remoteEndpoint = new DefaultEndpoint(!server, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isServer() {
|
||||
return server;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream requireStream(int streamId) throws Http2Exception {
|
||||
Http2Stream stream = stream(streamId);
|
||||
if (stream == null) {
|
||||
throw protocolError("Stream does not exist %d", streamId);
|
||||
}
|
||||
return stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream stream(int streamId) {
|
||||
return streamMap.get(streamId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numActiveStreams() {
|
||||
return activeStreams.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Http2Stream> activeStreams() {
|
||||
return Collections.unmodifiableSet(activeStreams);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoint local() {
|
||||
return localEndpoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoint remote() {
|
||||
return remoteEndpoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void goAwaySent() {
|
||||
goAwaySent = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void goAwayReceived() {
|
||||
goAwayReceived = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGoAwaySent() {
|
||||
return goAwaySent;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGoAwayReceived() {
|
||||
return goAwayReceived;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGoAway() {
|
||||
return isGoAwaySent() || isGoAwayReceived();
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple stream implementation. Streams can be compared to each other by priority.
|
||||
*/
|
||||
private final class DefaultStream implements Http2Stream {
|
||||
private final int id;
|
||||
private State state = State.IDLE;
|
||||
|
||||
DefaultStream(int id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int id() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public State state() {
|
||||
return state;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream verifyState(Http2Error error, State... allowedStates) throws Http2Exception {
|
||||
for (State allowedState : allowedStates) {
|
||||
if (state == allowedState) {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
throw format(error, "Stream %d in unexpected state: %s", id, state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream openForPush() throws Http2Exception {
|
||||
switch (state) {
|
||||
case RESERVED_LOCAL:
|
||||
state = State.HALF_CLOSED_REMOTE;
|
||||
break;
|
||||
case RESERVED_REMOTE:
|
||||
state = State.HALF_CLOSED_LOCAL;
|
||||
break;
|
||||
default:
|
||||
throw protocolError("Attempting to open non-reserved stream for push");
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream close() {
|
||||
if (state == State.CLOSED) {
|
||||
return this;
|
||||
}
|
||||
|
||||
state = State.CLOSED;
|
||||
activeStreams.remove(this);
|
||||
streamMap.remove(id);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream closeLocalSide() {
|
||||
switch (state) {
|
||||
case OPEN:
|
||||
state = State.HALF_CLOSED_LOCAL;
|
||||
break;
|
||||
case HALF_CLOSED_LOCAL:
|
||||
break;
|
||||
default:
|
||||
close();
|
||||
break;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream closeRemoteSide() {
|
||||
switch (state) {
|
||||
case OPEN:
|
||||
state = State.HALF_CLOSED_REMOTE;
|
||||
break;
|
||||
case HALF_CLOSED_REMOTE:
|
||||
break;
|
||||
default:
|
||||
close();
|
||||
break;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean remoteSideOpen() {
|
||||
return state == HALF_CLOSED_LOCAL || state == OPEN || state == RESERVED_REMOTE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean localSideOpen() {
|
||||
return state == HALF_CLOSED_REMOTE || state == OPEN || state == RESERVED_LOCAL;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple endpoint implementation.
|
||||
*/
|
||||
private final class DefaultEndpoint implements Endpoint {
|
||||
private int nextStreamId;
|
||||
private int lastStreamCreated;
|
||||
private int maxStreams = Integer.MAX_VALUE;
|
||||
private boolean pushToAllowed = true;
|
||||
private boolean allowCompressedData;
|
||||
|
||||
DefaultEndpoint(boolean serverEndpoint, boolean allowCompressedData) {
|
||||
// Determine the starting stream ID for this endpoint. Zero is reserved for the
|
||||
// connection and 1 is reserved for responding to an upgrade from HTTP 1.1.
|
||||
// Client-initiated streams use odd identifiers and server-initiated streams use
|
||||
// even.
|
||||
nextStreamId = serverEndpoint ? 2 : 3;
|
||||
this.allowCompressedData = allowCompressedData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultStream createStream(int streamId, boolean halfClosed) throws Http2Exception {
|
||||
checkNewStreamAllowed(streamId);
|
||||
|
||||
// Create and initialize the stream.
|
||||
DefaultStream stream = new DefaultStream(streamId);
|
||||
if (halfClosed) {
|
||||
stream.state = isLocal() ? State.HALF_CLOSED_LOCAL : State.HALF_CLOSED_REMOTE;
|
||||
} else {
|
||||
stream.state = State.OPEN;
|
||||
}
|
||||
|
||||
// Update the next and last stream IDs.
|
||||
nextStreamId += 2;
|
||||
lastStreamCreated = streamId;
|
||||
|
||||
// Register the stream and mark it as active.
|
||||
streamMap.put(streamId, stream);
|
||||
activeStreams.add(stream);
|
||||
return stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultStream reservePushStream(int streamId, Http2Stream parent)
|
||||
throws Http2Exception {
|
||||
if (parent == null) {
|
||||
throw protocolError("Parent stream missing");
|
||||
}
|
||||
if (isLocal() ? !parent.localSideOpen() : !parent.remoteSideOpen()) {
|
||||
throw protocolError("Stream %d is not open for sending push promise", parent.id());
|
||||
}
|
||||
if (!opposite().allowPushTo()) {
|
||||
throw protocolError("Server push not allowed to opposite endpoint.");
|
||||
}
|
||||
|
||||
// Create and initialize the stream.
|
||||
DefaultStream stream = new DefaultStream(streamId);
|
||||
stream.state = isLocal() ? State.RESERVED_LOCAL : State.RESERVED_REMOTE;
|
||||
|
||||
// Update the next and last stream IDs.
|
||||
nextStreamId += 2;
|
||||
lastStreamCreated = streamId;
|
||||
|
||||
// Register the stream.
|
||||
streamMap.put(streamId, stream);
|
||||
return stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void allowPushTo(boolean allow) {
|
||||
pushToAllowed = allow;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean allowPushTo() {
|
||||
return pushToAllowed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxStreams() {
|
||||
return maxStreams;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maxStreams(int maxStreams) {
|
||||
this.maxStreams = maxStreams;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean allowCompressedData() {
|
||||
return allowCompressedData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void allowCompressedData(boolean allow) {
|
||||
allowCompressedData = allow;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lastStreamCreated() {
|
||||
return lastStreamCreated;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoint opposite() {
|
||||
return isLocal() ? remoteEndpoint : localEndpoint;
|
||||
}
|
||||
|
||||
private void checkNewStreamAllowed(int streamId) throws Http2Exception {
|
||||
if (isGoAway()) {
|
||||
throw protocolError("Cannot create a stream since the connection is going away");
|
||||
}
|
||||
if (nextStreamId < 0) {
|
||||
throw protocolError("No more streams can be created on this connection");
|
||||
}
|
||||
if (streamId != nextStreamId) {
|
||||
throw protocolError("Incorrect next stream ID requested: %d", streamId);
|
||||
}
|
||||
if (streamMap.size() + 1 > maxStreams) {
|
||||
throw protocolError("Maximum streams exceeded for this endpoint.");
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isLocal() {
|
||||
return this == localEndpoint;
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,745 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.FRAME_LENGTH_MASK;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.INT_FIELD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.PRIORITY_ENTRY_LENGTH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_COMPRESS_DATA;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_ENABLE_PUSH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_HEADER_TABLE_SIZE;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_INITIAL_WINDOW_SIZE;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_MAX_CONCURRENT_STREAMS;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTING_ENTRY_LENGTH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.readUnsignedInt;
|
||||
import static io.netty.handler.codec.http2.Http2Exception.protocolError;
|
||||
import static io.netty.util.CharsetUtil.UTF_8;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
|
||||
/**
|
||||
* A {@link Http2FrameReader} that supports all frame types defined by the HTTP/2 specification.
|
||||
*/
|
||||
public class DefaultHttp2FrameReader implements Http2FrameReader {
|
||||
|
||||
private enum State {
|
||||
FRAME_HEADER,
|
||||
FRAME_PAYLOAD,
|
||||
ERROR
|
||||
}
|
||||
|
||||
private final boolean server;
|
||||
private final Http2HeadersDecoder headersDecoder;
|
||||
|
||||
private State state = State.FRAME_HEADER;
|
||||
private Http2FrameType frameType;
|
||||
private int streamId;
|
||||
private Http2Flags flags;
|
||||
private int payloadLength;
|
||||
private HeadersContinuation headersContinuation;
|
||||
|
||||
public DefaultHttp2FrameReader(boolean server) {
|
||||
this(server, new DefaultHttp2HeadersDecoder());
|
||||
}
|
||||
|
||||
public DefaultHttp2FrameReader(boolean server, Http2HeadersDecoder headersDecoder) {
|
||||
this.server = server;
|
||||
this.headersDecoder = headersDecoder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maxHeaderTableSize(int max) {
|
||||
headersDecoder.maxHeaderTableSize(max);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxHeaderTableSize() {
|
||||
return headersDecoder.maxHeaderTableSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (headersContinuation != null) {
|
||||
headersContinuation.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrame(ByteBufAllocator alloc, ByteBuf input, Http2FrameObserver observer)
|
||||
throws Http2Exception {
|
||||
try {
|
||||
while (input.isReadable()) {
|
||||
switch (state) {
|
||||
case FRAME_HEADER:
|
||||
processHeaderState(input);
|
||||
if (state == State.FRAME_HEADER) {
|
||||
// Wait until the entire header has arrived.
|
||||
return;
|
||||
}
|
||||
|
||||
// The header is complete, fall into the next case to process the payload.
|
||||
// This is to ensure the proper handling of zero-length payloads. In this
|
||||
// case, we don't want to loop around because there may be no more data
|
||||
// available, causing us to exit the loop. Instead, we just want to perform
|
||||
// the first pass at payload processing now.
|
||||
case FRAME_PAYLOAD:
|
||||
processPayloadState(alloc, input, observer);
|
||||
if (state == State.FRAME_PAYLOAD) {
|
||||
// Wait until the entire payload has arrived.
|
||||
return;
|
||||
}
|
||||
break;
|
||||
case ERROR:
|
||||
input.skipBytes(input.readableBytes());
|
||||
return;
|
||||
default:
|
||||
throw new IllegalStateException("Should never get here");
|
||||
}
|
||||
}
|
||||
} catch (Http2Exception e) {
|
||||
state = State.ERROR;
|
||||
throw e;
|
||||
} catch (RuntimeException e) {
|
||||
state = State.ERROR;
|
||||
throw e;
|
||||
} catch (Error e) {
|
||||
state = State.ERROR;
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private void processHeaderState(ByteBuf in) throws Http2Exception {
|
||||
if (in.readableBytes() < FRAME_HEADER_LENGTH) {
|
||||
// Wait until the entire frame header has been read.
|
||||
return;
|
||||
}
|
||||
|
||||
// Read the header and prepare the unmarshaller to read the frame.
|
||||
payloadLength = in.readUnsignedShort() & FRAME_LENGTH_MASK;
|
||||
frameType = Http2FrameType.forTypeCode(in.readUnsignedByte());
|
||||
flags = new Http2Flags(in.readUnsignedByte());
|
||||
streamId = readUnsignedInt(in);
|
||||
|
||||
switch (frameType) {
|
||||
case DATA:
|
||||
verifyDataFrame();
|
||||
break;
|
||||
case HEADERS:
|
||||
verifyHeadersFrame();
|
||||
break;
|
||||
case PRIORITY:
|
||||
verifyPriorityFrame();
|
||||
break;
|
||||
case RST_STREAM:
|
||||
verifyRstStreamFrame();
|
||||
break;
|
||||
case SETTINGS:
|
||||
verifySettingsFrame();
|
||||
break;
|
||||
case PUSH_PROMISE:
|
||||
verifyPushPromiseFrame();
|
||||
break;
|
||||
case PING:
|
||||
verifyPingFrame();
|
||||
break;
|
||||
case GO_AWAY:
|
||||
verifyGoAwayFrame();
|
||||
break;
|
||||
case WINDOW_UPDATE:
|
||||
verifyWindowUpdateFrame();
|
||||
break;
|
||||
case CONTINUATION:
|
||||
verifyContinuationFrame();
|
||||
break;
|
||||
case ALT_SVC:
|
||||
verifyAltSvcFrame();
|
||||
break;
|
||||
case BLOCKED:
|
||||
verifyBlockedFrame();
|
||||
break;
|
||||
default:
|
||||
throw protocolError("Unsupported frame type: %s", frameType);
|
||||
}
|
||||
|
||||
// Start reading the payload for the frame.
|
||||
state = State.FRAME_PAYLOAD;
|
||||
}
|
||||
|
||||
private void
|
||||
processPayloadState(ByteBufAllocator alloc, ByteBuf in, Http2FrameObserver observer)
|
||||
throws Http2Exception {
|
||||
if (in.readableBytes() < payloadLength) {
|
||||
// Wait until the entire payload has been read.
|
||||
return;
|
||||
}
|
||||
|
||||
// Get a view of the buffer for the size of the payload.
|
||||
ByteBuf payload = in.readSlice(payloadLength);
|
||||
|
||||
// Read the payload and fire the frame event to the observer.
|
||||
switch (frameType) {
|
||||
case DATA:
|
||||
readDataFrame(payload, observer);
|
||||
break;
|
||||
case HEADERS:
|
||||
readHeadersFrame(alloc, payload, observer);
|
||||
break;
|
||||
case PRIORITY:
|
||||
readPriorityFrame(payload, observer);
|
||||
break;
|
||||
case RST_STREAM:
|
||||
readRstStreamFrame(payload, observer);
|
||||
break;
|
||||
case SETTINGS:
|
||||
readSettingsFrame(payload, observer);
|
||||
break;
|
||||
case PUSH_PROMISE:
|
||||
readPushPromiseFrame(alloc, payload, observer);
|
||||
break;
|
||||
case PING:
|
||||
readPingFrame(payload, observer);
|
||||
break;
|
||||
case GO_AWAY:
|
||||
readGoAwayFrame(payload, observer);
|
||||
break;
|
||||
case WINDOW_UPDATE:
|
||||
readWindowUpdateFrame(payload, observer);
|
||||
break;
|
||||
case CONTINUATION:
|
||||
readContinuationFrame(payload, observer);
|
||||
break;
|
||||
case ALT_SVC:
|
||||
readAltSvcFrame(payload, observer);
|
||||
break;
|
||||
case BLOCKED:
|
||||
observer.onBlockedRead(streamId);
|
||||
break;
|
||||
default:
|
||||
// Should never happen.
|
||||
throw protocolError("Unsupported frame type: %s", frameType);
|
||||
}
|
||||
|
||||
// Go back to reading the next frame header.
|
||||
state = State.FRAME_HEADER;
|
||||
}
|
||||
|
||||
private void verifyDataFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
verifyPayloadLength(payloadLength);
|
||||
|
||||
if (!flags.isPaddingLengthValid()) {
|
||||
throw protocolError("Pad high is set but pad low is not");
|
||||
}
|
||||
if (payloadLength < flags.getNumPaddingLengthBytes()) {
|
||||
throw protocolError("Frame length %d too small.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyHeadersFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
verifyPayloadLength(payloadLength);
|
||||
|
||||
int lengthWithoutPriority = flags.getNumPaddingLengthBytes();
|
||||
if (lengthWithoutPriority < 0) {
|
||||
throw protocolError("Frame length too small." + payloadLength);
|
||||
}
|
||||
|
||||
if (!flags.isPaddingLengthValid()) {
|
||||
throw protocolError("Pad high is set but pad low is not");
|
||||
}
|
||||
|
||||
if (lengthWithoutPriority < flags.getNumPaddingLengthBytes()) {
|
||||
throw protocolError("Frame length %d too small for padding.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyPriorityFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
|
||||
if (payloadLength != PRIORITY_ENTRY_LENGTH) {
|
||||
throw protocolError("Invalid frame length %d.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyRstStreamFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
|
||||
if (payloadLength != INT_FIELD_LENGTH) {
|
||||
throw protocolError("Invalid frame length %d.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifySettingsFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
verifyPayloadLength(payloadLength);
|
||||
if (streamId != 0) {
|
||||
throw protocolError("A stream ID must be zero.");
|
||||
}
|
||||
if (flags.ack() && payloadLength > 0) {
|
||||
throw protocolError("Ack settings frame must have an empty payload.");
|
||||
}
|
||||
if (payloadLength % SETTING_ENTRY_LENGTH > 0) {
|
||||
throw protocolError("Frame length %d invalid.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyPushPromiseFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
verifyPayloadLength(payloadLength);
|
||||
|
||||
if (!flags.isPaddingLengthValid()) {
|
||||
throw protocolError("Pad high is set but pad low is not");
|
||||
}
|
||||
|
||||
// Subtract the length of the promised stream ID field, to determine the length of the
|
||||
// rest of the payload (header block fragment + payload).
|
||||
int lengthWithoutPromisedId = payloadLength - INT_FIELD_LENGTH;
|
||||
if (lengthWithoutPromisedId < flags.getNumPaddingLengthBytes()) {
|
||||
throw protocolError("Frame length %d too small for padding.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyPingFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
if (streamId != 0) {
|
||||
throw protocolError("A stream ID must be zero.");
|
||||
}
|
||||
if (payloadLength != 8) {
|
||||
throw protocolError("Frame length %d incorrect size for ping.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyGoAwayFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
verifyPayloadLength(payloadLength);
|
||||
|
||||
if (streamId != 0) {
|
||||
throw protocolError("A stream ID must be zero.");
|
||||
}
|
||||
if (payloadLength < 8) {
|
||||
throw protocolError("Frame length %d too small.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyWindowUpdateFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
verifyStreamOrConnectionId(streamId, "Stream ID");
|
||||
|
||||
if (payloadLength != INT_FIELD_LENGTH) {
|
||||
throw protocolError("Invalid frame length %d.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyContinuationFrame() throws Http2Exception {
|
||||
verifyPayloadLength(payloadLength);
|
||||
|
||||
if (headersContinuation == null) {
|
||||
throw protocolError("Received %s frame but not currently processing headers.",
|
||||
frameType);
|
||||
}
|
||||
|
||||
if (streamId != headersContinuation.getStreamId()) {
|
||||
throw protocolError("Continuation stream ID does not match pending headers. "
|
||||
+ "Expected %d, but received %d.", headersContinuation.getStreamId(), streamId);
|
||||
}
|
||||
|
||||
if (!flags.isPaddingLengthValid()) {
|
||||
throw protocolError("Pad high is set but pad low is not");
|
||||
}
|
||||
|
||||
if (payloadLength < flags.getNumPaddingLengthBytes()) {
|
||||
throw protocolError("Frame length %d too small for padding.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyAltSvcFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
verifyStreamOrConnectionId(streamId, "Stream ID");
|
||||
verifyPayloadLength(payloadLength);
|
||||
|
||||
if (server) {
|
||||
throw protocolError("ALT_SVC frames must not be received by servers");
|
||||
}
|
||||
|
||||
if (payloadLength < 8) {
|
||||
throw protocolError("Frame length too small." + payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyBlockedFrame() throws Http2Exception {
|
||||
verifyNotProcessingHeaders();
|
||||
verifyStreamOrConnectionId(streamId, "Stream ID");
|
||||
|
||||
if (payloadLength != 0) {
|
||||
throw protocolError("Invalid frame length %d.", payloadLength);
|
||||
}
|
||||
}
|
||||
|
||||
private void readDataFrame(ByteBuf payload, Http2FrameObserver observer) throws Http2Exception {
|
||||
int dataPadding = flags.readPaddingLength(payload);
|
||||
|
||||
// Determine how much data there is to read by removing the trailing
|
||||
// padding.
|
||||
int dataLength = payload.readableBytes() - dataPadding;
|
||||
if (dataLength < 0) {
|
||||
throw protocolError("Frame payload too small for padding.");
|
||||
}
|
||||
|
||||
ByteBuf data = payload.readSlice(dataLength);
|
||||
observer.onDataRead(streamId, data, dataPadding, flags.endOfStream(), flags.endOfSegment(),
|
||||
flags.compressed());
|
||||
payload.skipBytes(payload.readableBytes());
|
||||
}
|
||||
|
||||
private void readHeadersFrame(final ByteBufAllocator alloc, ByteBuf payload,
|
||||
Http2FrameObserver observer) throws Http2Exception {
|
||||
final int headersStreamId = streamId;
|
||||
final Http2Flags headersFlags = flags;
|
||||
int padding = flags.readPaddingLength(payload);
|
||||
|
||||
// The callback that is invoked is different depending on whether priority information
|
||||
// is present in the headers frame.
|
||||
if (flags.priorityPresent()) {
|
||||
long word1 = payload.readUnsignedInt();
|
||||
final boolean exclusive = (word1 & 0x80000000L) > 0;
|
||||
final int streamDependency = (int) (word1 & 0x7FFFFFFFL);
|
||||
final short headersWeight = payload.readUnsignedByte();
|
||||
final ByteBuf fragment = payload.readSlice(payload.readableBytes() - padding);
|
||||
|
||||
// Create a handler that invokes the observer when the header block is complete.
|
||||
headersContinuation = new HeadersContinuation() {
|
||||
@Override
|
||||
public int getStreamId() {
|
||||
return headersStreamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processFragment(boolean endOfHeaders, ByteBuf fragment, int padding,
|
||||
Http2FrameObserver observer) throws Http2Exception {
|
||||
builder().addFragment(fragment, alloc, endOfHeaders);
|
||||
if (endOfHeaders) {
|
||||
Http2Headers headers = builder().buildHeaders();
|
||||
observer.onHeadersRead(headersStreamId, headers, streamDependency,
|
||||
headersWeight, exclusive, padding, headersFlags.endOfStream(),
|
||||
headersFlags.endOfSegment());
|
||||
close();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// Process the initial fragment, invoking the observer's callback if end of headers.
|
||||
headersContinuation.processFragment(flags.endOfHeaders(), fragment, padding, observer);
|
||||
return;
|
||||
}
|
||||
|
||||
// The priority fields are not present in the frame. Prepare a continuation that invokes
|
||||
// the observer callback without priority information.
|
||||
headersContinuation = new HeadersContinuation() {
|
||||
@Override
|
||||
public int getStreamId() {
|
||||
return headersStreamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processFragment(boolean endOfHeaders, ByteBuf fragment, int padding,
|
||||
Http2FrameObserver observer) throws Http2Exception {
|
||||
builder().addFragment(fragment, alloc, endOfHeaders);
|
||||
if (endOfHeaders) {
|
||||
Http2Headers headers = builder().buildHeaders();
|
||||
observer.onHeadersRead(headersStreamId, headers, padding,
|
||||
headersFlags.endOfStream(), headersFlags.endOfSegment());
|
||||
close();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// Process the initial fragment, invoking the observer's callback if end of headers.
|
||||
final ByteBuf fragment = payload.readSlice(payload.readableBytes() - padding);
|
||||
headersContinuation.processFragment(flags.endOfHeaders(), fragment, padding, observer);
|
||||
}
|
||||
|
||||
private void readPriorityFrame(ByteBuf payload, Http2FrameObserver observer)
|
||||
throws Http2Exception {
|
||||
long word1 = payload.readUnsignedInt();
|
||||
boolean exclusive = (word1 & 0x80000000L) > 0;
|
||||
int streamDependency = (int) (word1 & 0x7FFFFFFFL);
|
||||
short weight = payload.readUnsignedByte();
|
||||
observer.onPriorityRead(streamId, streamDependency, weight, exclusive);
|
||||
}
|
||||
|
||||
private void readRstStreamFrame(ByteBuf payload, Http2FrameObserver observer)
|
||||
throws Http2Exception {
|
||||
long errorCode = payload.readUnsignedInt();
|
||||
observer.onRstStreamRead(streamId, errorCode);
|
||||
}
|
||||
|
||||
private void readSettingsFrame(ByteBuf payload, Http2FrameObserver observer)
|
||||
throws Http2Exception {
|
||||
if (flags.ack()) {
|
||||
observer.onSettingsAckRead();
|
||||
} else {
|
||||
int numSettings = payloadLength / SETTING_ENTRY_LENGTH;
|
||||
Http2Settings settings = new Http2Settings();
|
||||
for (int index = 0; index < numSettings; ++index) {
|
||||
short id = payload.readUnsignedByte();
|
||||
long value = payload.readUnsignedInt();
|
||||
switch (id) {
|
||||
case SETTINGS_HEADER_TABLE_SIZE:
|
||||
if (value < 0 || value > Integer.MAX_VALUE) {
|
||||
throw protocolError("Invalid value for HEADER_TABLE_SIZE: %d", value);
|
||||
}
|
||||
settings.maxHeaderTableSize((int) value);
|
||||
break;
|
||||
case SETTINGS_COMPRESS_DATA:
|
||||
if (value != 0 && value != 1) {
|
||||
throw protocolError("Invalid value for COMPRESS_DATA: %d", value);
|
||||
}
|
||||
settings.allowCompressedData(value == 1);
|
||||
break;
|
||||
case SETTINGS_ENABLE_PUSH:
|
||||
if (value != 0 && value != 1) {
|
||||
throw protocolError("Invalid value for ENABLE_PUSH: %d", value);
|
||||
}
|
||||
settings.pushEnabled(value == 1);
|
||||
break;
|
||||
case SETTINGS_INITIAL_WINDOW_SIZE:
|
||||
if (value < 0 || value > Integer.MAX_VALUE) {
|
||||
throw protocolError("Invalid value for INITIAL_WINDOW_SIZE: %d", value);
|
||||
}
|
||||
settings.initialWindowSize((int) value);
|
||||
break;
|
||||
case SETTINGS_MAX_CONCURRENT_STREAMS:
|
||||
if (value < 0 || value > Integer.MAX_VALUE) {
|
||||
throw protocolError("Invalid value for MAX_CONCURRENT_STREAMS: %d",
|
||||
value);
|
||||
}
|
||||
settings.maxConcurrentStreams((int) value);
|
||||
break;
|
||||
default:
|
||||
throw protocolError("Unsupport setting: %d", id);
|
||||
}
|
||||
}
|
||||
observer.onSettingsRead(settings);
|
||||
}
|
||||
}
|
||||
|
||||
private void readPushPromiseFrame(final ByteBufAllocator alloc, ByteBuf payload,
|
||||
Http2FrameObserver observer) throws Http2Exception {
|
||||
final int pushPromiseStreamId = streamId;
|
||||
int padding = flags.readPaddingLength(payload);
|
||||
final int promisedStreamId = readUnsignedInt(payload);
|
||||
|
||||
// Create a handler that invokes the observer when the header block is complete.
|
||||
headersContinuation = new HeadersContinuation() {
|
||||
@Override
|
||||
public int getStreamId() {
|
||||
return pushPromiseStreamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processFragment(boolean endOfHeaders, ByteBuf fragment, int padding,
|
||||
Http2FrameObserver observer) throws Http2Exception {
|
||||
builder().addFragment(fragment, alloc, endOfHeaders);
|
||||
if (endOfHeaders) {
|
||||
Http2Headers headers = builder().buildHeaders();
|
||||
observer.onPushPromiseRead(pushPromiseStreamId, promisedStreamId, headers,
|
||||
padding);
|
||||
close();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// Process the initial fragment, invoking the observer's callback if end of headers.
|
||||
final ByteBuf fragment = payload.readSlice(payload.readableBytes() - padding);
|
||||
headersContinuation.processFragment(flags.endOfHeaders(), fragment, padding, observer);
|
||||
}
|
||||
|
||||
private void readPingFrame(ByteBuf payload, Http2FrameObserver observer) throws Http2Exception {
|
||||
ByteBuf data = payload.readSlice(payload.readableBytes());
|
||||
if (flags.ack()) {
|
||||
observer.onPingAckRead(data);
|
||||
} else {
|
||||
observer.onPingRead(data);
|
||||
}
|
||||
}
|
||||
|
||||
private void readGoAwayFrame(ByteBuf payload, Http2FrameObserver observer)
|
||||
throws Http2Exception {
|
||||
int lastStreamId = readUnsignedInt(payload);
|
||||
long errorCode = payload.readUnsignedInt();
|
||||
ByteBuf debugData = payload.readSlice(payload.readableBytes());
|
||||
observer.onGoAwayRead(lastStreamId, errorCode, debugData);
|
||||
}
|
||||
|
||||
private void readWindowUpdateFrame(ByteBuf payload, Http2FrameObserver observer)
|
||||
throws Http2Exception {
|
||||
int windowSizeIncrement = readUnsignedInt(payload);
|
||||
observer.onWindowUpdateRead(streamId, windowSizeIncrement);
|
||||
}
|
||||
|
||||
private void readContinuationFrame(ByteBuf payload, Http2FrameObserver observer)
|
||||
throws Http2Exception {
|
||||
int padding = flags.readPaddingLength(payload);
|
||||
|
||||
// Process the initial fragment, invoking the observer's callback if end of headers.
|
||||
final ByteBuf continuationFragment = payload.readSlice(payload.readableBytes() - padding);
|
||||
headersContinuation.processFragment(flags.endOfHeaders(), continuationFragment, padding,
|
||||
observer);
|
||||
}
|
||||
|
||||
private void readAltSvcFrame(ByteBuf payload, Http2FrameObserver observer)
|
||||
throws Http2Exception {
|
||||
long maxAge = payload.readUnsignedInt();
|
||||
int port = payload.readUnsignedShort();
|
||||
payload.skipBytes(1);
|
||||
short protocolIdLength = payload.readUnsignedByte();
|
||||
ByteBuf protocolId = payload.readSlice(protocolIdLength);
|
||||
short hostLength = payload.readUnsignedByte();
|
||||
String host = payload.toString(payload.readerIndex(), hostLength, UTF_8);
|
||||
payload.skipBytes(hostLength);
|
||||
String origin = null;
|
||||
if (payload.isReadable()) {
|
||||
origin = payload.toString(UTF_8);
|
||||
payload.skipBytes(payload.readableBytes());
|
||||
}
|
||||
observer.onAltSvcRead(streamId, maxAge, port, protocolId, host, origin);
|
||||
}
|
||||
|
||||
/**
|
||||
* Base class for processing of HEADERS and PUSH_PROMISE header blocks that potentially span
|
||||
* multiple frames. The implementation of this interface will perform the final callback to the
|
||||
* {@linkHttp2FrameObserver} once the end of headers is reached.
|
||||
*/
|
||||
private abstract class HeadersContinuation {
|
||||
private final HeadersBuilder builder = new HeadersBuilder();
|
||||
|
||||
/**
|
||||
* Returns the stream for which headers are currently being processed.
|
||||
*/
|
||||
abstract int getStreamId();
|
||||
|
||||
/**
|
||||
* Processes the next fragment for the current header block.
|
||||
*
|
||||
* @param endOfHeaders whether the fragment is the last in the header block.
|
||||
* @param fragment the fragment of the header block to be added.
|
||||
* @param padding the amount of padding to be supplied to the {@linkHttp2FrameObserver}
|
||||
* callback.
|
||||
* @param observer the observer to be notified if the header block is completed.
|
||||
*/
|
||||
abstract void processFragment(boolean endOfHeaders, ByteBuf fragment, int padding,
|
||||
Http2FrameObserver observer) throws Http2Exception;
|
||||
|
||||
final HeadersBuilder builder() {
|
||||
return builder;
|
||||
}
|
||||
|
||||
/**
|
||||
* Free any allocated resources.
|
||||
*/
|
||||
final void close() {
|
||||
builder.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility class to help with construction of the headers block that may potentially span
|
||||
* multiple frames.
|
||||
*/
|
||||
private class HeadersBuilder {
|
||||
private ByteBuf headerBlock;
|
||||
|
||||
/**
|
||||
* Adds a fragment to the block.
|
||||
*
|
||||
* @param fragment the fragment of the headers block to be added.
|
||||
* @param alloc allocator for new blocks if needed.
|
||||
* @param endOfHeaders flag indicating whether the current frame is the end of the headers.
|
||||
* This is used for an optimization for when the first fragment is the full
|
||||
* block. In that case, the buffer is used directly without copying.
|
||||
*/
|
||||
final void addFragment(ByteBuf fragment, ByteBufAllocator alloc, boolean endOfHeaders) {
|
||||
if (headerBlock == null) {
|
||||
if (endOfHeaders) {
|
||||
// Optimization - don't bother copying, just use the buffer as-is. Need
|
||||
// to retain since we release when the header block is built.
|
||||
headerBlock = fragment.retain();
|
||||
} else {
|
||||
headerBlock = alloc.buffer(fragment.readableBytes());
|
||||
headerBlock.writeBytes(fragment);
|
||||
}
|
||||
return;
|
||||
}
|
||||
if (headerBlock.isWritable(fragment.readableBytes())) {
|
||||
// The buffer can hold the requeste bytes, just write it directly.
|
||||
headerBlock.writeBytes(fragment);
|
||||
} else {
|
||||
// Allocate a new buffer that is big enough to hold the entire header block so far.
|
||||
ByteBuf buf = alloc.buffer(headerBlock.readableBytes() + fragment.readableBytes());
|
||||
buf.writeBytes(headerBlock);
|
||||
buf.writeBytes(fragment);
|
||||
headerBlock.release();
|
||||
headerBlock = buf;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds the headers from the completed headers block. After this is called, this builder
|
||||
* should not be called again.
|
||||
*/
|
||||
Http2Headers buildHeaders() throws Http2Exception {
|
||||
try {
|
||||
return headersDecoder.decodeHeaders(headerBlock);
|
||||
} finally {
|
||||
close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes this builder and frees any resources.
|
||||
*/
|
||||
void close() {
|
||||
if (headerBlock != null) {
|
||||
headerBlock.release();
|
||||
headerBlock = null;
|
||||
}
|
||||
|
||||
// Clear the member variable pointing at this instance.
|
||||
headersContinuation = null;
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyNotProcessingHeaders() throws Http2Exception {
|
||||
if (headersContinuation != null) {
|
||||
throw protocolError("Received frame of type %s while processing headers.", frameType);
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyStreamOrConnectionId(int streamId, String argumentName)
|
||||
throws Http2Exception {
|
||||
if (streamId < 0) {
|
||||
throw protocolError("%s must be >= 0", argumentName);
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyPayloadLength(int payloadLength) throws Http2Exception {
|
||||
if (payloadLength > MAX_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Total payload length %d exceeds max frame length.", payloadLength);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,565 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.INT_FIELD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_UNSIGNED_BYTE;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_UNSIGNED_INT;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_UNSIGNED_SHORT;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.PRIORITY_ENTRY_LENGTH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_COMPRESS_DATA;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_ENABLE_PUSH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_HEADER_TABLE_SIZE;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_INITIAL_WINDOW_SIZE;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_MAX_CONCURRENT_STREAMS;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.SETTING_ENTRY_LENGTH;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.failAndThrow;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.writeUnsignedInt;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.writeUnsignedShort;
|
||||
import static io.netty.util.CharsetUtil.UTF_8;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.CompositeByteBuf;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
|
||||
/**
|
||||
* A {@link Http2FrameWriter} that supports all frame types defined by the HTTP/2 specification.
|
||||
*/
|
||||
public class DefaultHttp2FrameWriter implements Http2FrameWriter {
|
||||
|
||||
private final boolean server;
|
||||
private final Http2HeadersEncoder headersEncoder;
|
||||
|
||||
public DefaultHttp2FrameWriter(boolean server) {
|
||||
this(server, new DefaultHttp2HeadersEncoder());
|
||||
}
|
||||
|
||||
public DefaultHttp2FrameWriter(boolean server, Http2HeadersEncoder headersEncoder) {
|
||||
this.server = server;
|
||||
this.headersEncoder = headersEncoder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maxHeaderTableSize(int max) throws Http2Exception {
|
||||
headersEncoder.maxHeaderTableSize(max);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxHeaderTableSize() {
|
||||
return headersEncoder.maxHeaderTableSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
// Nothing to do.
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeData(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
|
||||
ByteBuf data, int padding, boolean endStream, boolean endSegment, boolean compressed) {
|
||||
try {
|
||||
verifyStreamId(streamId, "Stream ID");
|
||||
verifyPadding(padding);
|
||||
|
||||
Http2Flags flags =
|
||||
Http2Flags.newBuilder().setPaddingFlags(padding).endOfStream(endStream)
|
||||
.endOfSegment(endSegment).compressed(compressed).build();
|
||||
|
||||
int payloadLength = data.readableBytes() + padding + flags.getNumPaddingLengthBytes();
|
||||
verifyPayloadLength(payloadLength);
|
||||
|
||||
ByteBuf out = ctx.alloc().buffer(FRAME_HEADER_LENGTH + payloadLength);
|
||||
|
||||
writeFrameHeader(out, payloadLength, Http2FrameType.DATA, flags, streamId);
|
||||
|
||||
writePaddingLength(padding, out);
|
||||
|
||||
// Write the data.
|
||||
out.writeBytes(data, data.readerIndex(), data.readableBytes());
|
||||
|
||||
// Write the required padding.
|
||||
out.writeZero(padding);
|
||||
return ctx.writeAndFlush(out, promise);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception {
|
||||
return writeHeadersInternal(ctx, promise, streamId, headers, padding, endStream,
|
||||
endSegment, false, 0, (short) 0, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, Http2Headers headers, int streamDependency, short weight,
|
||||
boolean exclusive, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception {
|
||||
return writeHeadersInternal(ctx, promise, streamId, headers, padding, endStream,
|
||||
endSegment, true, streamDependency, weight, exclusive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writePriority(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int streamDependency, short weight, boolean exclusive) {
|
||||
try {
|
||||
verifyStreamId(streamId, "Stream ID");
|
||||
verifyStreamId(streamDependency, "Stream Dependency");
|
||||
verifyWeight(weight);
|
||||
|
||||
ByteBuf frame = ctx.alloc().buffer(FRAME_HEADER_LENGTH + PRIORITY_ENTRY_LENGTH);
|
||||
writeFrameHeader(frame, PRIORITY_ENTRY_LENGTH, Http2FrameType.PRIORITY,
|
||||
Http2Flags.EMPTY, streamId);
|
||||
long word1 = exclusive ? (0x80000000L | streamDependency) : streamDependency;
|
||||
writeUnsignedInt(word1, frame);
|
||||
frame.writeByte(weight);
|
||||
return ctx.writeAndFlush(frame, promise);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeRstStream(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, long errorCode) {
|
||||
try {
|
||||
verifyStreamId(streamId, "Stream ID");
|
||||
verifyErrorCode(errorCode);
|
||||
|
||||
ByteBuf frame = ctx.alloc().buffer(FRAME_HEADER_LENGTH + INT_FIELD_LENGTH);
|
||||
writeFrameHeader(frame, INT_FIELD_LENGTH, Http2FrameType.RST_STREAM, Http2Flags.EMPTY,
|
||||
streamId);
|
||||
writeUnsignedInt(errorCode, frame);
|
||||
return ctx.writeAndFlush(frame, promise);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeSettings(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
Http2Settings settings) {
|
||||
try {
|
||||
int numFields = 0;
|
||||
numFields += settings.hasAllowCompressedData() ? 1 : 0;
|
||||
numFields += settings.hasMaxHeaderTableSize() ? 1 : 0;
|
||||
numFields += settings.hasInitialWindowSize() ? 1 : 0;
|
||||
numFields += settings.hasMaxConcurrentStreams() ? 1 : 0;
|
||||
numFields += settings.hasPushEnabled() ? 1 : 0;
|
||||
|
||||
int payloadLength = SETTING_ENTRY_LENGTH * numFields;
|
||||
ByteBuf frame = ctx.alloc().buffer(FRAME_HEADER_LENGTH + payloadLength);
|
||||
writeFrameHeader(frame, payloadLength, Http2FrameType.SETTINGS, Http2Flags.EMPTY, 0);
|
||||
|
||||
if (settings.hasAllowCompressedData()) {
|
||||
frame.writeByte(SETTINGS_COMPRESS_DATA);
|
||||
writeUnsignedInt(settings.allowCompressedData() ? 1L : 0L, frame);
|
||||
}
|
||||
if (settings.hasMaxHeaderTableSize()) {
|
||||
frame.writeByte(SETTINGS_HEADER_TABLE_SIZE);
|
||||
writeUnsignedInt(settings.maxHeaderTableSize(), frame);
|
||||
}
|
||||
if (settings.hasInitialWindowSize()) {
|
||||
frame.writeByte(SETTINGS_INITIAL_WINDOW_SIZE);
|
||||
writeUnsignedInt(settings.initialWindowSize(), frame);
|
||||
}
|
||||
if (settings.hasMaxConcurrentStreams()) {
|
||||
frame.writeByte(SETTINGS_MAX_CONCURRENT_STREAMS);
|
||||
writeUnsignedInt(settings.maxConcurrentStreams(), frame);
|
||||
}
|
||||
if (settings.hasPushEnabled()) {
|
||||
frame.writeByte(SETTINGS_ENABLE_PUSH);
|
||||
writeUnsignedInt(settings.pushEnabled() ? 1L : 0L, frame);
|
||||
}
|
||||
return ctx.writeAndFlush(frame, promise);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeSettingsAck(ChannelHandlerContext ctx, ChannelPromise promise) {
|
||||
try {
|
||||
ByteBuf frame = ctx.alloc().buffer(FRAME_HEADER_LENGTH);
|
||||
writeFrameHeader(frame, 0, Http2FrameType.SETTINGS, Http2Flags.ACK_ONLY, 0);
|
||||
return ctx.writeAndFlush(frame, promise);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writePing(ChannelHandlerContext ctx, ChannelPromise promise, boolean ack,
|
||||
ByteBuf data) {
|
||||
try {
|
||||
ByteBuf frame = ctx.alloc().buffer(FRAME_HEADER_LENGTH + data.readableBytes());
|
||||
Http2Flags flags = ack ? Http2Flags.ACK_ONLY : Http2Flags.EMPTY;
|
||||
writeFrameHeader(frame, data.readableBytes(), Http2FrameType.PING, flags, 0);
|
||||
|
||||
// Write the debug data.
|
||||
frame.writeBytes(data, data.readerIndex(), data.readableBytes());
|
||||
return ctx.writeAndFlush(frame, promise);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writePushPromise(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int promisedStreamId, Http2Headers headers, int padding)
|
||||
throws Http2Exception {
|
||||
ByteBuf headerBlock = null;
|
||||
try {
|
||||
verifyStreamId(streamId, "Stream ID");
|
||||
verifyStreamId(promisedStreamId, "Promised Stream ID");
|
||||
verifyPadding(padding);
|
||||
|
||||
// Encode the entire header block into an intermediate buffer.
|
||||
headerBlock = ctx.alloc().buffer();
|
||||
headersEncoder.encodeHeaders(headers, headerBlock);
|
||||
|
||||
// Read the first fragment (possibly everything).
|
||||
Http2Flags.Builder flags = Http2Flags.newBuilder().setPaddingFlags(padding);
|
||||
int promisedStreamIdLength = INT_FIELD_LENGTH;
|
||||
int maxFragmentLength =
|
||||
MAX_FRAME_PAYLOAD_LENGTH
|
||||
- (promisedStreamIdLength + padding + flags.getNumPaddingLengthBytes());
|
||||
ByteBuf fragment =
|
||||
headerBlock.readSlice(Math.min(headerBlock.readableBytes(), maxFragmentLength));
|
||||
|
||||
flags = flags.endOfHeaders(headerBlock.readableBytes() == 0);
|
||||
|
||||
int payloadLength =
|
||||
fragment.readableBytes() + promisedStreamIdLength + padding
|
||||
+ flags.getNumPaddingLengthBytes();
|
||||
ByteBuf firstFrame = ctx.alloc().buffer(FRAME_HEADER_LENGTH + payloadLength);
|
||||
writeFrameHeader(firstFrame, payloadLength, Http2FrameType.PUSH_PROMISE, flags.build(),
|
||||
streamId);
|
||||
|
||||
writePaddingLength(padding, firstFrame);
|
||||
|
||||
// Write out the promised stream ID.
|
||||
firstFrame.writeInt(promisedStreamId);
|
||||
|
||||
// Write the first fragment.
|
||||
firstFrame.writeBytes(fragment);
|
||||
|
||||
// Write out the padding, if any.
|
||||
firstFrame.writeZero(padding);
|
||||
|
||||
if (headerBlock.readableBytes() == 0) {
|
||||
return ctx.writeAndFlush(firstFrame, promise);
|
||||
}
|
||||
|
||||
// Create a composite buffer wrapping the first frame and any continuation frames.
|
||||
return continueHeaders(ctx, promise, streamId, padding, headerBlock, firstFrame);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
} finally {
|
||||
if (headerBlock != null) {
|
||||
headerBlock.release();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeGoAway(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int lastStreamId, long errorCode, ByteBuf debugData) {
|
||||
try {
|
||||
verifyStreamOrConnectionId(lastStreamId, "Last Stream ID");
|
||||
verifyErrorCode(errorCode);
|
||||
|
||||
int payloadLength = 8 + debugData.readableBytes();
|
||||
ByteBuf frame = ctx.alloc().buffer(FRAME_HEADER_LENGTH + payloadLength);
|
||||
writeFrameHeader(frame, payloadLength, Http2FrameType.GO_AWAY, Http2Flags.EMPTY, 0);
|
||||
frame.writeInt(lastStreamId);
|
||||
writeUnsignedInt(errorCode, frame);
|
||||
frame.writeBytes(debugData, debugData.readerIndex(), debugData.readableBytes());
|
||||
return ctx.writeAndFlush(frame, promise);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeWindowUpdate(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int windowSizeIncrement) {
|
||||
try {
|
||||
verifyStreamOrConnectionId(streamId, "Stream ID");
|
||||
verifyWindowSizeIncrement(windowSizeIncrement);
|
||||
|
||||
ByteBuf frame = ctx.alloc().buffer(FRAME_HEADER_LENGTH + INT_FIELD_LENGTH);
|
||||
writeFrameHeader(frame, INT_FIELD_LENGTH, Http2FrameType.WINDOW_UPDATE,
|
||||
Http2Flags.EMPTY, streamId);
|
||||
frame.writeInt(windowSizeIncrement);
|
||||
return ctx.writeAndFlush(frame, promise);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeAltSvc(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, long maxAge, int port, ByteBuf protocolId, String host, String origin) {
|
||||
try {
|
||||
if (!server) {
|
||||
throw new IllegalArgumentException("ALT_SVC frames must not be sent by clients");
|
||||
}
|
||||
verifyStreamOrConnectionId(streamId, "Stream ID");
|
||||
verifyMaxAge(maxAge);
|
||||
verifyPort(port);
|
||||
|
||||
// 9 bytes is the total of all fields except for the protocol ID and host.
|
||||
// Breakdown: Max-Age(4) + Port(2) + reserved(1) + Proto-Len(1) + Host-Len(1) = 9
|
||||
int payloadLength = 9 + protocolId.readableBytes() + host.length();
|
||||
if (origin != null) {
|
||||
payloadLength += origin.length();
|
||||
}
|
||||
|
||||
ByteBuf frame = ctx.alloc().buffer(FRAME_HEADER_LENGTH + payloadLength);
|
||||
writeFrameHeader(frame, payloadLength, Http2FrameType.ALT_SVC, Http2Flags.EMPTY,
|
||||
streamId);
|
||||
writeUnsignedInt(maxAge, frame);
|
||||
writeUnsignedShort(port, frame);
|
||||
frame.writeZero(1);
|
||||
frame.writeByte(protocolId.readableBytes());
|
||||
frame.writeBytes(protocolId);
|
||||
frame.writeByte(host.length());
|
||||
frame.writeBytes(host.getBytes(UTF_8));
|
||||
if (origin != null) {
|
||||
frame.writeBytes(origin.getBytes(UTF_8));
|
||||
}
|
||||
return ctx.writeAndFlush(frame, promise);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeBlocked(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId) {
|
||||
try {
|
||||
verifyStreamOrConnectionId(streamId, "Stream ID");
|
||||
|
||||
ByteBuf frame = ctx.alloc().buffer(FRAME_HEADER_LENGTH);
|
||||
writeFrameHeader(frame, 0, Http2FrameType.BLOCKED, Http2Flags.EMPTY, streamId);
|
||||
return ctx.writeAndFlush(frame, promise);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
}
|
||||
}
|
||||
|
||||
private static void writeFrameHeader(ByteBuf out, int payloadLength, Http2FrameType type,
|
||||
Http2Flags flags, int streamId) {
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + payloadLength);
|
||||
out.writeShort(payloadLength);
|
||||
out.writeByte(type.typeCode());
|
||||
out.writeByte(flags.value());
|
||||
out.writeInt(streamId);
|
||||
}
|
||||
|
||||
private ChannelFuture writeHeadersInternal(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment,
|
||||
boolean hasPriority, int streamDependency, short weight, boolean exclusive)
|
||||
throws Http2Exception {
|
||||
ByteBuf headerBlock = null;
|
||||
try {
|
||||
verifyStreamId(streamId, "Stream ID");
|
||||
if (hasPriority) {
|
||||
verifyStreamOrConnectionId(streamDependency, "Stream Dependency");
|
||||
verifyPadding(padding);
|
||||
verifyWeight(weight);
|
||||
}
|
||||
|
||||
// Encode the entire header block.
|
||||
headerBlock = ctx.alloc().buffer();
|
||||
headersEncoder.encodeHeaders(headers, headerBlock);
|
||||
|
||||
Http2Flags.Builder flags =
|
||||
Http2Flags.newBuilder().endOfStream(endStream).endOfSegment(endSegment)
|
||||
.priorityPresent(hasPriority).setPaddingFlags(padding);
|
||||
|
||||
// Read the first fragment (possibly everything).
|
||||
int nonFragmentBytes =
|
||||
padding + flags.getNumPriorityBytes() + flags.getNumPaddingLengthBytes();
|
||||
int maxFragmentLength = MAX_FRAME_PAYLOAD_LENGTH - nonFragmentBytes;
|
||||
ByteBuf fragment =
|
||||
headerBlock.readSlice(Math.min(headerBlock.readableBytes(), maxFragmentLength));
|
||||
|
||||
// Set the end of headers flag for the first frame.
|
||||
flags = flags.endOfHeaders(headerBlock.readableBytes() == 0);
|
||||
|
||||
int payloadLength = fragment.readableBytes() + nonFragmentBytes;
|
||||
ByteBuf firstFrame = ctx.alloc().buffer(FRAME_HEADER_LENGTH + payloadLength);
|
||||
writeFrameHeader(firstFrame, payloadLength, Http2FrameType.HEADERS, flags.build(),
|
||||
streamId);
|
||||
|
||||
// Write the padding length.
|
||||
writePaddingLength(padding, firstFrame);
|
||||
|
||||
// Write the priority.
|
||||
if (hasPriority) {
|
||||
long word1 = exclusive ? (0x80000000L | streamDependency) : streamDependency;
|
||||
writeUnsignedInt(word1, firstFrame);
|
||||
firstFrame.writeByte(weight);
|
||||
}
|
||||
|
||||
// Write the first fragment.
|
||||
firstFrame.writeBytes(fragment);
|
||||
|
||||
// Write out the padding, if any.
|
||||
firstFrame.writeZero(padding);
|
||||
|
||||
if (flags.endOfHeaders()) {
|
||||
return ctx.writeAndFlush(firstFrame, promise);
|
||||
}
|
||||
|
||||
// Create a composite buffer wrapping the first frame and any continuation frames.
|
||||
return continueHeaders(ctx, promise, streamId, padding, headerBlock, firstFrame);
|
||||
} catch (RuntimeException e) {
|
||||
throw failAndThrow(promise, e);
|
||||
} finally {
|
||||
if (headerBlock != null) {
|
||||
headerBlock.release();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Drains the header block and creates a composite buffer containing the first frame and a
|
||||
* number of CONTINUATION frames.
|
||||
*/
|
||||
private static ChannelFuture continueHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int padding, ByteBuf headerBlock, ByteBuf firstFrame) {
|
||||
// Create a composite buffer wrapping the first frame and any continuation frames.
|
||||
CompositeByteBuf out = ctx.alloc().compositeBuffer();
|
||||
out.addComponent(firstFrame);
|
||||
int numBytes = firstFrame.readableBytes();
|
||||
|
||||
// Process any continuation frames there might be.
|
||||
while (headerBlock.isReadable()) {
|
||||
ByteBuf frame = createContinuationFrame(ctx, streamId, headerBlock, padding);
|
||||
out.addComponent(frame);
|
||||
numBytes += frame.readableBytes();
|
||||
}
|
||||
|
||||
out.writerIndex(numBytes);
|
||||
return ctx.writeAndFlush(out, promise);
|
||||
}
|
||||
|
||||
/**
|
||||
* Allocates a new buffer and writes a single continuation frame with a fragment of the header
|
||||
* block to the output buffer.
|
||||
*/
|
||||
private static ByteBuf createContinuationFrame(ChannelHandlerContext ctx, int streamId,
|
||||
ByteBuf headerBlock, int padding) {
|
||||
Http2Flags.Builder flags = Http2Flags.newBuilder().setPaddingFlags(padding);
|
||||
int maxFragmentLength =
|
||||
MAX_FRAME_PAYLOAD_LENGTH - (padding + flags.getNumPaddingLengthBytes());
|
||||
ByteBuf fragment =
|
||||
headerBlock.readSlice(Math.min(headerBlock.readableBytes(), maxFragmentLength));
|
||||
|
||||
int payloadLength = fragment.readableBytes() + padding + flags.getNumPaddingLengthBytes();
|
||||
ByteBuf frame = ctx.alloc().buffer(FRAME_HEADER_LENGTH + payloadLength);
|
||||
flags = flags.endOfHeaders(headerBlock.readableBytes() == 0);
|
||||
|
||||
writeFrameHeader(frame, payloadLength, Http2FrameType.CONTINUATION, flags.build(), streamId);
|
||||
|
||||
writePaddingLength(padding, frame);
|
||||
|
||||
frame.writeBytes(fragment);
|
||||
|
||||
// Write out the padding, if any.
|
||||
frame.writeZero(padding);
|
||||
return frame;
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes the padding length field to the output buffer.
|
||||
*/
|
||||
private static void writePaddingLength(int paddingLength, ByteBuf out) {
|
||||
if (paddingLength > MAX_UNSIGNED_BYTE) {
|
||||
int padHigh = paddingLength / 256;
|
||||
out.writeByte(padHigh);
|
||||
}
|
||||
// Always include PadLow if there is any padding at all.
|
||||
if (paddingLength > 0) {
|
||||
int padLow = paddingLength % 256;
|
||||
out.writeByte(padLow);
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyStreamId(int streamId, String argumentName) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException(argumentName + " must be > 0");
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyStreamOrConnectionId(int streamId, String argumentName) {
|
||||
if (streamId < 0) {
|
||||
throw new IllegalArgumentException(argumentName + " must be >= 0");
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyPadding(int padding) {
|
||||
if (padding < 0 || padding > MAX_UNSIGNED_SHORT) {
|
||||
throw new IllegalArgumentException("Invalid padding value: " + padding);
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyPayloadLength(int payloadLength) {
|
||||
if (payloadLength > MAX_FRAME_PAYLOAD_LENGTH) {
|
||||
throw new IllegalArgumentException("Total payload length " + payloadLength
|
||||
+ " exceeds max frame length.");
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyWeight(short weight) {
|
||||
if (weight < 1 || weight > MAX_UNSIGNED_BYTE) {
|
||||
throw new IllegalArgumentException("Invalid weight: " + weight);
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyErrorCode(long errorCode) {
|
||||
if (errorCode < 0 || errorCode > MAX_UNSIGNED_INT) {
|
||||
throw new IllegalArgumentException("Invalid errorCode: " + errorCode);
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyWindowSizeIncrement(int windowSizeIncrement) {
|
||||
if (windowSizeIncrement < 0) {
|
||||
throw new IllegalArgumentException("WindowSizeIncrement must be >= 0");
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyMaxAge(long maxAge) {
|
||||
if (maxAge < 0 || maxAge > MAX_UNSIGNED_INT) {
|
||||
throw new IllegalArgumentException("Invalid Max Age: " + maxAge);
|
||||
}
|
||||
}
|
||||
|
||||
private static void verifyPort(int port) {
|
||||
if (port < 0 || port > MAX_UNSIGNED_SHORT) {
|
||||
throw new IllegalArgumentException("Invalid port: " + port);
|
||||
}
|
||||
}
|
||||
}
|
@ -13,7 +13,7 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
@ -80,8 +80,7 @@ public final class DefaultHttp2Headers extends Http2Headers {
|
||||
|
||||
@Override
|
||||
public List<Entry<String, String>> entries() {
|
||||
List<Map.Entry<String, String>> all =
|
||||
new LinkedList<Map.Entry<String, String>>();
|
||||
List<Map.Entry<String, String>> all = new LinkedList<Map.Entry<String, String>>();
|
||||
|
||||
HeaderEntry e = head.after;
|
||||
while (e != head) {
|
||||
@ -208,7 +207,7 @@ public final class DefaultHttp2Headers extends Http2Headers {
|
||||
int hashTableIndex = index(nameHash);
|
||||
|
||||
remove0(nameHash, hashTableIndex, lowerCaseName);
|
||||
for (Object v: values) {
|
||||
for (Object v : values) {
|
||||
if (v == null) {
|
||||
break;
|
||||
}
|
||||
@ -231,35 +230,35 @@ public final class DefaultHttp2Headers extends Http2Headers {
|
||||
/**
|
||||
* Sets the {@link HttpName#METHOD} header.
|
||||
*/
|
||||
public Builder setMethod(String method) {
|
||||
public Builder method(String method) {
|
||||
return set(HttpName.METHOD.value(), method);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link HttpName#SCHEME} header.
|
||||
*/
|
||||
public Builder setScheme(String scheme) {
|
||||
public Builder scheme(String scheme) {
|
||||
return set(HttpName.SCHEME.value(), scheme);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link HttpName#AUTHORITY} header.
|
||||
*/
|
||||
public Builder setAuthority(String authority) {
|
||||
public Builder authority(String authority) {
|
||||
return set(HttpName.AUTHORITY.value(), authority);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link HttpName#PATH} header.
|
||||
*/
|
||||
public Builder setPath(String path) {
|
||||
public Builder path(String path) {
|
||||
return set(HttpName.PATH.value(), path);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link HttpName#STATUS} header.
|
||||
*/
|
||||
public Builder setStatus(String status) {
|
||||
public Builder status(String status) {
|
||||
return set(HttpName.STATUS.value(), status);
|
||||
}
|
||||
|
||||
@ -280,8 +279,8 @@ public final class DefaultHttp2Headers extends Http2Headers {
|
||||
|
||||
/**
|
||||
* Performs a lazy copy of the last build results, if there are any. For the typical use
|
||||
* case, headers will only be built once so no copy will be required. If the any method
|
||||
* is called on the builder after that, it will force a copy of the most recently created
|
||||
* case, headers will only be built once so no copy will be required. If the any method is
|
||||
* called on the builder after that, it will force a copy of the most recently created
|
||||
* headers object.
|
||||
*/
|
||||
private void lazyCopy() {
|
||||
@ -352,11 +351,10 @@ public final class DefaultHttp2Headers extends Http2Headers {
|
||||
if (value == null) {
|
||||
throw new NullPointerException("value");
|
||||
}
|
||||
for (int i = 0; i < value.length(); i ++) {
|
||||
for (int i = 0; i < value.length(); i++) {
|
||||
char c = value.charAt(i);
|
||||
if (c == 0) {
|
||||
throw new IllegalArgumentException(
|
||||
"value contains null character: " + value);
|
||||
throw new IllegalArgumentException("value contains null character: " + value);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -369,24 +367,20 @@ public final class DefaultHttp2Headers extends Http2Headers {
|
||||
throw new NullPointerException("name");
|
||||
}
|
||||
if (name.isEmpty()) {
|
||||
throw new IllegalArgumentException(
|
||||
"name cannot be length zero");
|
||||
throw new IllegalArgumentException("name cannot be length zero");
|
||||
}
|
||||
// Since name may only contain ascii characters, for valid names
|
||||
// name.length() returns the number of bytes when UTF-8 encoded.
|
||||
if (name.length() > MAX_VALUE_LENGTH) {
|
||||
throw new IllegalArgumentException(
|
||||
"name exceeds allowable length: " + name);
|
||||
throw new IllegalArgumentException("name exceeds allowable length: " + name);
|
||||
}
|
||||
for (int i = 0; i < name.length(); i ++) {
|
||||
for (int i = 0; i < name.length(); i++) {
|
||||
char c = name.charAt(i);
|
||||
if (c == 0) {
|
||||
throw new IllegalArgumentException(
|
||||
"name contains null character: " + name);
|
||||
throw new IllegalArgumentException("name contains null character: " + name);
|
||||
}
|
||||
if (c > 127) {
|
||||
throw new IllegalArgumentException(
|
||||
"name contains non-ascii character: " + name);
|
||||
throw new IllegalArgumentException("name contains non-ascii character: " + name);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -394,7 +388,7 @@ public final class DefaultHttp2Headers extends Http2Headers {
|
||||
|
||||
private static int hash(String name) {
|
||||
int h = 0;
|
||||
for (int i = name.length() - 1; i >= 0; i --) {
|
||||
for (int i = name.length() - 1; i >= 0; i--) {
|
||||
char c = name.charAt(i);
|
||||
if (c >= 'A' && c <= 'Z') {
|
||||
c += 32;
|
||||
@ -417,7 +411,7 @@ public final class DefaultHttp2Headers extends Http2Headers {
|
||||
return false;
|
||||
}
|
||||
|
||||
for (int i = nameLen - 1; i >= 0; i --) {
|
||||
for (int i = nameLen - 1; i >= 0; i--) {
|
||||
char c1 = name1.charAt(i);
|
||||
char c2 = name2.charAt(i);
|
||||
if (c1 != c2) {
|
@ -13,17 +13,14 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.connection.Http2ConnectionUtil.DEFAULT_HEADER_TABLE_SIZE;
|
||||
import static io.netty.handler.codec.http2.draft10.connection.Http2ConnectionUtil.DEFAULT_MAX_HEADER_SIZE;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_TABLE_SIZE;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_MAX_HEADER_SIZE;
|
||||
import static io.netty.handler.codec.http2.Http2Error.COMPRESSION_ERROR;
|
||||
import static io.netty.util.CharsetUtil.UTF_8;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufInputStream;
|
||||
import io.netty.handler.codec.http2.draft10.DefaultHttp2Headers;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Error;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Headers;
|
||||
import io.netty.util.CharsetUtil;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
@ -35,12 +32,21 @@ public class DefaultHttp2HeadersDecoder implements Http2HeadersDecoder {
|
||||
private final Decoder decoder;
|
||||
|
||||
public DefaultHttp2HeadersDecoder() {
|
||||
decoder = new Decoder(DEFAULT_MAX_HEADER_SIZE, DEFAULT_HEADER_TABLE_SIZE);
|
||||
this(DEFAULT_MAX_HEADER_SIZE, DEFAULT_HEADER_TABLE_SIZE);
|
||||
}
|
||||
|
||||
public DefaultHttp2HeadersDecoder(int maxHeaderSize, int maxHeaderTableSize) {
|
||||
decoder = new Decoder(maxHeaderSize, maxHeaderTableSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setHeaderTableSize(int size) throws Http2Exception {
|
||||
// TODO: can we throw away the decoder and create a new one?
|
||||
public void maxHeaderTableSize(int size) {
|
||||
decoder.setMaxHeaderTableSize(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxHeaderTableSize() {
|
||||
return decoder.getMaxHeaderTableSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -49,18 +55,20 @@ public class DefaultHttp2HeadersDecoder implements Http2HeadersDecoder {
|
||||
final DefaultHttp2Headers.Builder headersBuilder = new DefaultHttp2Headers.Builder();
|
||||
HeaderListener listener = new HeaderListener() {
|
||||
@Override
|
||||
public void emitHeader(byte[] key, byte[] value) {
|
||||
headersBuilder.add(new String(key, CharsetUtil.UTF_8), new String(value,
|
||||
CharsetUtil.UTF_8));
|
||||
public void emitHeader(byte[] key, byte[] value, boolean sensitive) {
|
||||
headersBuilder.add(new String(key, UTF_8), new String(value, UTF_8));
|
||||
}
|
||||
};
|
||||
|
||||
decoder.decode(new ByteBufInputStream(headerBlock), listener);
|
||||
decoder.endHeaderBlock(listener);
|
||||
boolean truncated = decoder.endHeaderBlock(listener);
|
||||
if (truncated) {
|
||||
// TODO: what's the right thing to do here?
|
||||
}
|
||||
|
||||
return headersBuilder.build();
|
||||
} catch (IOException e) {
|
||||
throw new Http2Exception(Http2Error.COMPRESSION_ERROR, e.getMessage());
|
||||
throw new Http2Exception(COMPRESSION_ERROR, e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
@ -13,15 +13,13 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.connection.Http2ConnectionUtil.DEFAULT_HEADER_TABLE_SIZE;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_HEADER_TABLE_SIZE;
|
||||
import static io.netty.util.CharsetUtil.UTF_8;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufOutputStream;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Error;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Headers;
|
||||
import io.netty.buffer.Unpooled;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
@ -30,32 +28,53 @@ import java.util.Map.Entry;
|
||||
import com.twitter.hpack.Encoder;
|
||||
|
||||
public class DefaultHttp2HeadersEncoder implements Http2HeadersEncoder {
|
||||
|
||||
private final Encoder encoder;
|
||||
private final ByteBuf tableSizeChangeOutput = Unpooled.buffer();
|
||||
|
||||
public DefaultHttp2HeadersEncoder() {
|
||||
encoder = new Encoder(DEFAULT_HEADER_TABLE_SIZE);
|
||||
this(DEFAULT_HEADER_TABLE_SIZE);
|
||||
}
|
||||
|
||||
public DefaultHttp2HeadersEncoder(int maxHeaderTableSize) {
|
||||
encoder = new Encoder(maxHeaderTableSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void encodeHeaders(Http2Headers headers, ByteBuf buffer) throws Http2Exception {
|
||||
try {
|
||||
// If there was a change in the table size, serialize the output from the encoder
|
||||
// resulting from that change.
|
||||
if (tableSizeChangeOutput.isReadable()) {
|
||||
buffer.writeBytes(tableSizeChangeOutput);
|
||||
tableSizeChangeOutput.clear();
|
||||
}
|
||||
|
||||
OutputStream stream = new ByteBufOutputStream(buffer);
|
||||
for (Entry<String, String> header : headers) {
|
||||
byte[] key = header.getKey().getBytes(UTF_8);
|
||||
byte[] value = header.getValue().getBytes(UTF_8);
|
||||
encoder.encodeHeader(stream, key, value);
|
||||
encoder.encodeHeader(stream, key, value, false);
|
||||
}
|
||||
encoder.endHeaders(stream);
|
||||
} catch (IOException e) {
|
||||
throw Http2Exception.format(Http2Error.COMPRESSION_ERROR, "Failed encoding headers block: %s",
|
||||
e.getMessage());
|
||||
throw Http2Exception.format(Http2Error.COMPRESSION_ERROR,
|
||||
"Failed encoding headers block: %s", e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setHeaderTableSize(int size) throws Http2Exception {
|
||||
// TODO: can we throw away the encoder and create a new one?
|
||||
public void maxHeaderTableSize(int size) throws Http2Exception {
|
||||
try {
|
||||
// No headers should be emitted. If they are, we throw.
|
||||
encoder.setMaxHeaderTableSize(new ByteBufOutputStream(tableSizeChangeOutput), size);
|
||||
} catch (IOException e) {
|
||||
throw new Http2Exception(Http2Error.COMPRESSION_ERROR, e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxHeaderTableSize() {
|
||||
return encoder.getMaxHeaderTableSize();
|
||||
}
|
||||
|
||||
}
|
@ -13,102 +13,100 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.connection;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.flowControlError;
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.connection.Http2ConnectionUtil.DEFAULT_FLOW_CONTROL_WINDOW_SIZE;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.CONNECTION_STREAM_ID;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2WindowUpdateFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2DataFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2WindowUpdateFrame;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.CONNECTION_STREAM_ID;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_FLOW_CONTROL_WINDOW_SIZE;
|
||||
import static io.netty.handler.codec.http2.Http2Exception.flowControlError;
|
||||
import static io.netty.handler.codec.http2.Http2Exception.protocolError;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Basic implementation of {@link InboundFlowController}.
|
||||
* Basic implementation of {@link Http2InboundFlowController}.
|
||||
*/
|
||||
public class DefaultInboundFlowController implements InboundFlowController {
|
||||
public class DefaultHttp2InboundFlowController implements Http2InboundFlowController {
|
||||
|
||||
private int initialWindowSize = DEFAULT_FLOW_CONTROL_WINDOW_SIZE;
|
||||
private final StreamWindow connectionWindow = new StreamWindow(CONNECTION_STREAM_ID);
|
||||
private final Map<Integer, StreamWindow> streamWindows = new HashMap<Integer, StreamWindow>();
|
||||
private final FlowState connectionState = new FlowState(CONNECTION_STREAM_ID);
|
||||
private final Map<Integer, FlowState> streamStates = new HashMap<Integer, FlowState>();
|
||||
|
||||
public DefaultInboundFlowController(Http2Connection connection) {
|
||||
if (connection == null) {
|
||||
throw new NullPointerException("connecton");
|
||||
@Override
|
||||
public void addStream(int streamId) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("Stream ID must be > 0");
|
||||
}
|
||||
connection.addListener(new Http2Connection.Listener() {
|
||||
@Override
|
||||
public void streamCreated(int streamId) {
|
||||
streamWindows.put(streamId, new StreamWindow(streamId));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void streamClosed(int streamId) {
|
||||
streamWindows.remove(streamId);
|
||||
}
|
||||
});
|
||||
if (streamStates.containsKey(streamId)) {
|
||||
throw new IllegalArgumentException("Stream " + streamId + " already exists.");
|
||||
}
|
||||
streamStates.put(streamId, new FlowState(streamId));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setInitialInboundWindowSize(int newWindowSize) throws Http2Exception {
|
||||
public void removeStream(int streamId) {
|
||||
streamStates.remove(streamId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialInboundWindowSize(int newWindowSize) throws Http2Exception {
|
||||
int deltaWindowSize = newWindowSize - initialWindowSize;
|
||||
initialWindowSize = newWindowSize;
|
||||
|
||||
// Apply the delta to all of the windows.
|
||||
connectionWindow.addAndGet(deltaWindowSize);
|
||||
for (StreamWindow window : streamWindows.values()) {
|
||||
connectionState.addAndGet(deltaWindowSize);
|
||||
for (FlowState window : streamStates.values()) {
|
||||
window.updatedInitialWindowSize(deltaWindowSize);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInitialInboundWindowSize() {
|
||||
public int initialInboundWindowSize() {
|
||||
return initialWindowSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void applyInboundFlowControl(Http2DataFrame dataFrame, FrameWriter frameWriter)
|
||||
public void applyInboundFlowControl(int streamId, ByteBuf data, int padding,
|
||||
boolean endOfStream, boolean endOfSegment, boolean compressed, FrameWriter frameWriter)
|
||||
throws Http2Exception {
|
||||
applyConnectionFlowControl(dataFrame, frameWriter);
|
||||
applyStreamFlowControl(dataFrame, frameWriter);
|
||||
int dataLength = data.readableBytes();
|
||||
applyConnectionFlowControl(dataLength, frameWriter);
|
||||
applyStreamFlowControl(streamId, dataLength, endOfStream, frameWriter);
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply connection-wide flow control to the incoming data frame.
|
||||
*/
|
||||
private void applyConnectionFlowControl(Http2DataFrame dataFrame, FrameWriter frameWriter)
|
||||
private void applyConnectionFlowControl(int dataLength, FrameWriter frameWriter)
|
||||
throws Http2Exception {
|
||||
// Remove the data length from the available window size. Throw if the lower bound
|
||||
// was exceeded.
|
||||
connectionWindow.addAndGet(-dataFrame.content().readableBytes());
|
||||
connectionState.addAndGet(-dataLength);
|
||||
|
||||
// If less than the window update threshold remains, restore the window size
|
||||
// to the initial value and send a window update to the remote endpoint indicating
|
||||
// the new window size.
|
||||
if (connectionWindow.getSize() <= getWindowUpdateThreshold()) {
|
||||
connectionWindow.updateWindow(frameWriter);
|
||||
if (connectionState.windowSize() <= getWindowUpdateThreshold()) {
|
||||
connectionState.updateWindow(frameWriter);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply stream-based flow control to the incoming data frame.
|
||||
*/
|
||||
private void applyStreamFlowControl(Http2DataFrame dataFrame, FrameWriter frameWriter)
|
||||
throws Http2Exception {
|
||||
private void applyStreamFlowControl(int streamId, int dataLength, boolean endOfStream,
|
||||
FrameWriter frameWriter) throws Http2Exception {
|
||||
// Remove the data length from the available window size. Throw if the lower bound
|
||||
// was exceeded.
|
||||
StreamWindow window = getWindowOrFail(dataFrame.getStreamId());
|
||||
window.addAndGet(-dataFrame.content().readableBytes());
|
||||
FlowState state = getStateOrFail(streamId);
|
||||
state.addAndGet(-dataLength);
|
||||
|
||||
// If less than the window update threshold remains, restore the window size
|
||||
// to the initial value and send a window update to the remote endpoint indicating
|
||||
// the new window size.
|
||||
if (window.getSize() <= getWindowUpdateThreshold() && !dataFrame.isEndOfStream()) {
|
||||
window.updateWindow(frameWriter);
|
||||
if (state.windowSize() <= getWindowUpdateThreshold() && !endOfStream) {
|
||||
state.updateWindow(frameWriter);
|
||||
}
|
||||
}
|
||||
|
||||
@ -122,8 +120,8 @@ public class DefaultInboundFlowController implements InboundFlowController {
|
||||
/**
|
||||
* Gets the window for the stream or raises a {@code PROTOCOL_ERROR} if not found.
|
||||
*/
|
||||
private StreamWindow getWindowOrFail(int streamId) throws Http2Exception {
|
||||
StreamWindow window = streamWindows.get(streamId);
|
||||
private FlowState getStateOrFail(int streamId) throws Http2Exception {
|
||||
FlowState window = streamStates.get(streamId);
|
||||
if (window == null) {
|
||||
throw protocolError("Flow control window missing for stream: %d", streamId);
|
||||
}
|
||||
@ -133,17 +131,17 @@ public class DefaultInboundFlowController implements InboundFlowController {
|
||||
/**
|
||||
* Flow control window state for an individual stream.
|
||||
*/
|
||||
private final class StreamWindow {
|
||||
private final class FlowState {
|
||||
private int windowSize;
|
||||
private int lowerBound;
|
||||
private final int streamId;
|
||||
|
||||
public StreamWindow(int streamId) {
|
||||
FlowState(int streamId) {
|
||||
this.streamId = streamId;
|
||||
windowSize = initialWindowSize;
|
||||
}
|
||||
|
||||
public int getSize() {
|
||||
int windowSize() {
|
||||
return windowSize;
|
||||
}
|
||||
|
||||
@ -153,7 +151,7 @@ public class DefaultInboundFlowController implements InboundFlowController {
|
||||
* @param delta the delta in the initial window size.
|
||||
* @throws Http2Exception thrown if the new window is less than the allowed lower bound.
|
||||
*/
|
||||
public int addAndGet(int delta) throws Http2Exception {
|
||||
int addAndGet(int delta) throws Http2Exception {
|
||||
// Apply the delta. Even if we throw an exception we want to have taken this delta into
|
||||
// account.
|
||||
windowSize += delta;
|
||||
@ -178,20 +176,20 @@ public class DefaultInboundFlowController implements InboundFlowController {
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when sending a SETTINGS frame with a new initial window size. If the window has gotten
|
||||
* smaller (i.e. deltaWindowSize < 0), the lower bound is set to that value. This will
|
||||
* temporarily allow for receipt of data frames which were sent by the remote endpoint before
|
||||
* receiving the SETTINGS frame.
|
||||
* Called when sending a SETTINGS frame with a new initial window size. If the window has
|
||||
* gotten smaller (i.e. deltaWindowSize < 0), the lower bound is set to that value. This
|
||||
* will temporarily allow for receipt of data frames which were sent by the remote endpoint
|
||||
* before receiving the SETTINGS frame.
|
||||
*
|
||||
* @param delta the delta in the initial window size.
|
||||
* @throws Http2Exception thrown if integer overflow occurs on the window.
|
||||
*/
|
||||
public void updatedInitialWindowSize(int delta) throws Http2Exception {
|
||||
windowSize += delta;
|
||||
if (delta > 0 && windowSize < Integer.MIN_VALUE + delta) {
|
||||
void updatedInitialWindowSize(int delta) throws Http2Exception {
|
||||
if (delta > 0 && windowSize > Integer.MAX_VALUE - delta) {
|
||||
// Integer overflow.
|
||||
throw flowControlError("Flow control window overflowed for stream: %d", streamId);
|
||||
}
|
||||
windowSize += delta;
|
||||
|
||||
if (delta < 0) {
|
||||
lowerBound = delta;
|
||||
@ -199,19 +197,17 @@ public class DefaultInboundFlowController implements InboundFlowController {
|
||||
}
|
||||
|
||||
/**
|
||||
* Called to perform a window update for this stream (or connection). Updates the window size
|
||||
* back to the size of the initial window and sends a window update frame to the remote
|
||||
* Called to perform a window update for this stream (or connection). Updates the window
|
||||
* size back to the size of the initial window and sends a window update frame to the remote
|
||||
* endpoint.
|
||||
*/
|
||||
public void updateWindow(FrameWriter frameWriter) throws Http2Exception {
|
||||
void updateWindow(FrameWriter frameWriter) throws Http2Exception {
|
||||
// Expand the window for this stream back to the size of the initial window.
|
||||
int deltaWindowSize = initialWindowSize - getSize();
|
||||
int deltaWindowSize = initialWindowSize - windowSize();
|
||||
addAndGet(deltaWindowSize);
|
||||
|
||||
// Send a window update for the stream/connection.
|
||||
Http2WindowUpdateFrame updateFrame = new DefaultHttp2WindowUpdateFrame.Builder()
|
||||
.setStreamId(streamId).setWindowSizeIncrement(deltaWindowSize).build();
|
||||
frameWriter.writeFrame(updateFrame);
|
||||
frameWriter.writeFrame(streamId, deltaWindowSize);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,635 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.CONNECTION_STREAM_ID;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_FLOW_CONTROL_WINDOW_SIZE;
|
||||
import static io.netty.handler.codec.http2.Http2Error.FLOW_CONTROL_ERROR;
|
||||
import static io.netty.handler.codec.http2.Http2Error.STREAM_CLOSED;
|
||||
import static io.netty.handler.codec.http2.Http2Exception.format;
|
||||
import static io.netty.handler.codec.http2.Http2Exception.protocolError;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.handler.codec.http2.Http2PriorityTree.Priority;
|
||||
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* Basic implementation of {@link Http2OutboundFlowController}.
|
||||
*/
|
||||
public class DefaultHttp2OutboundFlowController implements Http2OutboundFlowController {
|
||||
/**
|
||||
* The interval (in ns) at which the removed priority garbage collector runs.
|
||||
*/
|
||||
private final long GARBAGE_COLLECTION_INTERVAL = TimeUnit.SECONDS.toNanos(2);
|
||||
|
||||
private final Http2PriorityTree<FlowState> priorityTree =
|
||||
new DefaultHttp2PriorityTree<FlowState>();
|
||||
private final Queue<Priority<FlowState>> garbage = new ArrayDeque<Priority<FlowState>>();
|
||||
private int initialWindowSize = DEFAULT_FLOW_CONTROL_WINDOW_SIZE;
|
||||
private int connectionWindowSize = DEFAULT_FLOW_CONTROL_WINDOW_SIZE;
|
||||
private long lastGarbageCollection;
|
||||
|
||||
@Override
|
||||
public void addStream(int streamId, int parent, short weight, boolean exclusive) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("Stream ID must be > 0");
|
||||
}
|
||||
Priority<FlowState> priority = priorityTree.get(streamId);
|
||||
if (priority != null) {
|
||||
throw new IllegalArgumentException("stream " + streamId + " already exists");
|
||||
}
|
||||
|
||||
FlowState state = new FlowState();
|
||||
priority = priorityTree.prioritize(streamId, parent, weight, exclusive, state);
|
||||
state.init(priority);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateStream(int streamId, int parent, short weight, boolean exclusive) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("Stream ID must be > 0");
|
||||
}
|
||||
Priority<FlowState> priority = priorityTree.get(streamId);
|
||||
if (priority == null) {
|
||||
throw new IllegalArgumentException("stream " + streamId + " does not exist");
|
||||
}
|
||||
|
||||
priorityTree.prioritize(streamId, parent, weight, exclusive, priority.data());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeStream(int streamId) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("Stream ID must be > 0");
|
||||
}
|
||||
|
||||
Priority<FlowState> priority = priorityTree.get(streamId);
|
||||
if (priority != null) {
|
||||
priority.data().markForRemoval();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialOutboundWindowSize(int newWindowSize) throws Http2Exception {
|
||||
int delta = newWindowSize - initialWindowSize;
|
||||
initialWindowSize = newWindowSize;
|
||||
addAndGetConnectionWindowSize(delta);
|
||||
for (Priority<FlowState> priority : priorityTree) {
|
||||
FlowState state = priority.data();
|
||||
if (!state.isMarkedForRemoval()) {
|
||||
// Verify that the maximum value is not exceeded by this change.
|
||||
state.addAndGetWindow(delta);
|
||||
}
|
||||
}
|
||||
|
||||
if (delta > 0) {
|
||||
// The window size increased, send any pending frames for all streams.
|
||||
writePendingBytes();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int initialOutboundWindowSize() {
|
||||
return initialWindowSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateOutboundWindowSize(int streamId, int delta) throws Http2Exception {
|
||||
if (streamId == CONNECTION_STREAM_ID) {
|
||||
// Update the connection window and write any pending frames for all streams.
|
||||
addAndGetConnectionWindowSize(delta);
|
||||
writePendingBytes();
|
||||
} else {
|
||||
// Update the stream window and write any pending frames for the stream.
|
||||
FlowState state = getStateOrFail(streamId);
|
||||
state.addAndGetWindow(delta);
|
||||
state.writeBytes(state.writableWindow());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBlocked(int streamId) throws Http2Exception {
|
||||
// Ignore blocked frames. Just rely on window updates.
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendFlowControlled(int streamId, ByteBuf data, int padding, boolean endStream,
|
||||
boolean endSegment, boolean compressed, FrameWriter frameWriter) throws Http2Exception {
|
||||
FlowState state = getStateOrFail(streamId);
|
||||
FlowState.Frame frame =
|
||||
state.newFrame(data, padding, endStream, endSegment, compressed, frameWriter);
|
||||
|
||||
int dataLength = data.readableBytes();
|
||||
if (state.writableWindow() >= dataLength) {
|
||||
// Window size is large enough to send entire data frame
|
||||
frame.write();
|
||||
return;
|
||||
}
|
||||
|
||||
// Enqueue the frame to be written when the window size permits.
|
||||
frame.enqueue();
|
||||
|
||||
if (state.writableWindow() <= 0) {
|
||||
// Stream is stalled, don't send anything now.
|
||||
return;
|
||||
}
|
||||
|
||||
// Create and send a partial frame up to the window size.
|
||||
frame.split(state.writableWindow()).write();
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempts to get the {@link FlowState} for the given stream. If not available, raises a
|
||||
* {@code PROTOCOL_ERROR}.
|
||||
*/
|
||||
private FlowState getStateOrFail(int streamId) throws Http2Exception {
|
||||
FlowState state = getFlowState(streamId);
|
||||
if (state == null) {
|
||||
throw protocolError("Missing flow control window for stream: %d", streamId);
|
||||
}
|
||||
return state;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether applying the delta to the given value will cause an integer overflow.
|
||||
*/
|
||||
private static boolean isIntegerOverflow(int previousValue, int delta) {
|
||||
return delta > 0 && (Integer.MAX_VALUE - delta) < previousValue;
|
||||
}
|
||||
|
||||
/**
|
||||
* Increments the connectionWindowSize and returns the new value.
|
||||
*/
|
||||
private int addAndGetConnectionWindowSize(int delta) throws Http2Exception {
|
||||
if (isIntegerOverflow(connectionWindowSize, delta)) {
|
||||
throw format(FLOW_CONTROL_ERROR, "Window update exceeds maximum for connection");
|
||||
}
|
||||
return connectionWindowSize += delta;
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes as many pending bytes as possible, according to stream priority.
|
||||
*/
|
||||
private void writePendingBytes() throws Http2Exception {
|
||||
// Perform garbage collection to remove any priorities marked for deletion from the tree.
|
||||
garbageCollect();
|
||||
|
||||
// Calculate the total writable bytes for each stream in the tree.
|
||||
// TODO: consider maintaining a running total of the writable bytes at each node so we
|
||||
// don't have to walk the entire tree here.
|
||||
int totalWritableBytes = 0;
|
||||
for (Priority<FlowState> priority : priorityTree.root().children()) {
|
||||
totalWritableBytes += priority.data().calcWritableBytes();
|
||||
}
|
||||
|
||||
// Recursively write as many of the total writable bytes as possible.
|
||||
writeAllowedBytes(totalWritableBytes, priorityTree.root());
|
||||
}
|
||||
|
||||
/**
|
||||
* Recursively traverses the priority tree rooted at the given node. Attempts to write the
|
||||
* allowed bytes for the streams in this sub tree based on their weighted priorities.
|
||||
*
|
||||
* @param allowance an allowed number of bytes that may be written to the streams in this sub
|
||||
* tree.
|
||||
* @param priority the sub tree to write to. On the first invocation, this will be the root of
|
||||
* the priority tree (i.e. the connection node).
|
||||
*/
|
||||
private void writeAllowedBytes(int allowance, Priority<FlowState> priority)
|
||||
throws Http2Exception {
|
||||
// Get the flow control state for this priority node. It may be null if processing the root
|
||||
// node (i.e. the connection) or if the priority is being retained for a short time
|
||||
// after the stream was closed.
|
||||
FlowState state = priority.data();
|
||||
if (state != null) {
|
||||
// Write the allowed bytes for this node. If not all of the allowance was used,
|
||||
// restore what's left so that it can be propagated to future nodes.
|
||||
int bytesWritten = state.writeBytes(allowance);
|
||||
allowance -= bytesWritten;
|
||||
}
|
||||
|
||||
if (allowance <= 0 || priority.isLeaf()) {
|
||||
// Nothing left to do in this sub tree.
|
||||
return;
|
||||
}
|
||||
|
||||
// Clip the remaining connection flow control window by the allowance.
|
||||
int remainingWindow = Math.min(allowance, connectionWindowSize);
|
||||
|
||||
// Decreasing totals for all nodes yet to be processed in the current pass.
|
||||
// For the root node, the allowance is the total number of writable bytes.
|
||||
int unallocatedBytes = priority.isRoot() ? allowance : priority.data().unallocatedBytes();
|
||||
|
||||
// Optimization. If the window is big enough to fit all the data. Just write everything
|
||||
// and skip the priority algorithm.
|
||||
if (unallocatedBytes <= remainingWindow) {
|
||||
for (Priority<FlowState> child : priority.children()) {
|
||||
child.data().writeBytes(child.data().unallocatedBytes());
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
// Get the total weight of all children directly under the current node.
|
||||
int remainingWeight = priority.totalChildWeights();
|
||||
|
||||
// Iterate over the children and spread the remaining bytes across them as is appropriate
|
||||
// based on the weights. This algorithm loops over all of the children more than once,
|
||||
// although it should typically only take a few passes to complete. In each pass we
|
||||
// give a node its share of the current remaining bytes. The node's weight and bytes
|
||||
// allocated are then decremented from the totals, so that the subsequent
|
||||
// nodes split the difference. If after being processed, a node still has writable data,
|
||||
// it is added to the end of the queue and will be processed again in the next pass.
|
||||
|
||||
// Increasing totals for nodes that have been re-added to the queue for the next pass.
|
||||
int unallocatedBytesForNextPass = 0;
|
||||
int remainingWeightForNextPass = 0;
|
||||
|
||||
// Copy the children to a deque
|
||||
ArrayDeque<Priority<FlowState>> deque =
|
||||
new ArrayDeque<Priority<FlowState>>(priority.children());
|
||||
for (;;) {
|
||||
Priority<FlowState> next = deque.poll();
|
||||
if (next == null) {
|
||||
break;
|
||||
}
|
||||
FlowState node = next.data();
|
||||
|
||||
if (remainingWeight == 0) {
|
||||
// We've reached the end one pass of the nodes. Reset the totals based on
|
||||
// the nodes that were re-added to the deque since they still have data available.
|
||||
unallocatedBytes = unallocatedBytesForNextPass;
|
||||
remainingWeight = remainingWeightForNextPass;
|
||||
unallocatedBytesForNextPass = 0;
|
||||
remainingWeightForNextPass = 0;
|
||||
}
|
||||
|
||||
int weight = node.priority().weight();
|
||||
|
||||
// Determine the amount of data that's still unallocated and will fit into
|
||||
// the current connection window.
|
||||
int writableData = Math.min(unallocatedBytes, remainingWindow);
|
||||
if (writableData > 0 && node.unallocatedBytes() > 0) {
|
||||
|
||||
// Determine the value (in bytes) of a single unit of weight.
|
||||
double dataToWeightRatio = writableData / (double) remainingWeight;
|
||||
|
||||
// Determine the portion of the current writable data that is assigned to this
|
||||
// node.
|
||||
int writableChunk = (int) (weight * dataToWeightRatio);
|
||||
|
||||
// Clip the chunk allocated by the total amount of unallocated data remaining in
|
||||
// the node.
|
||||
int allocatedChunk = Math.min(writableChunk, node.unallocatedBytes());
|
||||
|
||||
// Update the remaining connection window size.
|
||||
remainingWindow -= allocatedChunk;
|
||||
|
||||
// This node has been processed for this loop. Remove it from the loop totals.
|
||||
unallocatedBytes -= node.unallocatedBytes();
|
||||
remainingWeight -= weight;
|
||||
|
||||
// Update the node state.
|
||||
node.allocateBytes(allocatedChunk);
|
||||
if (node.unallocatedBytes() > 0) {
|
||||
// There is still data remaining for this stream. Add it to the end of the
|
||||
// deque to be processed in the next loop.
|
||||
unallocatedBytesForNextPass += node.unallocatedBytes();
|
||||
remainingWeightForNextPass += weight;
|
||||
deque.add(node.priority());
|
||||
|
||||
// Don't write the data for this node yet - there may be more that will
|
||||
// be allocated in the next loop.
|
||||
continue;
|
||||
}
|
||||
} else {
|
||||
// This node has been processed for this loop. Remove it from the loop totals.
|
||||
unallocatedBytes -= node.unallocatedBytes();
|
||||
remainingWeight -= weight;
|
||||
}
|
||||
|
||||
if (node.allocatedBytes() > 0) {
|
||||
// Write the allocated data for this stream.
|
||||
writeAllowedBytes(node.allocatedBytes(), node.priority());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private FlowState getFlowState(int streamId) {
|
||||
Priority<FlowState> priority = priorityTree.get(streamId);
|
||||
return priority != null ? priority.data() : null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes any priorities from the tree that were marked for removal greater than
|
||||
* {@link #GARBAGE_COLLECTION_INTERVAL} milliseconds ago. Garbage collection will run at most on
|
||||
* the interval {@link #GARBAGE_COLLECTION_INTERVAL}, so calling it more frequently will have no
|
||||
* effect.
|
||||
*/
|
||||
private void garbageCollect() {
|
||||
if (garbage.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
long time = System.nanoTime();
|
||||
if (time - lastGarbageCollection < GARBAGE_COLLECTION_INTERVAL) {
|
||||
// Only run the garbage collection on the threshold interval (at most).
|
||||
return;
|
||||
}
|
||||
lastGarbageCollection = time;
|
||||
|
||||
for (;;) {
|
||||
Priority<FlowState> next = garbage.peek();
|
||||
if (next == null) {
|
||||
break;
|
||||
}
|
||||
long removeTime = next.data().removalTime();
|
||||
if (time - removeTime > GARBAGE_COLLECTION_INTERVAL) {
|
||||
Priority<FlowState> priority = garbage.remove();
|
||||
priorityTree.remove(priority.streamId());
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The outbound flow control state for a single stream.
|
||||
*/
|
||||
private final class FlowState {
|
||||
private final Queue<Frame> pendingWriteQueue = new ArrayDeque<Frame>(2);
|
||||
private Priority<FlowState> priority;
|
||||
private int windowSize = initialWindowSize;
|
||||
private long removalTime;
|
||||
private int writableBytes;
|
||||
private int allocatedBytes;
|
||||
|
||||
/**
|
||||
* Initializes this flow state with the stream priority.
|
||||
*/
|
||||
void init(Priority<FlowState> priority) {
|
||||
this.priority = priority;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the priority in the tree associated with this flow state.
|
||||
*/
|
||||
Priority<FlowState> priority() {
|
||||
return priority;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates that this priority has been marked for removal, thus making it a candidate for
|
||||
* garbage collection.
|
||||
*/
|
||||
boolean isMarkedForRemoval() {
|
||||
return removalTime > 0L;
|
||||
}
|
||||
|
||||
/**
|
||||
* If marked for removal, indicates the removal time of this priority.
|
||||
*/
|
||||
long removalTime() {
|
||||
return removalTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* Marks this state for removal, thus making it a candidate for garbage collection. Sets the
|
||||
* removal time to the current system time.
|
||||
*/
|
||||
void markForRemoval() {
|
||||
if (!isMarkedForRemoval()) {
|
||||
removalTime = System.nanoTime();
|
||||
garbage.add(priority);
|
||||
clear();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Increments the flow control window for this stream by the given delta and returns the new
|
||||
* value.
|
||||
*/
|
||||
int addAndGetWindow(int delta) throws Http2Exception {
|
||||
if (isIntegerOverflow(windowSize, delta)) {
|
||||
throw new Http2StreamException(priority.streamId(), FLOW_CONTROL_ERROR,
|
||||
"Window size overflow for stream");
|
||||
}
|
||||
windowSize += delta;
|
||||
return windowSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the maximum writable window (minimum of the stream and connection windows).
|
||||
*/
|
||||
int writableWindow() {
|
||||
return Math.min(windowSize, connectionWindowSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by the priority algorithm to allocate bytes to this stream.
|
||||
*/
|
||||
void allocateBytes(int bytes) {
|
||||
allocatedBytes += bytes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by the priority algorithm to get the intermediate allocation of bytes to this
|
||||
* stream.
|
||||
*/
|
||||
int allocatedBytes() {
|
||||
return allocatedBytes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by the priority algorithm to determine the number of writable bytes that have not
|
||||
* yet been allocated.
|
||||
*/
|
||||
int unallocatedBytes() {
|
||||
return writableBytes - allocatedBytes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by the priority algorithm to calculate the number of writable bytes for this
|
||||
* sub-tree. Writable bytes takes into account the connection window and the stream windows
|
||||
* for each node.
|
||||
*/
|
||||
int calcWritableBytes() {
|
||||
writableBytes = 0;
|
||||
|
||||
// Calculate the writable bytes for this node.
|
||||
if (!isMarkedForRemoval()) {
|
||||
int window = writableWindow();
|
||||
for (Frame frame : pendingWriteQueue) {
|
||||
writableBytes += Math.min(window, frame.data.readableBytes());
|
||||
if (writableBytes == window) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Calculate the writable bytes for all children.
|
||||
for (Priority<FlowState> child : priority.children()) {
|
||||
writableBytes += child.data().calcWritableBytes();
|
||||
}
|
||||
|
||||
return writableBytes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new frame with the given values but does not add it to the pending queue.
|
||||
*/
|
||||
Frame newFrame(ByteBuf data, int padding, boolean endStream, boolean endSegment,
|
||||
boolean compressed, FrameWriter writer) {
|
||||
return new Frame(data, padding, endStream, endSegment, compressed, writer);
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether or not there are frames in the pending queue.
|
||||
*/
|
||||
boolean hasFrame() {
|
||||
return !pendingWriteQueue.isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the the head of the pending queue, or {@code null} if empty or the current window
|
||||
* size is zero.
|
||||
*/
|
||||
Frame peek() {
|
||||
if (windowSize > 0) {
|
||||
return pendingWriteQueue.peek();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Clears the pending queue and writes errors for each remaining frame.
|
||||
*/
|
||||
void clear() {
|
||||
while (true) {
|
||||
Frame frame = pendingWriteQueue.poll();
|
||||
if (frame == null) {
|
||||
break;
|
||||
}
|
||||
frame.writeError(format(STREAM_CLOSED,
|
||||
"Stream closed before write could take place"));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes up to the number of bytes from the pending queue. May write less if limited by the
|
||||
* writable window, by the number of pending writes available, or because a frame does not
|
||||
* support splitting on arbitrary boundaries.
|
||||
*/
|
||||
int writeBytes(int bytes) throws Http2Exception {
|
||||
int bytesWritten = 0;
|
||||
if (isMarkedForRemoval()) {
|
||||
return bytesWritten;
|
||||
}
|
||||
|
||||
int maxBytes = Math.min(bytes, writableWindow());
|
||||
while (bytesWritten < maxBytes && hasFrame()) {
|
||||
Frame pendingWrite = peek();
|
||||
if (maxBytes >= pendingWrite.size()) {
|
||||
// Window size is large enough to send entire data frame
|
||||
bytesWritten += pendingWrite.size();
|
||||
pendingWrite.write();
|
||||
} else {
|
||||
// We can send a partial frame
|
||||
Frame partialFrame = pendingWrite.split(maxBytes);
|
||||
bytesWritten += partialFrame.size();
|
||||
partialFrame.write();
|
||||
}
|
||||
|
||||
// Update the threshold.
|
||||
maxBytes = Math.min(bytes - bytesWritten, writableWindow());
|
||||
}
|
||||
return bytesWritten;
|
||||
}
|
||||
|
||||
/**
|
||||
* A wrapper class around the content of a data frame.
|
||||
*/
|
||||
private final class Frame {
|
||||
private final ByteBuf data;
|
||||
private final int padding;
|
||||
private final boolean endStream;
|
||||
private final boolean endSegment;
|
||||
private final boolean compressed;
|
||||
private final FrameWriter writer;
|
||||
private boolean enqueued;
|
||||
|
||||
Frame(ByteBuf data, int padding, boolean endStream, boolean endSegment,
|
||||
boolean compressed, FrameWriter writer) {
|
||||
this.data = data;
|
||||
this.padding = padding;
|
||||
this.endStream = endStream;
|
||||
this.endSegment = endSegment;
|
||||
this.compressed = compressed;
|
||||
this.writer = writer;
|
||||
}
|
||||
|
||||
int size() {
|
||||
return data.readableBytes();
|
||||
}
|
||||
|
||||
void enqueue() {
|
||||
if (!enqueued) {
|
||||
enqueued = true;
|
||||
pendingWriteQueue.offer(this);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes the frame and decrements the stream and connection window sizes. If the frame
|
||||
* is in the pending queue, the written bytes are removed from this branch of the
|
||||
* priority tree.
|
||||
*/
|
||||
void write() throws Http2Exception {
|
||||
int dataLength = data.readableBytes();
|
||||
connectionWindowSize -= dataLength;
|
||||
addAndGetWindow(-dataLength);
|
||||
writer.writeFrame(priority.streamId(), data, padding, endStream, endSegment,
|
||||
compressed);
|
||||
}
|
||||
|
||||
/**
|
||||
* Discards this frame, writing an error. If this frame is in the pending queue, the
|
||||
* unwritten bytes are removed from this branch of the priority tree.
|
||||
*/
|
||||
void writeError(Http2Exception cause) {
|
||||
data.release();
|
||||
writer.setFailure(cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new frame that is a view of this frame's data buffer starting at the
|
||||
* current read index with the given number of bytes. The reader index on the input
|
||||
* frame is then advanced by the number of bytes. The returned frame will not have
|
||||
* end-of-stream set and it will not be automatically placed in the pending queue.
|
||||
*
|
||||
* @param maxBytes the maximum number of bytes that is allowed in the created frame.
|
||||
* @return the partial frame.
|
||||
*/
|
||||
Frame split(int maxBytes) {
|
||||
// TODO: Should padding be included in the chunks or only the last frame?
|
||||
maxBytes = Math.min(maxBytes, data.readableBytes());
|
||||
return new Frame(data.readSlice(maxBytes).retain(), 0, false, false, compressed,
|
||||
writer);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,321 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.DEFAULT_PRIORITY_WEIGHT;
|
||||
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Collections;
|
||||
import java.util.Deque;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link Http2PriorityTree}.
|
||||
*/
|
||||
public class DefaultHttp2PriorityTree<T> implements Http2PriorityTree<T> {
|
||||
|
||||
private final DefaultPriority<T> root = new DefaultPriority<T>(0, (short) 0, null);
|
||||
private Map<Integer, Priority<T>> priorityMap = new HashMap<Integer, Priority<T>>();
|
||||
|
||||
@Override
|
||||
public Priority<T> get(int streamId) {
|
||||
return priorityMap.get(streamId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Priority<T>> iterator() {
|
||||
return Collections.unmodifiableCollection(priorityMap.values()).iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Priority<T> prioritizeUsingDefaults(int streamId, T data) {
|
||||
return prioritize(streamId, 0, DEFAULT_PRIORITY_WEIGHT, false, data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Priority<T> prioritize(int streamId, int parent, short weight, boolean exclusive,
|
||||
T data) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("Stream ID must be > 0");
|
||||
}
|
||||
if (streamId == parent) {
|
||||
throw new IllegalArgumentException("A stream cannot depend on itself");
|
||||
}
|
||||
if (parent < 0) {
|
||||
throw new IllegalArgumentException("Parent stream ID must be >= 0");
|
||||
}
|
||||
if (weight < 1 || weight > 256) {
|
||||
throw new IllegalArgumentException("Invalid weight: " + weight);
|
||||
}
|
||||
|
||||
// Get the parent.
|
||||
DefaultPriority<T> newParent = root;
|
||||
if (parent > 0) {
|
||||
newParent = internalGet(parent);
|
||||
}
|
||||
if (newParent == null) {
|
||||
throw new IllegalArgumentException("Parent priority does not exist: " + parent);
|
||||
}
|
||||
|
||||
DefaultPriority<T> priority = internalGet(streamId);
|
||||
if (priority == null) {
|
||||
// Add a new priority.
|
||||
priority = new DefaultPriority<T>(streamId, weight, data);
|
||||
newParent.addChild(priority, exclusive);
|
||||
priorityMap.put(streamId, priority);
|
||||
return priority;
|
||||
}
|
||||
|
||||
// Already have a priority. Re-prioritize the stream.
|
||||
priority.setWeight(weight);
|
||||
priority.setData(data);
|
||||
|
||||
if (newParent == priority.parent() && !exclusive) {
|
||||
// No changes were made to the tree structure.
|
||||
return priority;
|
||||
}
|
||||
|
||||
// Break off the priority branch from it's current parent.
|
||||
DefaultPriority<T> oldParent = priority.parent();
|
||||
oldParent.removeChildBranch(priority);
|
||||
|
||||
if (newParent.isDescendantOf(priority)) {
|
||||
// Adding a circular dependency (priority<->newParent). Break off the new parent's
|
||||
// branch and add it above this priority.
|
||||
newParent.parent().removeChildBranch(newParent);
|
||||
oldParent.addChild(newParent, false);
|
||||
}
|
||||
|
||||
// Add the priority under the new parent.
|
||||
newParent.addChild(priority, exclusive);
|
||||
return priority;
|
||||
}
|
||||
|
||||
@Override
|
||||
public T remove(int streamId) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("Stream ID must be > 0");
|
||||
}
|
||||
|
||||
// Remove the priority from the map.
|
||||
DefaultPriority<T> priority = internalGet(streamId);
|
||||
if (priority != null) {
|
||||
// Remove it from the tree as well.
|
||||
priority.parent().removeChild(priority);
|
||||
return priority.data();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return priorityMap.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Priority<T> root() {
|
||||
return root;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
|
||||
Deque<DefaultPriority<T>> nodes = new ArrayDeque<DefaultPriority<T>>();
|
||||
nodes.addLast(root);
|
||||
while (!nodes.isEmpty()) {
|
||||
DefaultPriority<T> p = nodes.pop();
|
||||
builder.append(p.streamId()).append("->")
|
||||
.append(p.parent() == null ? "null" : p.parent().streamId).append(", ");
|
||||
for (DefaultPriority<T> child : p.children) {
|
||||
nodes.addLast(child);
|
||||
}
|
||||
}
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
private DefaultPriority<T> internalGet(int streamId) {
|
||||
return (DefaultPriority<T>) priorityMap.get(streamId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal implementation of {@link Priority}.
|
||||
*/
|
||||
private static final class DefaultPriority<T> implements Priority<T> {
|
||||
private final int streamId;
|
||||
private short weight;
|
||||
private T data;
|
||||
private Set<DefaultPriority<T>> children = newChildSet();
|
||||
|
||||
private DefaultPriority<T> parent;
|
||||
private int totalChildWeights;
|
||||
|
||||
DefaultPriority(int streamId, short weight, T data) {
|
||||
this.streamId = streamId;
|
||||
this.weight = weight;
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRoot() {
|
||||
return parent == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int streamId() {
|
||||
return streamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short weight() {
|
||||
return weight;
|
||||
}
|
||||
|
||||
@Override
|
||||
public T data() {
|
||||
return data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int totalChildWeights() {
|
||||
return totalChildWeights;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultPriority<T> parent() {
|
||||
return parent;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDescendantOf(Priority<T> priority) {
|
||||
Priority<T> next = parent;
|
||||
while (next != null) {
|
||||
if (next == priority) {
|
||||
return true;
|
||||
}
|
||||
next = next.parent();
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLeaf() {
|
||||
return numChildren() == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numChildren() {
|
||||
return children.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<? extends Priority<T>> children() {
|
||||
return Collections.unmodifiableSet(children);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasChild(int streamId) {
|
||||
return getChild(streamId) != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Priority<T> getChild(int streamId) {
|
||||
for (DefaultPriority<T> child : children) {
|
||||
if (child.streamId() == streamId) {
|
||||
return child;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
void setWeight(short weight) {
|
||||
if (parent != null && weight != this.weight) {
|
||||
int delta = weight - this.weight;
|
||||
parent.totalChildWeights += delta;
|
||||
}
|
||||
this.weight = weight;
|
||||
}
|
||||
|
||||
void setData(T data) {
|
||||
this.data = data;
|
||||
}
|
||||
|
||||
Set<DefaultPriority<T>> removeAllChildren() {
|
||||
if (children.isEmpty()) {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
totalChildWeights = 0;
|
||||
Set<DefaultPriority<T>> prevChildren = children;
|
||||
children = newChildSet();
|
||||
return prevChildren;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a child to this priority. If exclusive is set, any children of this node are moved
|
||||
* to being dependent on the child.
|
||||
*/
|
||||
void addChild(DefaultPriority<T> child, boolean exclusive) {
|
||||
if (exclusive) {
|
||||
// If it was requested that this child be the exclusive dependency of this node,
|
||||
// move any previous children to the child node, becoming grand children
|
||||
// of this node.
|
||||
for (DefaultPriority<T> grandchild : removeAllChildren()) {
|
||||
child.addChild(grandchild, false);
|
||||
}
|
||||
}
|
||||
|
||||
child.parent = this;
|
||||
if (children.add(child)) {
|
||||
totalChildWeights += child.weight();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the child priority and moves any of its dependencies to being direct dependencies
|
||||
* on this node.
|
||||
*/
|
||||
void removeChild(DefaultPriority<T> child) {
|
||||
if (children.remove(child)) {
|
||||
child.parent = null;
|
||||
totalChildWeights -= child.weight();
|
||||
|
||||
// Move up any grand children to be directly dependent on this node.
|
||||
for (DefaultPriority<T> grandchild : child.children) {
|
||||
addChild(grandchild, false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the child priority but unlike {@link #removeChild}, leaves its branch unaffected.
|
||||
*/
|
||||
void removeChildBranch(DefaultPriority<T> child) {
|
||||
if (children.remove(child)) {
|
||||
child.parent = null;
|
||||
totalChildWeights -= child.weight();
|
||||
}
|
||||
}
|
||||
|
||||
private static <T> Set<DefaultPriority<T>> newChildSet() {
|
||||
return new LinkedHashSet<DefaultPriority<T>>(2);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,201 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
|
||||
/**
|
||||
* Helper class that facilitates use of {@link AbstractHttp2ConnectionHandler} in compositional
|
||||
* models, rather than having to subclass it directly.
|
||||
* <p>
|
||||
* Exposes all {@code writeXXX} methods as public and delegates all frame read events to a provided
|
||||
* {@link Http2FrameObserver}.
|
||||
* <p>
|
||||
* The {@link #channelActive} and {@link #handlerAdded} should called when appropriate to ensure
|
||||
* that the initial SETTINGS frame is sent to the remote endpoint.
|
||||
*/
|
||||
public class DelegatingHttp2ConnectionHandler extends AbstractHttp2ConnectionHandler {
|
||||
private final Http2FrameObserver observer;
|
||||
|
||||
public DelegatingHttp2ConnectionHandler(boolean server, Http2FrameObserver observer) {
|
||||
super(server);
|
||||
this.observer = observer;
|
||||
}
|
||||
|
||||
public DelegatingHttp2ConnectionHandler(boolean server, boolean allowCompression,
|
||||
Http2FrameObserver observer) {
|
||||
super(server, allowCompression);
|
||||
this.observer = observer;
|
||||
}
|
||||
|
||||
public DelegatingHttp2ConnectionHandler(Http2Connection connection,
|
||||
Http2FrameReader frameReader, Http2FrameWriter frameWriter,
|
||||
Http2InboundFlowController inboundFlow, Http2OutboundFlowController outboundFlow,
|
||||
Http2FrameObserver observer) {
|
||||
super(connection, frameReader, frameWriter, inboundFlow, outboundFlow);
|
||||
this.observer = observer;
|
||||
}
|
||||
|
||||
public DelegatingHttp2ConnectionHandler(Http2Connection connection, Http2FrameObserver observer) {
|
||||
super(connection);
|
||||
this.observer = observer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeData(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
|
||||
ByteBuf data, int padding, boolean endStream, boolean endSegment, boolean compressed)
|
||||
throws Http2Exception {
|
||||
return super.writeData(ctx, promise, streamId, data, padding, endStream, endSegment,
|
||||
compressed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception {
|
||||
return super.writeHeaders(ctx, promise, streamId, headers, padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, Http2Headers headers, int streamDependency, short weight,
|
||||
boolean exclusive, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception {
|
||||
return super.writeHeaders(ctx, promise, streamId, headers, streamDependency, weight,
|
||||
exclusive, padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writePriority(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int streamDependency, short weight, boolean exclusive)
|
||||
throws Http2Exception {
|
||||
return super.writePriority(ctx, promise, streamId, streamDependency, weight, exclusive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeRstStream(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, long errorCode) {
|
||||
return super.writeRstStream(ctx, promise, streamId, errorCode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeSettings(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
Http2Settings settings) throws Http2Exception {
|
||||
return super.writeSettings(ctx, promise, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writePing(ChannelHandlerContext ctx, ChannelPromise promise, ByteBuf data)
|
||||
throws Http2Exception {
|
||||
return super.writePing(ctx, promise, data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writePushPromise(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int promisedStreamId, Http2Headers headers, int padding)
|
||||
throws Http2Exception {
|
||||
return super.writePushPromise(ctx, promise, streamId, promisedStreamId, headers, padding);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeAltSvc(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, long maxAge, int port, ByteBuf protocolId, String host, String origin)
|
||||
throws Http2Exception {
|
||||
return super.writeAltSvc(ctx, promise, streamId, maxAge, port, protocolId, host, origin);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onDataRead(int streamId, ByteBuf data, int padding, boolean endOfStream,
|
||||
boolean endOfSegment, boolean compressed) throws Http2Exception {
|
||||
observer.onDataRead(streamId, data, padding, endOfStream, endOfSegment, compressed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onHeadersRead(int streamId, Http2Headers headers, int padding, boolean endStream,
|
||||
boolean endSegment) throws Http2Exception {
|
||||
observer.onHeadersRead(streamId, headers, padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onHeadersRead(int streamId, Http2Headers headers, int streamDependency,
|
||||
short weight, boolean exclusive, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception {
|
||||
observer.onHeadersRead(streamId, headers, streamDependency, weight, exclusive, padding,
|
||||
endStream, endSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPriorityRead(int streamId, int streamDependency, short weight, boolean exclusive)
|
||||
throws Http2Exception {
|
||||
observer.onPriorityRead(streamId, streamDependency, weight, exclusive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onRstStreamRead(int streamId, long errorCode) throws Http2Exception {
|
||||
observer.onRstStreamRead(streamId, errorCode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onSettingsAckRead() throws Http2Exception {
|
||||
observer.onSettingsAckRead();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onSettingsRead(Http2Settings settings) throws Http2Exception {
|
||||
observer.onSettingsRead(settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPingRead(ByteBuf data) throws Http2Exception {
|
||||
observer.onPingRead(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPingAckRead(ByteBuf data) throws Http2Exception {
|
||||
observer.onPingAckRead(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPushPromiseRead(int streamId, int promisedStreamId, Http2Headers headers,
|
||||
int padding) throws Http2Exception {
|
||||
observer.onPushPromiseRead(streamId, promisedStreamId, headers, padding);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onGoAwayRead(int lastStreamId, long errorCode, ByteBuf debugData)
|
||||
throws Http2Exception {
|
||||
observer.onGoAwayRead(lastStreamId, errorCode, debugData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onWindowUpdateRead(int streamId, int windowSizeIncrement) throws Http2Exception {
|
||||
observer.onWindowUpdateRead(streamId, windowSizeIncrement);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onAltSvcRead(int streamId, long maxAge, int port, ByteBuf protocolId, String host,
|
||||
String origin) throws Http2Exception {
|
||||
observer.onAltSvcRead(streamId, maxAge, port, protocolId, host, origin);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onBlockedRead(int streamId) throws Http2Exception {
|
||||
observer.onBlockedRead(streamId);
|
||||
}
|
||||
}
|
@ -0,0 +1,141 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2Error.INTERNAL_ERROR;
|
||||
import static io.netty.handler.codec.http2.Http2Exception.format;
|
||||
import static io.netty.util.CharsetUtil.UTF_8;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
|
||||
/**
|
||||
* Constants and utility method used for encoding/decoding HTTP2 frames.
|
||||
*/
|
||||
public final class Http2CodecUtil {
|
||||
private static final ByteBuf CONNECTION_PREFACE_BUF = Unpooled.unmodifiableBuffer(Unpooled
|
||||
.copiedBuffer("PRI * HTTP/2.0\r\n\r\nSM\r\n\r\n", UTF_8));
|
||||
private static final ByteBuf EMPTY_PING_BUF = Unpooled.unmodifiableBuffer(Unpooled
|
||||
.copiedBuffer(new byte[8]));
|
||||
|
||||
public static final int CONNECTION_STREAM_ID = 0;
|
||||
public static final int MAX_FRAME_PAYLOAD_LENGTH = 16383;
|
||||
public static final int PING_FRAME_PAYLOAD_LENGTH = 8;
|
||||
public static final short MAX_UNSIGNED_BYTE = 0xFF;
|
||||
public static final int MAX_UNSIGNED_SHORT = 0xFFFF;
|
||||
public static final long MAX_UNSIGNED_INT = 0xFFFFFFFFL;
|
||||
public static final int FRAME_HEADER_LENGTH = 8;
|
||||
public static final int FRAME_LENGTH_MASK = 0x3FFF;
|
||||
public static final int SETTING_ENTRY_LENGTH = 5;
|
||||
public static final int PRIORITY_ENTRY_LENGTH = 5;
|
||||
public static final int INT_FIELD_LENGTH = 4;
|
||||
|
||||
public static final short SETTINGS_HEADER_TABLE_SIZE = 1;
|
||||
public static final short SETTINGS_ENABLE_PUSH = 2;
|
||||
public static final short SETTINGS_MAX_CONCURRENT_STREAMS = 3;
|
||||
public static final short SETTINGS_INITIAL_WINDOW_SIZE = 4;
|
||||
public static final short SETTINGS_COMPRESS_DATA = 5;
|
||||
|
||||
public static final int DEFAULT_FLOW_CONTROL_WINDOW_SIZE = 65535;
|
||||
public static final short DEFAULT_PRIORITY_WEIGHT = 16;
|
||||
public static final int DEFAULT_HEADER_TABLE_SIZE = 4096;
|
||||
public static final int DEFAULT_MAX_HEADER_SIZE = 8192;
|
||||
|
||||
/**
|
||||
* Returns a buffer containing the the {@link #CONNECTION_PREFACE}.
|
||||
*/
|
||||
public static ByteBuf connectionPrefaceBuf() {
|
||||
// Return a duplicate so that modifications to the reader index will not affect the original
|
||||
// buffer.
|
||||
return CONNECTION_PREFACE_BUF.duplicate().retain();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a buffer filled with all zeros that is the appropriate length for a PING frame.
|
||||
*/
|
||||
public static ByteBuf emptyPingBuf() {
|
||||
// Return a duplicate so that modifications to the reader index will not affect the original
|
||||
// buffer.
|
||||
return EMPTY_PING_BUF.duplicate().retain();
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts the given cause to a {@link Http2Exception} if it isn't already.
|
||||
*/
|
||||
public static Http2Exception toHttp2Exception(Throwable cause) {
|
||||
if (cause instanceof Http2Exception) {
|
||||
return (Http2Exception) cause;
|
||||
}
|
||||
String msg = cause != null ? cause.getMessage() : "Failed writing the data frame.";
|
||||
return format(INTERNAL_ERROR, msg);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a buffer containing the error message from the given exception. If the cause is
|
||||
* {@code null} returns an empty buffer.
|
||||
*/
|
||||
public static ByteBuf toByteBuf(ChannelHandlerContext ctx, Throwable cause) {
|
||||
if (cause == null || cause.getMessage() == null) {
|
||||
return Unpooled.EMPTY_BUFFER;
|
||||
}
|
||||
|
||||
// Create the debug message.
|
||||
byte[] msg = cause.getMessage().getBytes(UTF_8);
|
||||
ByteBuf debugData = ctx.alloc().buffer(msg.length);
|
||||
debugData.writeBytes(msg);
|
||||
return debugData;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a big-endian (31-bit) integer from the buffer.
|
||||
*/
|
||||
public static int readUnsignedInt(ByteBuf buf) {
|
||||
return (buf.readByte() & 0x7F) << 24 | (buf.readByte() & 0xFF) << 16
|
||||
| (buf.readByte() & 0xFF) << 8 | buf.readByte() & 0xFF;
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a big-endian (32-bit) unsigned integer to the buffer.
|
||||
*/
|
||||
public static void writeUnsignedInt(long value, ByteBuf out) {
|
||||
out.writeByte((int) ((value >> 24) & 0xFF));
|
||||
out.writeByte((int) ((value >> 16) & 0xFF));
|
||||
out.writeByte((int) ((value >> 8) & 0xFF));
|
||||
out.writeByte((int) ((value & 0xFF)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a big-endian (16-bit) unsigned integer to the buffer.
|
||||
*/
|
||||
public static void writeUnsignedShort(int value, ByteBuf out) {
|
||||
out.writeByte((int) ((value >> 8) & 0xFF));
|
||||
out.writeByte((int) ((value & 0xFF)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Fails the given promise with the cause and then re-throws the cause.
|
||||
*/
|
||||
public static <T extends Throwable> T failAndThrow(ChannelPromise promise, T cause) throws T {
|
||||
if (!promise.isDone()) {
|
||||
promise.setFailure(cause);
|
||||
}
|
||||
throw cause;
|
||||
}
|
||||
|
||||
private Http2CodecUtil() {
|
||||
}
|
||||
}
|
@ -13,11 +13,7 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.connection;
|
||||
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
@ -29,21 +25,18 @@ public interface Http2Connection {
|
||||
interface Endpoint {
|
||||
|
||||
/**
|
||||
* Creates a stream initiated by this endpoint and notifies all listeners. This could fail for
|
||||
* the following reasons:
|
||||
* Creates a stream initiated by this endpoint. This could fail for the following reasons:
|
||||
* <p/>
|
||||
* - The requested stream ID is not the next sequential ID for this endpoint. <br>
|
||||
* - The stream already exists. <br>
|
||||
* - The number of concurrent streams is above the allowed threshold for this endpoint. <br>
|
||||
* - The connection is marked as going away}. <br>
|
||||
* - The provided priority is < 0.
|
||||
*
|
||||
* @param streamId The ID of the stream
|
||||
* @param priority the priority of the stream
|
||||
* @param streamId The ID of the stream
|
||||
* @param halfClosed if true, the stream is created in the half-closed state with respect to
|
||||
* this endpoint. Otherwise it's created in the open state.
|
||||
* this endpoint. Otherwise it's created in the open state.
|
||||
*/
|
||||
Http2Stream createStream(int streamId, int priority, boolean halfClosed) throws Http2Exception;
|
||||
Http2Stream createStream(int streamId, boolean halfClosed) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Creates a push stream in the reserved state for this endpoint and notifies all listeners.
|
||||
@ -53,39 +46,49 @@ public interface Http2Connection {
|
||||
* - The requested stream ID is not the next sequential stream ID for this endpoint. <br>
|
||||
* - The number of concurrent streams is above the allowed threshold for this endpoint. <br>
|
||||
* - The connection is marked as going away. <br>
|
||||
* - The parent stream ID does not exist or is not open from the side sending the push promise.
|
||||
* <br>
|
||||
* - The parent stream ID does not exist or is not open from the side sending the push
|
||||
* promise. <br>
|
||||
* - Could not set a valid priority for the new stream.
|
||||
*
|
||||
* @param streamId the ID of the push stream
|
||||
* @param parent the parent stream used to initiate the push stream.
|
||||
* @param parent the parent stream used to initiate the push stream.
|
||||
*/
|
||||
Http2Stream reservePushStream(int streamId, Http2Stream parent) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Sets whether server push is allowed to this endpoint.
|
||||
*/
|
||||
void setPushToAllowed(boolean allow);
|
||||
void allowPushTo(boolean allow);
|
||||
|
||||
/**
|
||||
* Gets whether or not server push is allowed to this endpoint.
|
||||
*/
|
||||
boolean isPushToAllowed();
|
||||
boolean allowPushTo();
|
||||
|
||||
/**
|
||||
* Gets the maximum number of concurrent streams allowed by this endpoint.
|
||||
*/
|
||||
int getMaxStreams();
|
||||
int maxStreams();
|
||||
|
||||
/**
|
||||
* Sets the maximum number of concurrent streams allowed by this endpoint.
|
||||
*/
|
||||
void setMaxStreams(int maxStreams);
|
||||
void maxStreams(int maxStreams);
|
||||
|
||||
/**
|
||||
* Indicates whether or not this endpoint allows compression.
|
||||
*/
|
||||
boolean allowCompressedData();
|
||||
|
||||
/**
|
||||
* Sets whether or not this endpoint allows compression.
|
||||
*/
|
||||
void allowCompressedData(boolean allow);
|
||||
|
||||
/**
|
||||
* Gets the ID of the stream last successfully created by this endpoint.
|
||||
*/
|
||||
int getLastStreamCreated();
|
||||
int lastStreamCreated();
|
||||
|
||||
/**
|
||||
* Gets the {@link Endpoint} opposite this one.
|
||||
@ -94,45 +97,30 @@ public interface Http2Connection {
|
||||
}
|
||||
|
||||
/**
|
||||
* A listener of the connection for stream events.
|
||||
* Indicates whether or not this endpoint is the server-side of the connection.
|
||||
*/
|
||||
interface Listener {
|
||||
/**
|
||||
* Called when a new stream with the given ID is created.
|
||||
*/
|
||||
void streamCreated(int streamId);
|
||||
|
||||
/**
|
||||
* Called when the stream with the given ID is closed.
|
||||
*/
|
||||
void streamClosed(int streamId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a listener of this connection.
|
||||
*/
|
||||
void addListener(Listener listener);
|
||||
|
||||
/**
|
||||
* Removes a listener of this connection.
|
||||
*/
|
||||
void removeListener(Listener listener);
|
||||
boolean isServer();
|
||||
|
||||
/**
|
||||
* Attempts to get the stream for the given ID. If it doesn't exist, throws.
|
||||
*/
|
||||
Http2Stream getStreamOrFail(int streamId) throws Http2Exception;
|
||||
Http2Stream requireStream(int streamId) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Gets the stream if it exists. If not, returns {@code null}.
|
||||
*/
|
||||
Http2Stream getStream(int streamId);
|
||||
Http2Stream stream(int streamId);
|
||||
|
||||
/**
|
||||
* Gets the number of active streams in this connection.
|
||||
*/
|
||||
int numActiveStreams();
|
||||
|
||||
/**
|
||||
* Gets all streams that are currently either open or half-closed. The returned collection is
|
||||
* sorted by priority.
|
||||
*/
|
||||
Collection<Http2Stream> getActiveStreams();
|
||||
Collection<Http2Stream> activeStreams();
|
||||
|
||||
/**
|
||||
* Gets a view of this connection from the local {@link Endpoint}.
|
||||
@ -148,7 +136,7 @@ public interface Http2Connection {
|
||||
* Marks that a GoAway frame has been sent on this connection. After calling this, both
|
||||
* {@link #isGoAwaySent()} and {@link #isGoAway()} will be {@code true}.
|
||||
*/
|
||||
void sendGoAway(ChannelHandlerContext ctx, ChannelPromise promise, Http2Exception cause);
|
||||
void goAwaySent();
|
||||
|
||||
/**
|
||||
* Marks that a GoAway frame has been received on this connection. After calling this, both
|
@ -13,10 +13,10 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
/**
|
||||
* All error codes identified by the HTTP2 spec.
|
||||
* All error codes identified by the HTTP/2 spec.
|
||||
*/
|
||||
public enum Http2Error {
|
||||
NO_ERROR(0),
|
||||
@ -39,7 +39,10 @@ public enum Http2Error {
|
||||
this.code = code;
|
||||
}
|
||||
|
||||
public int getCode() {
|
||||
/**
|
||||
* Gets the code for this error used on the wire.
|
||||
*/
|
||||
public int code() {
|
||||
return code;
|
||||
}
|
||||
}
|
@ -13,7 +13,7 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
/**
|
||||
* Exception thrown when an HTTP2 error was encountered.
|
||||
@ -32,7 +32,12 @@ public class Http2Exception extends Exception {
|
||||
this.error = error;
|
||||
}
|
||||
|
||||
public Http2Error getError() {
|
||||
public Http2Exception(Http2Error error, String message, Throwable cause) {
|
||||
super(message, cause);
|
||||
this.error = error;
|
||||
}
|
||||
|
||||
public Http2Error error() {
|
||||
return error;
|
||||
}
|
||||
|
@ -0,0 +1,486 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
|
||||
/**
|
||||
* Provides utility methods for accessing specific flags as defined by the HTTP/2 spec.
|
||||
*/
|
||||
public class Http2Flags {
|
||||
public static final Http2Flags EMPTY = new Http2Flags();
|
||||
public static final Http2Flags ACK_ONLY = new Builder().ack(true).build();
|
||||
|
||||
public static final short END_STREAM = 0x1;
|
||||
public static final short END_SEGMENT = 0x2;
|
||||
public static final short END_HEADERS = 0x4;
|
||||
public static final short ACK = 0x1;
|
||||
public static final short PAD_LOW = 0x8;
|
||||
public static final short PAD_HIGH = 0x10;
|
||||
public static final short PRIORITY = 0x20;
|
||||
public static final short COMPRESSED = 0x20;
|
||||
|
||||
private final short value;
|
||||
|
||||
private Http2Flags() {
|
||||
this((short) 0);
|
||||
}
|
||||
|
||||
public Http2Flags(short value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the underlying flags value.
|
||||
*/
|
||||
public short value() {
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the {@link #END_STREAM} flag is set. Only applies to DATA and HEADERS
|
||||
* frames.
|
||||
*/
|
||||
public boolean endOfStream() {
|
||||
return endOfStream(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the {@link #END_SEGMENT} flag is set. Only applies to DATA and HEADERS
|
||||
* frames.
|
||||
*/
|
||||
public boolean endOfSegment() {
|
||||
return endOfSegment(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the {@link #END_HEADERS} flag is set. Only applies for HEADERS,
|
||||
* PUSH_PROMISE, and CONTINUATION frames.
|
||||
*/
|
||||
public boolean endOfHeaders() {
|
||||
return endOfHeaders(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the flag is set indicating the presence of the exclusive, stream
|
||||
* dependency, and weight fields in a HEADERS frame.
|
||||
*/
|
||||
public boolean priorityPresent() {
|
||||
return priorityPresent(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the flag is set indicating that this frame is an ACK. Only applies for
|
||||
* SETTINGS and PING frames.
|
||||
*/
|
||||
public boolean ack() {
|
||||
return ack(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* For DATA frames, indicates that the data is compressed using gzip compression.
|
||||
*/
|
||||
public boolean compressed() {
|
||||
return compressed(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* For frames that include padding, indicates if the {@link #PAD_LOW} field is present. Only
|
||||
* applies to DATA, HEADERS, PUSH_PROMISE and CONTINUATION frames.
|
||||
*/
|
||||
public boolean padLowPresent() {
|
||||
return padLowPresent(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* For frames that include padding, indicates if the {@link #PAD_HIGH} field is present. Only
|
||||
* applies to DATA, HEADERS, PUSH_PROMISE and CONTINUATION frames.
|
||||
*/
|
||||
public boolean padHighPresent() {
|
||||
return padHighPresent(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether the padding flags are set properly. If pad high is set, pad low must also
|
||||
* be set.
|
||||
*/
|
||||
public boolean isPaddingLengthValid() {
|
||||
return isPaddingLengthValid(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of bytes expected in the padding length field of the payload. This is
|
||||
* determined by the {@link #padHighPresent()} and {@link #padLowPresent()} flags.
|
||||
*/
|
||||
public int getNumPaddingLengthBytes() {
|
||||
return getNumPaddingLengthBytes(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of bytes expected for the priority fields of the payload. This is determined
|
||||
* by the {@link #priorityPresent()} flag.
|
||||
*/
|
||||
public int getNumPriorityBytes() {
|
||||
return getNumPriorityBytes(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the variable-length padding length field from the payload.
|
||||
*/
|
||||
public int readPaddingLength(ByteBuf payload) {
|
||||
return readPaddingLength(value, payload);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + value;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
Http2Flags other = (Http2Flags) obj;
|
||||
if (value != other.value) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("value = ").append(value).append(" (");
|
||||
if (ack()) {
|
||||
builder.append("ACK,");
|
||||
}
|
||||
if (endOfHeaders()) {
|
||||
builder.append("END_OF_HEADERS,");
|
||||
}
|
||||
if (endOfStream()) {
|
||||
builder.append("END_OF_STREAM,");
|
||||
}
|
||||
if (priorityPresent()) {
|
||||
builder.append("PRIORITY_PRESENT,");
|
||||
}
|
||||
if (endOfSegment()) {
|
||||
builder.append("END_OF_SEGMENT,");
|
||||
}
|
||||
if (padHighPresent()) {
|
||||
builder.append("PAD_HIGH,");
|
||||
}
|
||||
if (padLowPresent()) {
|
||||
builder.append("PAD_LOW,");
|
||||
}
|
||||
builder.append(")");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Shortcut for creating a new {@link Builder} instance.
|
||||
*/
|
||||
public static Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder for instances of {@link Http2Flags}.
|
||||
*/
|
||||
public static final class Builder {
|
||||
private short value;
|
||||
|
||||
/**
|
||||
* Sets the {@link #END_STREAM} flag.
|
||||
*/
|
||||
public Builder endOfStream(boolean endOfStream) {
|
||||
return setFlag(endOfStream, END_STREAM);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link #END_SEGMENT} flag.
|
||||
*/
|
||||
public Builder endOfSegment(boolean endOfSegment) {
|
||||
return setFlag(endOfSegment, END_SEGMENT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link #END_HEADERS} flag.
|
||||
*/
|
||||
public Builder endOfHeaders(boolean endOfHeaders) {
|
||||
return setFlag(endOfHeaders, END_HEADERS);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link #PRIORITY} flag.
|
||||
*/
|
||||
public Builder priorityPresent(boolean priorityPresent) {
|
||||
return setFlag(priorityPresent, PRIORITY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link #ACK} flag.
|
||||
*/
|
||||
public Builder ack(boolean ack) {
|
||||
return setFlag(ack, ACK);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link #COMPRESSED} flag.
|
||||
*/
|
||||
public Builder compressed(boolean compressed) {
|
||||
return setFlag(compressed, COMPRESSED);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the padding flags in the given flags value as appropriate based on the padding
|
||||
* length.
|
||||
*/
|
||||
public Builder setPaddingFlags(int paddingLength) {
|
||||
if (paddingLength > 255) {
|
||||
value |= PAD_HIGH;
|
||||
}
|
||||
if (paddingLength > 0) {
|
||||
value |= PAD_LOW;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the {@link #END_STREAM} flag is set. Only applies to DATA and HEADERS
|
||||
* frames.
|
||||
*/
|
||||
public boolean endOfStream() {
|
||||
return Http2Flags.endOfStream(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the {@link #END_SEGMENT} flag is set. Only applies to DATA and HEADERS
|
||||
* frames.
|
||||
*/
|
||||
public boolean endOfSegment() {
|
||||
return Http2Flags.endOfSegment(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the {@link #END_HEADERS} flag is set. Only applies for HEADERS,
|
||||
* PUSH_PROMISE, and CONTINUATION frames.
|
||||
*/
|
||||
public boolean endOfHeaders() {
|
||||
return Http2Flags.endOfHeaders(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the flag is set indicating the presence of the exclusive, stream
|
||||
* dependency, and weight fields in a HEADERS frame.
|
||||
*/
|
||||
public boolean priorityPresent() {
|
||||
return Http2Flags.priorityPresent(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the flag is set indicating that this frame is an ACK. Only applies for
|
||||
* SETTINGS and PING frames.
|
||||
*/
|
||||
public boolean ack() {
|
||||
return Http2Flags.ack(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* For DATA frames, indicates that the data is compressed using gzip compression.
|
||||
*/
|
||||
public boolean compressed() {
|
||||
return Http2Flags.compressed(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* For frames that include padding, indicates if the {@link #PAD_LOW} field is present. Only
|
||||
* applies to DATA, HEADERS, PUSH_PROMISE and CONTINUATION frames.
|
||||
*/
|
||||
public boolean padLowPresent() {
|
||||
return Http2Flags.padLowPresent(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* For frames that include padding, indicates if the {@link #PAD_HIGH} field is present. Only
|
||||
* applies to DATA, HEADERS, PUSH_PROMISE and CONTINUATION frames.
|
||||
*/
|
||||
public boolean padHighPresent() {
|
||||
return Http2Flags.padHighPresent(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether the padding flags are set properly. If pad high is set, pad low must also
|
||||
* be set.
|
||||
*/
|
||||
public boolean isPaddingLengthValid() {
|
||||
return Http2Flags.isPaddingLengthValid(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of bytes expected in the padding length field of the payload. This is
|
||||
* determined by the {@link #padHighPresent()} and {@link #padLowPresent()} flags.
|
||||
*/
|
||||
public int getNumPaddingLengthBytes() {
|
||||
return Http2Flags.getNumPaddingLengthBytes(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of bytes expected for the priority fields of the payload. This is determined
|
||||
* by the {@link #priorityPresent()} flag.
|
||||
*/
|
||||
public int getNumPriorityBytes() {
|
||||
return Http2Flags.getNumPriorityBytes(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the variable-length padding length field from the payload.
|
||||
*/
|
||||
public int readPaddingLength(ByteBuf payload) {
|
||||
return Http2Flags.readPaddingLength(value, payload);
|
||||
}
|
||||
/**
|
||||
* Builds a new {@link Http2Flags} instance.
|
||||
*/
|
||||
public Http2Flags build() {
|
||||
return new Http2Flags(value);
|
||||
}
|
||||
|
||||
private Builder setFlag(boolean on, short mask) {
|
||||
if (on) {
|
||||
value |= mask;
|
||||
} else {
|
||||
value &= ~mask;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the {@link #END_STREAM} flag is set. Only applies to DATA and HEADERS
|
||||
* frames.
|
||||
*/
|
||||
public static boolean endOfStream(short value) {
|
||||
return isFlagSet(value, END_STREAM);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the {@link #END_SEGMENT} flag is set. Only applies to DATA and HEADERS
|
||||
* frames.
|
||||
*/
|
||||
public static boolean endOfSegment(short value) {
|
||||
return isFlagSet(value, END_SEGMENT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the {@link #END_HEADERS} flag is set. Only applies for HEADERS,
|
||||
* PUSH_PROMISE, and CONTINUATION frames.
|
||||
*/
|
||||
public static boolean endOfHeaders(short value) {
|
||||
return isFlagSet(value, END_HEADERS);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the flag is set indicating the presence of the exclusive, stream
|
||||
* dependency, and weight fields in a HEADERS frame.
|
||||
*/
|
||||
public static boolean priorityPresent(short value) {
|
||||
return isFlagSet(value, PRIORITY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the flag is set indicating that this frame is an ACK. Only applies for
|
||||
* SETTINGS and PING frames.
|
||||
*/
|
||||
public static boolean ack(short value) {
|
||||
return isFlagSet(value, ACK);
|
||||
}
|
||||
|
||||
/**
|
||||
* For DATA frames, indicates that the data is compressed using gzip compression.
|
||||
*/
|
||||
public static boolean compressed(short value) {
|
||||
return isFlagSet(value, COMPRESSED);
|
||||
}
|
||||
|
||||
/**
|
||||
* For frames that include padding, indicates if the {@link #PAD_LOW} field is present. Only
|
||||
* applies to DATA, HEADERS, PUSH_PROMISE and CONTINUATION frames.
|
||||
*/
|
||||
public static boolean padLowPresent(short value) {
|
||||
return isFlagSet(value, PAD_LOW);
|
||||
}
|
||||
|
||||
/**
|
||||
* For frames that include padding, indicates if the {@link #PAD_HIGH} field is present. Only
|
||||
* applies to DATA, HEADERS, PUSH_PROMISE and CONTINUATION frames.
|
||||
*/
|
||||
public static boolean padHighPresent(short value) {
|
||||
return isFlagSet(value, PAD_HIGH);
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether the padding flags are set properly. If pad high is set, pad low must also
|
||||
* be set.
|
||||
*/
|
||||
public static boolean isPaddingLengthValid(short value) {
|
||||
return padHighPresent(value) ? padLowPresent(value) : true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of bytes expected in the padding length field of the payload. This is
|
||||
* determined by the {@link #padHighPresent()} and {@link #padLowPresent()} flags.
|
||||
*/
|
||||
public static int getNumPaddingLengthBytes(short value) {
|
||||
return (padHighPresent(value) ? 1 : 0) + (padLowPresent(value) ? 1 : 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of bytes expected for the priority fields of the payload. This is determined
|
||||
* by the {@link #priorityPresent()} flag.
|
||||
*/
|
||||
public static int getNumPriorityBytes(short value) {
|
||||
return priorityPresent(value) ? 5 : 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the variable-length padding length field from the payload.
|
||||
*/
|
||||
public static int readPaddingLength(short value, ByteBuf payload) {
|
||||
int paddingLength = 0;
|
||||
if (padHighPresent(value)) {
|
||||
paddingLength += payload.readUnsignedByte() * 256;
|
||||
}
|
||||
if (padLowPresent(value)) {
|
||||
paddingLength += payload.readUnsignedByte();
|
||||
}
|
||||
|
||||
return paddingLength;
|
||||
}
|
||||
|
||||
private static boolean isFlagSet(short value, short mask) {
|
||||
return (value & mask) != 0;
|
||||
}
|
||||
}
|
@ -0,0 +1,137 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelHandlerAdapter;
|
||||
import io.netty.util.internal.logging.InternalLogLevel;
|
||||
import io.netty.util.internal.logging.InternalLogger;
|
||||
import io.netty.util.internal.logging.InternalLoggerFactory;
|
||||
|
||||
/**
|
||||
* Logs HTTP2 frames for debugging purposes.
|
||||
*/
|
||||
public class Http2FrameLogger extends ChannelHandlerAdapter {
|
||||
|
||||
public enum Direction {
|
||||
INBOUND,
|
||||
OUTBOUND
|
||||
}
|
||||
|
||||
private final InternalLogger logger;
|
||||
private final InternalLogLevel level;
|
||||
|
||||
public Http2FrameLogger(InternalLogLevel level) {
|
||||
this(level, InternalLoggerFactory.getInstance(Http2FrameLogger.class));
|
||||
}
|
||||
|
||||
public Http2FrameLogger(InternalLogLevel level, InternalLogger logger) {
|
||||
if (level == null) {
|
||||
throw new NullPointerException("level");
|
||||
}
|
||||
if (logger == null) {
|
||||
throw new NullPointerException("logger");
|
||||
}
|
||||
this.level = level;
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
public void logData(Direction direction, int streamId, ByteBuf data, int padding,
|
||||
boolean endStream, boolean endSegment, boolean compressed) {
|
||||
log(direction,
|
||||
"DATA: streamId=%d, dataLen=%d, padding=%d, endStream=%b, endSegment=%b, compressed=%b",
|
||||
streamId, data.readableBytes(), padding, endStream, endSegment, compressed);
|
||||
}
|
||||
|
||||
public void logHeaders(Direction direction, int streamId, Http2Headers headers, int padding,
|
||||
boolean endStream, boolean endSegment) {
|
||||
log(direction, "HEADERS: steramId:%d, headers=%s, padding=%d, endStream=%b, endSegment=%b",
|
||||
streamId, headers, padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
public void logHeaders(Direction direction, int streamId, Http2Headers headers,
|
||||
int streamDependency, short weight, boolean exclusive, int padding, boolean endStream,
|
||||
boolean endSegment) {
|
||||
log(direction,
|
||||
"HEADERS: steramId:%d, headers=%s, streamDependency=%d, weight=%d, exclusive=%b, "
|
||||
+ "padding=%d, endStream=%b, endSegment=%b", streamId, headers,
|
||||
streamDependency, weight, exclusive, padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
public void logPriority(Direction direction, int streamId, int streamDependency, short weight,
|
||||
boolean exclusive) {
|
||||
log(direction, "PRIORITY: streamId=%d, streamDependency=%d, weight=%d, exclusive=%b",
|
||||
streamId, streamDependency, weight, exclusive);
|
||||
}
|
||||
|
||||
public void logRstStream(Direction direction, int streamId, long errorCode) {
|
||||
log(direction, "RST_STREAM: streamId=%d, errorCode=%d", streamId, errorCode);
|
||||
}
|
||||
|
||||
public void logSettingsAck(Direction direction) {
|
||||
log(direction, "SETTINGS ack=true");
|
||||
}
|
||||
|
||||
public void logSettings(Direction direction, Http2Settings settings) {
|
||||
log(direction, "SETTINGS: ack=false, settings=%s", settings);
|
||||
}
|
||||
|
||||
public void logPing(Direction direction, ByteBuf data) {
|
||||
log(direction, "PING: ack=false, dataLen=%d", data.readableBytes());
|
||||
}
|
||||
|
||||
public void logPingAck(Direction direction, ByteBuf data) {
|
||||
log(direction, "PING: ack=true, dataLen=%d", data.readableBytes());
|
||||
}
|
||||
|
||||
public void logPushPromise(Direction direction, int streamId, int promisedStreamId,
|
||||
Http2Headers headers, int padding) {
|
||||
log(direction, "PUSH_PROMISE: streamId=%d, promisedStreamId=%d, headers=%s, padding=%d",
|
||||
streamId, promisedStreamId, headers, padding);
|
||||
}
|
||||
|
||||
public void logGoAway(Direction direction, int lastStreamId, long errorCode, ByteBuf debugData) {
|
||||
log(direction, "GO_AWAY: lastStreamId=%d, errorCode=%d, dataLen=%d", lastStreamId,
|
||||
errorCode, debugData.readableBytes());
|
||||
}
|
||||
|
||||
public void logWindowsUpdate(Direction direction, int streamId, int windowSizeIncrement) {
|
||||
log(direction, "WINDOW_UPDATE: stream=%d, windowSizeIncrement=%d", streamId,
|
||||
windowSizeIncrement);
|
||||
}
|
||||
|
||||
public void logAltSvc(Direction direction, int streamId, long maxAge, int port,
|
||||
ByteBuf protocolId, String host, String origin) {
|
||||
log(direction,
|
||||
"ALT_SVC: streamId=%d, maxAge=%d, port=%d, protocolIdLen=%d, host=%s, origin=%s",
|
||||
streamId, maxAge, port, protocolId.readableBytes(), host, origin);
|
||||
}
|
||||
|
||||
public void logBlocked(Direction direction, int streamId) {
|
||||
log(direction, "BLOCKED: streamId=%d", streamId);
|
||||
}
|
||||
|
||||
private void log(Direction direction, String format, Object... args) {
|
||||
if (logger.isEnabled(level)) {
|
||||
StringBuilder b = new StringBuilder("\n----------------");
|
||||
b.append(direction.name());
|
||||
b.append("--------------------\n");
|
||||
b.append(String.format(format, args));
|
||||
b.append("\n------------------------------------");
|
||||
logger.log(level, b.toString());
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,165 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
|
||||
/**
|
||||
* An observer of HTTP/2 frames.
|
||||
*/
|
||||
public interface Http2FrameObserver {
|
||||
|
||||
/**
|
||||
* Handles an inbound DATA frame.
|
||||
*
|
||||
* @param streamId the subject stream for the frame.
|
||||
* @param data the payload of the frame.
|
||||
* @param padding the number of padding bytes found at the end of the frame.
|
||||
* @param endOfStream Indicates whether this is the last frame to be sent from the remote
|
||||
* endpoint for this stream.
|
||||
* @param endOfSegment Indicates whether this frame is the end of the current segment.
|
||||
* @param compressed Indicates whether or not the payload is compressed with gzip encoding.
|
||||
*/
|
||||
void onDataRead(int streamId, ByteBuf data, int padding, boolean endOfStream,
|
||||
boolean endOfSegment, boolean compressed) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound HEADERS frame.
|
||||
*
|
||||
* @param streamId the subject stream for the frame.
|
||||
* @param headers the received headers.
|
||||
* @param padding the number of padding bytes found at the end of the frame.
|
||||
* @param endStream Indicates whether this is the last frame to be sent from the remote endpoint
|
||||
* for this stream.
|
||||
* @param endSegment Indicates whether this frame is the end of the current segment.
|
||||
*/
|
||||
void onHeadersRead(int streamId, Http2Headers headers, int padding, boolean endStream,
|
||||
boolean endSegment) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound HEADERS frame with priority information specified.
|
||||
*
|
||||
* @param streamId the subject stream for the frame.
|
||||
* @param headers the received headers.
|
||||
* @param streamDependency the stream on which this stream depends, or 0 if dependent on the
|
||||
* connection.
|
||||
* @param weight the new weight for the stream.
|
||||
* @param exclusive whether or not the stream should be the exclusive dependent of its parent.
|
||||
* @param padding the number of padding bytes found at the end of the frame.
|
||||
* @param endStream Indicates whether this is the last frame to be sent from the remote endpoint
|
||||
* for this stream.
|
||||
* @param endSegment Indicates whether this frame is the end of the current segment.
|
||||
*/
|
||||
void onHeadersRead(int streamId, Http2Headers headers, int streamDependency, short weight,
|
||||
boolean exclusive, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound PRIORITY frame.
|
||||
*
|
||||
* @param streamId the subject stream for the frame.
|
||||
* @param streamDependency the stream on which this stream depends, or 0 if dependent on the
|
||||
* connection.
|
||||
* @param weight the new weight for the stream.
|
||||
* @param exclusive whether or not the stream should be the exclusive dependent of its parent.
|
||||
*/
|
||||
void onPriorityRead(int streamId, int streamDependency, short weight, boolean exclusive)
|
||||
throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound RST_STREAM frame.
|
||||
*
|
||||
* @param streamId the stream that is terminating.
|
||||
* @param errorCode the error code identifying the type of failure.
|
||||
*/
|
||||
void onRstStreamRead(int streamId, long errorCode) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound SETTINGS acknowledgment frame.
|
||||
*/
|
||||
void onSettingsAckRead() throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound SETTINGS frame.
|
||||
*
|
||||
* @param settings the settings received from the remote endpoint.
|
||||
*/
|
||||
void onSettingsRead(Http2Settings settings) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound PING frame.
|
||||
*
|
||||
* @param data the payload of the frame.
|
||||
*/
|
||||
void onPingRead(ByteBuf data) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound PING acknowledgment.
|
||||
*
|
||||
* @param data the payload of the frame.
|
||||
*/
|
||||
void onPingAckRead(ByteBuf data) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound PUSH_PROMISE frame.
|
||||
*
|
||||
* @param streamId the stream the frame was sent on.
|
||||
* @param promisedStreamId the ID of the promised stream.
|
||||
* @param headers the received headers.
|
||||
* @param paddingthe number of padding bytes found at the end of the frame.
|
||||
*/
|
||||
void onPushPromiseRead(int streamId, int promisedStreamId, Http2Headers headers, int padding)
|
||||
throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound GO_AWAY frame.
|
||||
*
|
||||
* @param lastStreamId the last known stream of the remote endpoint.
|
||||
* @param errorCode the error code, if abnormal closure.
|
||||
* @param debugData application-defined debug data.
|
||||
*/
|
||||
void onGoAwayRead(int lastStreamId, long errorCode, ByteBuf debugData) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound WINDOW_UPDATE frame.
|
||||
*
|
||||
* @param streamId the stream the frame was sent on.
|
||||
* @param windowSizeIncrement the increased number of bytes of the remote endpoint's flow
|
||||
* control window.
|
||||
*/
|
||||
void onWindowUpdateRead(int streamId, int windowSizeIncrement) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound ALT_SVC frame.
|
||||
*
|
||||
* @param streamId the stream.
|
||||
* @param maxAge the freshness lifetime of the alternative service association.
|
||||
* @param port the port that the alternative service is available upon.
|
||||
* @param protocolId the ALPN protocol identifier of the alternative service.
|
||||
* @param host the host that the alternative service is available upon.
|
||||
* @param origin an optional origin that the alternative service is available upon. May be
|
||||
* {@code null}.
|
||||
*/
|
||||
void onAltSvcRead(int streamId, long maxAge, int port, ByteBuf protocolId, String host,
|
||||
String origin) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Handles an inbound BLOCKED frame.
|
||||
*
|
||||
* @param streamId the stream that is blocked or 0 if the entire connection is blocked.
|
||||
*/
|
||||
void onBlockedRead(int streamId) throws Http2Exception;
|
||||
}
|
@ -0,0 +1,51 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
* Reads HTTP/2 frames from an input {@link ByteBuf} and notifies the specified
|
||||
* {@link Http2FrameObserver} when frames are complete.
|
||||
*/
|
||||
public interface Http2FrameReader extends Closeable {
|
||||
|
||||
/**
|
||||
* Attempts to read the next frame from the input buffer. If enough data is available to fully
|
||||
* read the frame, notifies the observer of the read frame.
|
||||
*/
|
||||
void readFrame(ByteBufAllocator alloc, ByteBuf input, Http2FrameObserver observer)
|
||||
throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Sets the maximum size of the HPACK header table used for decoding HTTP/2 headers.
|
||||
*/
|
||||
void maxHeaderTableSize(int max);
|
||||
|
||||
/**
|
||||
* Gets the maximum size of the HPACK header table used for decoding HTTP/2 headers.
|
||||
*/
|
||||
int maxHeaderTableSize();
|
||||
|
||||
/**
|
||||
* Closes this reader and frees any allocated resources.
|
||||
*/
|
||||
@Override
|
||||
void close();
|
||||
}
|
@ -0,0 +1,77 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
/**
|
||||
* Enumeration of all frame types defined by the HTTP/2 specification.
|
||||
*/
|
||||
public enum Http2FrameType {
|
||||
DATA((short) 0x0),
|
||||
HEADERS((short) 0x1),
|
||||
PRIORITY((short) 0x2),
|
||||
RST_STREAM((short) 0x3),
|
||||
SETTINGS((short) 0x4),
|
||||
PUSH_PROMISE((short) 0x5),
|
||||
PING((short) 0x6),
|
||||
GO_AWAY((short) 0x7),
|
||||
WINDOW_UPDATE((short) 0x8),
|
||||
CONTINUATION((short) 0x9),
|
||||
ALT_SVC((short) 0xA),
|
||||
BLOCKED((short) 0xB);
|
||||
|
||||
/**
|
||||
* Create an array indexed by the frame type code for fast lookup of the enum value.
|
||||
*/
|
||||
private static final Http2FrameType[] codeToTypeMap;
|
||||
static {
|
||||
int maxIndex = 0;
|
||||
for (Http2FrameType type : Http2FrameType.values()) {
|
||||
maxIndex = Math.max(maxIndex, type.typeCode());
|
||||
}
|
||||
codeToTypeMap = new Http2FrameType[maxIndex + 1];
|
||||
for (Http2FrameType type : Http2FrameType.values()) {
|
||||
codeToTypeMap[type.typeCode()] = type;
|
||||
}
|
||||
}
|
||||
|
||||
private final short code;
|
||||
|
||||
private Http2FrameType(short code) {
|
||||
this.code = code;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the code used to represent this frame type on the wire.
|
||||
*/
|
||||
public short typeCode() {
|
||||
return code;
|
||||
}
|
||||
|
||||
/**
|
||||
* Looks up the frame type by it's type code.
|
||||
*/
|
||||
public static Http2FrameType forTypeCode(short typeCode) {
|
||||
Http2FrameType type = null;
|
||||
if (typeCode >= 0 && typeCode < codeToTypeMap.length) {
|
||||
type = codeToTypeMap[typeCode];
|
||||
}
|
||||
|
||||
if (type == null) {
|
||||
throw new IllegalArgumentException("Unsupported typeCode: " + typeCode);
|
||||
}
|
||||
return type;
|
||||
}
|
||||
}
|
@ -0,0 +1,228 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
* A writer responsible for marshalling HTTP/2 frames to the channel.
|
||||
*/
|
||||
public interface Http2FrameWriter extends Closeable {
|
||||
|
||||
/**
|
||||
* Writes a DATA frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param streamId the stream for which to send the frame.
|
||||
* @param data the payload of the frame.
|
||||
* @param padding the amount of padding to be added to the end of the frame
|
||||
* @param endStream indicates if this is the last frame to be sent for the stream.
|
||||
* @param endSegment indicates if this is the last frame in the current segment.
|
||||
* @param compressed indicates whether the data is compressed using gzip encoding.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writeData(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
|
||||
ByteBuf data, int padding, boolean endStream, boolean endSegment, boolean compressed)
|
||||
throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Writes a HEADERS frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param streamId the stream for which to send the frame.
|
||||
* @param headers the headers to be sent.
|
||||
* @param padding the amount of padding to be added to the end of the frame
|
||||
* @param endStream indicates if this is the last frame to be sent for the stream.
|
||||
* @param endSegment indicates if this is the last frame in the current segment.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
|
||||
Http2Headers headers, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Writes a HEADERS frame with priority specified to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param streamId the stream for which to send the frame.
|
||||
* @param headers the headers to be sent.
|
||||
* @param streamDependency the stream on which this stream should depend, or 0 if it should
|
||||
* depend on the connection.
|
||||
* @param weight the weight for this stream.
|
||||
* @param exclusive whether this stream should be the exclusive dependant of its parent.
|
||||
* @param padding the amount of padding to be added to the end of the frame
|
||||
* @param endStream indicates if this is the last frame to be sent for the stream.
|
||||
* @param endSegment indicates if this is the last frame in the current segment.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
|
||||
Http2Headers headers, int streamDependency, short weight, boolean exclusive,
|
||||
int padding, boolean endStream, boolean endSegment) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Writes a PRIORITY frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param streamId the stream for which to send the frame.
|
||||
* @param streamDependency the stream on which this stream should depend, or 0 if it should
|
||||
* depend on the connection.
|
||||
* @param weight the weight for this stream.
|
||||
* @param exclusive whether this stream should be the exclusive dependant of its parent.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writePriority(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
|
||||
int streamDependency, short weight, boolean exclusive) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Writes a RST_STREAM frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param streamId the stream for which to send the frame.
|
||||
* @param errorCode the error code indicating the nature of the failure.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writeRstStream(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
|
||||
long errorCode);
|
||||
|
||||
/**
|
||||
* Writes a SETTINGS frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param settings the settings to be sent.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writeSettings(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
Http2Settings settings) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Writes a SETTINGS acknowledgment to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writeSettingsAck(ChannelHandlerContext ctx, ChannelPromise promise)
|
||||
throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Writes a PING frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param ack indicates whether this is an ack of a PING frame previously received from the
|
||||
* remote endpoint.
|
||||
* @param data the payload of the frame.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writePing(ChannelHandlerContext ctx, ChannelPromise promise, boolean ack,
|
||||
ByteBuf data) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Writes a PUSH_PROMISE frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param streamId the stream for which to send the frame.
|
||||
* @param promisedStreamId the ID of the promised stream.
|
||||
* @param headers the headers to be sent.
|
||||
* @param padding the amount of padding to be added to the end of the frame
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writePushPromise(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
|
||||
int promisedStreamId, Http2Headers headers, int padding) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Writes a GO_AWAY frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param lastStreamId the last known stream of this endpoint.
|
||||
* @param errorCode the error code, if the connection was abnormally terminated.
|
||||
* @param debugData application-defined debug data.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writeGoAway(ChannelHandlerContext ctx, ChannelPromise promise, int lastStreamId,
|
||||
long errorCode, ByteBuf debugData);
|
||||
|
||||
/**
|
||||
* Writes a WINDOW_UPDATE frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param streamId the stream for which to send the frame.
|
||||
* @param windowSizeIncrement the number of bytes by which the local inbound flow control window
|
||||
* is increasing.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writeWindowUpdate(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int windowSizeIncrement) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Writes a ALT_SVC frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param streamId the stream.
|
||||
* @param maxAge the freshness lifetime of the alternative service association.
|
||||
* @param port the port that the alternative service is available upon.
|
||||
* @param protocolId the ALPN protocol identifier of the alternative service.
|
||||
* @param host the host that the alternative service is available upon.
|
||||
* @param origin an optional origin that the alternative service is available upon. May be
|
||||
* {@code null}.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writeAltSvc(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
|
||||
long maxAge, int port, ByteBuf protocolId, String host, String origin)
|
||||
throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Writes a BLOCKED frame to the remote endpoint.
|
||||
*
|
||||
* @param ctx the context to use for writing.
|
||||
* @param promise the promise for the write.
|
||||
* @param streamId the stream that is blocked or 0 if the entire connection is blocked.
|
||||
* @return the future for the write.
|
||||
*/
|
||||
ChannelFuture writeBlocked(ChannelHandlerContext ctx, ChannelPromise promise, int streamId)
|
||||
throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Closes this writer and frees any allocated resources.
|
||||
*/
|
||||
@Override
|
||||
void close();
|
||||
|
||||
/**
|
||||
* Sets the maximum size of the HPACK header table used for decoding HTTP/2 headers.
|
||||
*/
|
||||
void maxHeaderTableSize(int max) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Gets the maximum size of the HPACK header table used for decoding HTTP/2 headers.
|
||||
*/
|
||||
int maxHeaderTableSize();
|
||||
}
|
@ -13,16 +13,16 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeSet;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
/**
|
||||
* An immutable collection of headers sent or received via HTTP/2.
|
||||
@ -113,9 +113,8 @@ public abstract class Http2Headers implements Iterable<Entry<String, String>> {
|
||||
public abstract Set<String> names();
|
||||
|
||||
/**
|
||||
* Returns the header value with the specified header name. If there is
|
||||
* more than one header value for the specified header name, the first
|
||||
* value is returned.
|
||||
* Returns the header value with the specified header name. If there is more than one header
|
||||
* value for the specified header name, the first value is returned.
|
||||
*
|
||||
* @return the header value or {@code null} if there is no such header
|
||||
*/
|
||||
@ -124,22 +123,20 @@ public abstract class Http2Headers implements Iterable<Entry<String, String>> {
|
||||
/**
|
||||
* Returns the header values with the specified header name.
|
||||
*
|
||||
* @return the {@link List} of header values. An empty list if there is no
|
||||
* such header.
|
||||
* @return the {@link List} of header values. An empty list if there is no such header.
|
||||
*/
|
||||
public abstract List<String> getAll(String name);
|
||||
|
||||
/**
|
||||
* Returns all header names and values that this frame contains.
|
||||
*
|
||||
* @return the {@link List} of the header name-value pairs. An empty list
|
||||
* if there is no header in this message.
|
||||
* @return the {@link List} of the header name-value pairs. An empty list if there is no header
|
||||
* in this message.
|
||||
*/
|
||||
public abstract List<Map.Entry<String, String>> entries();
|
||||
|
||||
/**
|
||||
* Returns {@code true} if and only if there is a header with the specified
|
||||
* header name.
|
||||
* Returns {@code true} if and only if there is a header with the specified header name.
|
||||
*/
|
||||
public abstract boolean contains(String name);
|
||||
|
||||
@ -149,47 +146,37 @@ public abstract class Http2Headers implements Iterable<Entry<String, String>> {
|
||||
public abstract boolean isEmpty();
|
||||
|
||||
/**
|
||||
* Gets the {@link HttpName#METHOD} header.
|
||||
*
|
||||
* @return the header value or {@code null} if there is no such header
|
||||
* Gets the {@link HttpName#METHOD} header or {@code null} if there is no such header
|
||||
*/
|
||||
public final String getMethod() {
|
||||
public final String method() {
|
||||
return get(HttpName.METHOD.value());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the {@link HttpName#SCHEME} header.
|
||||
*
|
||||
* @return the header value or {@code null} if there is no such header
|
||||
* Gets the {@link HttpName#SCHEME} header or {@code null} if there is no such header
|
||||
*/
|
||||
public final String getScheme() {
|
||||
public final String scheme() {
|
||||
return get(HttpName.SCHEME.value());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the {@link HttpName#AUTHORITY} header.
|
||||
*
|
||||
* @return the header value or {@code null} if there is no such header
|
||||
* Gets the {@link HttpName#AUTHORITY} header or {@code null} if there is no such header
|
||||
*/
|
||||
public final String getAuthority() {
|
||||
public final String authority() {
|
||||
return get(HttpName.AUTHORITY.value());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the {@link HttpName#PATH} header.
|
||||
*
|
||||
* @return the header value or {@code null} if there is no such header
|
||||
* Gets the {@link HttpName#PATH} header or {@code null} if there is no such header
|
||||
*/
|
||||
public final String getPath() {
|
||||
public final String path() {
|
||||
return get(HttpName.PATH.value());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the {@link HttpName#STATUS} header.
|
||||
*
|
||||
* @return the header value or {@code null} if there is no such header
|
||||
* Gets the {@link HttpName#STATUS} header or {@code null} if there is no such header
|
||||
*/
|
||||
public final String getStatus() {
|
||||
public final String status() {
|
||||
return get(HttpName.STATUS.value());
|
||||
}
|
||||
|
||||
@ -239,4 +226,17 @@ public abstract class Http2Headers implements Iterable<Entry<String, String>> {
|
||||
// They match.
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder("Http2Headers[");
|
||||
for (Map.Entry<String, String> header : this) {
|
||||
builder.append(header.getKey());
|
||||
builder.append(":");
|
||||
builder.append(header.getValue());
|
||||
builder.append(",");
|
||||
}
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
@ -13,11 +13,9 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Headers;
|
||||
|
||||
/**
|
||||
* Decodes HPACK-encoded headers blocks into {@link Http2Headers}.
|
||||
@ -32,5 +30,10 @@ public interface Http2HeadersDecoder {
|
||||
/**
|
||||
* Sets the new max header table size for this decoder.
|
||||
*/
|
||||
void setHeaderTableSize(int size) throws Http2Exception;
|
||||
void maxHeaderTableSize(int size);
|
||||
|
||||
/**
|
||||
* Gets the maximum header table size for this decoder.
|
||||
*/
|
||||
int maxHeaderTableSize();
|
||||
}
|
@ -13,11 +13,9 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Headers;
|
||||
|
||||
/**
|
||||
* Encodes {@link Http2Headers} into HPACK-encoded headers blocks.
|
||||
@ -28,12 +26,17 @@ public interface Http2HeadersEncoder {
|
||||
* Encodes the given headers and writes the output headers block to the given output buffer.
|
||||
*
|
||||
* @param headers the headers to be encoded.
|
||||
* @param buffer the buffer to write the headers to.
|
||||
* @param buffer the buffer to receive the encoded headers.
|
||||
*/
|
||||
void encodeHeaders(Http2Headers headers, ByteBuf buffer) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Updates the maximum header table size for this encoder.
|
||||
*/
|
||||
void setHeaderTableSize(int size) throws Http2Exception;
|
||||
void maxHeaderTableSize(int size) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Gets the current maximum value for the header table size.
|
||||
*/
|
||||
int maxHeaderTableSize();
|
||||
}
|
@ -13,16 +13,14 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.connection;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2DataFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2WindowUpdateFrame;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
|
||||
/**
|
||||
* Controls the inbound flow of data frames from the remote endpoint.
|
||||
*/
|
||||
public interface InboundFlowController {
|
||||
public interface Http2InboundFlowController {
|
||||
|
||||
/**
|
||||
* A writer of window update frames.
|
||||
@ -32,9 +30,21 @@ public interface InboundFlowController {
|
||||
/**
|
||||
* Writes a window update frame to the remote endpoint.
|
||||
*/
|
||||
void writeFrame(Http2WindowUpdateFrame frame);
|
||||
void writeFrame(int streamId, int windowSizeIncrement) throws Http2Exception;
|
||||
}
|
||||
|
||||
/**
|
||||
* Informs the flow controller of the existence of a new stream, allowing it to allocate
|
||||
* resources as needed.
|
||||
*/
|
||||
void addStream(int streamId);
|
||||
|
||||
/**
|
||||
* Removes the given stream from flow control processing logic and frees resources as
|
||||
* appropriate.
|
||||
*/
|
||||
void removeStream(int streamId);
|
||||
|
||||
/**
|
||||
* Sets the initial inbound flow control window size and updates all stream window sizes by the
|
||||
* delta. This is called as part of the processing for an outbound SETTINGS frame.
|
||||
@ -42,20 +52,25 @@ public interface InboundFlowController {
|
||||
* @param newWindowSize the new initial window size.
|
||||
* @throws Http2Exception thrown if any protocol-related error occurred.
|
||||
*/
|
||||
void setInitialInboundWindowSize(int newWindowSize) throws Http2Exception;
|
||||
void initialInboundWindowSize(int newWindowSize) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Gets the initial inbound flow control window size.
|
||||
*/
|
||||
int getInitialInboundWindowSize();
|
||||
int initialInboundWindowSize();
|
||||
|
||||
/**
|
||||
* Applies flow control for the received data frame.
|
||||
*
|
||||
* @param dataFrame the flow controlled frame
|
||||
* @param streamId the ID of the stream receiving the data
|
||||
* @param data the data portion of the data frame. Does not contain padding.
|
||||
* @param padding the amount of padding received in the original frame.
|
||||
* @param endOfStream indicates whether this is the last frame for the stream.
|
||||
* @param endOfSegment indicates whether this is the last frame for the current segment.
|
||||
* @param frameWriter allows this flow controller to send window updates to the remote endpoint.
|
||||
* @throws Http2Exception thrown if any protocol-related error occurred.
|
||||
*/
|
||||
void applyInboundFlowControl(Http2DataFrame dataFrame, FrameWriter frameWriter)
|
||||
void applyInboundFlowControl(int streamId, ByteBuf data, int padding, boolean endOfStream,
|
||||
boolean endOfSegment, boolean compressed, FrameWriter frameWriter)
|
||||
throws Http2Exception;
|
||||
}
|
@ -0,0 +1,160 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2FrameLogger.Direction.INBOUND;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
|
||||
/**
|
||||
* Decorator around a {@link Http2FrameReader} that logs all inbound frames before calling
|
||||
* back the observer.
|
||||
*/
|
||||
public class Http2InboundFrameLogger implements Http2FrameReader {
|
||||
|
||||
private final Http2FrameReader reader;
|
||||
private final Http2FrameLogger logger;
|
||||
|
||||
public Http2InboundFrameLogger(Http2FrameReader reader, Http2FrameLogger logger) {
|
||||
if (reader == null) {
|
||||
throw new NullPointerException("reader");
|
||||
}
|
||||
if (logger == null) {
|
||||
throw new NullPointerException("logger");
|
||||
}
|
||||
this.reader = reader;
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrame(ByteBufAllocator alloc, ByteBuf input, final Http2FrameObserver observer)
|
||||
throws Http2Exception {
|
||||
reader.readFrame(alloc, input, new Http2FrameObserver() {
|
||||
|
||||
@Override
|
||||
public void onDataRead(int streamId, ByteBuf data, int padding, boolean endOfStream,
|
||||
boolean endOfSegment, boolean compressed) throws Http2Exception {
|
||||
logger.logData(INBOUND, streamId, data, padding, endOfStream, endOfSegment,
|
||||
compressed);
|
||||
observer.onDataRead(streamId, data, padding, endOfStream, endOfSegment, compressed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onHeadersRead(int streamId, Http2Headers headers, int padding,
|
||||
boolean endStream, boolean endSegment) throws Http2Exception {
|
||||
logger.logHeaders(INBOUND, streamId, headers, padding, endStream, endSegment);
|
||||
observer.onHeadersRead(streamId, headers, padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onHeadersRead(int streamId, Http2Headers headers, int streamDependency,
|
||||
short weight, boolean exclusive, int padding, boolean endStream,
|
||||
boolean endSegment) throws Http2Exception {
|
||||
logger.logHeaders(INBOUND, streamId, headers, streamDependency, weight, exclusive,
|
||||
padding, endStream, endSegment);
|
||||
observer.onHeadersRead(streamId, headers, streamDependency, weight, exclusive,
|
||||
padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPriorityRead(int streamId, int streamDependency, short weight,
|
||||
boolean exclusive) throws Http2Exception {
|
||||
logger.logPriority(INBOUND, streamId, streamDependency, weight, exclusive);
|
||||
observer.onPriorityRead(streamId, streamDependency, weight, exclusive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onRstStreamRead(int streamId, long errorCode) throws Http2Exception {
|
||||
logger.logRstStream(INBOUND, streamId, errorCode);
|
||||
observer.onRstStreamRead(streamId, errorCode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onSettingsAckRead() throws Http2Exception {
|
||||
logger.logSettingsAck(INBOUND);
|
||||
observer.onSettingsAckRead();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onSettingsRead(Http2Settings settings) throws Http2Exception {
|
||||
logger.logSettings(INBOUND, settings);
|
||||
observer.onSettingsRead(settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPingRead(ByteBuf data) throws Http2Exception {
|
||||
logger.logPing(INBOUND, data);
|
||||
observer.onPingRead(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPingAckRead(ByteBuf data) throws Http2Exception {
|
||||
logger.logPingAck(INBOUND, data);
|
||||
observer.onPingAckRead(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPushPromiseRead(int streamId, int promisedStreamId, Http2Headers headers,
|
||||
int padding) throws Http2Exception {
|
||||
logger.logPushPromise(INBOUND, streamId, promisedStreamId, headers, padding);
|
||||
observer.onPushPromiseRead(streamId, promisedStreamId, headers, padding);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onGoAwayRead(int lastStreamId, long errorCode, ByteBuf debugData)
|
||||
throws Http2Exception {
|
||||
logger.logGoAway(INBOUND, lastStreamId, errorCode, debugData);
|
||||
observer.onGoAwayRead(lastStreamId, errorCode, debugData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onWindowUpdateRead(int streamId, int windowSizeIncrement)
|
||||
throws Http2Exception {
|
||||
logger.logWindowsUpdate(INBOUND, streamId, windowSizeIncrement);
|
||||
observer.onWindowUpdateRead(streamId, windowSizeIncrement);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onAltSvcRead(int streamId, long maxAge, int port, ByteBuf protocolId,
|
||||
String host, String origin) throws Http2Exception {
|
||||
logger.logAltSvc(INBOUND, streamId, maxAge, port, protocolId, host, origin);
|
||||
observer.onAltSvcRead(streamId, maxAge, port, protocolId, host, origin);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onBlockedRead(int streamId) throws Http2Exception {
|
||||
logger.logBlocked(INBOUND, streamId);
|
||||
observer.onBlockedRead(streamId);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maxHeaderTableSize(int max) {
|
||||
reader.maxHeaderTableSize(max);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxHeaderTableSize() {
|
||||
return reader.maxHeaderTableSize();
|
||||
}
|
||||
|
||||
}
|
@ -13,7 +13,7 @@
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2.draft10;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelHandler;
|
||||
@ -23,18 +23,16 @@ import io.netty.handler.codec.ByteToMessageDecoder;
|
||||
import io.netty.handler.codec.http.HttpObjectAggregator;
|
||||
import io.netty.handler.codec.http.HttpRequestDecoder;
|
||||
import io.netty.handler.codec.http.HttpResponseEncoder;
|
||||
import io.netty.handler.codec.http2.draft10.connection.Http2ConnectionHandler;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameCodec;
|
||||
import io.netty.handler.codec.http2.draft10.frame.decoder.Http2ServerPrefaceReader;
|
||||
import io.netty.handler.ssl.SslHandler;
|
||||
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import java.util.List;
|
||||
|
||||
import javax.net.ssl.SSLEngine;
|
||||
|
||||
/**
|
||||
* {@link io.netty.channel.ChannelHandler} which is responsible to setup the {@link io.netty.channel.ChannelPipeline}
|
||||
* either for HTTP or HTTP2. This offers an easy way for users to support both at the same time while not care to much
|
||||
* about the low-level details.
|
||||
* {@link io.netty.channel.ChannelHandler} which is responsible to setup the
|
||||
* {@link io.netty.channel.ChannelPipeline} either for HTTP or HTTP2. This offers an easy way for
|
||||
* users to support both at the same time while not care to much about the low-level details.
|
||||
*/
|
||||
public abstract class Http2OrHttpChooser extends ByteToMessageDecoder {
|
||||
|
||||
@ -42,7 +40,7 @@ public abstract class Http2OrHttpChooser extends ByteToMessageDecoder {
|
||||
|
||||
public enum SelectedProtocol {
|
||||
/** Must be updated to match the HTTP/2 draft number. */
|
||||
HTTP_2("h2-10"),
|
||||
HTTP_2("h2-12"),
|
||||
HTTP_1_1("http/1.1"),
|
||||
HTTP_1_0("http/1.0"),
|
||||
UNKNOWN("Unknown");
|
||||
@ -75,15 +73,16 @@ public abstract class Http2OrHttpChooser extends ByteToMessageDecoder {
|
||||
}
|
||||
|
||||
private final int maxHttpContentLength;
|
||||
private final boolean server;
|
||||
|
||||
protected Http2OrHttpChooser(int maxHttpContentLength) {
|
||||
protected Http2OrHttpChooser(boolean server, int maxHttpContentLength) {
|
||||
this.server = server;
|
||||
this.maxHttpContentLength = maxHttpContentLength;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the {@link SelectedProtocol} for the {@link javax.net.ssl.SSLEngine}. If its not known yet implementations
|
||||
* MUST return {@link SelectedProtocol#UNKNOWN}.
|
||||
*
|
||||
* Return the {@link SelectedProtocol} for the {@link javax.net.ssl.SSLEngine}. If its not known
|
||||
* yet implementations MUST return {@link SelectedProtocol#UNKNOWN}.
|
||||
*/
|
||||
protected abstract SelectedProtocol getProtocol(SSLEngine engine);
|
||||
|
||||
@ -130,10 +129,8 @@ public abstract class Http2OrHttpChooser extends ByteToMessageDecoder {
|
||||
*/
|
||||
protected void addHttp2Handlers(ChannelHandlerContext ctx) {
|
||||
ChannelPipeline pipeline = ctx.pipeline();
|
||||
pipeline.addLast("http2ServerPrefaceReader", new Http2ServerPrefaceReader());
|
||||
pipeline.addLast("http2FrameCodec", new Http2FrameCodec());
|
||||
pipeline.addLast("http2ConnectionHandler", new Http2ConnectionHandler(true));
|
||||
pipeline.addLast("http2RequestHandler", createHttp2RequestHandler());
|
||||
pipeline.addLast("http2PrefaceHandler", new Http2PrefaceHandler(server));
|
||||
pipeline.addLast("http2ConnectionHandler", createHttp2RequestHandler());
|
||||
}
|
||||
|
||||
/**
|
||||
@ -148,14 +145,16 @@ public abstract class Http2OrHttpChooser extends ByteToMessageDecoder {
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the {@link io.netty.channel.ChannelHandler} that is responsible for handling the http requests when the
|
||||
* {@link SelectedProtocol} was {@link SelectedProtocol#HTTP_1_0} or {@link SelectedProtocol#HTTP_1_1}
|
||||
* Create the {@link io.netty.channel.ChannelHandler} that is responsible for handling the http
|
||||
* requests when the {@link SelectedProtocol} was {@link SelectedProtocol#HTTP_1_0} or
|
||||
* {@link SelectedProtocol#HTTP_1_1}
|
||||
*/
|
||||
protected abstract ChannelHandler createHttp1RequestHandler();
|
||||
|
||||
/**
|
||||
* Create the {@link io.netty.channel.ChannelHandler} that is responsible for handling the http responses when the
|
||||
* when the {@link SelectedProtocol} was {@link SelectedProtocol#HTTP_2}.
|
||||
* Create the {@link io.netty.channel.ChannelHandler} that is responsible for handling the http
|
||||
* responses when the when the {@link SelectedProtocol} was {@link SelectedProtocol#HTTP_2}. The
|
||||
* returned class should be a subclass of {@link AbstractHttp2ConnectionHandler}.
|
||||
*/
|
||||
protected abstract ChannelHandler createHttp2RequestHandler();
|
||||
}
|
@ -0,0 +1,129 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
|
||||
/**
|
||||
* Controls the outbound flow of data frames to the remote endpoint.
|
||||
*/
|
||||
public interface Http2OutboundFlowController {
|
||||
|
||||
/**
|
||||
* Interface that abstracts the writing of {@link Http2Frame} objects to the remote endpoint.
|
||||
*/
|
||||
interface FrameWriter {
|
||||
|
||||
/**
|
||||
* Writes a single data frame to the remote endpoint.
|
||||
*/
|
||||
void writeFrame(int streamId, ByteBuf data, int padding, boolean endStream,
|
||||
boolean endSegment, boolean compressed);
|
||||
|
||||
/**
|
||||
* Called if an error occurred before the write could take place. Sets the failure on the
|
||||
* channel promise.
|
||||
*/
|
||||
void setFailure(Throwable cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new priority for the a stream with respect to out-bound flow control.
|
||||
*
|
||||
* @param streamId the stream to be prioritized.
|
||||
* @param parent an optional stream that the given stream should depend on. Zero, if no
|
||||
* dependency.
|
||||
* @param weight the weight to be assigned to this stream relative to its parent. This value
|
||||
* must be between 1 and 256 (inclusive)
|
||||
* @param exclusive indicates that the stream should be the exclusive dependent on its parent.
|
||||
* This only applies if the stream has a parent.
|
||||
*/
|
||||
void addStream(int streamId, int parent, short weight, boolean exclusive);
|
||||
|
||||
/**
|
||||
* Updates the priority for a stream with respect to out-bound flow control.
|
||||
*
|
||||
* @param streamId the stream to be prioritized.
|
||||
* @param parent an optional stream that the given stream should depend on. Zero, if no
|
||||
* dependency.
|
||||
* @param weight the weight to be assigned to this stream relative to its parent. This value
|
||||
* must be between 1 and 256 (inclusive)
|
||||
* @param exclusive indicates that the stream should be the exclusive dependent on its parent.
|
||||
* This only applies if the stream has a parent.
|
||||
*/
|
||||
void updateStream(int streamId, int parent, short weight, boolean exclusive);
|
||||
|
||||
/**
|
||||
* Removes the given stream from those considered for out-bound flow control.
|
||||
*/
|
||||
void removeStream(int streamId);
|
||||
|
||||
/**
|
||||
* Sets the initial size of the connection's outbound flow control window. The outbound flow
|
||||
* control windows for all streams are updated by the delta in the initial window size. This is
|
||||
* called as part of the processing of a SETTINGS frame received from the remote endpoint.
|
||||
*
|
||||
* @param newWindowSize the new initial window size.
|
||||
*/
|
||||
void initialOutboundWindowSize(int newWindowSize) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Gets the initial size of the connection's outbound flow control window.
|
||||
*/
|
||||
int initialOutboundWindowSize();
|
||||
|
||||
/**
|
||||
* Updates the size of the stream's outbound flow control window. This is called upon receiving
|
||||
* a WINDOW_UPDATE frame from the remote endpoint.
|
||||
*
|
||||
* @param streamId the ID of the stream, or zero if the window is for the entire connection.
|
||||
* @param deltaWindowSize the change in size of the outbound flow control window.
|
||||
* @throws Http2Exception thrown if a protocol-related error occurred.
|
||||
*/
|
||||
void updateOutboundWindowSize(int streamId, int deltaWindowSize) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Indicates that the given stream or the entire connection is blocked and that no more messages
|
||||
* should be sent.
|
||||
*
|
||||
* @param streamId the stream ID that is blocked or zero if the entire connection is blocked.
|
||||
* @throws Http2Exception thrown if a protocol-related error occurred.
|
||||
*/
|
||||
void setBlocked(int streamId) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Sends the frame with outbound flow control applied. The frame may be written at a later time,
|
||||
* depending on whether the remote endpoint can receive the frame now.
|
||||
* <p/>
|
||||
* Data frame flow control processing requirements:
|
||||
* <p/>
|
||||
* Sender must not send a data frame with data length greater than the transfer window size.
|
||||
* After sending each data frame, the stream's transfer window size is decremented by the amount
|
||||
* of data transmitted. When the window size becomes less than or equal to 0, the sender must
|
||||
* pause transmitting data frames.
|
||||
*
|
||||
* @param streamId the ID of the stream on which the data is to be sent.
|
||||
* @param data the data be be sent to the remote endpoint.
|
||||
* @param padding the number of bytes of padding to be added to the frame.
|
||||
* @param endStream indicates whether this frames is to be the last sent on this stream.
|
||||
* @param endSegment indicates whether this is to be the last frame in the segment.
|
||||
* @param compressed whether the data is compressed using gzip compression.
|
||||
* @param frameWriter peforms to the write of the frame to the remote endpoint.
|
||||
* @throws Http2Exception thrown if a protocol-related error occurred.
|
||||
*/
|
||||
void sendFlowControlled(int streamId, ByteBuf data, int padding, boolean endStream,
|
||||
boolean endSegment, boolean compressed, FrameWriter frameWriter) throws Http2Exception;
|
||||
}
|
@ -0,0 +1,158 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License,
|
||||
* version 2.0 (the "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2FrameLogger.Direction.OUTBOUND;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
|
||||
/**
|
||||
* Decorator around a {@link Http2FrameWriter} that logs all outbound frames before calling the
|
||||
* writer.
|
||||
*/
|
||||
public class Http2OutboundFrameLogger implements Http2FrameWriter {
|
||||
|
||||
private final Http2FrameWriter writer;
|
||||
private final Http2FrameLogger logger;
|
||||
|
||||
public Http2OutboundFrameLogger(Http2FrameWriter writer, Http2FrameLogger logger) {
|
||||
if (writer == null) {
|
||||
throw new NullPointerException("writer");
|
||||
}
|
||||
if (logger == null) {
|
||||
throw new NullPointerException("logger");
|
||||
}
|
||||
this.writer = writer;
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeData(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
|
||||
ByteBuf data, int padding, boolean endStream, boolean endSegment, boolean compressed)
|
||||
throws Http2Exception {
|
||||
logger.logData(OUTBOUND, streamId, data, padding, endStream, endSegment, compressed);
|
||||
return writer.writeData(ctx, promise, streamId, data, padding, endStream, endSegment,
|
||||
compressed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception {
|
||||
logger.logHeaders(OUTBOUND, streamId, headers, padding, endStream, endSegment);
|
||||
return writer.writeHeaders(ctx, promise, streamId, headers, padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, Http2Headers headers, int streamDependency, short weight,
|
||||
boolean exclusive, int padding, boolean endStream, boolean endSegment)
|
||||
throws Http2Exception {
|
||||
logger.logHeaders(OUTBOUND, streamId, headers, streamDependency, weight, exclusive,
|
||||
padding, endStream, endSegment);
|
||||
return writer.writeHeaders(ctx, promise, streamId, headers, streamDependency, weight,
|
||||
exclusive, padding, endStream, endSegment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writePriority(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int streamDependency, short weight, boolean exclusive)
|
||||
throws Http2Exception {
|
||||
logger.logPriority(OUTBOUND, streamId, streamDependency, weight, exclusive);
|
||||
return writer.writePriority(ctx, promise, streamId, streamDependency, weight, exclusive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeRstStream(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, long errorCode) {
|
||||
return writer.writeRstStream(ctx, promise, streamId, errorCode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeSettings(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
Http2Settings settings) throws Http2Exception {
|
||||
logger.logSettings(OUTBOUND, settings);
|
||||
return writer.writeSettings(ctx, promise, settings);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeSettingsAck(ChannelHandlerContext ctx, ChannelPromise promise)
|
||||
throws Http2Exception {
|
||||
logger.logSettingsAck(OUTBOUND);
|
||||
return writer.writeSettingsAck(ctx, promise);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writePing(ChannelHandlerContext ctx, ChannelPromise promise, boolean ack,
|
||||
ByteBuf data) throws Http2Exception {
|
||||
logger.logPing(OUTBOUND, data);
|
||||
return writer.writePing(ctx, promise, ack, data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writePushPromise(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int promisedStreamId, Http2Headers headers, int padding)
|
||||
throws Http2Exception {
|
||||
logger.logPushPromise(OUTBOUND, streamId, promisedStreamId, headers, padding);
|
||||
return writer.writePushPromise(ctx, promise, streamId, promisedStreamId, headers, padding);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeGoAway(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int lastStreamId, long errorCode, ByteBuf debugData) {
|
||||
logger.logGoAway(OUTBOUND, lastStreamId, errorCode, debugData);
|
||||
return writer.writeGoAway(ctx, promise, lastStreamId, errorCode, debugData);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeWindowUpdate(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, int windowSizeIncrement) throws Http2Exception {
|
||||
logger.logWindowsUpdate(OUTBOUND, streamId, windowSizeIncrement);
|
||||
return writer.writeWindowUpdate(ctx, promise, streamId, windowSizeIncrement);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeAltSvc(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId, long maxAge, int port, ByteBuf protocolId, String host, String origin)
|
||||
throws Http2Exception {
|
||||
logger.logAltSvc(OUTBOUND, streamId, maxAge, port, protocolId, host, origin);
|
||||
return writer.writeAltSvc(ctx, promise, streamId, maxAge, port, protocolId, host, origin);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChannelFuture writeBlocked(ChannelHandlerContext ctx, ChannelPromise promise,
|
||||
int streamId) throws Http2Exception {
|
||||
logger.logBlocked(OUTBOUND, streamId);
|
||||
return writer.writeBlocked(ctx, promise, streamId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
writer.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void maxHeaderTableSize(int max) throws Http2Exception {
|
||||
writer.maxHeaderTableSize(max);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxHeaderTableSize() {
|
||||
return writer.maxHeaderTableSize();
|
||||
}
|
||||
}
|
@ -0,0 +1,120 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.connectionPrefaceBuf;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelFutureListener;
|
||||
import io.netty.channel.ChannelHandlerAdapter;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
|
||||
/**
|
||||
* Reads and writes the HTTP/2 connection preface, which must be the first bytes sent by both
|
||||
* endpoints upon successful establishment of an HTTP/2 connection. After receiving the preface from
|
||||
* the remote endpoint, this handler removes itself from the pipeline.
|
||||
*
|
||||
* https://tools.ietf.org/html/draft-ietf-httpbis-http2-12#section-3.5
|
||||
*/
|
||||
public class Http2PrefaceHandler extends ChannelHandlerAdapter {
|
||||
|
||||
private boolean server;
|
||||
private final ByteBuf preface = connectionPrefaceBuf();
|
||||
private boolean prefaceWritten;
|
||||
|
||||
public Http2PrefaceHandler(boolean server) {
|
||||
this.server = server;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelActive(ChannelHandlerContext ctx) throws Exception {
|
||||
// The channel just became active - send the HTTP2 connection preface to the remote
|
||||
// endpoint.
|
||||
sendPreface(ctx);
|
||||
|
||||
super.channelActive(ctx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
// This handler was just added to the context. In case it was handled after
|
||||
// the connection became active, send the HTTP2 connection preface now.
|
||||
sendPreface(ctx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
|
||||
if (server) {
|
||||
// Only servers receive the preface string.
|
||||
if (preface.isReadable() && msg instanceof ByteBuf) {
|
||||
ByteBuf buf = (ByteBuf) msg;
|
||||
processHttp2Preface(ctx, buf);
|
||||
if (preface.isReadable()) {
|
||||
// More preface left to process.
|
||||
buf.release();
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
super.channelRead(ctx, msg);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends the HTTP2 connection preface to the remote endpoint, if not already sent.
|
||||
*/
|
||||
private void sendPreface(final ChannelHandlerContext ctx) {
|
||||
if (server) {
|
||||
// The preface string is only written by clients.
|
||||
return;
|
||||
}
|
||||
if (!prefaceWritten && ctx.channel().isActive()) {
|
||||
prefaceWritten = true;
|
||||
ctx.writeAndFlush(connectionPrefaceBuf()).addListener(new ChannelFutureListener() {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
if (!future.isSuccess() && ctx.channel().isOpen()) {
|
||||
// The write failed, close the connection.
|
||||
ctx.close();
|
||||
} else {
|
||||
ctx.pipeline().remove(Http2PrefaceHandler.this);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
private void processHttp2Preface(ChannelHandlerContext ctx, ByteBuf in) throws Http2Exception {
|
||||
int prefaceRemaining = preface.readableBytes();
|
||||
int bytesRead = Math.min(in.readableBytes(), prefaceRemaining);
|
||||
|
||||
// Read the portion of the input up to the length of the preface, if reached.
|
||||
ByteBuf sourceSlice = in.readSlice(bytesRead);
|
||||
|
||||
// Read the same number of bytes from the preface buffer.
|
||||
ByteBuf prefaceSlice = preface.readSlice(bytesRead);
|
||||
|
||||
// If the input so far doesn't match the preface, break the connection.
|
||||
if (bytesRead == 0 || !prefaceSlice.equals(sourceSlice)) {
|
||||
ctx.close();
|
||||
return;
|
||||
}
|
||||
|
||||
if (!preface.isReadable()) {
|
||||
// Entire preface has been read, remove ourselves from the pipeline.
|
||||
ctx.pipeline().remove(this);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,142 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
* A tree for maintaining relative priority information among streams.
|
||||
*/
|
||||
public interface Http2PriorityTree<T> extends Iterable<Http2PriorityTree.Priority<T>> {
|
||||
|
||||
/**
|
||||
* Priority node for a single stream.
|
||||
*/
|
||||
interface Priority<T> {
|
||||
|
||||
/**
|
||||
* Indicates whether or not this priority is the root of the tree.
|
||||
*/
|
||||
boolean isRoot();
|
||||
|
||||
/**
|
||||
* Indicates whether or not this is a leaf node (i.e. {@link #numChildren} is 0).
|
||||
*/
|
||||
boolean isLeaf();
|
||||
|
||||
/**
|
||||
* Returns the subject stream for this priority information.
|
||||
*/
|
||||
int streamId();
|
||||
|
||||
/**
|
||||
* Returns optional data associated with this stream.
|
||||
*/
|
||||
T data();
|
||||
|
||||
/**
|
||||
* Returns weight assigned to the dependency with the parent. The weight will be a value
|
||||
* between 1 and 256.
|
||||
*/
|
||||
short weight();
|
||||
|
||||
/**
|
||||
* The total of the weights of all children of this node.
|
||||
*/
|
||||
int totalChildWeights();
|
||||
|
||||
/**
|
||||
* The parent (i.e. the node on which this node depends), or {@code null} if this is the
|
||||
* root node.
|
||||
*/
|
||||
Priority<T> parent();
|
||||
|
||||
/**
|
||||
* Indicates whether or not this priority is descended from the given priority.
|
||||
*/
|
||||
boolean isDescendantOf(Priority<T> priority);
|
||||
|
||||
/**
|
||||
* Returns the number of children directly dependent on this node.
|
||||
*/
|
||||
int numChildren();
|
||||
|
||||
/**
|
||||
* Indicates whether the priority for the given stream is a direct child of this node.
|
||||
*/
|
||||
boolean hasChild(int streamId);
|
||||
|
||||
/**
|
||||
* Attempts to find a child of this node for the given stream. If not found, returns
|
||||
* {@code null}.
|
||||
*/
|
||||
Priority<T> getChild(int streamId);
|
||||
|
||||
/**
|
||||
* Gets the children nodes that are dependent on this node.
|
||||
*/
|
||||
Collection<? extends Priority<T>> children();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new priority or updates an existing priority for the given stream, using default
|
||||
* priority values.
|
||||
*
|
||||
* @param streamId the stream to be prioritized
|
||||
* @param data optional user-defined data to associate to the stream
|
||||
* @return the priority for the stream.
|
||||
*/
|
||||
Priority<T> prioritizeUsingDefaults(int streamId, T data);
|
||||
|
||||
/**
|
||||
* Adds a new priority or updates an existing priority for the given stream.
|
||||
*
|
||||
* @param streamId the stream to be prioritized.
|
||||
* @param parent an optional stream that the given stream should depend on. Zero, if no
|
||||
* dependency.
|
||||
* @param weight the weight to be assigned to this stream relative to its parent. This value
|
||||
* must be between 1 and 256 (inclusive)
|
||||
* @param exclusive indicates that the stream should be the exclusive dependent on its parent.
|
||||
* This only applies if the stream has a parent.
|
||||
* @param data optional user-defined data to associate to the stream.
|
||||
* @return the priority for the stream.
|
||||
*/
|
||||
Priority<T> prioritize(int streamId, int parent, short weight, boolean exclusive, T data);
|
||||
|
||||
/**
|
||||
* Removes the priority information for the given stream. Adjusts other priorities if necessary.
|
||||
*
|
||||
* @return the data that was associated with the stream or {@code null} if the node was not
|
||||
* found or no data was found in the node.
|
||||
*/
|
||||
T remove(int streamId);
|
||||
|
||||
/**
|
||||
* Gets the total number of streams that have been prioritized in the tree (not counting the
|
||||
* root node).
|
||||
*/
|
||||
int size();
|
||||
|
||||
/**
|
||||
* Returns the root of the tree. The root always exists and represents the connection itself.
|
||||
*/
|
||||
Priority<T> root();
|
||||
|
||||
/**
|
||||
* Returns the priority for the given stream, or {@code null} if not available.
|
||||
*/
|
||||
Priority<T> get(int streamId);
|
||||
}
|
@ -0,0 +1,258 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
/**
|
||||
* Settings for one endpoint in an HTTP/2 connection. Each of the values are optional as defined in
|
||||
* the spec for the SETTINGS frame.
|
||||
*/
|
||||
public class Http2Settings {
|
||||
private static final byte MAX_HEADER_TABLE_SIZE_MASK = 0x1;
|
||||
private static final byte PUSH_ENABLED_MASK = 0x2;
|
||||
private static final byte MAX_CONCURRENT_STREAMS_MASK = 0x4;
|
||||
private static final byte INITIAL_WINDOW_SIZE_MASK = 0x8;
|
||||
private static final byte ALLOW_COMPRESSION_MASK = 0x10;
|
||||
|
||||
private byte enabled;
|
||||
private int maxHeaderTableSize;
|
||||
private boolean pushEnabled;
|
||||
private int maxConcurrentStreams;
|
||||
private int initialWindowSize;
|
||||
private boolean allowCompressedData;
|
||||
|
||||
/**
|
||||
* Indicates whether or not the headerTableSize value is available.
|
||||
*/
|
||||
public boolean hasMaxHeaderTableSize() {
|
||||
return isEnabled(MAX_HEADER_TABLE_SIZE_MASK);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the maximum HPACK header table size or throws {@link NoSuchElementException} if the
|
||||
* value has not been set.
|
||||
*/
|
||||
public int maxHeaderTableSize() {
|
||||
if (!hasMaxHeaderTableSize()) {
|
||||
throw new NoSuchElementException("headerTableSize");
|
||||
}
|
||||
return maxHeaderTableSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the maximum HPACK header table size to the specified value.
|
||||
*/
|
||||
public Http2Settings maxHeaderTableSize(int headerTableSize) {
|
||||
if (headerTableSize < 0) {
|
||||
throw new IllegalArgumentException("headerTableSize must be >= 0");
|
||||
}
|
||||
|
||||
enable(MAX_HEADER_TABLE_SIZE_MASK);
|
||||
this.maxHeaderTableSize = headerTableSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether or not the pushEnabled value is available.
|
||||
*/
|
||||
public boolean hasPushEnabled() {
|
||||
return isEnabled(PUSH_ENABLED_MASK);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets whether or not server push is enabled or throws {@link NoSuchElementException} if the
|
||||
* value has not been set.
|
||||
*/
|
||||
public boolean pushEnabled() {
|
||||
if (!hasPushEnabled()) {
|
||||
throw new NoSuchElementException("pushEnabled");
|
||||
}
|
||||
return pushEnabled;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets whether or not server push is enabled.
|
||||
*/
|
||||
public Http2Settings pushEnabled(boolean pushEnabled) {
|
||||
enable(PUSH_ENABLED_MASK);
|
||||
this.pushEnabled = pushEnabled;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether or not the maxConcurrentStreams value is available.
|
||||
*/
|
||||
public boolean hasMaxConcurrentStreams() {
|
||||
return isEnabled(MAX_CONCURRENT_STREAMS_MASK);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the maximum allowed concurrent streams or throws {@link NoSuchElementException} if the
|
||||
* value has not been set.
|
||||
*/
|
||||
public int maxConcurrentStreams() {
|
||||
if (!hasMaxConcurrentStreams()) {
|
||||
throw new NoSuchElementException("maxConcurrentStreams");
|
||||
}
|
||||
return maxConcurrentStreams;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the maximum allowed concurrent streams to the specified value.
|
||||
*/
|
||||
public Http2Settings maxConcurrentStreams(int maxConcurrentStreams) {
|
||||
if (maxConcurrentStreams < 0) {
|
||||
throw new IllegalArgumentException("maxConcurrentStreams must be >= 0");
|
||||
}
|
||||
enable(MAX_CONCURRENT_STREAMS_MASK);
|
||||
this.maxConcurrentStreams = maxConcurrentStreams;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether or not the initialWindowSize value is available.
|
||||
*/
|
||||
public boolean hasInitialWindowSize() {
|
||||
return isEnabled(INITIAL_WINDOW_SIZE_MASK);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the initial flow control window size or throws {@link NoSuchElementException} if the
|
||||
* value has not been set.
|
||||
*/
|
||||
public int initialWindowSize() {
|
||||
if (!hasInitialWindowSize()) {
|
||||
throw new NoSuchElementException("initialWindowSize");
|
||||
}
|
||||
return initialWindowSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the initial flow control window size to the specified value.
|
||||
*/
|
||||
public Http2Settings initialWindowSize(int initialWindowSize) {
|
||||
if (initialWindowSize < 0) {
|
||||
throw new IllegalArgumentException("initialWindowSize must be >= 0");
|
||||
}
|
||||
enable(INITIAL_WINDOW_SIZE_MASK);
|
||||
this.initialWindowSize = initialWindowSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether or not the allowCompressedData value is available.
|
||||
*/
|
||||
public boolean hasAllowCompressedData() {
|
||||
return isEnabled(ALLOW_COMPRESSION_MASK);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets whether the endpoint allows compressed data or throws {@link NoSuchElementException} if
|
||||
* the value has not been set.
|
||||
*/
|
||||
public boolean allowCompressedData() {
|
||||
if (!hasAllowCompressedData()) {
|
||||
throw new NoSuchElementException("allowCompressedData");
|
||||
}
|
||||
return allowCompressedData;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets whether or not the endpoing allows compressed data.
|
||||
*/
|
||||
public Http2Settings allowCompressedData(boolean allowCompressedData) {
|
||||
enable(ALLOW_COMPRESSION_MASK);
|
||||
this.allowCompressedData = allowCompressedData;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + (allowCompressedData ? 1231 : 1237);
|
||||
result = prime * result + enabled;
|
||||
result = prime * result + maxHeaderTableSize;
|
||||
result = prime * result + initialWindowSize;
|
||||
result = prime * result + maxConcurrentStreams;
|
||||
result = prime * result + (pushEnabled ? 1231 : 1237);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
Http2Settings other = (Http2Settings) obj;
|
||||
if (allowCompressedData != other.allowCompressedData) {
|
||||
return false;
|
||||
}
|
||||
if (enabled != other.enabled) {
|
||||
return false;
|
||||
}
|
||||
if (maxHeaderTableSize != other.maxHeaderTableSize) {
|
||||
return false;
|
||||
}
|
||||
if (initialWindowSize != other.initialWindowSize) {
|
||||
return false;
|
||||
}
|
||||
if (maxConcurrentStreams != other.maxConcurrentStreams) {
|
||||
return false;
|
||||
}
|
||||
if (pushEnabled != other.pushEnabled) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder("Http2Settings [");
|
||||
if (hasMaxHeaderTableSize()) {
|
||||
builder.append("maxHeaderTableSize=").append(maxHeaderTableSize).append(",");
|
||||
}
|
||||
if (hasPushEnabled()) {
|
||||
builder.append("pushEnabled=").append(pushEnabled).append(",");
|
||||
}
|
||||
if (hasMaxConcurrentStreams()) {
|
||||
builder.append("maxConcurrentStreams=").append(maxConcurrentStreams).append(",");
|
||||
}
|
||||
if (hasInitialWindowSize()) {
|
||||
builder.append("initialWindowSize=").append(initialWindowSize).append(",");
|
||||
}
|
||||
if (hasAllowCompressedData()) {
|
||||
builder.append("allowCompressedData=").append(allowCompressedData).append(",");
|
||||
}
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
private void enable(int mask) {
|
||||
enabled |= mask;
|
||||
}
|
||||
|
||||
private boolean isEnabled(int mask) {
|
||||
return (enabled & mask) > 0;
|
||||
}
|
||||
}
|
@ -13,12 +13,7 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.connection;
|
||||
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Error;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
/**
|
||||
* A single stream within an HTTP2 connection. Streams are compared to each other by priority.
|
||||
@ -29,67 +24,61 @@ public interface Http2Stream {
|
||||
* The allowed states of an HTTP2 stream.
|
||||
*/
|
||||
enum State {
|
||||
IDLE, RESERVED_LOCAL, RESERVED_REMOTE, OPEN, HALF_CLOSED_LOCAL, HALF_CLOSED_REMOTE, CLOSED
|
||||
IDLE,
|
||||
RESERVED_LOCAL,
|
||||
RESERVED_REMOTE,
|
||||
OPEN,
|
||||
HALF_CLOSED_LOCAL,
|
||||
HALF_CLOSED_REMOTE,
|
||||
CLOSED
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the unique identifier for this stream within the connection.
|
||||
*/
|
||||
int getId();
|
||||
int id();
|
||||
|
||||
/**
|
||||
* Gets the state of this stream.
|
||||
*/
|
||||
State getState();
|
||||
State state();
|
||||
|
||||
/**
|
||||
* Verifies that the stream is in one of the given allowed states.
|
||||
*/
|
||||
void verifyState(Http2Error error, State... allowedStates) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Sets the priority of this stream. A value of zero is the highest priority and a value of
|
||||
* {@link Integer#MAX_VALUE} is the lowest.
|
||||
*/
|
||||
void setPriority(int priority) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Gets the priority of this stream. A value of zero is the highest priority and a value of
|
||||
* {@link Integer#MAX_VALUE} is the lowest.
|
||||
*/
|
||||
int getPriority();
|
||||
Http2Stream verifyState(Http2Error error, State... allowedStates) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* If this is a reserved push stream, opens the stream for push in one direction.
|
||||
*/
|
||||
void openForPush() throws Http2Exception;
|
||||
Http2Stream openForPush() throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Closes the stream.
|
||||
*/
|
||||
void close(ChannelHandlerContext ctx, ChannelFuture future);
|
||||
Http2Stream close();
|
||||
|
||||
/**
|
||||
* Closes the local side of this stream. If this makes the stream closed, the child is closed as
|
||||
* well.
|
||||
*/
|
||||
void closeLocalSide(ChannelHandlerContext ctx, ChannelFuture future);
|
||||
Http2Stream closeLocalSide();
|
||||
|
||||
/**
|
||||
* Closes the remote side of this stream. If this makes the stream closed, the child is closed as
|
||||
* well.
|
||||
* Closes the remote side of this stream. If this makes the stream closed, the child is closed
|
||||
* as well.
|
||||
*/
|
||||
void closeRemoteSide(ChannelHandlerContext ctx, ChannelFuture future);
|
||||
Http2Stream closeRemoteSide();
|
||||
|
||||
/**
|
||||
* Indicates whether the remote side of this stream is open (i.e. the state is either
|
||||
* {@link State#OPEN} or {@link State#HALF_CLOSED_LOCAL}).
|
||||
*/
|
||||
boolean isRemoteSideOpen();
|
||||
boolean remoteSideOpen();
|
||||
|
||||
/**
|
||||
* Indicates whether the local side of this stream is open (i.e. the state is either
|
||||
* {@link State#OPEN} or {@link State#HALF_CLOSED_REMOTE}).
|
||||
*/
|
||||
boolean isLocalSideOpen();
|
||||
boolean localSideOpen();
|
||||
}
|
@ -13,7 +13,7 @@
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
public class Http2StreamException extends Http2Exception {
|
||||
|
||||
@ -30,7 +30,7 @@ public class Http2StreamException extends Http2Exception {
|
||||
this.streamId = streamId;
|
||||
}
|
||||
|
||||
public int getStreamId() {
|
||||
public int streamId() {
|
||||
return streamId;
|
||||
}
|
||||
}
|
@ -1,463 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.connection;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Error.NO_ERROR;
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.format;
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.connection.Http2ConnectionUtil.toByteBuf;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.DEFAULT_STREAM_PRIORITY;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelFutureListener;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Error;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.connection.Http2Stream.State;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2GoAwayFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2GoAwayFrame;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
public class DefaultHttp2Connection implements Http2Connection {
|
||||
/**
|
||||
* Used to sort streams in the activeStreams set. Sort by priority first, then by stream ID.
|
||||
* Streams with the same ID are considered equal.
|
||||
*/
|
||||
private static final Comparator<Http2Stream> STREAM_COMPARATOR = new Comparator<Http2Stream>() {
|
||||
@Override
|
||||
public int compare(Http2Stream s1, Http2Stream s2) {
|
||||
int p1 = s1.getPriority();
|
||||
int p2 = s2.getPriority();
|
||||
|
||||
// Sort streams with the same priority by their ID.
|
||||
if (p1 == p2) {
|
||||
return s1.getId() - s2.getId();
|
||||
}
|
||||
return p1 - p2;
|
||||
}
|
||||
};
|
||||
|
||||
private final List<Listener> listeners = new ArrayList<Listener>();
|
||||
private final Map<Integer, Http2Stream> streamMap = new HashMap<Integer, Http2Stream>();
|
||||
private final Set<Http2Stream> activeStreams = new TreeSet<Http2Stream>(STREAM_COMPARATOR);
|
||||
private final DefaultEndpoint localEndpoint;
|
||||
private final DefaultEndpoint remoteEndpoint;
|
||||
private boolean goAwaySent;
|
||||
private boolean goAwayReceived;
|
||||
private ChannelFutureListener closeListener;
|
||||
|
||||
public DefaultHttp2Connection(boolean server) {
|
||||
localEndpoint = new DefaultEndpoint(server);
|
||||
remoteEndpoint = new DefaultEndpoint(!server);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addListener(Listener listener) {
|
||||
listeners.add(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeListener(Listener listener) {
|
||||
listeners.remove(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream getStreamOrFail(int streamId) throws Http2Exception {
|
||||
Http2Stream stream = getStream(streamId);
|
||||
if (stream == null) {
|
||||
throw protocolError("Stream does not exist %d", streamId);
|
||||
}
|
||||
return stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Stream getStream(int streamId) {
|
||||
return streamMap.get(streamId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Http2Stream> getActiveStreams() {
|
||||
// Copy the list in case any operation on the returned streams causes the activeStreams set
|
||||
// to change.
|
||||
return Collections.unmodifiableSet(activeStreams);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoint local() {
|
||||
return localEndpoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoint remote() {
|
||||
return remoteEndpoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendGoAway(ChannelHandlerContext ctx, ChannelPromise promise, Http2Exception cause) {
|
||||
closeListener = getOrCreateCloseListener(ctx, promise);
|
||||
ChannelFuture future;
|
||||
if (!goAwaySent) {
|
||||
goAwaySent = true;
|
||||
|
||||
int errorCode = cause != null ? cause.getError().getCode() : NO_ERROR.getCode();
|
||||
ByteBuf debugData = toByteBuf(ctx, cause);
|
||||
|
||||
Http2GoAwayFrame frame = new DefaultHttp2GoAwayFrame.Builder().setErrorCode(errorCode)
|
||||
.setLastStreamId(remote().getLastStreamCreated()).setDebugData(debugData).build();
|
||||
future = ctx.writeAndFlush(frame);
|
||||
} else {
|
||||
future = ctx.newSucceededFuture();
|
||||
}
|
||||
|
||||
// If there are no active streams, close immediately after the send is complete.
|
||||
// Otherwise wait until all streams are inactive.
|
||||
if (cause != null || activeStreams.isEmpty()) {
|
||||
future.addListener(closeListener);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void goAwayReceived() {
|
||||
goAwayReceived = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGoAwaySent() {
|
||||
return goAwaySent;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGoAwayReceived() {
|
||||
return goAwayReceived;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGoAway() {
|
||||
return isGoAwaySent() || isGoAwayReceived();
|
||||
}
|
||||
|
||||
private ChannelFutureListener getOrCreateCloseListener(final ChannelHandlerContext ctx,
|
||||
final ChannelPromise promise) {
|
||||
if (closeListener == null) {
|
||||
closeListener = new ChannelFutureListener() {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
ctx.close(promise);
|
||||
}
|
||||
};
|
||||
}
|
||||
return closeListener;
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple stream implementation. Streams can be compared to each other by priority.
|
||||
*/
|
||||
private class DefaultStream implements Http2Stream {
|
||||
private final int id;
|
||||
private State state = State.IDLE;
|
||||
private int priority;
|
||||
|
||||
public DefaultStream(int id) {
|
||||
this.id = id;
|
||||
priority = DEFAULT_STREAM_PRIORITY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public State getState() {
|
||||
return state;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void verifyState(Http2Error error, State... allowedStates) throws Http2Exception {
|
||||
for (State allowedState : allowedStates) {
|
||||
if (state == allowedState) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
throw format(error, "Stream %d in unexpected state: %s", id, state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPriority(int priority) throws Http2Exception {
|
||||
if (priority < 0) {
|
||||
throw protocolError("Invalid priority: %d", priority);
|
||||
}
|
||||
|
||||
// If it was active, we must remove it from the set before changing the priority.
|
||||
// Otherwise it won't be able to locate the stream in the set.
|
||||
boolean wasActive = activeStreams.remove(this);
|
||||
this.priority = priority;
|
||||
|
||||
// If this stream was in the active set, re-add it so that it's properly sorted.
|
||||
if (wasActive) {
|
||||
activeStreams.add(this);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPriority() {
|
||||
return priority;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void openForPush() throws Http2Exception {
|
||||
switch (state) {
|
||||
case RESERVED_LOCAL:
|
||||
state = State.HALF_CLOSED_REMOTE;
|
||||
break;
|
||||
case RESERVED_REMOTE:
|
||||
state = State.HALF_CLOSED_LOCAL;
|
||||
break;
|
||||
default:
|
||||
throw protocolError("Attempting to open non-reserved stream for push");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(ChannelHandlerContext ctx, ChannelFuture future) {
|
||||
if (state == State.CLOSED) {
|
||||
return;
|
||||
}
|
||||
|
||||
state = State.CLOSED;
|
||||
activeStreams.remove(this);
|
||||
streamMap.remove(id);
|
||||
notifyStreamClosed(id);
|
||||
|
||||
// If this connection is closing and there are no longer any
|
||||
// active streams, close after the current operation completes.
|
||||
if (closeListener != null && activeStreams.isEmpty()) {
|
||||
future.addListener(closeListener);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void closeLocalSide(ChannelHandlerContext ctx, ChannelFuture future) {
|
||||
switch (state) {
|
||||
case OPEN:
|
||||
case HALF_CLOSED_LOCAL:
|
||||
state = State.HALF_CLOSED_LOCAL;
|
||||
break;
|
||||
case HALF_CLOSED_REMOTE:
|
||||
case RESERVED_LOCAL:
|
||||
case RESERVED_REMOTE:
|
||||
case IDLE:
|
||||
case CLOSED:
|
||||
default:
|
||||
close(ctx, future);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void closeRemoteSide(ChannelHandlerContext ctx, ChannelFuture future) {
|
||||
switch (state) {
|
||||
case OPEN:
|
||||
case HALF_CLOSED_REMOTE:
|
||||
state = State.HALF_CLOSED_REMOTE;
|
||||
break;
|
||||
case RESERVED_LOCAL:
|
||||
case RESERVED_REMOTE:
|
||||
case IDLE:
|
||||
case HALF_CLOSED_LOCAL:
|
||||
case CLOSED:
|
||||
default:
|
||||
close(ctx, future);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRemoteSideOpen() {
|
||||
switch (state) {
|
||||
case HALF_CLOSED_LOCAL:
|
||||
case OPEN:
|
||||
case RESERVED_REMOTE:
|
||||
return true;
|
||||
case IDLE:
|
||||
case RESERVED_LOCAL:
|
||||
case HALF_CLOSED_REMOTE:
|
||||
case CLOSED:
|
||||
default:
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLocalSideOpen() {
|
||||
switch (state) {
|
||||
case HALF_CLOSED_REMOTE:
|
||||
case OPEN:
|
||||
case RESERVED_LOCAL:
|
||||
return true;
|
||||
case IDLE:
|
||||
case RESERVED_REMOTE:
|
||||
case HALF_CLOSED_LOCAL:
|
||||
case CLOSED:
|
||||
default:
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private void notifyStreamClosed(int id) {
|
||||
for (Listener listener : listeners) {
|
||||
listener.streamClosed(id);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple endpoint implementation.
|
||||
*/
|
||||
private class DefaultEndpoint implements Endpoint {
|
||||
private int nextStreamId;
|
||||
private int lastStreamCreated;
|
||||
private int maxStreams = Integer.MAX_VALUE;
|
||||
private boolean pushToAllowed = true;
|
||||
|
||||
public DefaultEndpoint(boolean serverEndpoint) {
|
||||
// Determine the starting stream ID for this endpoint. Zero is reserved for the
|
||||
// connection and 1 is reserved for responding to an upgrade from HTTP 1.1.
|
||||
// Client-initiated streams use odd identifiers and server-initiated streams use
|
||||
// even.
|
||||
nextStreamId = serverEndpoint ? 2 : 3;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultStream createStream(int streamId, int priority, boolean halfClosed)
|
||||
throws Http2Exception {
|
||||
checkNewStreamAllowed(streamId);
|
||||
|
||||
// Create and initialize the stream.
|
||||
DefaultStream stream = new DefaultStream(streamId);
|
||||
stream.setPriority(priority);
|
||||
if (halfClosed) {
|
||||
stream.state = isLocal() ? State.HALF_CLOSED_LOCAL : State.HALF_CLOSED_REMOTE;
|
||||
} else {
|
||||
stream.state = State.OPEN;
|
||||
}
|
||||
|
||||
// Update the next and last stream IDs.
|
||||
nextStreamId += 2;
|
||||
lastStreamCreated = streamId;
|
||||
|
||||
// Register the stream and mark it as active.
|
||||
streamMap.put(streamId, stream);
|
||||
activeStreams.add(stream);
|
||||
|
||||
notifyStreamCreated(streamId);
|
||||
return stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultStream reservePushStream(int streamId, Http2Stream parent) throws Http2Exception {
|
||||
if (parent == null) {
|
||||
throw protocolError("Parent stream missing");
|
||||
}
|
||||
if (isLocal() ? !parent.isLocalSideOpen() : !parent.isRemoteSideOpen()) {
|
||||
throw protocolError("Stream %d is not open for sending push promise", parent.getId());
|
||||
}
|
||||
if (!opposite().isPushToAllowed()) {
|
||||
throw protocolError("Server push not allowed to opposite endpoint.");
|
||||
}
|
||||
|
||||
// Create and initialize the stream.
|
||||
DefaultStream stream = new DefaultStream(streamId);
|
||||
stream.setPriority(parent.getPriority() + 1);
|
||||
stream.state = isLocal() ? State.RESERVED_LOCAL : State.RESERVED_REMOTE;
|
||||
|
||||
// Update the next and last stream IDs.
|
||||
nextStreamId += 2;
|
||||
lastStreamCreated = streamId;
|
||||
|
||||
// Register the stream.
|
||||
streamMap.put(streamId, stream);
|
||||
|
||||
notifyStreamCreated(streamId);
|
||||
return stream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPushToAllowed(boolean allow) {
|
||||
pushToAllowed = allow;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPushToAllowed() {
|
||||
return pushToAllowed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxStreams() {
|
||||
return maxStreams;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setMaxStreams(int maxStreams) {
|
||||
this.maxStreams = maxStreams;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLastStreamCreated() {
|
||||
return lastStreamCreated;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Endpoint opposite() {
|
||||
return isLocal() ? remoteEndpoint : localEndpoint;
|
||||
}
|
||||
|
||||
private void checkNewStreamAllowed(int streamId) throws Http2Exception {
|
||||
if (isGoAway()) {
|
||||
throw protocolError("Cannot create a stream since the connection is going away");
|
||||
}
|
||||
if (nextStreamId < 0) {
|
||||
throw protocolError("No more streams can be created on this connection");
|
||||
}
|
||||
if (streamId != nextStreamId) {
|
||||
throw protocolError("Incorrect next stream ID requested: %d", streamId);
|
||||
}
|
||||
if (streamMap.size() + 1 > maxStreams) {
|
||||
// TODO(nathanmittler): is this right?
|
||||
throw protocolError("Maximum streams exceeded for this endpoint.");
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isLocal() {
|
||||
return this == localEndpoint;
|
||||
}
|
||||
|
||||
private void notifyStreamCreated(int id) {
|
||||
for (Listener listener : listeners) {
|
||||
listener.streamCreated(id);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -1,298 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.connection;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Error.FLOW_CONTROL_ERROR;
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Error.STREAM_CLOSED;
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.format;
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.connection.Http2ConnectionUtil.DEFAULT_FLOW_CONTROL_WINDOW_SIZE;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.CONNECTION_STREAM_ID;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.Http2StreamException;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2DataFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2DataFrame;
|
||||
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
|
||||
/**
|
||||
* Basic implementation of {@link OutboundFlowController}.
|
||||
*/
|
||||
public class DefaultOutboundFlowController implements OutboundFlowController {
|
||||
|
||||
private final Http2Connection connection;
|
||||
private final Map<Integer, StreamState> streamStates = new HashMap<Integer, StreamState>();
|
||||
private int initialWindowSize = DEFAULT_FLOW_CONTROL_WINDOW_SIZE;
|
||||
private int connectionWindowSize = DEFAULT_FLOW_CONTROL_WINDOW_SIZE;
|
||||
|
||||
public DefaultOutboundFlowController(Http2Connection connection) {
|
||||
if (connection == null) {
|
||||
throw new NullPointerException("connection");
|
||||
}
|
||||
this.connection = connection;
|
||||
connection.addListener(new Http2Connection.Listener() {
|
||||
@Override
|
||||
public void streamCreated(int streamId) {
|
||||
streamStates.put(streamId, new StreamState(streamId));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void streamClosed(int streamId) {
|
||||
StreamState state = streamStates.remove(streamId);
|
||||
if (state != null) {
|
||||
state.clearPendingWrites();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setInitialOutboundWindowSize(int newWindowSize) throws Http2Exception {
|
||||
int delta = newWindowSize - initialWindowSize;
|
||||
initialWindowSize = newWindowSize;
|
||||
addAndGetConnectionWindowSize(delta);
|
||||
for (StreamState window : streamStates.values()) {
|
||||
// Verify that the maximum value is not exceeded by this change.
|
||||
window.addAndGetWindow(delta);
|
||||
}
|
||||
|
||||
if (delta > 0) {
|
||||
// The window size increased, send any pending frames for all streams.
|
||||
writePendingFrames();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInitialOutboundWindowSize() {
|
||||
return initialWindowSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateOutboundWindowSize(int streamId, int delta) throws Http2Exception {
|
||||
StreamState streamWindow;
|
||||
if (streamId == CONNECTION_STREAM_ID) {
|
||||
// Update the connection window and write any pending frames for all streams.
|
||||
addAndGetConnectionWindowSize(delta);
|
||||
writePendingFrames();
|
||||
} else {
|
||||
// Update the stream window and write any pending frames for the stream.
|
||||
streamWindow = getStateOrFail(streamId);
|
||||
streamWindow.addAndGetWindow(delta);
|
||||
streamWindow.writePendingFrames(Integer.MAX_VALUE);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendFlowControlled(Http2DataFrame frame, FrameWriter frameWriter)
|
||||
throws Http2Exception {
|
||||
|
||||
StreamState streamState = getStateOrFail(frame.getStreamId());
|
||||
|
||||
int dataLength = frame.content().readableBytes();
|
||||
if (streamState.writableWindow() >= dataLength) {
|
||||
// Window size is large enough to send entire data frame
|
||||
writeFrame(frame, streamState, frameWriter);
|
||||
return;
|
||||
}
|
||||
|
||||
// Enqueue the frame to be written when the window size permits.
|
||||
streamState.addPendingWrite(new PendingWrite(frame, frameWriter));
|
||||
|
||||
if (streamState.writableWindow() <= 0) {
|
||||
// Stream is stalled, don't send anything now.
|
||||
return;
|
||||
}
|
||||
|
||||
// Create and send a partial frame up to the window size.
|
||||
Http2DataFrame partialFrame = readPartialFrame(frame, streamState.writableWindow());
|
||||
writeFrame(partialFrame, streamState, frameWriter);
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempts to get the {@link StreamState} for the given stream. If not available, raises a
|
||||
* {@code PROTOCOL_ERROR}.
|
||||
*/
|
||||
private StreamState getStateOrFail(int streamId) throws Http2Exception {
|
||||
StreamState streamState = streamStates.get(streamId);
|
||||
if (streamState == null) {
|
||||
throw protocolError("Missing flow control window for stream: %d", streamId);
|
||||
}
|
||||
return streamState;
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes the frame and decrements the stream and connection window sizes.
|
||||
*/
|
||||
private void writeFrame(Http2DataFrame frame, StreamState state, FrameWriter frameWriter)
|
||||
throws Http2Exception {
|
||||
int dataLength = frame.content().readableBytes();
|
||||
connectionWindowSize -= dataLength;
|
||||
state.addAndGetWindow(-dataLength);
|
||||
frameWriter.writeFrame(frame);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a view of the given frame starting at the current read index with the given number of
|
||||
* bytes. The reader index on the input frame is then advanced by the number of bytes. The
|
||||
* returned frame will not have end-of-stream set.
|
||||
*/
|
||||
private static Http2DataFrame readPartialFrame(Http2DataFrame frame, int numBytes) {
|
||||
return new DefaultHttp2DataFrame.Builder().setStreamId(frame.getStreamId())
|
||||
.setContent(frame.content().readSlice(numBytes).retain()).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether applying the delta to the given value will cause an integer overflow.
|
||||
*/
|
||||
private static boolean isIntegerOverflow(int previousValue, int delta) {
|
||||
return delta > 0 && (Integer.MAX_VALUE - delta) < previousValue;
|
||||
}
|
||||
|
||||
/**
|
||||
* Increments the connectionWindowSize and returns the new value.
|
||||
*/
|
||||
private int addAndGetConnectionWindowSize(int delta) throws Http2Exception {
|
||||
if (isIntegerOverflow(connectionWindowSize, delta)) {
|
||||
throw format(FLOW_CONTROL_ERROR, "Window update exceeds maximum for connection");
|
||||
}
|
||||
return connectionWindowSize += delta;
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes any pending frames for the entire connection.
|
||||
*/
|
||||
private void writePendingFrames() throws Http2Exception {
|
||||
// The request for for the entire connection, write any pending frames across
|
||||
// all active streams. Active streams are already sorted by their priority.
|
||||
for (Http2Stream stream : connection.getActiveStreams()) {
|
||||
StreamState state = getStateOrFail(stream.getId());
|
||||
state.writePendingFrames(1);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The outbound flow control state for a single stream.
|
||||
*/
|
||||
private class StreamState {
|
||||
private final int streamId;
|
||||
private final Queue<PendingWrite> pendingWriteQueue = new ArrayDeque<PendingWrite>();
|
||||
private int windowSize = initialWindowSize;
|
||||
|
||||
public StreamState(int streamId) {
|
||||
this.streamId = streamId;
|
||||
}
|
||||
|
||||
public int addAndGetWindow(int delta) throws Http2Exception {
|
||||
if (isIntegerOverflow(windowSize, delta)) {
|
||||
throw new Http2StreamException(streamId, FLOW_CONTROL_ERROR,
|
||||
"Window size overflow for stream");
|
||||
}
|
||||
windowSize += delta;
|
||||
return windowSize;
|
||||
}
|
||||
|
||||
public int writableWindow() {
|
||||
return Math.min(windowSize, connectionWindowSize);
|
||||
}
|
||||
|
||||
public void addPendingWrite(PendingWrite pendingWrite) {
|
||||
pendingWriteQueue.offer(pendingWrite);
|
||||
}
|
||||
|
||||
public boolean hasPendingWrite() {
|
||||
return !pendingWriteQueue.isEmpty();
|
||||
}
|
||||
|
||||
public PendingWrite peekPendingWrite() {
|
||||
if (windowSize > 0) {
|
||||
return pendingWriteQueue.peek();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public void removePendingWrite() {
|
||||
pendingWriteQueue.poll();
|
||||
}
|
||||
|
||||
public void clearPendingWrites() {
|
||||
while (true) {
|
||||
PendingWrite pendingWrite = pendingWriteQueue.poll();
|
||||
if (pendingWrite == null) {
|
||||
break;
|
||||
}
|
||||
pendingWrite.writeError(
|
||||
format(STREAM_CLOSED, "Stream closed before write could take place"));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends all pending writes for this stream so long as there is space the the stream and
|
||||
* connection windows.
|
||||
*
|
||||
* @param maxFrames the maximum number of frames to send.
|
||||
*/
|
||||
public void writePendingFrames(int maxFrames) throws Http2Exception {
|
||||
while (maxFrames > 0 && writableWindow() > 0 && hasPendingWrite()) {
|
||||
maxFrames--;
|
||||
PendingWrite pendingWrite = peekPendingWrite();
|
||||
|
||||
if (writableWindow() >= pendingWrite.size()) {
|
||||
// Window size is large enough to send entire data frame
|
||||
removePendingWrite();
|
||||
writeFrame(pendingWrite.frame(), this, pendingWrite.writer());
|
||||
} else {
|
||||
// We can send a partial frame
|
||||
Http2DataFrame partialDataFrame =
|
||||
readPartialFrame(pendingWrite.frame(), writableWindow());
|
||||
writeFrame(partialDataFrame, this, pendingWrite.writer());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Pending write for a single data frame.
|
||||
*/
|
||||
private static class PendingWrite {
|
||||
private final Http2DataFrame frame;
|
||||
private final FrameWriter writer;
|
||||
|
||||
public PendingWrite(Http2DataFrame frame, FrameWriter writer) {
|
||||
this.frame = frame;
|
||||
this.writer = writer;
|
||||
}
|
||||
|
||||
public Http2DataFrame frame() {
|
||||
return frame;
|
||||
}
|
||||
|
||||
public FrameWriter writer() {
|
||||
return writer;
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return frame.content().readableBytes();
|
||||
}
|
||||
|
||||
public void writeError(Http2Exception cause) {
|
||||
frame.release();
|
||||
writer.setFailure(cause);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,672 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.connection;
|
||||
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelFutureListener;
|
||||
import io.netty.channel.ChannelHandlerAdapter;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.Http2StreamException;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2PingFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2RstStreamFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2SettingsFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2DataFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2GoAwayFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2HeadersFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PingFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PriorityFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PushPromiseFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2RstStreamFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2SettingsFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2StreamFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2WindowUpdateFrame;
|
||||
import io.netty.util.ReferenceCountUtil;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Error.*;
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.*;
|
||||
import static io.netty.handler.codec.http2.draft10.connection.Http2ConnectionUtil.*;
|
||||
import static io.netty.handler.codec.http2.draft10.connection.Http2Stream.State.*;
|
||||
|
||||
/**
|
||||
* Handler for HTTP/2 connection state. Manages inbound and outbound flow control for data frames.
|
||||
* Handles error conditions as defined by the HTTP/2 spec and controls appropriate shutdown of the
|
||||
* connection.
|
||||
* <p>
|
||||
* Propagates the following inbound frames to downstream handlers:<br>
|
||||
* {@link Http2DataFrame}<br>
|
||||
* {@link Http2HeadersFrame}<br>
|
||||
* {@link Http2PushPromiseFrame}<br>
|
||||
* {@link Http2PriorityFrame}<br>
|
||||
* {@link Http2RstStreamFrame}<br>
|
||||
* {@link Http2GoAwayFrame}<br>
|
||||
* {@link Http2WindowUpdateFrame}<br>
|
||||
* {@link Http2SettingsFrame}<br>
|
||||
* <p>
|
||||
* The following outbound frames are allowed from downstream handlers:<br>
|
||||
* {@link Http2DataFrame}<br>
|
||||
* {@link Http2HeadersFrame}<br>
|
||||
* {@link Http2PushPromiseFrame}<br>
|
||||
* {@link Http2PriorityFrame}<br>
|
||||
* {@link Http2RstStreamFrame}<br>
|
||||
* {@link Http2PingFrame} (non-ack)<br>
|
||||
* {@link Http2SettingsFrame} (non-ack)<br>
|
||||
* <p>
|
||||
* All outbound frames are disallowed after a connection shutdown has begun by sending a goAway
|
||||
* frame to the remote endpoint. In addition, no outbound frames are allowed until the first non-ack
|
||||
* settings frame is received from the remote endpoint.
|
||||
*/
|
||||
public class Http2ConnectionHandler extends ChannelHandlerAdapter {
|
||||
|
||||
private final Http2Connection connection;
|
||||
private final InboundFlowController inboundFlow;
|
||||
private final OutboundFlowController outboundFlow;
|
||||
private boolean initialSettingsSent;
|
||||
private boolean initialSettingsReceived;
|
||||
|
||||
public Http2ConnectionHandler(boolean server) {
|
||||
this(new DefaultHttp2Connection(server));
|
||||
}
|
||||
|
||||
public Http2ConnectionHandler(Http2Connection connection) {
|
||||
this(connection, new DefaultInboundFlowController(connection),
|
||||
new DefaultOutboundFlowController(connection));
|
||||
}
|
||||
|
||||
public Http2ConnectionHandler(final Http2Connection connection,
|
||||
final InboundFlowController inboundFlow, final OutboundFlowController outboundFlow) {
|
||||
if (connection == null) {
|
||||
throw new NullPointerException("connection");
|
||||
}
|
||||
if (inboundFlow == null) {
|
||||
throw new NullPointerException("inboundFlow");
|
||||
}
|
||||
if (outboundFlow == null) {
|
||||
throw new NullPointerException("outboundFlow");
|
||||
}
|
||||
this.connection = connection;
|
||||
this.inboundFlow = inboundFlow;
|
||||
this.outboundFlow = outboundFlow;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelActive(ChannelHandlerContext ctx) throws Exception {
|
||||
// The channel just became active - send the initial settings frame to the remote
|
||||
// endpoint.
|
||||
sendInitialSettings(ctx);
|
||||
super.handlerAdded(ctx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
// This handler was just added to the context. In case it was handled after
|
||||
// the connection became active, send the initial settings frame now.
|
||||
sendInitialSettings(ctx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
|
||||
// Avoid NotYetConnectedException
|
||||
if (!ctx.channel().isActive()) {
|
||||
ctx.close(promise);
|
||||
return;
|
||||
}
|
||||
|
||||
connection.sendGoAway(ctx, promise, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
|
||||
for (Http2Stream stream : connection.getActiveStreams()) {
|
||||
stream.close(ctx, ctx.newSucceededFuture());
|
||||
}
|
||||
ctx.fireChannelInactive();
|
||||
}
|
||||
|
||||
/**
|
||||
* Handles {@link Http2Exception} objects that were thrown from other handlers. Ignores all other
|
||||
* exceptions.
|
||||
*/
|
||||
@Override
|
||||
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
|
||||
if (cause instanceof Http2Exception) {
|
||||
processHttp2Exception(ctx, (Http2Exception) cause);
|
||||
}
|
||||
|
||||
super.exceptionCaught(ctx, cause);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelRead(ChannelHandlerContext ctx, Object inMsg) throws Exception {
|
||||
try {
|
||||
if (inMsg instanceof Http2DataFrame) {
|
||||
handleInboundData(ctx, (Http2DataFrame) inMsg);
|
||||
} else if (inMsg instanceof Http2HeadersFrame) {
|
||||
handleInboundHeaders(ctx, (Http2HeadersFrame) inMsg);
|
||||
} else if (inMsg instanceof Http2PushPromiseFrame) {
|
||||
handleInboundPushPromise(ctx, (Http2PushPromiseFrame) inMsg);
|
||||
} else if (inMsg instanceof Http2PriorityFrame) {
|
||||
handleInboundPriority(ctx, (Http2PriorityFrame) inMsg);
|
||||
} else if (inMsg instanceof Http2RstStreamFrame) {
|
||||
handleInboundRstStream(ctx, (Http2RstStreamFrame) inMsg);
|
||||
} else if (inMsg instanceof Http2PingFrame) {
|
||||
handleInboundPing(ctx, (Http2PingFrame) inMsg);
|
||||
} else if (inMsg instanceof Http2GoAwayFrame) {
|
||||
handleInboundGoAway(ctx, (Http2GoAwayFrame) inMsg);
|
||||
} else if (inMsg instanceof Http2WindowUpdateFrame) {
|
||||
handleInboundWindowUpdate(ctx, (Http2WindowUpdateFrame) inMsg);
|
||||
} else if (inMsg instanceof Http2SettingsFrame) {
|
||||
handleInboundSettings(ctx, (Http2SettingsFrame) inMsg);
|
||||
} else {
|
||||
ctx.fireChannelRead(inMsg);
|
||||
}
|
||||
|
||||
} catch (Http2Exception e) {
|
||||
ReferenceCountUtil.release(inMsg);
|
||||
processHttp2Exception(ctx, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
|
||||
throws Exception {
|
||||
try {
|
||||
if (msg instanceof Http2DataFrame) {
|
||||
handleOutboundData(ctx, (Http2DataFrame) msg, promise);
|
||||
} else if (msg instanceof Http2HeadersFrame) {
|
||||
handleOutboundHeaders(ctx, (Http2HeadersFrame) msg, promise);
|
||||
} else if (msg instanceof Http2PushPromiseFrame) {
|
||||
handleOutboundPushPromise(ctx, (Http2PushPromiseFrame) msg, promise);
|
||||
} else if (msg instanceof Http2PriorityFrame) {
|
||||
handleOutboundPriority(ctx, (Http2PriorityFrame) msg, promise);
|
||||
} else if (msg instanceof Http2RstStreamFrame) {
|
||||
handleOutboundRstStream(ctx, (Http2RstStreamFrame) msg, promise);
|
||||
} else if (msg instanceof Http2PingFrame) {
|
||||
handleOutboundPing(ctx, (Http2PingFrame) msg, promise);
|
||||
} else if (msg instanceof Http2GoAwayFrame) {
|
||||
handleOutboundGoAway();
|
||||
} else if (msg instanceof Http2WindowUpdateFrame) {
|
||||
handleOutboundWindowUpdate();
|
||||
} else if (msg instanceof Http2SettingsFrame) {
|
||||
handleOutboundSettings(ctx, (Http2SettingsFrame) msg, promise);
|
||||
} else {
|
||||
ctx.write(msg, promise);
|
||||
}
|
||||
|
||||
} catch (Throwable e) {
|
||||
ReferenceCountUtil.release(msg);
|
||||
promise.setFailure(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Processes the given exception. Depending on the type of exception, delegates to either
|
||||
* {@link #processConnectionError} or {@link #processStreamError}.
|
||||
*/
|
||||
private void processHttp2Exception(ChannelHandlerContext ctx, Http2Exception e) {
|
||||
if (e instanceof Http2StreamException) {
|
||||
processStreamError(ctx, (Http2StreamException) e);
|
||||
} else {
|
||||
processConnectionError(ctx, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void processConnectionError(ChannelHandlerContext ctx, Http2Exception cause) {
|
||||
connection.sendGoAway(ctx, ctx.newPromise(), cause);
|
||||
}
|
||||
|
||||
private void processStreamError(ChannelHandlerContext ctx, Http2StreamException cause) {
|
||||
// Close the stream if it was open.
|
||||
int streamId = cause.getStreamId();
|
||||
ChannelPromise promise = ctx.newPromise();
|
||||
Http2Stream stream = connection.getStream(streamId);
|
||||
if (stream != null) {
|
||||
stream.close(ctx, promise);
|
||||
}
|
||||
|
||||
// Send the Rst frame to the remote endpoint.
|
||||
Http2RstStreamFrame frame = new DefaultHttp2RstStreamFrame.Builder().setStreamId(streamId)
|
||||
.setErrorCode(cause.getError().getCode()).build();
|
||||
ctx.writeAndFlush(frame, promise);
|
||||
}
|
||||
|
||||
private void handleInboundData(final ChannelHandlerContext ctx, Http2DataFrame frame)
|
||||
throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
// Check if we received a data frame for a stream which is half-closed
|
||||
Http2Stream stream = connection.getStreamOrFail(frame.getStreamId());
|
||||
stream.verifyState(STREAM_CLOSED, OPEN, HALF_CLOSED_LOCAL);
|
||||
|
||||
// Apply flow control.
|
||||
inboundFlow.applyInboundFlowControl(frame, new InboundFlowController.FrameWriter() {
|
||||
@Override
|
||||
public void writeFrame(Http2WindowUpdateFrame frame) {
|
||||
ctx.writeAndFlush(frame);
|
||||
}
|
||||
});
|
||||
|
||||
if (isInboundStreamAfterGoAway(frame)) {
|
||||
// Ignore frames for any stream created after we sent a go-away.
|
||||
frame.release();
|
||||
return;
|
||||
}
|
||||
|
||||
if (frame.isEndOfStream()) {
|
||||
stream.closeRemoteSide(ctx, ctx.newSucceededFuture());
|
||||
}
|
||||
|
||||
// Allow this frame to continue other handlers.
|
||||
ctx.fireChannelRead(frame);
|
||||
}
|
||||
|
||||
private void handleInboundHeaders(ChannelHandlerContext ctx, Http2HeadersFrame frame)
|
||||
throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(frame)) {
|
||||
return;
|
||||
}
|
||||
|
||||
int streamId = frame.getStreamId();
|
||||
Http2Stream stream = connection.getStream(streamId);
|
||||
if (stream == null) {
|
||||
// Create the new stream.
|
||||
connection.remote().createStream(frame.getStreamId(), frame.getPriority(),
|
||||
frame.isEndOfStream());
|
||||
} else {
|
||||
if (stream.getState() == RESERVED_REMOTE) {
|
||||
// Received headers for a reserved push stream ... open it for push to the
|
||||
// local endpoint.
|
||||
stream.verifyState(PROTOCOL_ERROR, RESERVED_REMOTE);
|
||||
stream.openForPush();
|
||||
} else {
|
||||
// Receiving headers on an existing stream. Make sure the stream is in an allowed
|
||||
// state.
|
||||
stream.verifyState(PROTOCOL_ERROR, OPEN, HALF_CLOSED_LOCAL);
|
||||
}
|
||||
|
||||
// If the headers completes this stream, close it.
|
||||
if (frame.isEndOfStream()) {
|
||||
stream.closeRemoteSide(ctx, ctx.newSucceededFuture());
|
||||
}
|
||||
}
|
||||
|
||||
ctx.fireChannelRead(frame);
|
||||
}
|
||||
|
||||
private void handleInboundPushPromise(ChannelHandlerContext ctx, Http2PushPromiseFrame frame)
|
||||
throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(frame)) {
|
||||
// Ignore frames for any stream created after we sent a go-away.
|
||||
return;
|
||||
}
|
||||
|
||||
// Reserve the push stream based with a priority based on the current stream's priority.
|
||||
Http2Stream parentStream = connection.getStreamOrFail(frame.getStreamId());
|
||||
connection.remote().reservePushStream(frame.getPromisedStreamId(), parentStream);
|
||||
|
||||
ctx.fireChannelRead(frame);
|
||||
}
|
||||
|
||||
private void handleInboundPriority(ChannelHandlerContext ctx, Http2PriorityFrame frame)
|
||||
throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(frame)) {
|
||||
// Ignore frames for any stream created after we sent a go-away.
|
||||
return;
|
||||
}
|
||||
|
||||
Http2Stream stream = connection.getStream(frame.getStreamId());
|
||||
if (stream == null) {
|
||||
// Priority frames must be ignored for closed streams.
|
||||
return;
|
||||
}
|
||||
|
||||
stream.verifyState(PROTOCOL_ERROR, HALF_CLOSED_LOCAL, HALF_CLOSED_REMOTE, OPEN, RESERVED_LOCAL);
|
||||
|
||||
// Set the priority on the frame.
|
||||
stream.setPriority(frame.getPriority());
|
||||
|
||||
ctx.fireChannelRead(frame);
|
||||
}
|
||||
|
||||
private void handleInboundWindowUpdate(ChannelHandlerContext ctx, Http2WindowUpdateFrame frame)
|
||||
throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(frame)) {
|
||||
// Ignore frames for any stream created after we sent a go-away.
|
||||
return;
|
||||
}
|
||||
|
||||
int streamId = frame.getStreamId();
|
||||
if (streamId > 0) {
|
||||
Http2Stream stream = connection.getStream(streamId);
|
||||
if (stream == null) {
|
||||
// Window Update frames must be ignored for closed streams.
|
||||
return;
|
||||
}
|
||||
stream.verifyState(PROTOCOL_ERROR, OPEN, HALF_CLOSED_REMOTE);
|
||||
}
|
||||
|
||||
// Update the outbound flow controller.
|
||||
outboundFlow.updateOutboundWindowSize(streamId, frame.getWindowSizeIncrement());
|
||||
|
||||
ctx.fireChannelRead(frame);
|
||||
}
|
||||
|
||||
private void handleInboundRstStream(ChannelHandlerContext ctx, Http2RstStreamFrame frame)
|
||||
throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (isInboundStreamAfterGoAway(frame)) {
|
||||
// Ignore frames for any stream created after we sent a go-away.
|
||||
return;
|
||||
}
|
||||
|
||||
Http2Stream stream = connection.getStream(frame.getStreamId());
|
||||
if (stream == null) {
|
||||
// RstStream frames must be ignored for closed streams.
|
||||
return;
|
||||
}
|
||||
|
||||
stream.close(ctx, ctx.newSucceededFuture());
|
||||
|
||||
ctx.fireChannelRead(frame);
|
||||
}
|
||||
|
||||
private void handleInboundPing(ChannelHandlerContext ctx, Http2PingFrame frame)
|
||||
throws Http2Exception {
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
if (frame.isAck()) {
|
||||
// The remote enpoint is responding to an Ack that we sent.
|
||||
ctx.fireChannelRead(frame);
|
||||
return;
|
||||
}
|
||||
|
||||
// The remote endpoint is sending the ping. Acknowledge receipt.
|
||||
DefaultHttp2PingFrame ack = new DefaultHttp2PingFrame.Builder().setAck(true)
|
||||
.setData(frame.content().duplicate().retain()).build();
|
||||
ctx.writeAndFlush(ack);
|
||||
}
|
||||
|
||||
private void handleInboundSettings(ChannelHandlerContext ctx, Http2SettingsFrame frame)
|
||||
throws Http2Exception {
|
||||
if (frame.isAck()) {
|
||||
// Should not get an ack before receiving the initial settings from the remote
|
||||
// endpoint.
|
||||
verifyInitialSettingsReceived();
|
||||
|
||||
// The remote endpoint is acknowledging the settings - fire this up to the next
|
||||
// handler.
|
||||
ctx.fireChannelRead(frame);
|
||||
return;
|
||||
}
|
||||
|
||||
// It's not an ack, apply the settings.
|
||||
if (frame.getHeaderTableSize() != null) {
|
||||
// TODO(nathanmittler): what's the right thing handle this?
|
||||
// headersEncoder.setHeaderTableSize(frame.getHeaderTableSize());
|
||||
}
|
||||
|
||||
if (frame.getPushEnabled() != null) {
|
||||
connection.remote().setPushToAllowed(frame.getPushEnabled());
|
||||
}
|
||||
|
||||
if (frame.getMaxConcurrentStreams() != null) {
|
||||
int value = Math.max(0, (int) Math.min(Integer.MAX_VALUE, frame.getMaxConcurrentStreams()));
|
||||
connection.local().setMaxStreams(value);
|
||||
}
|
||||
|
||||
if (frame.getInitialWindowSize() != null) {
|
||||
outboundFlow.setInitialOutboundWindowSize(frame.getInitialWindowSize());
|
||||
}
|
||||
|
||||
// Acknowledge receipt of the settings.
|
||||
Http2Frame ack = new DefaultHttp2SettingsFrame.Builder().setAck(true).build();
|
||||
ctx.writeAndFlush(ack);
|
||||
|
||||
// We've received at least one non-ack settings frame from the remote endpoint.
|
||||
initialSettingsReceived = true;
|
||||
ctx.fireChannelRead(frame);
|
||||
}
|
||||
|
||||
private void handleInboundGoAway(ChannelHandlerContext ctx, Http2GoAwayFrame frame) {
|
||||
// Don't allow any more connections to be created.
|
||||
connection.goAwayReceived();
|
||||
ctx.fireChannelRead(frame);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether or not the stream was created after we sent a go-away frame. Frames from
|
||||
* streams created after we sent a go-away should be ignored. Frames for the connection stream ID
|
||||
* (i.e. 0) will always be allowed.
|
||||
*/
|
||||
private boolean isInboundStreamAfterGoAway(Http2StreamFrame frame) {
|
||||
return connection.isGoAwaySent()
|
||||
&& connection.remote().getLastStreamCreated() <= frame.getStreamId();
|
||||
}
|
||||
|
||||
private void handleOutboundData(final ChannelHandlerContext ctx, Http2DataFrame frame,
|
||||
final ChannelPromise promise) throws Http2Exception {
|
||||
if (connection.isGoAway()) {
|
||||
throw format(PROTOCOL_ERROR, "Sending data after connection going away.");
|
||||
}
|
||||
|
||||
Http2Stream stream = connection.getStreamOrFail(frame.getStreamId());
|
||||
stream.verifyState(PROTOCOL_ERROR, OPEN, HALF_CLOSED_REMOTE);
|
||||
|
||||
// Hand control of the frame to the flow controller.
|
||||
outboundFlow.sendFlowControlled(frame, new OutboundFlowController.FrameWriter() {
|
||||
@Override
|
||||
public void writeFrame(Http2DataFrame frame) {
|
||||
ChannelFuture future = ctx.writeAndFlush(frame, promise);
|
||||
|
||||
// Close the connection on write failures that leave the outbound flow control
|
||||
// window in a corrupt state.
|
||||
future.addListener(new ChannelFutureListener() {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
if (!future.isSuccess()) {
|
||||
processHttp2Exception(ctx, toHttp2Exception(future.cause()));
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
// Close the local side of the stream if this is the last frame
|
||||
if (frame.isEndOfStream()) {
|
||||
Http2Stream stream = connection.getStream(frame.getStreamId());
|
||||
stream.closeLocalSide(ctx, promise);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setFailure(Throwable cause) {
|
||||
promise.setFailure(cause);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private void handleOutboundHeaders(ChannelHandlerContext ctx, Http2HeadersFrame frame,
|
||||
ChannelPromise promise) throws Http2Exception {
|
||||
if (connection.isGoAway()) {
|
||||
throw format(PROTOCOL_ERROR, "Sending headers after connection going away.");
|
||||
}
|
||||
|
||||
Http2Stream stream = connection.getStream(frame.getStreamId());
|
||||
if (stream == null) {
|
||||
// Creates a new locally-initiated stream.
|
||||
stream = connection.local().createStream(frame.getStreamId(), frame.getPriority(),
|
||||
frame.isEndOfStream());
|
||||
} else {
|
||||
if (stream.getState() == RESERVED_LOCAL) {
|
||||
// Sending headers on a reserved push stream ... open it for push to the remote
|
||||
// endpoint.
|
||||
stream.openForPush();
|
||||
} else {
|
||||
// The stream already exists, make sure it's in an allowed state.
|
||||
stream.verifyState(PROTOCOL_ERROR, OPEN, HALF_CLOSED_REMOTE);
|
||||
}
|
||||
|
||||
// If the headers are the end of the stream, close it now.
|
||||
if (frame.isEndOfStream()) {
|
||||
stream.closeLocalSide(ctx, promise);
|
||||
}
|
||||
}
|
||||
// Flush to send all of the frames.
|
||||
ctx.writeAndFlush(frame, promise);
|
||||
}
|
||||
|
||||
private void handleOutboundPushPromise(ChannelHandlerContext ctx, Http2PushPromiseFrame frame,
|
||||
ChannelPromise promise) throws Http2Exception {
|
||||
if (connection.isGoAway()) {
|
||||
throw format(PROTOCOL_ERROR, "Sending push promise after connection going away.");
|
||||
}
|
||||
|
||||
// Reserve the promised stream.
|
||||
Http2Stream stream = connection.getStreamOrFail(frame.getStreamId());
|
||||
connection.local().reservePushStream(frame.getPromisedStreamId(), stream);
|
||||
|
||||
// Write the frame.
|
||||
ctx.writeAndFlush(frame, promise);
|
||||
}
|
||||
|
||||
private void handleOutboundPriority(ChannelHandlerContext ctx, Http2PriorityFrame frame,
|
||||
ChannelPromise promise) throws Http2Exception {
|
||||
if (connection.isGoAway()) {
|
||||
throw format(PROTOCOL_ERROR, "Sending priority after connection going away.");
|
||||
}
|
||||
|
||||
// Set the priority on the stream and forward the frame.
|
||||
Http2Stream stream = connection.getStreamOrFail(frame.getStreamId());
|
||||
stream.setPriority(frame.getPriority());
|
||||
ctx.writeAndFlush(frame, promise);
|
||||
}
|
||||
|
||||
private void handleOutboundRstStream(ChannelHandlerContext ctx, Http2RstStreamFrame frame,
|
||||
ChannelPromise promise) {
|
||||
Http2Stream stream = connection.getStream(frame.getStreamId());
|
||||
if (stream == null) {
|
||||
// The stream may already have been closed ... ignore.
|
||||
promise.setSuccess();
|
||||
return;
|
||||
}
|
||||
|
||||
stream.close(ctx, promise);
|
||||
ctx.writeAndFlush(frame, promise);
|
||||
}
|
||||
|
||||
private void handleOutboundPing(ChannelHandlerContext ctx, Http2PingFrame frame,
|
||||
ChannelPromise promise) throws Http2Exception {
|
||||
if (connection.isGoAway()) {
|
||||
throw format(PROTOCOL_ERROR, "Sending ping after connection going away.");
|
||||
}
|
||||
|
||||
if (frame.isAck()) {
|
||||
throw format(PROTOCOL_ERROR, "Another handler attempting to send ping ack");
|
||||
}
|
||||
|
||||
// Just pass the frame through.
|
||||
ctx.writeAndFlush(frame, promise);
|
||||
}
|
||||
|
||||
private static void handleOutboundGoAway() throws Http2Exception {
|
||||
// Why is this being sent? Intercept it and fail the write.
|
||||
// Should have sent a CLOSE ChannelStateEvent
|
||||
throw format(PROTOCOL_ERROR, "Another handler attempted to send GoAway.");
|
||||
}
|
||||
|
||||
private static void handleOutboundWindowUpdate() throws Http2Exception {
|
||||
// Why is this being sent? Intercept it and fail the write.
|
||||
throw format(PROTOCOL_ERROR, "Another handler attempted to send window update.");
|
||||
}
|
||||
|
||||
private void handleOutboundSettings(ChannelHandlerContext ctx, Http2SettingsFrame frame,
|
||||
ChannelPromise promise) throws Http2Exception {
|
||||
if (connection.isGoAway()) {
|
||||
throw format(PROTOCOL_ERROR, "Sending settings after connection going away.");
|
||||
}
|
||||
|
||||
if (frame.isAck()) {
|
||||
throw format(PROTOCOL_ERROR, "Another handler attempting to send settings ack");
|
||||
}
|
||||
|
||||
if (frame.getPushEnabled() != null) {
|
||||
// Enable/disable server push to this endpoint.
|
||||
connection.local().setPushToAllowed(frame.getPushEnabled());
|
||||
}
|
||||
if (frame.getHeaderTableSize() != null) {
|
||||
// TODO(nathanmittler): what's the right way to handle this?
|
||||
// headersDecoder.setHeaderTableSize(frame.getHeaderTableSize());
|
||||
}
|
||||
if (frame.getMaxConcurrentStreams() != null) {
|
||||
// Update maximum number of streams the remote endpoint can initiate.
|
||||
if (frame.getMaxConcurrentStreams() < 0L
|
||||
|| frame.getMaxConcurrentStreams() > Integer.MAX_VALUE) {
|
||||
throw format(PROTOCOL_ERROR, "Invalid value for max concurrent streams: %d",
|
||||
frame.getMaxConcurrentStreams());
|
||||
}
|
||||
connection.remote().setMaxStreams(frame.getMaxConcurrentStreams().intValue());
|
||||
}
|
||||
if (frame.getInitialWindowSize() != null) {
|
||||
// Update the initial window size for inbound traffic.
|
||||
if (frame.getInitialWindowSize() < 0) {
|
||||
throw format(PROTOCOL_ERROR, "Invalid value for initial window size: %d",
|
||||
frame.getInitialWindowSize());
|
||||
}
|
||||
inboundFlow.setInitialInboundWindowSize(frame.getInitialWindowSize());
|
||||
}
|
||||
ctx.writeAndFlush(frame, promise);
|
||||
}
|
||||
|
||||
private void verifyInitialSettingsReceived() throws Http2Exception {
|
||||
if (!initialSettingsReceived) {
|
||||
throw protocolError("Received non-SETTINGS as first frame.");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends the initial settings frame upon establishment of the connection, if not already sent.
|
||||
*/
|
||||
private void sendInitialSettings(final ChannelHandlerContext ctx) throws Http2Exception {
|
||||
if (!initialSettingsSent && ctx.channel().isActive()) {
|
||||
initialSettingsSent = true;
|
||||
|
||||
// Create and send the frame to the remote endpoint.
|
||||
DefaultHttp2SettingsFrame frame =
|
||||
new DefaultHttp2SettingsFrame.Builder()
|
||||
.setInitialWindowSize(inboundFlow.getInitialInboundWindowSize())
|
||||
.setMaxConcurrentStreams(connection.remote().getMaxStreams())
|
||||
.setPushEnabled(connection.local().isPushToAllowed()).build();
|
||||
ctx.writeAndFlush(frame).addListener(new ChannelFutureListener() {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
if (!future.isSuccess() && ctx.channel().isOpen()) {
|
||||
// The write failed, close the connection.
|
||||
ctx.close();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
@ -1,64 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.connection;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Error.INTERNAL_ERROR;
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.format;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.util.CharsetUtil;
|
||||
|
||||
/**
|
||||
* Constants and utility method used for encoding/decoding HTTP2 frames.
|
||||
*/
|
||||
public final class Http2ConnectionUtil {
|
||||
|
||||
public static final int DEFAULT_FLOW_CONTROL_WINDOW_SIZE = 65535;
|
||||
public static final int DEFAULT_HEADER_TABLE_SIZE = 4096;
|
||||
public static final int DEFAULT_MAX_HEADER_SIZE = 4096;
|
||||
|
||||
/**
|
||||
* Converts the given cause to a {@link Http2Exception} if it isn't already.
|
||||
*/
|
||||
public static Http2Exception toHttp2Exception(Throwable cause) {
|
||||
if (cause instanceof Http2Exception) {
|
||||
return (Http2Exception) cause;
|
||||
}
|
||||
String msg = cause != null ? cause.getMessage() : "Failed writing the data frame.";
|
||||
return format(INTERNAL_ERROR, msg);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a buffer containing the error message from the given exception. If the cause is
|
||||
* {@code null} returns an empty buffer.
|
||||
*/
|
||||
public static ByteBuf toByteBuf(ChannelHandlerContext ctx, Throwable cause) {
|
||||
ByteBuf debugData = Unpooled.EMPTY_BUFFER;
|
||||
if (cause != null) {
|
||||
// Create the debug message.
|
||||
byte[] msg = cause.getMessage().getBytes(CharsetUtil.UTF_8);
|
||||
debugData = ctx.alloc().buffer(msg.length);
|
||||
debugData.writeBytes(msg);
|
||||
}
|
||||
return debugData;
|
||||
}
|
||||
|
||||
private Http2ConnectionUtil() {
|
||||
}
|
||||
}
|
@ -1,84 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.connection;
|
||||
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2DataFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
|
||||
/**
|
||||
* Controls the outbound flow of data frames to the remote endpoint.
|
||||
*/
|
||||
public interface OutboundFlowController {
|
||||
|
||||
/**
|
||||
* Interface that abstracts the writing of {@link Http2Frame} objects to the remote endpoint.
|
||||
*/
|
||||
interface FrameWriter {
|
||||
|
||||
/**
|
||||
* Writes a single data frame to the remote endpoint.
|
||||
*/
|
||||
void writeFrame(Http2DataFrame frame);
|
||||
|
||||
/**
|
||||
* Called if an error occurred before the write could take place. Sets the failure on the
|
||||
* channel promise.
|
||||
*/
|
||||
void setFailure(Throwable cause);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the initial size of the connection's outbound flow control window. The outbound flow
|
||||
* control windows for all streams are updated by the delta in the initial window size. This is
|
||||
* called as part of the processing of a SETTINGS frame received from the remote endpoint.
|
||||
*
|
||||
* @param newWindowSize the new initial window size.
|
||||
*/
|
||||
void setInitialOutboundWindowSize(int newWindowSize) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Gets the initial size of the connection's outbound flow control window.
|
||||
*/
|
||||
int getInitialOutboundWindowSize();
|
||||
|
||||
/**
|
||||
* Updates the size of the stream's outbound flow control window. This is called upon receiving a
|
||||
* WINDOW_UPDATE frame from the remote endpoint.
|
||||
*
|
||||
* @param streamId the ID of the stream, or zero if the window is for the entire connection.
|
||||
* @param deltaWindowSize the change in size of the outbound flow control window.
|
||||
* @throws Http2Exception thrown if a protocol-related error occurred.
|
||||
*/
|
||||
void updateOutboundWindowSize(int streamId, int deltaWindowSize) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Sends the frame with outbound flow control applied. The frame may be written at a later time,
|
||||
* depending on whether the remote endpoint can receive the frame now.
|
||||
* <p/>
|
||||
* Data frame flow control processing requirements:
|
||||
* <p/>
|
||||
* Sender must not send a data frame with data length greater than the transfer window size. After
|
||||
* sending each data frame, the stream's transfer window size is decremented by the amount of data
|
||||
* transmitted. When the window size becomes less than or equal to 0, the sender must pause
|
||||
* transmitting data frames.
|
||||
*
|
||||
* @param frame the frame to send.
|
||||
* @param frameWriter peforms to the write of the frame to the remote endpoint.
|
||||
* @throws Http2Exception thrown if a protocol-related error occurred.
|
||||
*/
|
||||
void sendFlowControlled(Http2DataFrame frame, FrameWriter frameWriter) throws Http2Exception;
|
||||
}
|
@ -1,20 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Connection-level services (stream management, flow control) for HTTP2.
|
||||
*/
|
||||
package io.netty.handler.codec.http2.draft10.connection;
|
||||
|
@ -1,201 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_UNSIGNED_SHORT;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.DefaultByteBufHolder;
|
||||
import io.netty.buffer.Unpooled;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link Http2DataFrame}.
|
||||
*/
|
||||
public final class DefaultHttp2DataFrame extends DefaultByteBufHolder implements Http2DataFrame {
|
||||
|
||||
private final int paddingLength;
|
||||
private final int streamId;
|
||||
private final boolean endOfStream;
|
||||
|
||||
private DefaultHttp2DataFrame(Builder builder) {
|
||||
super(builder.content);
|
||||
streamId = builder.streamId;
|
||||
endOfStream = builder.endOfStream;
|
||||
paddingLength = builder.paddingLength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getStreamId() {
|
||||
return streamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEndOfStream() {
|
||||
return endOfStream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPaddingLength() {
|
||||
return paddingLength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2DataFrame copy() {
|
||||
return copyBuilder().setContent(content().copy()).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2DataFrame duplicate() {
|
||||
return copyBuilder().setContent(content().duplicate()).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2DataFrame retain() {
|
||||
super.retain();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2DataFrame retain(int increment) {
|
||||
super.retain(increment);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2DataFrame touch() {
|
||||
super.touch();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2DataFrame touch(Object hint) {
|
||||
super.touch(hint);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = content().hashCode();
|
||||
result = prime * result + (endOfStream ? 1231 : 1237);
|
||||
result = prime * result + paddingLength;
|
||||
result = prime * result + streamId;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
DefaultHttp2DataFrame other = (DefaultHttp2DataFrame) obj;
|
||||
if (endOfStream != other.endOfStream) {
|
||||
return false;
|
||||
}
|
||||
if (paddingLength != other.paddingLength) {
|
||||
return false;
|
||||
}
|
||||
if (streamId != other.streamId) {
|
||||
return false;
|
||||
}
|
||||
if (!content().equals(other.content())) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
|
||||
builder.append("streamId=").append(streamId);
|
||||
builder.append(", endOfStream=").append(endOfStream);
|
||||
builder.append(", paddingLength=").append(paddingLength);
|
||||
builder.append(", contentLength=").append(content().readableBytes());
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
private Builder copyBuilder() {
|
||||
return new Builder().setStreamId(streamId).setPaddingLength(paddingLength)
|
||||
.setEndOfStream(endOfStream);
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds instances of {@link DefaultHttp2DataFrame}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private int streamId;
|
||||
private boolean endOfStream;
|
||||
private ByteBuf content = Unpooled.EMPTY_BUFFER;
|
||||
private int paddingLength;
|
||||
|
||||
public Builder setStreamId(int streamId) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("StreamId must be > 0.");
|
||||
}
|
||||
this.streamId = streamId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setEndOfStream(boolean endOfStream) {
|
||||
this.endOfStream = endOfStream;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the content for the data frame, excluding any padding. This buffer will be retained when
|
||||
* the frame is built.
|
||||
*/
|
||||
public Builder setContent(ByteBuf content) {
|
||||
if (content == null) {
|
||||
throw new IllegalArgumentException("content must not be null");
|
||||
}
|
||||
verifyLength(paddingLength, content);
|
||||
this.content = content;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setPaddingLength(int paddingLength) {
|
||||
if (paddingLength < 0 || paddingLength > MAX_UNSIGNED_SHORT) {
|
||||
throw new IllegalArgumentException("Padding length invalid.");
|
||||
}
|
||||
verifyLength(paddingLength, content);
|
||||
this.paddingLength = paddingLength;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DefaultHttp2DataFrame build() {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("StreamId must be set.");
|
||||
}
|
||||
|
||||
verifyLength(paddingLength, content);
|
||||
|
||||
return new DefaultHttp2DataFrame(this);
|
||||
}
|
||||
|
||||
private static void verifyLength(int paddingLength, ByteBuf data) {
|
||||
int maxLength = MAX_FRAME_PAYLOAD_LENGTH;
|
||||
maxLength -= paddingLength;
|
||||
if (data.readableBytes() > maxLength) {
|
||||
throw new IllegalArgumentException("Header block fragment length too big.");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -1,172 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_UNSIGNED_INT;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.DefaultByteBufHolder;
|
||||
import io.netty.buffer.Unpooled;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link Http2GoAwayFrame}.
|
||||
*/
|
||||
public final class DefaultHttp2GoAwayFrame extends DefaultByteBufHolder implements
|
||||
Http2GoAwayFrame {
|
||||
private final int lastStreamId;
|
||||
private final long errorCode;
|
||||
|
||||
private DefaultHttp2GoAwayFrame(Builder builder) {
|
||||
super(builder.debugData);
|
||||
lastStreamId = builder.lastStreamId;
|
||||
errorCode = builder.errorCode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLastStreamId() {
|
||||
return lastStreamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getErrorCode() {
|
||||
return errorCode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2GoAwayFrame copy() {
|
||||
return copyBuilder().setDebugData(content().copy()).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2GoAwayFrame duplicate() {
|
||||
return copyBuilder().setDebugData(content().duplicate()).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2GoAwayFrame retain() {
|
||||
super.retain();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2GoAwayFrame retain(int increment) {
|
||||
super.retain(increment);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2GoAwayFrame touch() {
|
||||
super.touch();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2GoAwayFrame touch(Object hint) {
|
||||
super.touch(hint);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = content().hashCode();
|
||||
result = prime * result + (int) (errorCode ^ (errorCode >>> 32));
|
||||
result = prime * result + lastStreamId;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
DefaultHttp2GoAwayFrame other = (DefaultHttp2GoAwayFrame) obj;
|
||||
if (errorCode != other.errorCode) {
|
||||
return false;
|
||||
}
|
||||
if (lastStreamId != other.lastStreamId) {
|
||||
return false;
|
||||
}
|
||||
if (!content().equals(other.content())) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
|
||||
builder.append("lastStreamId=").append(lastStreamId);
|
||||
builder.append(", errorCode=").append(errorCode);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
private Builder copyBuilder() {
|
||||
return new Builder().setErrorCode(errorCode).setLastStreamId(lastStreamId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds instances of {@link DefaultHttp2GoAwayFrame}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private int lastStreamId = -1;
|
||||
private long errorCode = -1;
|
||||
private ByteBuf debugData = Unpooled.EMPTY_BUFFER;
|
||||
|
||||
public Builder setLastStreamId(int lastStreamId) {
|
||||
if (lastStreamId < 0) {
|
||||
throw new IllegalArgumentException("Invalid lastStreamId.");
|
||||
}
|
||||
this.lastStreamId = lastStreamId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setErrorCode(long errorCode) {
|
||||
if (errorCode < 0 || errorCode > MAX_UNSIGNED_INT) {
|
||||
throw new IllegalArgumentException("Invalid error code.");
|
||||
}
|
||||
this.errorCode = errorCode;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setDebugData(ByteBuf debugData) {
|
||||
if (debugData == null) {
|
||||
throw new IllegalArgumentException("debugData must not be null");
|
||||
}
|
||||
if (debugData.readableBytes() > MAX_FRAME_PAYLOAD_LENGTH - 8) {
|
||||
throw new IllegalArgumentException("Invalid debug data size.");
|
||||
}
|
||||
|
||||
this.debugData = debugData;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DefaultHttp2GoAwayFrame build() {
|
||||
if (lastStreamId < 0) {
|
||||
throw new IllegalArgumentException("LastStreamId must be set");
|
||||
}
|
||||
if (errorCode < 0) {
|
||||
throw new IllegalArgumentException("ErrorCode must be set.");
|
||||
}
|
||||
|
||||
return new DefaultHttp2GoAwayFrame(this);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,152 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.DEFAULT_STREAM_PRIORITY;
|
||||
import io.netty.handler.codec.http2.draft10.DefaultHttp2Headers;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Headers;
|
||||
|
||||
public final class DefaultHttp2HeadersFrame implements Http2HeadersFrame {
|
||||
|
||||
private final int streamId;
|
||||
private final int priority;
|
||||
private final boolean endOfStream;
|
||||
private final Http2Headers headers;
|
||||
|
||||
private DefaultHttp2HeadersFrame(Builder builder) {
|
||||
streamId = builder.streamId;
|
||||
priority = builder.priority;
|
||||
headers = builder.headersBuilder.build();
|
||||
endOfStream = builder.endOfStream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getStreamId() {
|
||||
return streamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEndOfStream() {
|
||||
return endOfStream;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPriority() {
|
||||
return priority;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Headers getHeaders() {
|
||||
return headers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + (endOfStream ? 1231 : 1237);
|
||||
result = prime * result + ((headers == null) ? 0 : headers.hashCode());
|
||||
result = prime * result + priority;
|
||||
result = prime * result + streamId;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
DefaultHttp2HeadersFrame other = (DefaultHttp2HeadersFrame) obj;
|
||||
if (endOfStream != other.endOfStream) {
|
||||
return false;
|
||||
}
|
||||
if (headers == null) {
|
||||
if (other.headers != null) {
|
||||
return false;
|
||||
}
|
||||
} else if (!headers.equals(other.headers)) {
|
||||
return false;
|
||||
}
|
||||
if (priority != other.priority) {
|
||||
return false;
|
||||
}
|
||||
if (streamId != other.streamId) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
|
||||
builder.append("streamId=").append(streamId);
|
||||
builder.append(", priority=").append(priority);
|
||||
builder.append(", endOfStream=").append(endOfStream);
|
||||
builder.append(", headers=").append(headers);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
private int streamId;
|
||||
private int priority = DEFAULT_STREAM_PRIORITY;
|
||||
private final DefaultHttp2Headers.Builder headersBuilder = new DefaultHttp2Headers.Builder();
|
||||
private boolean endOfStream;
|
||||
|
||||
public Builder setStreamId(int streamId) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("StreamId must be > 0.");
|
||||
}
|
||||
this.streamId = streamId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setEndOfStream(boolean endOfStream) {
|
||||
this.endOfStream = endOfStream;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setPriority(int priority) {
|
||||
if (priority < 0) {
|
||||
throw new IllegalArgumentException("Priority must be >= 0");
|
||||
}
|
||||
this.priority = priority;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DefaultHttp2Headers.Builder headers() {
|
||||
return headersBuilder;
|
||||
}
|
||||
|
||||
public Builder setHeaders(Http2Headers headers) {
|
||||
headersBuilder.set(headers);
|
||||
return this;
|
||||
}
|
||||
|
||||
public DefaultHttp2HeadersFrame build() {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("StreamId must be set.");
|
||||
}
|
||||
return new DefaultHttp2HeadersFrame(this);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,146 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.PING_FRAME_PAYLOAD_LENGTH;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.DefaultByteBufHolder;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link Http2PingFrame}.
|
||||
*/
|
||||
public final class DefaultHttp2PingFrame extends DefaultByteBufHolder implements Http2PingFrame {
|
||||
|
||||
private final boolean ack;
|
||||
|
||||
private DefaultHttp2PingFrame(Builder builder) {
|
||||
super(builder.data);
|
||||
ack = builder.ack;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAck() {
|
||||
return ack;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2PingFrame copy() {
|
||||
return new Builder().setAck(ack).setData(content().copy()).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2PingFrame duplicate() {
|
||||
return new Builder().setAck(ack).setData(content().duplicate()).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2PingFrame retain() {
|
||||
super.retain();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2PingFrame retain(int increment) {
|
||||
super.retain(increment);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2PingFrame touch() {
|
||||
super.touch();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DefaultHttp2PingFrame touch(Object hint) {
|
||||
super.touch(hint);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = content().hashCode();
|
||||
result = prime * result + (ack ? 1231 : 1237);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
DefaultHttp2PingFrame other = (DefaultHttp2PingFrame) obj;
|
||||
if (ack != other.ack) {
|
||||
return false;
|
||||
}
|
||||
if (!content().equals(other.content())) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
|
||||
builder.append("ack=").append(ack);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds instances of {@link DefaultHttp2PingFrame}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private boolean ack;
|
||||
private ByteBuf data;
|
||||
|
||||
/**
|
||||
* Sets the data for this ping. This buffer will be retained when the frame is built.
|
||||
*/
|
||||
public Builder setData(ByteBuf data) {
|
||||
if (data == null) {
|
||||
throw new IllegalArgumentException("data must not be null.");
|
||||
}
|
||||
if (data.readableBytes() != PING_FRAME_PAYLOAD_LENGTH) {
|
||||
throw new IllegalArgumentException(String.format(
|
||||
"Incorrect data length for ping. Expected %d, found %d",
|
||||
PING_FRAME_PAYLOAD_LENGTH, data.readableBytes()));
|
||||
}
|
||||
this.data = data;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setAck(boolean ack) {
|
||||
this.ack = ack;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DefaultHttp2PingFrame build() {
|
||||
if (data == null) {
|
||||
throw new IllegalArgumentException("debug data must be provided");
|
||||
}
|
||||
|
||||
return new DefaultHttp2PingFrame(this);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,117 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link Http2PriorityFrame}.
|
||||
*/
|
||||
public final class DefaultHttp2PriorityFrame implements Http2PriorityFrame {
|
||||
|
||||
private final int streamId;
|
||||
private final int priority;
|
||||
|
||||
private DefaultHttp2PriorityFrame(Builder builder) {
|
||||
streamId = builder.streamId;
|
||||
priority = builder.priority;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getStreamId() {
|
||||
return streamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPriority() {
|
||||
return priority;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEndOfStream() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + priority;
|
||||
result = prime * result + streamId;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
DefaultHttp2PriorityFrame other = (DefaultHttp2PriorityFrame) obj;
|
||||
if (priority != other.priority) {
|
||||
return false;
|
||||
}
|
||||
if (streamId != other.streamId) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
|
||||
builder.append("streamId=").append(streamId);
|
||||
builder.append(", priority=").append(priority);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
/**
|
||||
* Builds instances of {@link DefaultHttp2PriorityFrame}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private int streamId;
|
||||
private int priority = -1;
|
||||
|
||||
public Builder setStreamId(int streamId) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("StreamId must be > 0.");
|
||||
}
|
||||
this.streamId = streamId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setPriority(int priority) {
|
||||
if (priority < 0) {
|
||||
throw new IllegalArgumentException("Invalid priority.");
|
||||
}
|
||||
this.priority = priority;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DefaultHttp2PriorityFrame build() {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("StreamId must be set.");
|
||||
}
|
||||
if (priority < 0) {
|
||||
throw new IllegalArgumentException("Priority must be set.");
|
||||
}
|
||||
return new DefaultHttp2PriorityFrame(this);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,142 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import io.netty.handler.codec.http2.draft10.Http2Headers;
|
||||
|
||||
public final class DefaultHttp2PushPromiseFrame implements Http2PushPromiseFrame {
|
||||
|
||||
private final int streamId;
|
||||
private final int promisedStreamId;
|
||||
private final Http2Headers headers;
|
||||
|
||||
private DefaultHttp2PushPromiseFrame(Builder builder) {
|
||||
streamId = builder.streamId;
|
||||
promisedStreamId = builder.promisedStreamId;
|
||||
headers = builder.headers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getStreamId() {
|
||||
return streamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEndOfStream() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPromisedStreamId() {
|
||||
return promisedStreamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Headers getHeaders() {
|
||||
return headers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + ((headers == null) ? 0 : headers.hashCode());
|
||||
result = prime * result + promisedStreamId;
|
||||
result = prime * result + streamId;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
DefaultHttp2PushPromiseFrame other = (DefaultHttp2PushPromiseFrame) obj;
|
||||
if (headers == null) {
|
||||
if (other.headers != null) {
|
||||
return false;
|
||||
}
|
||||
} else if (!headers.equals(other.headers)) {
|
||||
return false;
|
||||
}
|
||||
if (promisedStreamId != other.promisedStreamId) {
|
||||
return false;
|
||||
}
|
||||
if (streamId != other.streamId) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
|
||||
builder.append("streamId=").append(streamId);
|
||||
builder.append(", promisedStreamId=").append(promisedStreamId);
|
||||
builder.append(", headers=").append(headers);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
private int streamId;
|
||||
private int promisedStreamId;
|
||||
private Http2Headers headers;
|
||||
|
||||
public Builder setStreamId(int streamId) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("StreamId must be > 0.");
|
||||
}
|
||||
this.streamId = streamId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setPromisedStreamId(int promisedStreamId) {
|
||||
if (promisedStreamId <= 0) {
|
||||
throw new IllegalArgumentException("promisedStreamId must be > 0.");
|
||||
}
|
||||
this.promisedStreamId = promisedStreamId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setHeaders(Http2Headers headers) {
|
||||
if (headers == null) {
|
||||
throw new IllegalArgumentException("headers must not be null.");
|
||||
}
|
||||
this.headers = headers;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DefaultHttp2PushPromiseFrame build() {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("StreamId must be set.");
|
||||
}
|
||||
if (promisedStreamId <= 0) {
|
||||
throw new IllegalArgumentException("promisedStreamId must be set.");
|
||||
}
|
||||
if (headers == null) {
|
||||
throw new IllegalArgumentException("headers must be set.");
|
||||
}
|
||||
return new DefaultHttp2PushPromiseFrame(this);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,119 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_UNSIGNED_INT;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link Http2RstStreamFrame}.
|
||||
*/
|
||||
public final class DefaultHttp2RstStreamFrame implements Http2RstStreamFrame {
|
||||
private final int streamId;
|
||||
private final long errorCode;
|
||||
|
||||
private DefaultHttp2RstStreamFrame(Builder builder) {
|
||||
streamId = builder.streamId;
|
||||
errorCode = builder.errorCode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getStreamId() {
|
||||
return streamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getErrorCode() {
|
||||
return errorCode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEndOfStream() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + (int) (errorCode ^ (errorCode >>> 32));
|
||||
result = prime * result + streamId;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
DefaultHttp2RstStreamFrame other = (DefaultHttp2RstStreamFrame) obj;
|
||||
if (errorCode != other.errorCode) {
|
||||
return false;
|
||||
}
|
||||
if (streamId != other.streamId) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
|
||||
builder.append("streamId=").append(streamId);
|
||||
builder.append(", errorCode=").append(errorCode);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds instances of {@link DefaultHttp2RstStreamFrame}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private int streamId;
|
||||
private long errorCode = -1L;
|
||||
|
||||
public Builder setStreamId(int streamId) {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("StreamId must be > 0.");
|
||||
}
|
||||
this.streamId = streamId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setErrorCode(long errorCode) {
|
||||
if (errorCode < 0 || errorCode > MAX_UNSIGNED_INT) {
|
||||
throw new IllegalArgumentException("Invalid errorCode value.");
|
||||
}
|
||||
this.errorCode = errorCode;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DefaultHttp2RstStreamFrame build() {
|
||||
if (streamId <= 0) {
|
||||
throw new IllegalArgumentException("StreamId must be set.");
|
||||
}
|
||||
if (errorCode < 0L) {
|
||||
throw new IllegalArgumentException("ErrorCode must be set.");
|
||||
}
|
||||
return new DefaultHttp2RstStreamFrame(this);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,184 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link Http2SettingsFrame}.
|
||||
*/
|
||||
public final class DefaultHttp2SettingsFrame implements Http2SettingsFrame {
|
||||
|
||||
private final boolean ack;
|
||||
private final Integer headerTableSize;
|
||||
private final Boolean pushEnabled;
|
||||
private final Long maxConcurrentStreams;
|
||||
private final Integer initialWindowSize;
|
||||
|
||||
private DefaultHttp2SettingsFrame(Builder builder) {
|
||||
ack = builder.ack;
|
||||
headerTableSize = builder.headerTableSize;
|
||||
pushEnabled = builder.pushEnabled;
|
||||
maxConcurrentStreams = builder.maxConcurrentStreams;
|
||||
initialWindowSize = builder.initialWindowSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAck() {
|
||||
return ack;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getHeaderTableSize() {
|
||||
return headerTableSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean getPushEnabled() {
|
||||
return pushEnabled;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getMaxConcurrentStreams() {
|
||||
return maxConcurrentStreams;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getInitialWindowSize() {
|
||||
return initialWindowSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + (ack ? 1231 : 1237);
|
||||
result = prime * result + ((headerTableSize == null) ? 0 : headerTableSize.hashCode());
|
||||
result = prime * result + ((initialWindowSize == null) ? 0 : initialWindowSize.hashCode());
|
||||
result =
|
||||
prime * result + ((maxConcurrentStreams == null) ? 0 : maxConcurrentStreams.hashCode());
|
||||
result = prime * result + ((pushEnabled == null) ? 0 : pushEnabled.hashCode());
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
DefaultHttp2SettingsFrame other = (DefaultHttp2SettingsFrame) obj;
|
||||
if (ack != other.ack) {
|
||||
return false;
|
||||
}
|
||||
if (headerTableSize == null) {
|
||||
if (other.headerTableSize != null) {
|
||||
return false;
|
||||
}
|
||||
} else if (!headerTableSize.equals(other.headerTableSize)) {
|
||||
return false;
|
||||
}
|
||||
if (initialWindowSize == null) {
|
||||
if (other.initialWindowSize != null) {
|
||||
return false;
|
||||
}
|
||||
} else if (!initialWindowSize.equals(other.initialWindowSize)) {
|
||||
return false;
|
||||
}
|
||||
if (maxConcurrentStreams == null) {
|
||||
if (other.maxConcurrentStreams != null) {
|
||||
return false;
|
||||
}
|
||||
} else if (!maxConcurrentStreams.equals(other.maxConcurrentStreams)) {
|
||||
return false;
|
||||
}
|
||||
if (pushEnabled == null) {
|
||||
if (other.pushEnabled != null) {
|
||||
return false;
|
||||
}
|
||||
} else if (!pushEnabled.equals(other.pushEnabled)) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
|
||||
builder.append("ack=").append(ack);
|
||||
if (headerTableSize != null) {
|
||||
builder.append(", headerTableSize=").append(headerTableSize);
|
||||
}
|
||||
if (pushEnabled != null) {
|
||||
builder.append(", pushEnabled=").append(pushEnabled);
|
||||
}
|
||||
if (maxConcurrentStreams != null) {
|
||||
builder.append(", maxConcurrentStreams=").append(maxConcurrentStreams);
|
||||
}
|
||||
if (initialWindowSize != null) {
|
||||
builder.append(", initialWindowSize=").append(initialWindowSize);
|
||||
}
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds instances of {@link DefaultHttp2SettingsFrame}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private boolean ack;
|
||||
private Integer headerTableSize;
|
||||
private Boolean pushEnabled;
|
||||
private Long maxConcurrentStreams;
|
||||
private Integer initialWindowSize;
|
||||
|
||||
public Builder setAck(boolean ack) {
|
||||
this.ack = ack;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setHeaderTableSize(int headerTableSize) {
|
||||
this.headerTableSize = headerTableSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setPushEnabled(boolean pushEnabled) {
|
||||
this.pushEnabled = pushEnabled;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setMaxConcurrentStreams(long maxConcurrentStreams) {
|
||||
this.maxConcurrentStreams = maxConcurrentStreams;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setInitialWindowSize(int initialWindowSize) {
|
||||
this.initialWindowSize = initialWindowSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DefaultHttp2SettingsFrame build() {
|
||||
if (ack && (headerTableSize != null || pushEnabled != null || maxConcurrentStreams != null
|
||||
|| initialWindowSize != null)) {
|
||||
throw new IllegalArgumentException("Ack frame must not contain settings");
|
||||
}
|
||||
return new DefaultHttp2SettingsFrame(this);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,112 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
/**
|
||||
* Default implementation of {@link Http2WindowUpdateFrame}.
|
||||
*/
|
||||
public final class DefaultHttp2WindowUpdateFrame implements Http2WindowUpdateFrame {
|
||||
|
||||
private final int streamId;
|
||||
private final int windowSizeIncrement;
|
||||
|
||||
private DefaultHttp2WindowUpdateFrame(Builder builder) {
|
||||
streamId = builder.streamId;
|
||||
windowSizeIncrement = builder.windowSizeIncrement;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getStreamId() {
|
||||
return streamId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEndOfStream() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getWindowSizeIncrement() {
|
||||
return windowSizeIncrement;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + streamId;
|
||||
result = prime * result + windowSizeIncrement;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
DefaultHttp2WindowUpdateFrame other = (DefaultHttp2WindowUpdateFrame) obj;
|
||||
if (streamId != other.streamId) {
|
||||
return false;
|
||||
}
|
||||
if (windowSizeIncrement != other.windowSizeIncrement) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
|
||||
builder.append("streamId=").append(streamId);
|
||||
builder.append(", windowSizeIncrement=").append(windowSizeIncrement);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds instances of {@link DefaultHttp2WindowUpdateFrame}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private int streamId;
|
||||
private int windowSizeIncrement;
|
||||
|
||||
public Builder setStreamId(int streamId) {
|
||||
this.streamId = streamId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setWindowSizeIncrement(int windowSizeIncrement) {
|
||||
this.windowSizeIncrement = windowSizeIncrement;
|
||||
return this;
|
||||
}
|
||||
|
||||
public DefaultHttp2WindowUpdateFrame build() {
|
||||
if (streamId < 0) {
|
||||
throw new IllegalArgumentException("StreamId must be >= 0.");
|
||||
}
|
||||
if (windowSizeIncrement < 0) {
|
||||
throw new IllegalArgumentException("SindowSizeIncrement must be >= 0.");
|
||||
}
|
||||
return new DefaultHttp2WindowUpdateFrame(this);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,54 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufHolder;
|
||||
|
||||
/**
|
||||
* An HTTP2 data frame.
|
||||
*/
|
||||
public interface Http2DataFrame extends Http2StreamFrame, ByteBufHolder {
|
||||
|
||||
/**
|
||||
* The amount of padding to follow the header data in the frame.
|
||||
*/
|
||||
int getPaddingLength();
|
||||
|
||||
/**
|
||||
* Returns the data payload of this frame.
|
||||
*/
|
||||
@Override
|
||||
ByteBuf content();
|
||||
|
||||
@Override
|
||||
Http2DataFrame copy();
|
||||
|
||||
@Override
|
||||
Http2DataFrame duplicate();
|
||||
|
||||
@Override
|
||||
Http2DataFrame retain();
|
||||
|
||||
@Override
|
||||
Http2DataFrame retain(int increment);
|
||||
|
||||
@Override
|
||||
Http2DataFrame touch();
|
||||
|
||||
@Override
|
||||
Http2DataFrame touch(Object hint);
|
||||
}
|
@ -1,167 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_ACK;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_END_HEADERS;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_END_SEGMENT;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_END_STREAM;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_PAD_HIGH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_PAD_LOW;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_PRIORITY;
|
||||
|
||||
/**
|
||||
* Provides utility methods for accessing specific flags as defined by the HTTP2 spec.
|
||||
*/
|
||||
public class Http2Flags {
|
||||
private final short value;
|
||||
|
||||
public Http2Flags(short value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the underlying flags value.
|
||||
*/
|
||||
public short getValue() {
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the end-of-stream flag is set.
|
||||
*/
|
||||
public boolean isEndOfStream() {
|
||||
return isSet(FLAG_END_STREAM);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the end-of-segment flag is set.
|
||||
*/
|
||||
public boolean isEndOfSegment() {
|
||||
return isSet(FLAG_END_SEGMENT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the end-of-headers flag is set.
|
||||
*/
|
||||
public boolean isEndOfHeaders() {
|
||||
return isSet(FLAG_END_HEADERS);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the flag is set indicating the presence of the priority field in a HEADERS
|
||||
* frame.
|
||||
*/
|
||||
public boolean isPriorityPresent() {
|
||||
return isSet(FLAG_PRIORITY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the flag is set indicating that this frame is an ACK.
|
||||
*/
|
||||
public boolean isAck() {
|
||||
return isSet(FLAG_ACK);
|
||||
}
|
||||
|
||||
/**
|
||||
* For frames that include padding, indicates if the pad low field is present.
|
||||
*/
|
||||
public boolean isPadLowPresent() {
|
||||
return isSet(FLAG_PAD_LOW);
|
||||
}
|
||||
|
||||
/**
|
||||
* For frames that include padding, indicates if the pad high field is present.
|
||||
*/
|
||||
public boolean isPadHighPresent() {
|
||||
return isSet(FLAG_PAD_HIGH);
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether the padding flags are set properly. If pad high is set, pad low must also be
|
||||
* set.
|
||||
*/
|
||||
public boolean isPaddingLengthValid() {
|
||||
return isPadHighPresent() ? isPadLowPresent() : true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of bytes expected in the padding length field of the payload. This is
|
||||
* determined by the {@link #isPadHighPresent()} and {@link #isPadLowPresent()} flags.
|
||||
*/
|
||||
public int getNumPaddingLengthBytes() {
|
||||
return (isPadHighPresent() ? 1 : 0) + (isPadLowPresent() ? 1 : 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result + value;
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
Http2Flags other = (Http2Flags) obj;
|
||||
if (value != other.value) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("value = ").append(value).append(" (");
|
||||
if (isAck()) {
|
||||
builder.append("ACK,");
|
||||
}
|
||||
if (isEndOfHeaders()) {
|
||||
builder.append("END_OF_HEADERS,");
|
||||
}
|
||||
if (isEndOfStream()) {
|
||||
builder.append("END_OF_STREAM,");
|
||||
}
|
||||
if (isPriorityPresent()) {
|
||||
builder.append("PRIORITY_PRESENT,");
|
||||
}
|
||||
if (isEndOfSegment()) {
|
||||
builder.append("END_OF_SEGMENT,");
|
||||
}
|
||||
if (isPadHighPresent()) {
|
||||
builder.append("PAD_HIGH,");
|
||||
}
|
||||
if (isPadLowPresent()) {
|
||||
builder.append("PAD_LOW,");
|
||||
}
|
||||
builder.append(")");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
private boolean isSet(short mask) {
|
||||
return (value & mask) != 0;
|
||||
}
|
||||
}
|
@ -1,22 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
/**
|
||||
* Marker interface for all HTTP2 frame types.
|
||||
*/
|
||||
public interface Http2Frame {
|
||||
}
|
@ -1,37 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import io.netty.channel.ChannelHandlerAppender;
|
||||
import io.netty.handler.codec.http2.draft10.frame.decoder.Http2FrameDecoder;
|
||||
import io.netty.handler.codec.http2.draft10.frame.decoder.Http2FrameUnmarshaller;
|
||||
import io.netty.handler.codec.http2.draft10.frame.encoder.Http2FrameEncoder;
|
||||
import io.netty.handler.codec.http2.draft10.frame.encoder.Http2FrameMarshaller;
|
||||
|
||||
/**
|
||||
* A combination of {@link Http2FrameEncoder} and {@link Http2FrameDecoder}.
|
||||
*/
|
||||
public class Http2FrameCodec extends ChannelHandlerAppender {
|
||||
|
||||
public Http2FrameCodec(Http2FrameMarshaller frameMarshaller,
|
||||
Http2FrameUnmarshaller frameUnmarshaller) {
|
||||
super(new Http2FrameEncoder(frameMarshaller), new Http2FrameDecoder(frameUnmarshaller));
|
||||
}
|
||||
|
||||
public Http2FrameCodec() {
|
||||
super(new Http2FrameEncoder(), new Http2FrameDecoder());
|
||||
}
|
||||
}
|
@ -1,141 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.util.CharsetUtil;
|
||||
|
||||
/**
|
||||
* Constants and utility method used for encoding/decoding HTTP2 frames.
|
||||
*/
|
||||
public final class Http2FrameCodecUtil {
|
||||
public static final int CONNECTION_STREAM_ID = 0;
|
||||
|
||||
public static final String CONNECTION_PREFACE = "PRI * HTTP/2.0\r\n\r\nSM\r\n\r\n";
|
||||
|
||||
private static final ByteBuf CONNECTION_PREFACE_BUF = Unpooled.unmodifiableBuffer(Unpooled
|
||||
.copiedBuffer(CONNECTION_PREFACE, CharsetUtil.UTF_8));
|
||||
|
||||
/**
|
||||
* Returns a buffer containing the the {@link #CONNECTION_PREFACE}.
|
||||
*/
|
||||
public static ByteBuf connectionPrefaceBuf() {
|
||||
// Return a duplicate so that modifications to the reader index will not affect the original
|
||||
// buffer.
|
||||
return CONNECTION_PREFACE_BUF.duplicate().retain();
|
||||
}
|
||||
|
||||
public static final int DEFAULT_STREAM_PRIORITY = 0x40000000; // 2^30
|
||||
|
||||
public static final int MAX_FRAME_PAYLOAD_LENGTH = 16383;
|
||||
public static final int PING_FRAME_PAYLOAD_LENGTH = 8;
|
||||
public static final short MAX_UNSIGNED_BYTE = 0xFF;
|
||||
public static final int MAX_UNSIGNED_SHORT = 0xFFFF;
|
||||
public static final long MAX_UNSIGNED_INT = 0xFFFFFFFFL;
|
||||
public static final int FRAME_HEADER_LENGTH = 8;
|
||||
public static final int FRAME_LENGTH_MASK = 0x3FFF;
|
||||
|
||||
public static final short FRAME_TYPE_DATA = 0x0;
|
||||
public static final short FRAME_TYPE_HEADERS = 0x1;
|
||||
public static final short FRAME_TYPE_PRIORITY = 0x2;
|
||||
public static final short FRAME_TYPE_RST_STREAM = 0x3;
|
||||
public static final short FRAME_TYPE_SETTINGS = 0x4;
|
||||
public static final short FRAME_TYPE_PUSH_PROMISE = 0x5;
|
||||
public static final short FRAME_TYPE_PING = 0x6;
|
||||
public static final short FRAME_TYPE_GO_AWAY = 0x7;
|
||||
public static final short FRAME_TYPE_WINDOW_UPDATE = 0x8;
|
||||
public static final short FRAME_TYPE_CONTINUATION = 0x9;
|
||||
|
||||
public static final short SETTINGS_HEADER_TABLE_SIZE = 1;
|
||||
public static final short SETTINGS_ENABLE_PUSH = 2;
|
||||
public static final short SETTINGS_MAX_CONCURRENT_STREAMS = 3;
|
||||
public static final short SETTINGS_INITIAL_WINDOW_SIZE = 4;
|
||||
|
||||
public static final short FLAG_END_STREAM = 0x1;
|
||||
public static final short FLAG_END_SEGMENT = 0x2;
|
||||
public static final short FLAG_END_HEADERS = 0x4;
|
||||
public static final short FLAG_PRIORITY = 0x8;
|
||||
public static final short FLAG_ACK = 0x1;
|
||||
public static final short FLAG_PAD_LOW = 0x10;
|
||||
public static final short FLAG_PAD_HIGH = 0x20;
|
||||
|
||||
/**
|
||||
* Reads a big-endian (31-bit) integer from the buffer.
|
||||
*/
|
||||
public static int readUnsignedInt(ByteBuf buf) {
|
||||
int offset = buf.readerIndex();
|
||||
int value = (buf.getByte(offset) & 0x7F) << 24 | (buf.getByte(offset + 1) & 0xFF) << 16
|
||||
| (buf.getByte(offset + 2) & 0xFF) << 8 | buf.getByte(offset + 3) & 0xFF;
|
||||
buf.skipBytes(4);
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a big-endian (32-bit) unsigned integer to the buffer.
|
||||
*/
|
||||
public static void writeUnsignedInt(long value, ByteBuf out) {
|
||||
out.writeByte((int) ((value >> 24) & 0xFF));
|
||||
out.writeByte((int) ((value >> 16) & 0xFF));
|
||||
out.writeByte((int) ((value >> 8) & 0xFF));
|
||||
out.writeByte((int) ((value & 0xFF)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the variable-length padding length field from the payload.
|
||||
*/
|
||||
public static int readPaddingLength(Http2Flags flags, ByteBuf payload) {
|
||||
int paddingLength = 0;
|
||||
if (flags.isPadHighPresent()) {
|
||||
paddingLength += payload.readUnsignedByte() * 256;
|
||||
}
|
||||
if (flags.isPadLowPresent()) {
|
||||
paddingLength += payload.readUnsignedByte();
|
||||
}
|
||||
return paddingLength;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the padding flags in the given flags value as appropriate based on the padding length.
|
||||
* Returns the new flags value after any padding flags have been set.
|
||||
*/
|
||||
public static short setPaddingFlags(short flags, int paddingLength) {
|
||||
if (paddingLength > 255) {
|
||||
flags |= FLAG_PAD_HIGH;
|
||||
}
|
||||
if (paddingLength > 0) {
|
||||
flags |= FLAG_PAD_LOW;
|
||||
}
|
||||
return flags;
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes the padding length field to the output buffer.
|
||||
*/
|
||||
public static void writePaddingLength(int paddingLength, ByteBuf out) {
|
||||
if (paddingLength > 255) {
|
||||
int padHigh = paddingLength / 256;
|
||||
out.writeByte(padHigh);
|
||||
}
|
||||
if (paddingLength > 0) {
|
||||
int padLow = paddingLength % 256;
|
||||
out.writeByte(padLow);
|
||||
}
|
||||
}
|
||||
|
||||
private Http2FrameCodecUtil() {
|
||||
}
|
||||
}
|
@ -1,84 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
|
||||
/**
|
||||
* Encapsulates the content of an HTTP2 frame header.
|
||||
*/
|
||||
public final class Http2FrameHeader {
|
||||
private final int payloadLength;
|
||||
private final int type;
|
||||
private final Http2Flags flags;
|
||||
private final int streamId;
|
||||
|
||||
private Http2FrameHeader(Builder builder) {
|
||||
payloadLength = builder.payloadLength;
|
||||
type = builder.type;
|
||||
flags = builder.flags;
|
||||
streamId = builder.streamId;
|
||||
}
|
||||
|
||||
public int getPayloadLength() {
|
||||
return payloadLength;
|
||||
}
|
||||
|
||||
public int getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
public Http2Flags getFlags() {
|
||||
return flags;
|
||||
}
|
||||
|
||||
public int getStreamId() {
|
||||
return streamId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds instances of {@link Http2FrameHeader}.
|
||||
*/
|
||||
public static class Builder {
|
||||
private int payloadLength;
|
||||
private int type;
|
||||
private Http2Flags flags = new Http2Flags((short) 0);
|
||||
private int streamId;
|
||||
|
||||
public Builder setPayloadLength(int payloadLength) {
|
||||
this.payloadLength = payloadLength;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setType(int type) {
|
||||
this.type = type;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setFlags(Http2Flags flags) {
|
||||
this.flags = flags;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setStreamId(int streamId) {
|
||||
this.streamId = streamId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Http2FrameHeader build() {
|
||||
return new Http2FrameHeader(this);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,60 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufHolder;
|
||||
|
||||
/**
|
||||
* An HTTP2 GO_AWAY frame indicating that the remote peer should stop creating streams for the
|
||||
* connection.
|
||||
*/
|
||||
public interface Http2GoAwayFrame extends Http2Frame, ByteBufHolder {
|
||||
/**
|
||||
* The highest numbered stream identifier for which the sender of the GOAWAY frame has received
|
||||
* frames on and might have taken some action on.
|
||||
*/
|
||||
int getLastStreamId();
|
||||
|
||||
/**
|
||||
* The error code containing the reason for closing the connection.
|
||||
*/
|
||||
long getErrorCode();
|
||||
|
||||
/**
|
||||
* Returns the debug data.
|
||||
*/
|
||||
@Override
|
||||
ByteBuf content();
|
||||
|
||||
@Override
|
||||
Http2GoAwayFrame copy();
|
||||
|
||||
@Override
|
||||
Http2GoAwayFrame duplicate();
|
||||
|
||||
@Override
|
||||
Http2GoAwayFrame retain();
|
||||
|
||||
@Override
|
||||
Http2GoAwayFrame retain(int increment);
|
||||
|
||||
@Override
|
||||
Http2GoAwayFrame touch();
|
||||
|
||||
@Override
|
||||
Http2GoAwayFrame touch(Object hint);
|
||||
}
|
@ -1,34 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import io.netty.handler.codec.http2.draft10.Http2Headers;
|
||||
|
||||
/**
|
||||
* The decoded form of a complete headers block for a HEADERS frame.
|
||||
*/
|
||||
public interface Http2HeadersFrame extends Http2StreamFrame {
|
||||
|
||||
/**
|
||||
* Gets the priority of the stream being created.
|
||||
*/
|
||||
int getPriority();
|
||||
|
||||
/**
|
||||
* Gets the decoded HTTP headers.
|
||||
*/
|
||||
Http2Headers getHeaders();
|
||||
}
|
@ -1,53 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufHolder;
|
||||
|
||||
/**
|
||||
* An HTTP2 connection PING frame.
|
||||
*/
|
||||
public interface Http2PingFrame extends Http2Frame, ByteBufHolder {
|
||||
/**
|
||||
* Indicates whether this frame is an acknowledgment of a PING sent by the peer.
|
||||
*/
|
||||
boolean isAck();
|
||||
|
||||
/**
|
||||
* Returns the opaque data of this frame.
|
||||
*/
|
||||
@Override
|
||||
ByteBuf content();
|
||||
|
||||
@Override
|
||||
Http2PingFrame copy();
|
||||
|
||||
@Override
|
||||
Http2PingFrame duplicate();
|
||||
|
||||
@Override
|
||||
Http2PingFrame retain();
|
||||
|
||||
@Override
|
||||
Http2PingFrame retain(int increment);
|
||||
|
||||
@Override
|
||||
Http2PingFrame touch();
|
||||
|
||||
@Override
|
||||
Http2PingFrame touch(Object hint);
|
||||
}
|
@ -1,26 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
/**
|
||||
* An HTTP2 priority frame indicating the sender-advised priority for the stream.
|
||||
*/
|
||||
public interface Http2PriorityFrame extends Http2StreamFrame {
|
||||
/**
|
||||
* The advised priority for the stream.
|
||||
*/
|
||||
int getPriority();
|
||||
}
|
@ -1,34 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
import io.netty.handler.codec.http2.draft10.Http2Headers;
|
||||
|
||||
/**
|
||||
* A decoded form of the completed headers block for a PUSH_PROMISE frame.
|
||||
*/
|
||||
public interface Http2PushPromiseFrame extends Http2StreamFrame {
|
||||
|
||||
/**
|
||||
* The ID of the stream that the endpoint intends to start sending frames for.
|
||||
*/
|
||||
int getPromisedStreamId();
|
||||
|
||||
/**
|
||||
* Gets the decoded HTTP headers.
|
||||
*/
|
||||
Http2Headers getHeaders();
|
||||
}
|
@ -1,26 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
/**
|
||||
* HTTP2 RST_STREAM frame that indicates abnormal termination of a stream.
|
||||
*/
|
||||
public interface Http2RstStreamFrame extends Http2StreamFrame {
|
||||
/**
|
||||
* The error code containing the reason for the stream being terminated.
|
||||
*/
|
||||
long getErrorCode();
|
||||
}
|
@ -1,48 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
/**
|
||||
* HTTP2 SETTINGS frame providing configuration parameters that affect how endpoints communicate.
|
||||
*/
|
||||
public interface Http2SettingsFrame extends Http2Frame {
|
||||
|
||||
/**
|
||||
* Indicates whether this is an acknowledgment of the settings sent by the peer.
|
||||
*/
|
||||
boolean isAck();
|
||||
|
||||
/**
|
||||
* Gets the sender's header compression table size, or {@code null} if not set.
|
||||
*/
|
||||
Integer getHeaderTableSize();
|
||||
|
||||
/**
|
||||
* Gets whether or not the sender allows server push, or {@code null} if not set.
|
||||
*/
|
||||
Boolean getPushEnabled();
|
||||
|
||||
/**
|
||||
* Gets the maximum number of streams the receiver is allowed to create, or {@code null} if not
|
||||
* set.
|
||||
*/
|
||||
Long getMaxConcurrentStreams();
|
||||
|
||||
/**
|
||||
* Gets the sender's initial flow control window in bytes, or {@code null} if not set.
|
||||
*/
|
||||
Integer getInitialWindowSize();
|
||||
}
|
@ -1,31 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
/**
|
||||
* Base interface for all frames that are associated to a stream.
|
||||
*/
|
||||
public interface Http2StreamFrame extends Http2Frame {
|
||||
/**
|
||||
* Gets the identifier of the associated stream.
|
||||
*/
|
||||
int getStreamId();
|
||||
|
||||
/**
|
||||
* Indicates whether this frame represents the last frame for the stream.
|
||||
*/
|
||||
boolean isEndOfStream();
|
||||
}
|
@ -1,27 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
||||
/**
|
||||
* HTTP2 WINDOW_UPDATE frame used to implement flow control.
|
||||
*/
|
||||
public interface Http2WindowUpdateFrame extends Http2StreamFrame {
|
||||
/**
|
||||
* Gets the number of bytes that the sender can transmit in addition to the existing flow control
|
||||
* window.
|
||||
*/
|
||||
int getWindowSizeIncrement();
|
||||
}
|
@ -1,126 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_CONTINUATION;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.readPaddingLength;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Flags;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
|
||||
public abstract class AbstractHeadersUnmarshaller extends AbstractHttp2FrameUnmarshaller {
|
||||
|
||||
/**
|
||||
* A builder for a headers/push_promise frame.
|
||||
*/
|
||||
protected abstract static class FrameBuilder {
|
||||
protected ByteBuf headerBlock;
|
||||
|
||||
abstract int getStreamId();
|
||||
|
||||
final void addHeaderFragment(ByteBuf fragment, ByteBufAllocator alloc) {
|
||||
if (headerBlock == null) {
|
||||
headerBlock = alloc.buffer(fragment.readableBytes());
|
||||
headerBlock.writeBytes(fragment);
|
||||
} else {
|
||||
ByteBuf buf = alloc.buffer(headerBlock.readableBytes() + fragment.readableBytes());
|
||||
buf.writeBytes(headerBlock);
|
||||
buf.writeBytes(fragment);
|
||||
headerBlock.release();
|
||||
headerBlock = buf;
|
||||
}
|
||||
}
|
||||
|
||||
abstract Http2Frame buildFrame() throws Http2Exception;
|
||||
}
|
||||
|
||||
private FrameBuilder frameBuilder;
|
||||
|
||||
@Override
|
||||
protected final void validate(Http2FrameHeader frameHeader) throws Http2Exception {
|
||||
if (frameBuilder == null) {
|
||||
// This frame is the beginning of a headers/push_promise.
|
||||
validateStartOfHeaderBlock(frameHeader);
|
||||
return;
|
||||
}
|
||||
|
||||
// Validate the continuation of a headers block.
|
||||
if (frameHeader.getType() != FRAME_TYPE_CONTINUATION) {
|
||||
throw protocolError("Unsupported frame type: %d.", frameHeader.getType());
|
||||
}
|
||||
if (frameBuilder.getStreamId() != frameHeader.getStreamId()) {
|
||||
throw protocolError("Continuation received for wrong stream. Expected %d, found %d",
|
||||
frameBuilder.getStreamId(), frameHeader.getStreamId());
|
||||
}
|
||||
Http2Flags flags = frameHeader.getFlags();
|
||||
if (!flags.isPaddingLengthValid()) {
|
||||
throw protocolError("Pad high is set but pad low is not");
|
||||
}
|
||||
if (frameHeader.getPayloadLength() < flags.getNumPaddingLengthBytes()) {
|
||||
throw protocolError("Frame length %d to small.", frameHeader.getPayloadLength());
|
||||
}
|
||||
if (frameHeader.getPayloadLength() > MAX_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Frame length %d too big.", frameHeader.getPayloadLength());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final Http2Frame doUnmarshall(Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception {
|
||||
Http2Flags flags = header.getFlags();
|
||||
if (frameBuilder == null) {
|
||||
// This is the start of a headers/push_promise frame. Delegate to the subclass to create
|
||||
// the appropriate builder for the frame.
|
||||
frameBuilder = createFrameBuilder(header, payload, alloc);
|
||||
} else {
|
||||
// Processing a continuation frame for a headers/push_promise. Update the current frame
|
||||
// builder with the new fragment.
|
||||
|
||||
int paddingLength = readPaddingLength(flags, payload);
|
||||
|
||||
// Determine how much data there is to read by removing the trailing
|
||||
// padding.
|
||||
int dataLength = payload.readableBytes() - paddingLength;
|
||||
if (dataLength < 0) {
|
||||
throw protocolError("Payload too small for padding.");
|
||||
}
|
||||
|
||||
// The remainder of this frame is the headers block.
|
||||
frameBuilder.addHeaderFragment(payload, alloc);
|
||||
}
|
||||
|
||||
// If the headers are complete, build the frame.
|
||||
Http2Frame frame = null;
|
||||
if (flags.isEndOfHeaders()) {
|
||||
frame = frameBuilder.buildFrame();
|
||||
frameBuilder = null;
|
||||
}
|
||||
|
||||
return frame;
|
||||
}
|
||||
|
||||
protected abstract void validateStartOfHeaderBlock(Http2FrameHeader frameHeader)
|
||||
throws Http2Exception;
|
||||
|
||||
protected abstract FrameBuilder createFrameBuilder(Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception;
|
||||
}
|
@ -1,66 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
|
||||
/**
|
||||
* Abstract base class for all {@link Http2FrameUnmarshaller} classes.
|
||||
*/
|
||||
public abstract class AbstractHttp2FrameUnmarshaller implements Http2FrameUnmarshaller {
|
||||
private Http2FrameHeader header;
|
||||
|
||||
@Override
|
||||
public final Http2FrameUnmarshaller unmarshall(Http2FrameHeader header) throws Http2Exception {
|
||||
if (header == null) {
|
||||
throw new NullPointerException("header");
|
||||
}
|
||||
|
||||
validate(header);
|
||||
this.header = header;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Http2Frame from(ByteBuf payload, ByteBufAllocator alloc) throws Http2Exception {
|
||||
if (header == null) {
|
||||
throw new IllegalStateException("header must be set before calling from().");
|
||||
}
|
||||
|
||||
return doUnmarshall(header, payload, alloc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Verifies that the given frame header is valid for the frame type(s) supported by this decoder.
|
||||
*/
|
||||
protected abstract void validate(Http2FrameHeader frameHeader) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Unmarshalls the frame.
|
||||
*
|
||||
* @param header the frame header
|
||||
* @param payload the payload of the frame.
|
||||
* @param alloc an allocator for new buffers
|
||||
* @return the frame
|
||||
* @throws Http2Exception thrown if any protocol error was encountered.
|
||||
*/
|
||||
protected abstract Http2Frame doUnmarshall(Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception;
|
||||
}
|
@ -1,88 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_DATA;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.readPaddingLength;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2DataFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2DataFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Flags;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
|
||||
/**
|
||||
* An unmarshaller for {@link Http2DataFrame} instances. The buffer contained in the frames is a
|
||||
* slice of the original input buffer. If the frame needs to be persisted it should be copied.
|
||||
*/
|
||||
public class Http2DataFrameUnmarshaller extends AbstractHttp2FrameUnmarshaller {
|
||||
|
||||
@Override
|
||||
protected void validate(Http2FrameHeader frameHeader) throws Http2Exception {
|
||||
if (frameHeader.getType() != FRAME_TYPE_DATA) {
|
||||
throw protocolError("Unsupported frame type: %d.", frameHeader.getType());
|
||||
}
|
||||
if (frameHeader.getStreamId() <= 0) {
|
||||
throw protocolError("A stream ID must be > 0.");
|
||||
}
|
||||
Http2Flags flags = frameHeader.getFlags();
|
||||
if (!flags.isPaddingLengthValid()) {
|
||||
throw protocolError("Pad high is set but pad low is not");
|
||||
}
|
||||
if (frameHeader.getPayloadLength() < flags.getNumPaddingLengthBytes()) {
|
||||
throw protocolError("Frame length %d too small.", frameHeader.getPayloadLength());
|
||||
}
|
||||
if (frameHeader.getPayloadLength() > MAX_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Frame length %d too big.", frameHeader.getPayloadLength());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Http2Frame doUnmarshall(Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception {
|
||||
DefaultHttp2DataFrame.Builder builder = new DefaultHttp2DataFrame.Builder();
|
||||
builder.setStreamId(header.getStreamId());
|
||||
|
||||
Http2Flags flags = header.getFlags();
|
||||
builder.setEndOfStream(flags.isEndOfStream());
|
||||
|
||||
// Read the padding length.
|
||||
int paddingLength = readPaddingLength(flags, payload);
|
||||
builder.setPaddingLength(paddingLength);
|
||||
|
||||
// Determine how much data there is to read by removing the trailing
|
||||
// padding.
|
||||
int dataLength = payload.readableBytes() - paddingLength;
|
||||
if (dataLength < 0) {
|
||||
throw protocolError("Frame payload too small for padding.");
|
||||
}
|
||||
|
||||
// Copy the remaining data into the frame.
|
||||
ByteBuf data = payload.slice(payload.readerIndex(), dataLength).retain();
|
||||
builder.setContent(data);
|
||||
|
||||
// Skip the rest of the bytes in the payload.
|
||||
payload.skipBytes(payload.readableBytes());
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
}
|
@ -1,140 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.handler.codec.ByteToMessageDecoder;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Flags;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.*;
|
||||
|
||||
/**
|
||||
* Decodes {@link Http2Frame} objects from an input {@link ByteBuf}. The frames that this handler
|
||||
* emits can be configured by providing a {@link Http2FrameUnmarshaller}. By default, the
|
||||
* {@link Http2StandardFrameUnmarshaller} is used to handle all frame types - see the documentation
|
||||
* for details.
|
||||
*
|
||||
* @see Http2StandardFrameUnmarshaller
|
||||
*/
|
||||
public class Http2FrameDecoder extends ByteToMessageDecoder {
|
||||
|
||||
private enum State {
|
||||
FRAME_HEADER,
|
||||
FRAME_PAYLOAD,
|
||||
ERROR
|
||||
}
|
||||
|
||||
private final Http2FrameUnmarshaller frameUnmarshaller;
|
||||
private State state;
|
||||
private int payloadLength;
|
||||
|
||||
public Http2FrameDecoder() {
|
||||
this(new Http2StandardFrameUnmarshaller());
|
||||
}
|
||||
|
||||
public Http2FrameDecoder(Http2FrameUnmarshaller frameUnmarshaller) {
|
||||
if (frameUnmarshaller == null) {
|
||||
throw new NullPointerException("frameUnmarshaller");
|
||||
}
|
||||
this.frameUnmarshaller = frameUnmarshaller;
|
||||
state = State.FRAME_HEADER;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
|
||||
try {
|
||||
switch (state) {
|
||||
case FRAME_HEADER:
|
||||
processFrameHeader(in);
|
||||
if (state == State.FRAME_HEADER) {
|
||||
// Still haven't read the entire frame header yet.
|
||||
break;
|
||||
}
|
||||
|
||||
// If we successfully read the entire frame header, drop down and start processing
|
||||
// the payload now.
|
||||
|
||||
case FRAME_PAYLOAD:
|
||||
processFramePayload(ctx, in, out);
|
||||
break;
|
||||
case ERROR:
|
||||
in.skipBytes(in.readableBytes());
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Should never get here");
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
ctx.fireExceptionCaught(t);
|
||||
state = State.ERROR;
|
||||
}
|
||||
}
|
||||
|
||||
private void processFrameHeader(ByteBuf in) throws Http2Exception {
|
||||
if (in.readableBytes() < FRAME_HEADER_LENGTH) {
|
||||
// Wait until the entire frame header has been read.
|
||||
return;
|
||||
}
|
||||
|
||||
// Read the header and prepare the unmarshaller to read the frame.
|
||||
Http2FrameHeader frameHeader = readFrameHeader(in);
|
||||
|
||||
payloadLength = frameHeader.getPayloadLength();
|
||||
frameUnmarshaller.unmarshall(frameHeader);
|
||||
|
||||
// Start reading the payload for the frame.
|
||||
state = State.FRAME_PAYLOAD;
|
||||
}
|
||||
|
||||
private void processFramePayload(ChannelHandlerContext ctx, ByteBuf in, List<Object> out)
|
||||
throws Http2Exception {
|
||||
if (in.readableBytes() < payloadLength) {
|
||||
// Wait until the entire payload has been read.
|
||||
return;
|
||||
}
|
||||
|
||||
// Get a view of the buffer for the size of the payload.
|
||||
ByteBuf payload = in.readSlice(payloadLength);
|
||||
|
||||
// Create the frame and add it to the output.
|
||||
Http2Frame frame = frameUnmarshaller.from(payload, ctx.alloc());
|
||||
if (frame != null) {
|
||||
out.add(frame);
|
||||
}
|
||||
|
||||
// Go back to reading the next frame header.
|
||||
state = State.FRAME_HEADER;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the frame header from the input buffer and creates an envelope initialized with those
|
||||
* values.
|
||||
*/
|
||||
private static Http2FrameHeader readFrameHeader(ByteBuf in) {
|
||||
int payloadLength = in.readUnsignedShort() & FRAME_LENGTH_MASK;
|
||||
short type = in.readUnsignedByte();
|
||||
short flags = in.readUnsignedByte();
|
||||
int streamId = readUnsignedInt(in);
|
||||
|
||||
return new Http2FrameHeader.Builder().setPayloadLength(payloadLength).setType(type)
|
||||
.setFlags(new Http2Flags(flags)).setStreamId(streamId).build();
|
||||
}
|
||||
}
|
@ -1,50 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
|
||||
/**
|
||||
* Used by the {@link Http2FrameDecoder} to unmarshall {@link Http2Frame} objects from an input
|
||||
* {@link ByteBuf}.
|
||||
*/
|
||||
public interface Http2FrameUnmarshaller {
|
||||
|
||||
/**
|
||||
* Prepares the unmarshaller for the next frame.
|
||||
*
|
||||
* @param header the header providing the detais of the frame to be unmarshalled.
|
||||
* @return this unmarshaller
|
||||
* @throws Http2Exception thrown if any of the information of the header violates the protocol.
|
||||
*/
|
||||
Http2FrameUnmarshaller unmarshall(Http2FrameHeader header) throws Http2Exception;
|
||||
|
||||
/**
|
||||
* Unmarshalls the frame from the payload.
|
||||
*
|
||||
* @param payload the payload from which the frame is to be unmarshalled.
|
||||
* @param alloc the allocator for any new buffers required by the unmarshaller.
|
||||
* @return the frame or {@code null} if the unmarshall operation is processing is incomplete and
|
||||
* requires additional data.
|
||||
* @throws Http2Exception thrown if any protocol error was encountered while unmarshalling the
|
||||
* frame.
|
||||
*/
|
||||
Http2Frame from(ByteBuf payload, ByteBufAllocator alloc) throws Http2Exception;
|
||||
}
|
@ -1,70 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_GO_AWAY;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.readUnsignedInt;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2GoAwayFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2GoAwayFrame;
|
||||
|
||||
/**
|
||||
* An unmarshaller for {@link Http2GoAwayFrame} instances. The buffer contained in the frames is a
|
||||
* slice of the original input buffer. If the frame needs to be persisted it should be copied.
|
||||
*/
|
||||
public class Http2GoAwayFrameUnmarshaller extends AbstractHttp2FrameUnmarshaller {
|
||||
|
||||
@Override
|
||||
protected void validate(Http2FrameHeader frameHeader) throws Http2Exception {
|
||||
if (frameHeader.getType() != FRAME_TYPE_GO_AWAY) {
|
||||
throw protocolError("Unsupported frame type: %d.", frameHeader.getType());
|
||||
}
|
||||
if (frameHeader.getStreamId() != 0) {
|
||||
throw protocolError("A stream ID must be zero.");
|
||||
}
|
||||
if (frameHeader.getPayloadLength() < 8) {
|
||||
throw protocolError("Frame length %d too small.", frameHeader.getPayloadLength());
|
||||
}
|
||||
if (frameHeader.getPayloadLength() > MAX_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Frame length %d too big.", frameHeader.getPayloadLength());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Http2Frame doUnmarshall(Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception {
|
||||
DefaultHttp2GoAwayFrame.Builder builder = new DefaultHttp2GoAwayFrame.Builder();
|
||||
|
||||
int lastStreamId = readUnsignedInt(payload);
|
||||
builder.setLastStreamId(lastStreamId);
|
||||
|
||||
long errorCode = payload.readUnsignedInt();
|
||||
builder.setErrorCode(errorCode);
|
||||
|
||||
// The remainder of this frame is the debug data.
|
||||
ByteBuf data = payload.slice().retain();
|
||||
builder.setDebugData(data);
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
}
|
@ -1,152 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_HEADERS;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.readPaddingLength;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.readUnsignedInt;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Headers;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2HeadersFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Flags;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2HeadersFrame;
|
||||
|
||||
/**
|
||||
* An unmarshaller for {@link Http2HeadersFrame} instances.
|
||||
*/
|
||||
public class Http2HeadersFrameUnmarshaller extends AbstractHeadersUnmarshaller {
|
||||
|
||||
private final Http2HeadersDecoder headersDecoder;
|
||||
|
||||
public Http2HeadersFrameUnmarshaller(Http2HeadersDecoder headersDecoder) {
|
||||
if (headersDecoder == null) {
|
||||
throw new NullPointerException("headersDecoder");
|
||||
}
|
||||
this.headersDecoder = headersDecoder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void validateStartOfHeaderBlock(Http2FrameHeader frameHeader) throws Http2Exception {
|
||||
if (frameHeader.getType() != FRAME_TYPE_HEADERS) {
|
||||
throw protocolError("Unsupported frame type: %d.", frameHeader.getType());
|
||||
}
|
||||
|
||||
if (frameHeader.getStreamId() <= 0) {
|
||||
throw protocolError("A stream ID must > 0.");
|
||||
}
|
||||
|
||||
Http2Flags flags = frameHeader.getFlags();
|
||||
if (flags.isPriorityPresent() && frameHeader.getPayloadLength() < 4) {
|
||||
throw protocolError("Frame length too small." + frameHeader.getPayloadLength());
|
||||
}
|
||||
|
||||
if (!flags.isPaddingLengthValid()) {
|
||||
throw protocolError("Pad high is set but pad low is not");
|
||||
}
|
||||
|
||||
if (frameHeader.getPayloadLength() < flags.getNumPaddingLengthBytes()) {
|
||||
throw protocolError("Frame length %d too small for padding.", frameHeader.getPayloadLength());
|
||||
}
|
||||
|
||||
if (frameHeader.getPayloadLength() > MAX_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Frame length %d too big.", frameHeader.getPayloadLength());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FrameBuilder createFrameBuilder(final Http2FrameHeader header, final ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception {
|
||||
try {
|
||||
final DefaultHttp2HeadersFrame.Builder builder = new DefaultHttp2HeadersFrame.Builder();
|
||||
builder.setStreamId(header.getStreamId());
|
||||
|
||||
Http2Flags flags = header.getFlags();
|
||||
builder.setEndOfStream(flags.isEndOfStream());
|
||||
|
||||
// Read the padding length.
|
||||
int paddingLength = readPaddingLength(flags, payload);
|
||||
|
||||
// Read the priority if it was included in the frame.
|
||||
if (flags.isPriorityPresent()) {
|
||||
int priority = readUnsignedInt(payload);
|
||||
builder.setPriority(priority);
|
||||
}
|
||||
|
||||
// Determine how much data there is to read by removing the trailing
|
||||
// padding.
|
||||
int dataLength = payload.readableBytes() - paddingLength;
|
||||
if (dataLength < 0) {
|
||||
throw protocolError("Payload too small for padding");
|
||||
}
|
||||
|
||||
// Get a view of the header block portion of the payload.
|
||||
final ByteBuf headerSlice = payload.readSlice(dataLength);
|
||||
|
||||
// The remainder of this frame is the headers block.
|
||||
if (flags.isEndOfHeaders()) {
|
||||
// Optimization: don't copy the buffer if we have the entire headers block.
|
||||
return new FrameBuilder() {
|
||||
@Override
|
||||
int getStreamId() {
|
||||
return header.getStreamId();
|
||||
}
|
||||
|
||||
@Override
|
||||
Http2Frame buildFrame() throws Http2Exception {
|
||||
Http2Headers headers = headersDecoder.decodeHeaders(headerSlice);
|
||||
builder.setHeaders(headers);
|
||||
return builder.build();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// The header block is not complete. Await one or more continuation frames
|
||||
// to complete the block before decoding.
|
||||
FrameBuilder frameBuilder = new FrameBuilder() {
|
||||
@Override
|
||||
int getStreamId() {
|
||||
return header.getStreamId();
|
||||
}
|
||||
|
||||
@Override
|
||||
Http2Frame buildFrame() throws Http2Exception {
|
||||
try {
|
||||
Http2Headers headers = headersDecoder.decodeHeaders(headerBlock);
|
||||
builder.setHeaders(headers);
|
||||
return builder.build();
|
||||
} finally {
|
||||
headerBlock.release();
|
||||
headerBlock = null;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// Copy and add the initial fragment of the header block.
|
||||
frameBuilder.addHeaderFragment(headerSlice, alloc);
|
||||
|
||||
return frameBuilder;
|
||||
} finally {
|
||||
payload.skipBytes(payload.readableBytes());
|
||||
}
|
||||
}
|
||||
}
|
@ -1,62 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_PING;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.PING_FRAME_PAYLOAD_LENGTH;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2PingFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PingFrame;
|
||||
|
||||
/**
|
||||
* An unmarshaller for {@link Http2PingFrame} instances. The buffer contained in the frames is a
|
||||
* slice of the original input buffer. If the frame needs to be persisted it should be copied.
|
||||
*/
|
||||
public class Http2PingFrameUnmarshaller extends AbstractHttp2FrameUnmarshaller {
|
||||
|
||||
@Override
|
||||
protected void validate(Http2FrameHeader frameHeader) throws Http2Exception {
|
||||
if (frameHeader.getType() != FRAME_TYPE_PING) {
|
||||
throw protocolError("Unsupported frame type: %d.", frameHeader.getType());
|
||||
}
|
||||
if (frameHeader.getStreamId() != 0) {
|
||||
throw protocolError("A stream ID must be zero.");
|
||||
}
|
||||
if (frameHeader.getPayloadLength() != PING_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Frame length %d incorrect size for ping.",
|
||||
frameHeader.getPayloadLength());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Http2Frame doUnmarshall(Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception {
|
||||
DefaultHttp2PingFrame.Builder builder = new DefaultHttp2PingFrame.Builder();
|
||||
builder.setAck(header.getFlags().isAck());
|
||||
|
||||
// The remainder of this frame is the opaque data.
|
||||
ByteBuf data = payload.slice().retain();
|
||||
builder.setData(data);
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
}
|
@ -1,63 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_PRIORITY;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.readUnsignedInt;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2PriorityFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PriorityFrame;
|
||||
|
||||
/**
|
||||
* An unmarshaller for {@link Http2PriorityFrame} instances.
|
||||
*/
|
||||
public class Http2PriorityFrameUnmarshaller extends AbstractHttp2FrameUnmarshaller {
|
||||
|
||||
@Override
|
||||
protected void validate(Http2FrameHeader frameHeader) throws Http2Exception {
|
||||
if (frameHeader.getType() != FRAME_TYPE_PRIORITY) {
|
||||
throw protocolError("Unsupported frame type: %d.", frameHeader.getType());
|
||||
}
|
||||
if (frameHeader.getStreamId() <= 0) {
|
||||
throw protocolError("A stream ID must be > 0.");
|
||||
}
|
||||
if (frameHeader.getPayloadLength() < 4) {
|
||||
throw protocolError("Frame length %d too small.", frameHeader.getPayloadLength());
|
||||
}
|
||||
if (frameHeader.getPayloadLength() > MAX_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Frame length %d too big.", frameHeader.getPayloadLength());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Http2Frame doUnmarshall(Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception {
|
||||
DefaultHttp2PriorityFrame.Builder builder = new DefaultHttp2PriorityFrame.Builder();
|
||||
builder.setStreamId(header.getStreamId());
|
||||
|
||||
int priority = readUnsignedInt(payload);
|
||||
builder.setPriority(priority);
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
}
|
@ -1,119 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_PUSH_PROMISE;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.readUnsignedInt;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Headers;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2PushPromiseFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Flags;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PushPromiseFrame;
|
||||
|
||||
/**
|
||||
* An unmarshaller for {@link Http2PushPromiseFrame} instances.
|
||||
*/
|
||||
public class Http2PushPromiseFrameUnmarshaller extends AbstractHeadersUnmarshaller {
|
||||
|
||||
private final Http2HeadersDecoder headersDecoder;
|
||||
|
||||
public Http2PushPromiseFrameUnmarshaller(Http2HeadersDecoder headersDecoder) {
|
||||
if (headersDecoder == null) {
|
||||
throw new NullPointerException("headersDecoder");
|
||||
}
|
||||
this.headersDecoder = headersDecoder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void validateStartOfHeaderBlock(Http2FrameHeader frameHeader) throws Http2Exception {
|
||||
if (frameHeader.getType() != FRAME_TYPE_PUSH_PROMISE) {
|
||||
throw protocolError("Unsupported frame type: %d.", frameHeader.getType());
|
||||
}
|
||||
if (frameHeader.getStreamId() <= 0) {
|
||||
throw protocolError("A stream ID must > 0.");
|
||||
}
|
||||
if (frameHeader.getPayloadLength() < 4) {
|
||||
throw protocolError("Frame length too small." + frameHeader.getPayloadLength());
|
||||
}
|
||||
if (frameHeader.getPayloadLength() > MAX_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Frame length %d too big.", frameHeader.getPayloadLength());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FrameBuilder createFrameBuilder(final Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception {
|
||||
final DefaultHttp2PushPromiseFrame.Builder builder = new DefaultHttp2PushPromiseFrame.Builder();
|
||||
builder.setStreamId(header.getStreamId());
|
||||
|
||||
int promisedStreamId = readUnsignedInt(payload);
|
||||
builder.setPromisedStreamId(promisedStreamId);
|
||||
|
||||
final ByteBuf headerSlice = payload.readSlice(payload.readableBytes());
|
||||
|
||||
// The remainder of this frame is the headers block.
|
||||
Http2Flags flags = header.getFlags();
|
||||
if (flags.isEndOfHeaders()) {
|
||||
// Optimization: don't copy the buffer if we have the entire headers block.
|
||||
return new FrameBuilder() {
|
||||
@Override
|
||||
int getStreamId() {
|
||||
return header.getStreamId();
|
||||
}
|
||||
|
||||
@Override
|
||||
Http2Frame buildFrame() throws Http2Exception {
|
||||
Http2Headers headers = headersDecoder.decodeHeaders(headerSlice);
|
||||
builder.setHeaders(headers);
|
||||
return builder.build();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// The header block is not complete. Await one or more continuation frames
|
||||
// to complete the block before decoding.
|
||||
FrameBuilder frameBuilder = new FrameBuilder() {
|
||||
@Override
|
||||
int getStreamId() {
|
||||
return header.getStreamId();
|
||||
}
|
||||
|
||||
@Override
|
||||
Http2Frame buildFrame() throws Http2Exception {
|
||||
try {
|
||||
Http2Headers headers = headersDecoder.decodeHeaders(headerBlock);
|
||||
builder.setHeaders(headers);
|
||||
return builder.build();
|
||||
} finally {
|
||||
headerBlock.release();
|
||||
headerBlock = null;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// Copy and add the initial fragment of the header block.
|
||||
frameBuilder.addHeaderFragment(headerSlice, alloc);
|
||||
|
||||
return frameBuilder;
|
||||
}
|
||||
}
|
@ -1,63 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_RST_STREAM;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2RstStreamFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2RstStreamFrame;
|
||||
|
||||
/**
|
||||
* An unmarshaller for {@link Http2RstStreamFrame} instances.
|
||||
*/
|
||||
public class Http2RstStreamFrameUnmarshaller extends AbstractHttp2FrameUnmarshaller {
|
||||
|
||||
@Override
|
||||
protected void validate(Http2FrameHeader frameHeader) throws Http2Exception {
|
||||
if (frameHeader.getType() != FRAME_TYPE_RST_STREAM) {
|
||||
throw protocolError("Unsupported frame type: %d.", frameHeader.getType());
|
||||
}
|
||||
if (frameHeader.getStreamId() <= 0) {
|
||||
throw protocolError("A stream ID must be > 0.");
|
||||
}
|
||||
if (frameHeader.getPayloadLength() < 4) {
|
||||
throw protocolError("Frame length %d too small.", frameHeader.getPayloadLength());
|
||||
}
|
||||
if (frameHeader.getPayloadLength() > MAX_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Frame length %d too big.", frameHeader.getPayloadLength());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Http2Frame doUnmarshall(Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception {
|
||||
DefaultHttp2RstStreamFrame.Builder builder = new DefaultHttp2RstStreamFrame.Builder();
|
||||
builder.setStreamId(header.getStreamId());
|
||||
|
||||
long errorCode = payload.readUnsignedInt();
|
||||
builder.setErrorCode(errorCode);
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
}
|
@ -1,72 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelHandlerAdapter;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Error.*;
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.*;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.*;
|
||||
|
||||
/**
|
||||
* Reads the initial client preface, then removes itself from the pipeline.
|
||||
* Only the server pipeline should do this.
|
||||
*
|
||||
* https://tools.ietf.org/html/draft-ietf-httpbis-http2-10#section-3.5
|
||||
*/
|
||||
public class Http2ServerPrefaceReader extends ChannelHandlerAdapter {
|
||||
|
||||
private final ByteBuf preface = connectionPrefaceBuf();
|
||||
|
||||
@Override
|
||||
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
|
||||
if (preface.isReadable() && msg instanceof ByteBuf) {
|
||||
ByteBuf buf = (ByteBuf) msg;
|
||||
processHttp2Preface(ctx, buf);
|
||||
if (preface.isReadable()) {
|
||||
// More preface left to process.
|
||||
buf.release();
|
||||
return;
|
||||
}
|
||||
}
|
||||
super.channelRead(ctx, msg);
|
||||
}
|
||||
|
||||
private void processHttp2Preface(ChannelHandlerContext ctx, ByteBuf in) throws Http2Exception {
|
||||
int prefaceRemaining = preface.readableBytes();
|
||||
int bytesRead = Math.min(in.readableBytes(), prefaceRemaining);
|
||||
|
||||
// Read the portion of the input up to the length of the preface, if reached.
|
||||
ByteBuf sourceSlice = in.readSlice(bytesRead);
|
||||
|
||||
// Read the same number of bytes from the preface buffer.
|
||||
ByteBuf prefaceSlice = preface.readSlice(bytesRead);
|
||||
|
||||
// If the input so far doesn't match the preface, break the connection.
|
||||
if (bytesRead == 0 || !prefaceSlice.equals(sourceSlice)) {
|
||||
throw format(PROTOCOL_ERROR, "Invalid HTTP2 preface");
|
||||
}
|
||||
|
||||
if (!preface.isReadable()) {
|
||||
// Entire preface has been read, remove ourselves from the pipeline.
|
||||
ctx.pipeline().remove(this);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -1,100 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_SETTINGS;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.SETTINGS_ENABLE_PUSH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.SETTINGS_HEADER_TABLE_SIZE;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.SETTINGS_INITIAL_WINDOW_SIZE;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.SETTINGS_MAX_CONCURRENT_STREAMS;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2SettingsFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2SettingsFrame;
|
||||
|
||||
/**
|
||||
* An unmarshaller for {@link Http2SettingsFrame} instances.
|
||||
*/
|
||||
public class Http2SettingsFrameUnmarshaller extends AbstractHttp2FrameUnmarshaller {
|
||||
|
||||
@Override
|
||||
protected void validate(Http2FrameHeader frameHeader) throws Http2Exception {
|
||||
if (frameHeader.getType() != FRAME_TYPE_SETTINGS) {
|
||||
throw protocolError("Unsupported frame type: %d.", frameHeader.getType());
|
||||
}
|
||||
if (frameHeader.getStreamId() != 0) {
|
||||
throw protocolError("A stream ID must be zero.");
|
||||
}
|
||||
if (frameHeader.getFlags().isAck() && frameHeader.getPayloadLength() > 0) {
|
||||
throw protocolError("Ack settings frame must have an empty payload.");
|
||||
}
|
||||
if (frameHeader.getPayloadLength() % 5 > 0) {
|
||||
throw protocolError("Frame length %d invalid.", frameHeader.getPayloadLength());
|
||||
}
|
||||
if (frameHeader.getPayloadLength() > MAX_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Frame length %d too big.", frameHeader.getPayloadLength());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Http2Frame doUnmarshall(Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception {
|
||||
DefaultHttp2SettingsFrame.Builder builder = new DefaultHttp2SettingsFrame.Builder();
|
||||
builder.setAck(header.getFlags().isAck());
|
||||
|
||||
int numSettings = header.getPayloadLength() / 5;
|
||||
for (int index = 0; index < numSettings; ++index) {
|
||||
short id = payload.readUnsignedByte();
|
||||
long value = payload.readUnsignedInt();
|
||||
switch (id) {
|
||||
case SETTINGS_HEADER_TABLE_SIZE:
|
||||
if (value <= 0L || value > Integer.MAX_VALUE) {
|
||||
throw protocolError("Invalid header table size setting: %d", value);
|
||||
}
|
||||
builder.setHeaderTableSize((int) value);
|
||||
break;
|
||||
case SETTINGS_ENABLE_PUSH:
|
||||
if (value != 0L && value != 1L) {
|
||||
throw protocolError("Invalid enable push setting: %d", value);
|
||||
}
|
||||
builder.setPushEnabled(value == 1);
|
||||
break;
|
||||
case SETTINGS_MAX_CONCURRENT_STREAMS:
|
||||
if (value < 0L) {
|
||||
throw protocolError("Invalid max concurrent streams setting: %d", value);
|
||||
}
|
||||
builder.setMaxConcurrentStreams(value);
|
||||
break;
|
||||
case SETTINGS_INITIAL_WINDOW_SIZE:
|
||||
if (value < 0L || value > Integer.MAX_VALUE) {
|
||||
throw protocolError("Invalid initial window size setting: %d", value);
|
||||
}
|
||||
builder.setInitialWindowSize((int) value);
|
||||
break;
|
||||
default:
|
||||
throw protocolError("Unsupported setting: %d", id);
|
||||
}
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
}
|
@ -1,125 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_CONTINUATION;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_DATA;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_GO_AWAY;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_HEADERS;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_PING;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_PRIORITY;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_PUSH_PROMISE;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_RST_STREAM;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_SETTINGS;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_WINDOW_UPDATE;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2DataFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2GoAwayFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2HeadersFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PingFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PriorityFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PushPromiseFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2RstStreamFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2SettingsFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2WindowUpdateFrame;
|
||||
|
||||
/**
|
||||
* A composite {@link Http2FrameUnmarshaller} that supports all frames identified by the HTTP2 spec.
|
||||
* This unmarshalls the following frames:
|
||||
* <p/>
|
||||
* {@link Http2DataFrame} (buffer is a slice of input buffer - must be copied if persisted)<br>
|
||||
* {@link Http2GoAwayFrame} (buffer is a slice of input buffer - must be copied if persisted)<br>
|
||||
* {@link Http2HeadersFrame}<br>
|
||||
* {@link Http2PingFrame} (buffer is a slice of input buffer - must be copied if persisted)<br>
|
||||
* {@link Http2PriorityFrame}<br>
|
||||
* {@link Http2PushPromiseFrame}<br>
|
||||
* {@link Http2RstStreamFrame}<br>
|
||||
* {@link Http2SettingsFrame}<br>
|
||||
* {@link Http2WindowUpdateFrame}<br>
|
||||
*/
|
||||
public class Http2StandardFrameUnmarshaller implements Http2FrameUnmarshaller {
|
||||
|
||||
private final Http2FrameUnmarshaller[] unmarshallers;
|
||||
private Http2FrameUnmarshaller activeUnmarshaller;
|
||||
|
||||
public Http2StandardFrameUnmarshaller() {
|
||||
this(new DefaultHttp2HeadersDecoder());
|
||||
}
|
||||
|
||||
public Http2StandardFrameUnmarshaller(Http2HeadersDecoder headersDecoder) {
|
||||
unmarshallers = new Http2FrameUnmarshaller[FRAME_TYPE_CONTINUATION + 1];
|
||||
unmarshallers[FRAME_TYPE_DATA] = new Http2DataFrameUnmarshaller();
|
||||
unmarshallers[FRAME_TYPE_HEADERS] = new Http2HeadersFrameUnmarshaller(headersDecoder);
|
||||
unmarshallers[FRAME_TYPE_PRIORITY] = new Http2PriorityFrameUnmarshaller();
|
||||
unmarshallers[FRAME_TYPE_RST_STREAM] = new Http2RstStreamFrameUnmarshaller();
|
||||
unmarshallers[FRAME_TYPE_SETTINGS] = new Http2SettingsFrameUnmarshaller();
|
||||
unmarshallers[FRAME_TYPE_PUSH_PROMISE] = new Http2PushPromiseFrameUnmarshaller(headersDecoder);
|
||||
unmarshallers[FRAME_TYPE_PING] = new Http2PingFrameUnmarshaller();
|
||||
unmarshallers[FRAME_TYPE_GO_AWAY] = new Http2GoAwayFrameUnmarshaller();
|
||||
unmarshallers[FRAME_TYPE_WINDOW_UPDATE] = new Http2WindowUpdateFrameUnmarshaller();
|
||||
unmarshallers[FRAME_TYPE_CONTINUATION] = new Http2FrameUnmarshaller() {
|
||||
private static final String MSG = "Received continuation without headers or push_promise";
|
||||
|
||||
@Override
|
||||
public Http2FrameUnmarshaller unmarshall(Http2FrameHeader header) throws Http2Exception {
|
||||
throw protocolError(MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Frame from(ByteBuf payload, ByteBufAllocator alloc) throws Http2Exception {
|
||||
throw protocolError(MSG);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2FrameUnmarshaller unmarshall(Http2FrameHeader header) throws Http2Exception {
|
||||
// If we're not in the middle of unmarshalling a continued frame (e.g. headers,
|
||||
// push_promise), select the appropriate marshaller for the frame type.
|
||||
if (activeUnmarshaller == null) {
|
||||
int type = header.getType();
|
||||
if (type < 0 || type >= unmarshallers.length || unmarshallers[type] == null) {
|
||||
throw protocolError("Unsupported frame type: %d", type);
|
||||
}
|
||||
|
||||
activeUnmarshaller = unmarshallers[type];
|
||||
}
|
||||
|
||||
// Prepare the unmarshaller.
|
||||
activeUnmarshaller.unmarshall(header);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Http2Frame from(ByteBuf payload, ByteBufAllocator alloc) throws Http2Exception {
|
||||
if (activeUnmarshaller == null) {
|
||||
throw new IllegalStateException("Must call unmarshall() before calling from().");
|
||||
}
|
||||
Http2Frame frame = activeUnmarshaller.from(payload, alloc);
|
||||
if (frame != null) {
|
||||
// The unmarshall is complete and does not require more frames. Clear the active
|
||||
// marshaller so that we select a fresh marshaller next time.
|
||||
activeUnmarshaller = null;
|
||||
}
|
||||
return frame;
|
||||
}
|
||||
}
|
@ -1,59 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_WINDOW_UPDATE;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.PING_FRAME_PAYLOAD_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.readUnsignedInt;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2WindowUpdateFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameHeader;
|
||||
|
||||
public class Http2WindowUpdateFrameUnmarshaller extends AbstractHttp2FrameUnmarshaller {
|
||||
|
||||
@Override
|
||||
protected void validate(Http2FrameHeader frameHeader) throws Http2Exception {
|
||||
if (frameHeader.getType() != FRAME_TYPE_WINDOW_UPDATE) {
|
||||
throw protocolError("Unsupported frame type: %d.", frameHeader.getType());
|
||||
}
|
||||
if (frameHeader.getStreamId() < 0) {
|
||||
throw protocolError("Stream Id must be >=0: ", frameHeader.getStreamId());
|
||||
}
|
||||
if (frameHeader.getPayloadLength() < 4) {
|
||||
throw protocolError("Frame length %d too small.", frameHeader.getPayloadLength());
|
||||
}
|
||||
if (frameHeader.getPayloadLength() > PING_FRAME_PAYLOAD_LENGTH) {
|
||||
throw protocolError("Frame length %d too big.", frameHeader.getPayloadLength());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Http2Frame doUnmarshall(Http2FrameHeader header, ByteBuf payload,
|
||||
ByteBufAllocator alloc) throws Http2Exception {
|
||||
DefaultHttp2WindowUpdateFrame.Builder builder = new DefaultHttp2WindowUpdateFrame.Builder();
|
||||
builder.setStreamId(header.getStreamId());
|
||||
|
||||
int windowSizeIncrement = readUnsignedInt(payload);
|
||||
builder.setWindowSizeIncrement(windowSizeIncrement);
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
}
|
@ -1,20 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Decoder and related classes for HTTP2 frames.
|
||||
*/
|
||||
package io.netty.handler.codec.http2.draft10.frame.decoder;
|
||||
|
@ -1,66 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
|
||||
/**
|
||||
* Abstract base class for all {@link Http2FrameMarshaller}s.
|
||||
*/
|
||||
public abstract class AbstractHttp2FrameMarshaller<T extends Http2Frame> implements
|
||||
Http2FrameMarshaller {
|
||||
|
||||
private final Class<T> frameType;
|
||||
|
||||
protected AbstractHttp2FrameMarshaller(Class<T> frameType) {
|
||||
if (frameType == null) {
|
||||
throw new NullPointerException("frameType");
|
||||
}
|
||||
this.frameType = frameType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void marshall(Http2Frame frame, ByteBuf out, ByteBufAllocator alloc)
|
||||
throws Http2Exception {
|
||||
if (frame == null) {
|
||||
throw new NullPointerException("frame");
|
||||
}
|
||||
|
||||
if (!frameType.isAssignableFrom(frame.getClass())) {
|
||||
throw protocolError("Unsupported frame type: %s", frame.getClass().getName());
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
T frameT = (T) frame;
|
||||
doMarshall(frameT, out, alloc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Marshals the frame to the output buffer.
|
||||
*
|
||||
* @param frame the frame to be marshalled
|
||||
* @param out the buffer to marshall the frame to.
|
||||
* @param alloc an allocator that this marshaller may use for creating intermediate buffers as
|
||||
* needed.
|
||||
*/
|
||||
protected abstract void doMarshall(T frame, ByteBuf out, ByteBufAllocator alloc)
|
||||
throws Http2Exception;
|
||||
}
|
@ -1,73 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import io.netty.channel.ChannelFutureListener;
|
||||
import io.netty.channel.ChannelHandlerAdapter;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.*;
|
||||
|
||||
/**
|
||||
* Sends the initial client preface, then removes itself from the pipeline.
|
||||
* Only the client pipeline should do this.
|
||||
*
|
||||
* https://tools.ietf.org/html/draft-ietf-httpbis-http2-10#section-3.5
|
||||
*/
|
||||
public class Http2ClientPrefaceWriter extends ChannelHandlerAdapter {
|
||||
|
||||
private boolean prefaceWritten;
|
||||
|
||||
public Http2ClientPrefaceWriter() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void channelActive(ChannelHandlerContext ctx) throws Exception {
|
||||
// The channel just became active - send the HTTP2 connection preface to the remote
|
||||
// endpoint.
|
||||
sendPreface(ctx);
|
||||
|
||||
super.channelActive(ctx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
|
||||
// This handler was just added to the context. In case it was handled after
|
||||
// the connection became active, send the HTTP2 connection preface now.
|
||||
sendPreface(ctx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends the HTTP2 connection preface to the remote endpoint, if not already sent.
|
||||
*/
|
||||
private void sendPreface(final ChannelHandlerContext ctx) {
|
||||
if (!prefaceWritten && ctx.channel().isActive()) {
|
||||
prefaceWritten = true;
|
||||
ctx.writeAndFlush(connectionPrefaceBuf()).addListener(new ChannelFutureListener() {
|
||||
@Override
|
||||
public void operationComplete(ChannelFuture future) throws Exception {
|
||||
if (!future.isSuccess() && ctx.channel().isOpen()) {
|
||||
// The write failed, close the connection.
|
||||
ctx.close();
|
||||
} else {
|
||||
ctx.pipeline().remove(Http2ClientPrefaceWriter.this);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
@ -1,68 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_END_STREAM;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_DATA;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.setPaddingFlags;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.writePaddingLength;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2DataFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Flags;
|
||||
|
||||
public class Http2DataFrameMarshaller extends AbstractHttp2FrameMarshaller<Http2DataFrame> {
|
||||
|
||||
public Http2DataFrameMarshaller() {
|
||||
super(Http2DataFrame.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doMarshall(Http2DataFrame frame, ByteBuf out, ByteBufAllocator alloc) {
|
||||
ByteBuf data = frame.content();
|
||||
|
||||
Http2Flags flags = getFlags(frame);
|
||||
|
||||
// Write the frame header.
|
||||
int payloadLength = data.readableBytes() + frame.getPaddingLength()
|
||||
+ (flags.isPadHighPresent() ? 1 : 0) + (flags.isPadLowPresent() ? 1 : 0);
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + payloadLength);
|
||||
out.writeShort(payloadLength);
|
||||
out.writeByte(FRAME_TYPE_DATA);
|
||||
out.writeByte(flags.getValue());
|
||||
out.writeInt(frame.getStreamId());
|
||||
|
||||
writePaddingLength(frame.getPaddingLength(), out);
|
||||
|
||||
// Write the data.
|
||||
out.writeBytes(data, data.readerIndex(), data.readableBytes());
|
||||
|
||||
// Write the required padding.
|
||||
out.writeZero(frame.getPaddingLength());
|
||||
}
|
||||
|
||||
private static Http2Flags getFlags(Http2DataFrame frame) {
|
||||
short flags = 0;
|
||||
if (frame.isEndOfStream()) {
|
||||
flags |= FLAG_END_STREAM;
|
||||
}
|
||||
|
||||
flags = setPaddingFlags(flags, frame.getPaddingLength());
|
||||
return new Http2Flags(flags);
|
||||
}
|
||||
}
|
@ -1,54 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.handler.codec.MessageToByteEncoder;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
|
||||
/**
|
||||
* Encodes {@link Http2Frame} objects and writes them to an output {@link ByteBuf}. The set of frame
|
||||
* types that is handled by this encoder is given by the {@link Http2FrameMarshaller}. By default,
|
||||
* the {@link Http2StandardFrameMarshaller} is used.
|
||||
*
|
||||
* @see Http2StandardFrameMarshaller
|
||||
*/
|
||||
public class Http2FrameEncoder extends MessageToByteEncoder<Http2Frame> {
|
||||
|
||||
private final Http2FrameMarshaller frameMarshaller;
|
||||
|
||||
public Http2FrameEncoder() {
|
||||
this(new Http2StandardFrameMarshaller());
|
||||
}
|
||||
|
||||
public Http2FrameEncoder(Http2FrameMarshaller frameMarshaller) {
|
||||
if (frameMarshaller == null) {
|
||||
throw new NullPointerException("frameMarshaller");
|
||||
}
|
||||
this.frameMarshaller = frameMarshaller;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void encode(ChannelHandlerContext ctx, Http2Frame frame, ByteBuf out)
|
||||
throws Exception {
|
||||
try {
|
||||
frameMarshaller.marshall(frame, out, ctx.alloc());
|
||||
} catch (Throwable t) {
|
||||
ctx.fireExceptionCaught(t);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,38 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
|
||||
/**
|
||||
* Marshalls {@link Http2Frame} objects to a {@link ByteBuf}.
|
||||
*/
|
||||
public interface Http2FrameMarshaller {
|
||||
|
||||
/**
|
||||
* Marshalls the given frame to the output buffer.
|
||||
*
|
||||
* @param frame the frame to be marshalled.
|
||||
* @param out the buffer to marshall the frame to.
|
||||
* @param alloc an allocator that this marshaller may use for creating intermediate buffers as
|
||||
* needed.
|
||||
* @throws Http2Exception thrown if the given fram is not supported by this marshaller.
|
||||
*/
|
||||
void marshall(Http2Frame frame, ByteBuf out, ByteBufAllocator alloc) throws Http2Exception;
|
||||
}
|
@ -1,50 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_GO_AWAY;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.writeUnsignedInt;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2GoAwayFrame;
|
||||
|
||||
public class Http2GoAwayFrameMarshaller extends AbstractHttp2FrameMarshaller<Http2GoAwayFrame> {
|
||||
|
||||
public Http2GoAwayFrameMarshaller() {
|
||||
super(Http2GoAwayFrame.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doMarshall(Http2GoAwayFrame frame, ByteBuf out, ByteBufAllocator alloc) {
|
||||
ByteBuf data = frame.content();
|
||||
|
||||
// Write the frame header.
|
||||
int payloadLength = data.readableBytes() + 8;
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + payloadLength);
|
||||
out.writeShort(payloadLength);
|
||||
out.writeByte(FRAME_TYPE_GO_AWAY);
|
||||
out.writeByte(0);
|
||||
out.writeInt(0);
|
||||
|
||||
out.writeInt(frame.getLastStreamId());
|
||||
writeUnsignedInt(frame.getErrorCode(), out);
|
||||
|
||||
// Write the debug data.
|
||||
out.writeBytes(data, data.readerIndex(), data.readableBytes());
|
||||
}
|
||||
}
|
@ -1,118 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.DEFAULT_STREAM_PRIORITY;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_END_HEADERS;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_END_STREAM;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_PRIORITY;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_CONTINUATION;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_HEADERS;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2HeadersFrame;
|
||||
|
||||
public class Http2HeadersFrameMarshaller extends AbstractHttp2FrameMarshaller<Http2HeadersFrame> {
|
||||
|
||||
private final Http2HeadersEncoder headersEncoder;
|
||||
|
||||
public Http2HeadersFrameMarshaller(Http2HeadersEncoder headersEncoder) {
|
||||
super(Http2HeadersFrame.class);
|
||||
if (headersEncoder == null) {
|
||||
throw new NullPointerException("headersEncoder");
|
||||
}
|
||||
this.headersEncoder = headersEncoder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doMarshall(Http2HeadersFrame frame, ByteBuf out, ByteBufAllocator alloc)
|
||||
throws Http2Exception {
|
||||
// TODO(nathanmittler): include padding?
|
||||
|
||||
int maxFragmentLength = MAX_FRAME_PAYLOAD_LENGTH;
|
||||
boolean hasPriority = frame.getPriority() != DEFAULT_STREAM_PRIORITY;
|
||||
if (hasPriority) {
|
||||
// The first frame will include the priority.
|
||||
maxFragmentLength -= 4;
|
||||
}
|
||||
|
||||
// Encode the entire header block into an intermediate buffer.
|
||||
ByteBuf headerBlock = alloc.buffer();
|
||||
headersEncoder.encodeHeaders(frame.getHeaders(), headerBlock);
|
||||
|
||||
ByteBuf fragment =
|
||||
headerBlock.readSlice(Math.min(headerBlock.readableBytes(), maxFragmentLength));
|
||||
int payloadLength = fragment.readableBytes() + (hasPriority ? 4 : 0);
|
||||
boolean endOfHeaders = headerBlock.readableBytes() == 0;
|
||||
|
||||
// Get the flags for the frame.
|
||||
short flags = 0;
|
||||
if (endOfHeaders) {
|
||||
flags |= FLAG_END_HEADERS;
|
||||
}
|
||||
if (frame.isEndOfStream()) {
|
||||
flags |= FLAG_END_STREAM;
|
||||
}
|
||||
if (hasPriority) {
|
||||
flags |= FLAG_PRIORITY;
|
||||
}
|
||||
|
||||
// Write the frame header.
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + payloadLength);
|
||||
out.writeShort(payloadLength);
|
||||
out.writeByte(FRAME_TYPE_HEADERS);
|
||||
out.writeByte(flags);
|
||||
out.writeInt(frame.getStreamId());
|
||||
|
||||
// Write out the priority if it's present.
|
||||
if (hasPriority) {
|
||||
out.writeInt(frame.getPriority());
|
||||
}
|
||||
|
||||
// Write the first fragment.
|
||||
out.writeBytes(fragment);
|
||||
|
||||
// Process any continuation frames there might be.
|
||||
while (headerBlock.readableBytes() > 0) {
|
||||
writeContinuationFrame(frame.getStreamId(), headerBlock, out);
|
||||
}
|
||||
|
||||
// Release the intermediate buffer.
|
||||
headerBlock.release();
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a single continuation frame with a fragment of the header block to the output buffer.
|
||||
*/
|
||||
private static void writeContinuationFrame(int streamId, ByteBuf headerBlock, ByteBuf out) {
|
||||
ByteBuf fragment =
|
||||
headerBlock.readSlice(Math.min(headerBlock.readableBytes(), MAX_FRAME_PAYLOAD_LENGTH));
|
||||
|
||||
// Write the frame header.
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + fragment.readableBytes());
|
||||
out.writeShort(fragment.readableBytes());
|
||||
out.writeByte(FRAME_TYPE_CONTINUATION);
|
||||
out.writeByte(headerBlock.readableBytes() == 0 ? FLAG_END_HEADERS : 0);
|
||||
out.writeInt(streamId);
|
||||
|
||||
// Write the headers block.
|
||||
out.writeBytes(fragment);
|
||||
}
|
||||
}
|
@ -1,47 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_ACK;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_PING;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PingFrame;
|
||||
|
||||
public class Http2PingFrameMarshaller extends AbstractHttp2FrameMarshaller<Http2PingFrame> {
|
||||
|
||||
public Http2PingFrameMarshaller() {
|
||||
super(Http2PingFrame.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doMarshall(Http2PingFrame frame, ByteBuf out, ByteBufAllocator alloc) {
|
||||
ByteBuf data = frame.content();
|
||||
|
||||
// Write the frame header.
|
||||
int payloadLength = data.readableBytes();
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + payloadLength);
|
||||
out.writeShort(payloadLength);
|
||||
out.writeByte(FRAME_TYPE_PING);
|
||||
out.writeByte(frame.isAck() ? FLAG_ACK : 0);
|
||||
out.writeInt(0);
|
||||
|
||||
// Write the debug data.
|
||||
out.writeBytes(data, data.readerIndex(), data.readableBytes());
|
||||
}
|
||||
}
|
@ -1,45 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_PRIORITY;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PriorityFrame;
|
||||
|
||||
public class Http2PriorityFrameMarshaller extends AbstractHttp2FrameMarshaller<Http2PriorityFrame> {
|
||||
|
||||
public Http2PriorityFrameMarshaller() {
|
||||
super(Http2PriorityFrame.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doMarshall(Http2PriorityFrame frame, ByteBuf out, ByteBufAllocator alloc) {
|
||||
|
||||
// Write the frame header.
|
||||
int payloadLength = 4;
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + payloadLength);
|
||||
out.writeShort(payloadLength);
|
||||
out.writeByte(FRAME_TYPE_PRIORITY);
|
||||
out.writeByte(0);
|
||||
out.writeInt(frame.getStreamId());
|
||||
|
||||
// Write out the priority if it's present.
|
||||
out.writeInt(frame.getPriority());
|
||||
}
|
||||
}
|
@ -1,95 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FLAG_END_HEADERS;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_CONTINUATION;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_PUSH_PROMISE;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_FRAME_PAYLOAD_LENGTH;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PushPromiseFrame;
|
||||
|
||||
public class Http2PushPromiseFrameMarshaller extends
|
||||
AbstractHttp2FrameMarshaller<Http2PushPromiseFrame> {
|
||||
|
||||
private final Http2HeadersEncoder headersEncoder;
|
||||
|
||||
public Http2PushPromiseFrameMarshaller(Http2HeadersEncoder headersEncoder) {
|
||||
super(Http2PushPromiseFrame.class);
|
||||
if (headersEncoder == null) {
|
||||
throw new NullPointerException("headersEncoder");
|
||||
}
|
||||
this.headersEncoder = headersEncoder;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doMarshall(Http2PushPromiseFrame frame, ByteBuf out, ByteBufAllocator alloc)
|
||||
throws Http2Exception {
|
||||
|
||||
// Max size minus the promised stream ID.
|
||||
int maxFragmentLength = MAX_FRAME_PAYLOAD_LENGTH - 4;
|
||||
|
||||
// Encode the entire header block into an intermediate buffer.
|
||||
ByteBuf headerBlock = alloc.buffer();
|
||||
headersEncoder.encodeHeaders(frame.getHeaders(), headerBlock);
|
||||
|
||||
ByteBuf fragment =
|
||||
headerBlock.readSlice(Math.min(headerBlock.readableBytes(), maxFragmentLength));
|
||||
|
||||
// Write the frame header.
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + fragment.readableBytes());
|
||||
out.writeShort(fragment.readableBytes() + 4);
|
||||
out.writeByte(FRAME_TYPE_PUSH_PROMISE);
|
||||
out.writeByte(headerBlock.readableBytes() == 0 ? FLAG_END_HEADERS : 0);
|
||||
out.writeInt(frame.getStreamId());
|
||||
|
||||
// Write out the promised stream ID.
|
||||
out.writeInt(frame.getPromisedStreamId());
|
||||
|
||||
// Write the first fragment.
|
||||
out.writeBytes(fragment);
|
||||
|
||||
// Process any continuation frames there might be.
|
||||
while (headerBlock.readableBytes() > 0) {
|
||||
writeContinuationFrame(frame.getStreamId(), headerBlock, out);
|
||||
}
|
||||
|
||||
// Release the intermediate buffer.
|
||||
headerBlock.release();
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes a single continuation frame with a fragment of the header block to the output buffer.
|
||||
*/
|
||||
private static void writeContinuationFrame(int streamId, ByteBuf headerBlock, ByteBuf out) {
|
||||
ByteBuf fragment =
|
||||
headerBlock.readSlice(Math.min(headerBlock.readableBytes(), MAX_FRAME_PAYLOAD_LENGTH));
|
||||
|
||||
// Write the frame header.
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + fragment.readableBytes());
|
||||
out.writeShort(fragment.readableBytes());
|
||||
out.writeByte(FRAME_TYPE_CONTINUATION);
|
||||
out.writeByte(headerBlock.readableBytes() == 0 ? FLAG_END_HEADERS : 0);
|
||||
out.writeInt(streamId);
|
||||
|
||||
// Write the headers block.
|
||||
out.writeBytes(fragment);
|
||||
}
|
||||
}
|
@ -1,46 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_RST_STREAM;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.writeUnsignedInt;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2RstStreamFrame;
|
||||
|
||||
public class Http2RstStreamFrameMarshaller extends
|
||||
AbstractHttp2FrameMarshaller<Http2RstStreamFrame> {
|
||||
|
||||
public Http2RstStreamFrameMarshaller() {
|
||||
super(Http2RstStreamFrame.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doMarshall(Http2RstStreamFrame frame, ByteBuf out, ByteBufAllocator alloc) {
|
||||
|
||||
// Write the frame header.
|
||||
int payloadLength = 4;
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + payloadLength);
|
||||
out.writeShort(payloadLength);
|
||||
out.writeByte(FRAME_TYPE_RST_STREAM);
|
||||
out.writeByte(0);
|
||||
out.writeInt(frame.getStreamId());
|
||||
|
||||
writeUnsignedInt(frame.getErrorCode(), out);
|
||||
}
|
||||
}
|
@ -1,70 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_SETTINGS;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.SETTINGS_ENABLE_PUSH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.SETTINGS_HEADER_TABLE_SIZE;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.SETTINGS_INITIAL_WINDOW_SIZE;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.SETTINGS_MAX_CONCURRENT_STREAMS;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.writeUnsignedInt;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2SettingsFrame;
|
||||
|
||||
public class Http2SettingsFrameMarshaller extends AbstractHttp2FrameMarshaller<Http2SettingsFrame> {
|
||||
|
||||
public Http2SettingsFrameMarshaller() {
|
||||
super(Http2SettingsFrame.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doMarshall(Http2SettingsFrame frame, ByteBuf out, ByteBufAllocator alloc) {
|
||||
int numSettings = 0;
|
||||
numSettings += frame.getPushEnabled() != null ? 1 : 0;
|
||||
numSettings += frame.getHeaderTableSize() != null ? 1 : 0;
|
||||
numSettings += frame.getInitialWindowSize() != null ? 1 : 0;
|
||||
numSettings += frame.getMaxConcurrentStreams() != null ? 1 : 0;
|
||||
|
||||
// Write the frame header.
|
||||
int payloadLength = 5 * numSettings;
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + payloadLength);
|
||||
out.writeShort(payloadLength);
|
||||
out.writeByte(FRAME_TYPE_SETTINGS);
|
||||
out.writeByte(frame.isAck() ? Http2FrameCodecUtil.FLAG_ACK : 0);
|
||||
out.writeInt(0);
|
||||
|
||||
if (frame.getPushEnabled() != null) {
|
||||
out.writeByte(SETTINGS_ENABLE_PUSH);
|
||||
writeUnsignedInt(frame.getPushEnabled() ? 1L : 0L, out);
|
||||
}
|
||||
if (frame.getHeaderTableSize() != null) {
|
||||
out.writeByte(SETTINGS_HEADER_TABLE_SIZE);
|
||||
writeUnsignedInt(frame.getHeaderTableSize(), out);
|
||||
}
|
||||
if (frame.getInitialWindowSize() != null) {
|
||||
out.writeByte(SETTINGS_INITIAL_WINDOW_SIZE);
|
||||
writeUnsignedInt(frame.getInitialWindowSize(), out);
|
||||
}
|
||||
if (frame.getMaxConcurrentStreams() != null) {
|
||||
out.writeByte(SETTINGS_MAX_CONCURRENT_STREAMS);
|
||||
writeUnsignedInt(frame.getMaxConcurrentStreams(), out);
|
||||
}
|
||||
}
|
||||
}
|
@ -1,111 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.Http2Exception.protocolError;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.Http2Exception;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2DataFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2HeadersFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PushPromiseFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2GoAwayFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PingFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2PriorityFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2RstStreamFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2SettingsFrame;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2WindowUpdateFrame;
|
||||
|
||||
/**
|
||||
* A composite {@link Http2FrameMarshaller} that supports all frames identified by the HTTP2 spec.
|
||||
* This handles marshalling for the following frame types:
|
||||
* <p/>
|
||||
* {@link Http2DataFrame} <br>
|
||||
* {@link Http2GoAwayFrame} <br>
|
||||
* {@link Http2HeadersFrame} <br>
|
||||
* {@link Http2PingFrame} <br>
|
||||
* {@link Http2PriorityFrame} <br>
|
||||
* {@link Http2PushPromiseFrame} <br>
|
||||
* {@link Http2RstStreamFrame} <br>
|
||||
* {@link Http2SettingsFrame} <br>
|
||||
* {@link Http2WindowUpdateFrame} <br>
|
||||
*/
|
||||
public class Http2StandardFrameMarshaller implements Http2FrameMarshaller {
|
||||
|
||||
private final Http2FrameMarshaller dataMarshaller;
|
||||
private final Http2FrameMarshaller headersMarshaller;
|
||||
private final Http2FrameMarshaller goAwayMarshaller;
|
||||
private final Http2FrameMarshaller pingMarshaller;
|
||||
private final Http2FrameMarshaller priorityMarshaller;
|
||||
private final Http2FrameMarshaller pushPromiseMarshaller;
|
||||
private final Http2FrameMarshaller rstStreamMarshaller;
|
||||
private final Http2FrameMarshaller settingsMarshaller;
|
||||
private final Http2FrameMarshaller windowUpdateMarshaller;
|
||||
|
||||
public Http2StandardFrameMarshaller() {
|
||||
this(new DefaultHttp2HeadersEncoder());
|
||||
}
|
||||
|
||||
public Http2StandardFrameMarshaller(Http2HeadersEncoder headersEncoder) {
|
||||
dataMarshaller = new Http2DataFrameMarshaller();
|
||||
headersMarshaller = new Http2HeadersFrameMarshaller(headersEncoder);
|
||||
goAwayMarshaller = new Http2GoAwayFrameMarshaller();
|
||||
pingMarshaller = new Http2PingFrameMarshaller();
|
||||
priorityMarshaller = new Http2PriorityFrameMarshaller();
|
||||
pushPromiseMarshaller = new Http2PushPromiseFrameMarshaller(headersEncoder);
|
||||
rstStreamMarshaller = new Http2RstStreamFrameMarshaller();
|
||||
settingsMarshaller = new Http2SettingsFrameMarshaller();
|
||||
windowUpdateMarshaller = new Http2WindowUpdateFrameMarshaller();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void marshall(Http2Frame frame, ByteBuf out, ByteBufAllocator alloc)
|
||||
throws Http2Exception {
|
||||
Http2FrameMarshaller marshaller = null;
|
||||
|
||||
if (frame == null) {
|
||||
throw new NullPointerException("frame");
|
||||
}
|
||||
|
||||
if (frame instanceof Http2DataFrame) {
|
||||
marshaller = dataMarshaller;
|
||||
} else if (frame instanceof Http2HeadersFrame) {
|
||||
marshaller = headersMarshaller;
|
||||
} else if (frame instanceof Http2GoAwayFrame) {
|
||||
marshaller = goAwayMarshaller;
|
||||
} else if (frame instanceof Http2PingFrame) {
|
||||
marshaller = pingMarshaller;
|
||||
} else if (frame instanceof Http2PriorityFrame) {
|
||||
marshaller = priorityMarshaller;
|
||||
} else if (frame instanceof Http2PushPromiseFrame) {
|
||||
marshaller = pushPromiseMarshaller;
|
||||
} else if (frame instanceof Http2RstStreamFrame) {
|
||||
marshaller = rstStreamMarshaller;
|
||||
} else if (frame instanceof Http2SettingsFrame) {
|
||||
marshaller = settingsMarshaller;
|
||||
} else if (frame instanceof Http2WindowUpdateFrame) {
|
||||
marshaller = windowUpdateMarshaller;
|
||||
}
|
||||
|
||||
if (marshaller == null) {
|
||||
throw protocolError("Unsupported frame type: %s", frame.getClass().getName());
|
||||
}
|
||||
|
||||
marshaller.marshall(frame, out, alloc);
|
||||
}
|
||||
}
|
@ -1,45 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_HEADER_LENGTH;
|
||||
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.FRAME_TYPE_WINDOW_UPDATE;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.handler.codec.http2.draft10.frame.Http2WindowUpdateFrame;
|
||||
|
||||
public class Http2WindowUpdateFrameMarshaller extends
|
||||
AbstractHttp2FrameMarshaller<Http2WindowUpdateFrame> {
|
||||
|
||||
public Http2WindowUpdateFrameMarshaller() {
|
||||
super(Http2WindowUpdateFrame.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doMarshall(Http2WindowUpdateFrame frame, ByteBuf out, ByteBufAllocator alloc) {
|
||||
|
||||
// Write the frame header.
|
||||
int payloadLength = 4;
|
||||
out.ensureWritable(FRAME_HEADER_LENGTH + payloadLength);
|
||||
out.writeShort(payloadLength);
|
||||
out.writeByte(FRAME_TYPE_WINDOW_UPDATE);
|
||||
out.writeByte(0);
|
||||
out.writeInt(frame.getStreamId());
|
||||
|
||||
out.writeInt(frame.getWindowSizeIncrement());
|
||||
}
|
||||
}
|
@ -1,20 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Encoder, decoder and their related message types for HTTP2 frames.
|
||||
*/
|
||||
package io.netty.handler.codec.http2.draft10.frame;
|
||||
|
@ -1,20 +0,0 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Encoder, decoder and their related message types for HTTP2.
|
||||
*/
|
||||
package io.netty.handler.codec.http2.draft10;
|
||||
|
@ -14,7 +14,7 @@
|
||||
*/
|
||||
|
||||
/**
|
||||
* Encoder and related classes for HTTP2.
|
||||
* Handlers for sending and receiving HTTP/2 frames.
|
||||
*/
|
||||
package io.netty.handler.codec.http2.draft10.frame.encoder;
|
||||
package io.netty.handler.codec.http2;
|
||||
|
@ -0,0 +1,182 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import io.netty.handler.codec.http2.Http2Stream.State;
|
||||
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.mockito.MockitoAnnotations;
|
||||
|
||||
/**
|
||||
* Tests for {@link DefaultHttp2Connection}.
|
||||
*/
|
||||
public class DefaultHttp2ConnectionTest {
|
||||
|
||||
private DefaultHttp2Connection server;
|
||||
private DefaultHttp2Connection client;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
MockitoAnnotations.initMocks(this);
|
||||
|
||||
server = new DefaultHttp2Connection(true, false);
|
||||
client = new DefaultHttp2Connection(false, false);
|
||||
}
|
||||
|
||||
@Test(expected = Http2Exception.class)
|
||||
public void getStreamOrFailWithoutStreamShouldFail() throws Http2Exception {
|
||||
server.requireStream(100);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void getStreamWithoutStreamShouldReturnNull() {
|
||||
assertNull(server.stream(100));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void serverCreateStreamShouldSucceed() throws Http2Exception {
|
||||
Http2Stream stream = server.local().createStream(2, false);
|
||||
assertEquals(2, stream.id());
|
||||
assertEquals(State.OPEN, stream.state());
|
||||
assertEquals(1, server.activeStreams().size());
|
||||
assertEquals(2, server.local().lastStreamCreated());
|
||||
|
||||
stream = server.local().createStream(4, true);
|
||||
assertEquals(4, stream.id());
|
||||
assertEquals(State.HALF_CLOSED_LOCAL, stream.state());
|
||||
assertEquals(2, server.activeStreams().size());
|
||||
assertEquals(4, server.local().lastStreamCreated());
|
||||
|
||||
stream = server.remote().createStream(3, true);
|
||||
assertEquals(3, stream.id());
|
||||
assertEquals(State.HALF_CLOSED_REMOTE, stream.state());
|
||||
assertEquals(3, server.activeStreams().size());
|
||||
assertEquals(3, server.remote().lastStreamCreated());
|
||||
|
||||
stream = server.remote().createStream(5, false);
|
||||
assertEquals(5, stream.id());
|
||||
assertEquals(State.OPEN, stream.state());
|
||||
assertEquals(4, server.activeStreams().size());
|
||||
assertEquals(5, server.remote().lastStreamCreated());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void clientCreateStreamShouldSucceed() throws Http2Exception {
|
||||
Http2Stream stream = client.remote().createStream(2, false);
|
||||
assertEquals(2, stream.id());
|
||||
assertEquals(State.OPEN, stream.state());
|
||||
assertEquals(1, client.activeStreams().size());
|
||||
assertEquals(2, client.remote().lastStreamCreated());
|
||||
|
||||
stream = client.remote().createStream(4, true);
|
||||
assertEquals(4, stream.id());
|
||||
assertEquals(State.HALF_CLOSED_REMOTE, stream.state());
|
||||
assertEquals(2, client.activeStreams().size());
|
||||
assertEquals(4, client.remote().lastStreamCreated());
|
||||
|
||||
stream = client.local().createStream(3, true);
|
||||
assertEquals(3, stream.id());
|
||||
assertEquals(State.HALF_CLOSED_LOCAL, stream.state());
|
||||
assertEquals(3, client.activeStreams().size());
|
||||
assertEquals(3, client.local().lastStreamCreated());
|
||||
|
||||
stream = client.local().createStream(5, false);
|
||||
assertEquals(5, stream.id());
|
||||
assertEquals(State.OPEN, stream.state());
|
||||
assertEquals(4, client.activeStreams().size());
|
||||
assertEquals(5, client.local().lastStreamCreated());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void serverReservePushStreamShouldSucceed() throws Http2Exception {
|
||||
Http2Stream stream = server.remote().createStream(3, true);
|
||||
Http2Stream pushStream = server.local().reservePushStream(2, stream);
|
||||
assertEquals(2, pushStream.id());
|
||||
assertEquals(State.RESERVED_LOCAL, pushStream.state());
|
||||
assertEquals(1, server.activeStreams().size());
|
||||
assertEquals(2, server.local().lastStreamCreated());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void clientReservePushStreamShouldSucceed() throws Http2Exception {
|
||||
Http2Stream stream = client.remote().createStream(2, true);
|
||||
Http2Stream pushStream = client.local().reservePushStream(3, stream);
|
||||
assertEquals(3, pushStream.id());
|
||||
assertEquals(State.RESERVED_LOCAL, pushStream.state());
|
||||
assertEquals(1, client.activeStreams().size());
|
||||
assertEquals(3, client.local().lastStreamCreated());
|
||||
}
|
||||
|
||||
@Test(expected = Http2Exception.class)
|
||||
public void createStreamWithInvalidIdShouldThrow() throws Http2Exception {
|
||||
server.remote().createStream(1, true);
|
||||
}
|
||||
|
||||
@Test(expected = Http2Exception.class)
|
||||
public void maxAllowedStreamsExceededShouldThrow() throws Http2Exception {
|
||||
server.local().maxStreams(0);
|
||||
server.local().createStream(2, true);
|
||||
}
|
||||
|
||||
@Test(expected = Http2Exception.class)
|
||||
public void reserveWithPushDisallowedShouldThrow() throws Http2Exception {
|
||||
Http2Stream stream = server.remote().createStream(3, true);
|
||||
server.remote().allowPushTo(false);
|
||||
server.local().reservePushStream(2, stream);
|
||||
}
|
||||
|
||||
@Test(expected = Http2Exception.class)
|
||||
public void goAwayReceivedShouldDisallowCreation() throws Http2Exception {
|
||||
server.goAwayReceived();
|
||||
server.remote().createStream(3, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void closeShouldSucceed() throws Http2Exception {
|
||||
Http2Stream stream = server.remote().createStream(3, true);
|
||||
stream.close();
|
||||
assertEquals(State.CLOSED, stream.state());
|
||||
assertTrue(server.activeStreams().isEmpty());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void closeLocalWhenOpenShouldSucceed() throws Http2Exception {
|
||||
Http2Stream stream = server.remote().createStream(3, false);
|
||||
stream.closeLocalSide();
|
||||
assertEquals(State.HALF_CLOSED_LOCAL, stream.state());
|
||||
assertEquals(1, server.activeStreams().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void closeRemoteWhenOpenShouldSucceed() throws Http2Exception {
|
||||
Http2Stream stream = server.remote().createStream(3, false);
|
||||
stream.closeRemoteSide();
|
||||
assertEquals(State.HALF_CLOSED_REMOTE, stream.state());
|
||||
assertEquals(1, server.activeStreams().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void closeOnlyOpenSideShouldClose() throws Http2Exception {
|
||||
Http2Stream stream = server.remote().createStream(3, true);
|
||||
stream.closeLocalSide();
|
||||
assertEquals(State.CLOSED, stream.state());
|
||||
assertTrue(server.activeStreams().isEmpty());
|
||||
}
|
||||
}
|
@ -0,0 +1,349 @@
|
||||
/*
|
||||
* Copyright 2014 The Netty Project
|
||||
*
|
||||
* The Netty Project licenses this file to you under the Apache License, version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
* or implied. See the License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
|
||||
package io.netty.handler.codec.http2;
|
||||
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_UNSIGNED_INT;
|
||||
import static io.netty.handler.codec.http2.Http2CodecUtil.MAX_UNSIGNED_SHORT;
|
||||
import static org.mockito.Matchers.eq;
|
||||
import static org.mockito.Matchers.isNull;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.ByteBufAllocator;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.buffer.UnpooledByteBufAllocator;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
||||
import io.netty.channel.ChannelPromise;
|
||||
import io.netty.util.CharsetUtil;
|
||||
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.mockito.ArgumentCaptor;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.MockitoAnnotations;
|
||||
|
||||
/**
|
||||
* Integration tests for {@link DefaultHttp2FrameReader} and {@link DefaultHttp2FrameWriter}.
|
||||
*/
|
||||
public class DefaultHttp2FrameIOTest {
|
||||
|
||||
private DefaultHttp2FrameReader reader;
|
||||
private DefaultHttp2FrameWriter writer;
|
||||
private ByteBufAllocator alloc;
|
||||
|
||||
@Mock
|
||||
private ChannelHandlerContext ctx;
|
||||
|
||||
@Mock
|
||||
private Http2FrameObserver observer;
|
||||
|
||||
@Mock
|
||||
private ChannelPromise promise;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
MockitoAnnotations.initMocks(this);
|
||||
|
||||
alloc = UnpooledByteBufAllocator.DEFAULT;
|
||||
|
||||
when(ctx.alloc()).thenReturn(alloc);
|
||||
|
||||
reader = new DefaultHttp2FrameReader(false);
|
||||
writer = new DefaultHttp2FrameWriter(true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void emptyDataShouldRoundtrip() throws Exception {
|
||||
ByteBuf data = Unpooled.EMPTY_BUFFER;
|
||||
writer.writeData(ctx, promise, 1000, data, 0, false, false, false);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onDataRead(eq(1000), eq(data), eq(0), eq(false), eq(false), eq(false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void dataShouldRoundtrip() throws Exception {
|
||||
ByteBuf data = dummyData();
|
||||
writer.writeData(ctx, promise, 1000, data, 0, false, false, false);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onDataRead(eq(1000), eq(data), eq(0), eq(false), eq(false), eq(false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void dataWithPaddingShouldRoundtrip() throws Exception {
|
||||
ByteBuf data = dummyData();
|
||||
writer.writeData(ctx, promise, 1, data, 256, true, true, true);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onDataRead(eq(1), eq(data), eq(256), eq(true), eq(true), eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void priorityShouldRoundtrip() throws Exception {
|
||||
writer.writePriority(ctx, promise, 1, 2, (short) 255, true);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onPriorityRead(eq(1), eq(2), eq((short) 255), eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void rstStreamShouldRoundtrip() throws Exception {
|
||||
writer.writeRstStream(ctx, promise, 1, MAX_UNSIGNED_INT);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onRstStreamRead(eq(1), eq(MAX_UNSIGNED_INT));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void emptySettingsShouldRoundtrip() throws Exception {
|
||||
writer.writeSettings(ctx, promise, new Http2Settings());
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onSettingsRead(eq(new Http2Settings()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void settingsShouldRoundtrip() throws Exception {
|
||||
Http2Settings settings = new Http2Settings();
|
||||
settings.pushEnabled(true);
|
||||
settings.maxHeaderTableSize(4096);
|
||||
settings.initialWindowSize(123);
|
||||
settings.maxConcurrentStreams(456);
|
||||
settings.allowCompressedData(false);
|
||||
|
||||
writer.writeSettings(ctx, promise, settings);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onSettingsRead(eq(settings));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void settingsAckShouldRoundtrip() throws Exception {
|
||||
writer.writeSettingsAck(ctx, promise);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onSettingsAckRead();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void pingShouldRoundtrip() throws Exception {
|
||||
ByteBuf data = dummyData();
|
||||
writer.writePing(ctx, promise, false, data);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onPingRead(eq(data));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void pingAckShouldRoundtrip() throws Exception {
|
||||
ByteBuf data = dummyData();
|
||||
writer.writePing(ctx, promise, true, data);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onPingAckRead(eq(data));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void goAwayShouldRoundtrip() throws Exception {
|
||||
ByteBuf data = dummyData();
|
||||
writer.writeGoAway(ctx, promise, 1, MAX_UNSIGNED_INT, data);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onGoAwayRead(eq(1), eq(MAX_UNSIGNED_INT), eq(data));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void windowUpdateShouldRoundtrip() throws Exception {
|
||||
writer.writeWindowUpdate(ctx, promise, 1, Integer.MAX_VALUE);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onWindowUpdateRead(eq(1), eq(Integer.MAX_VALUE));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void altSvcShouldRoundtrip() throws Exception {
|
||||
writer.writeAltSvc(ctx, promise, 1, MAX_UNSIGNED_INT, MAX_UNSIGNED_SHORT, dummyData(), "host",
|
||||
"origin");
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onAltSvcRead(eq(1), eq(MAX_UNSIGNED_INT), eq(MAX_UNSIGNED_SHORT),
|
||||
eq(dummyData()), eq("host"), eq("origin"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void altSvcWithoutOriginShouldRoundtrip() throws Exception {
|
||||
writer.writeAltSvc(ctx, promise, 1, MAX_UNSIGNED_INT, MAX_UNSIGNED_SHORT, dummyData(), "host", null);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onAltSvcRead(eq(1), eq(MAX_UNSIGNED_INT), eq(MAX_UNSIGNED_SHORT),
|
||||
eq(dummyData()), eq("host"), isNull(String.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void blockedShouldRoundtrip() throws Exception {
|
||||
writer.writeBlocked(ctx, promise, 1);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onBlockedRead(eq(1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void emptyHeadersShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = DefaultHttp2Headers.EMPTY_HEADERS;
|
||||
writer.writeHeaders(ctx, promise, 1, headers, 0, true, true);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onHeadersRead(eq(1), eq(headers), eq(0), eq(true), eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void emptyHeadersWithPaddingShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = DefaultHttp2Headers.EMPTY_HEADERS;
|
||||
writer.writeHeaders(ctx, promise, 1, headers, 256, true, true);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onHeadersRead(eq(1), eq(headers), eq(256), eq(true), eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void headersWithoutPriorityShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = dummyHeaders();
|
||||
writer.writeHeaders(ctx, promise, 1, headers, 0, true, true);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onHeadersRead(eq(1), eq(headers), eq(0), eq(true), eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void headersWithPaddingWithoutPriorityShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = dummyHeaders();
|
||||
writer.writeHeaders(ctx, promise, 1, headers, 256, true, true);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onHeadersRead(eq(1), eq(headers), eq(256), eq(true), eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void headersWithPriorityShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = dummyHeaders();
|
||||
writer.writeHeaders(ctx, promise, 1, headers, 2, (short) 3, true, 0, true, true);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onHeadersRead(eq(1), eq(headers), eq(2), eq((short) 3), eq(true), eq(0),
|
||||
eq(true), eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void headersWithPaddingWithPriorityShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = dummyHeaders();
|
||||
writer.writeHeaders(ctx, promise, 1, headers, 2, (short) 3, true, 256, true, true);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onHeadersRead(eq(1), eq(headers), eq(2), eq((short) 3), eq(true), eq(256),
|
||||
eq(true), eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void continuedHeadersShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = largeHeaders();
|
||||
writer.writeHeaders(ctx, promise, 1, headers, 2, (short) 3, true, 0, true, true);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onHeadersRead(eq(1), eq(headers), eq(2), eq((short) 3), eq(true), eq(0),
|
||||
eq(true), eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void continuedHeadersWithPaddingShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = largeHeaders();
|
||||
writer.writeHeaders(ctx, promise, 1, headers, 2, (short) 3, true, 256, true, true);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onHeadersRead(eq(1), eq(headers), eq(2), eq((short) 3), eq(true), eq(256),
|
||||
eq(true), eq(true));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void emptypushPromiseShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = Http2Headers.EMPTY_HEADERS;
|
||||
writer.writePushPromise(ctx, promise, 1, 2, headers, 0);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onPushPromiseRead(eq(1), eq(2), eq(headers), eq(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void pushPromiseShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = dummyHeaders();
|
||||
writer.writePushPromise(ctx, promise, 1, 2, headers, 0);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onPushPromiseRead(eq(1), eq(2), eq(headers), eq(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void pushPromiseWithPaddingShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = dummyHeaders();
|
||||
writer.writePushPromise(ctx, promise, 1, 2, headers, 256);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onPushPromiseRead(eq(1), eq(2), eq(headers), eq(256));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void continuedPushPromiseShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = largeHeaders();
|
||||
writer.writePushPromise(ctx, promise, 1, 2, headers, 0);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onPushPromiseRead(eq(1), eq(2), eq(headers), eq(0));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void continuedPushPromiseWithPaddingShouldRoundtrip() throws Exception {
|
||||
Http2Headers headers = largeHeaders();
|
||||
writer.writePushPromise(ctx, promise, 1, 2, headers, 256);
|
||||
ByteBuf frame = captureWrite();
|
||||
reader.readFrame(alloc, frame, observer);
|
||||
verify(observer).onPushPromiseRead(eq(1), eq(2), eq(headers), eq(256));
|
||||
}
|
||||
|
||||
private ByteBuf captureWrite() {
|
||||
ArgumentCaptor<ByteBuf> captor = ArgumentCaptor.forClass(ByteBuf.class);
|
||||
verify(ctx).writeAndFlush(captor.capture(), eq(promise));
|
||||
return captor.getValue();
|
||||
}
|
||||
|
||||
private ByteBuf dummyData() {
|
||||
return alloc.buffer().writeBytes("abcdefgh".getBytes(CharsetUtil.UTF_8));
|
||||
}
|
||||
|
||||
private Http2Headers dummyHeaders() {
|
||||
return DefaultHttp2Headers.newBuilder().method("GET").scheme("https")
|
||||
.authority("example.org").path("/some/path").add("accept", "*/*").build();
|
||||
}
|
||||
|
||||
private Http2Headers largeHeaders() {
|
||||
DefaultHttp2Headers.Builder builder = DefaultHttp2Headers.newBuilder();
|
||||
for (int i = 0; i < 100; ++i) {
|
||||
String key = "this-is-a-test-header-key-" + i;
|
||||
String value = "this-is-a-test-header-value-" + i;
|
||||
builder.add(key, value);
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user