Not throw a Http2Exception when a ChannelFuture is returned

Motivation:
At the moment we have some methods that return a ChannelFuture but still throw a Http2Exception too. This is confusing in terms of semantic. A method which returns a ChannelFuture should not throw an Http2Exception but just fail the ChannelFuture.

Modifications:
* Make sure we fail the returned ChannelFuture in cases of Http2Exception and remove the throws Http2Exception from the method signature.
* Also some cleanup

Result:
Make the API usage more clear.
This commit is contained in:
Norman Maurer 2014-05-11 13:33:14 +02:00
parent 1f68479e3c
commit 9429251aef
15 changed files with 164 additions and 191 deletions

View File

@ -17,7 +17,6 @@ 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.connectionPrefaceBuf;
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;
@ -189,7 +188,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
protected ChannelFuture writeData(final ChannelHandlerContext ctx,
final ChannelPromise promise, int streamId, final ByteBuf data, int padding,
boolean endStream, boolean endSegment, boolean compressed) throws Http2Exception {
boolean endStream, boolean endSegment, boolean compressed) {
try {
if (connection.isGoAway()) {
throw protocolError("Sending data after connection going away.");
@ -208,21 +207,19 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
return promise;
} catch (Http2Exception e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
protected ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment)
throws Http2Exception {
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment) {
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 {
boolean exclusive, int padding, boolean endStream, boolean endSegment) {
try {
if (connection.isGoAway()) {
throw protocolError("Sending headers after connection going away.");
@ -268,13 +265,12 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
return frameWriter.writeHeaders(ctx, promise, streamId, headers, streamDependency,
weight, exclusive, padding, endStream, endSegment);
} catch (Http2Exception e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
protected ChannelFuture writePriority(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, int streamDependency, short weight, boolean exclusive)
throws Http2Exception {
int streamId, int streamDependency, short weight, boolean exclusive) {
try {
if (connection.isGoAway()) {
throw protocolError("Sending priority after connection going away.");
@ -286,7 +282,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
return frameWriter.writePriority(ctx, promise, streamId, streamDependency, weight,
exclusive);
} catch (Http2Exception e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
@ -305,7 +301,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
}
protected ChannelFuture writeSettings(ChannelHandlerContext ctx, ChannelPromise promise,
Http2Settings settings) throws Http2Exception {
Http2Settings settings) {
outstandingLocalSettingsQueue.add(settings);
try {
if (connection.isGoAway()) {
@ -320,12 +316,12 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
return frameWriter.writeSettings(ctx, promise, settings);
} catch (Http2Exception e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
protected ChannelFuture writePing(ChannelHandlerContext ctx, ChannelPromise promise,
ByteBuf data) throws Http2Exception {
ByteBuf data) {
try {
if (connection.isGoAway()) {
throw protocolError("Sending ping after connection going away.");
@ -334,13 +330,12 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
// Just pass the frame through.
return frameWriter.writePing(ctx, promise, false, data);
} catch (Http2Exception e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
protected ChannelFuture writePushPromise(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, int promisedStreamId, Http2Headers headers, int padding)
throws Http2Exception {
int streamId, int promisedStreamId, Http2Headers headers, int padding) {
try {
if (connection.isGoAway()) {
throw protocolError("Sending push promise after connection going away.");
@ -354,15 +349,14 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
return frameWriter.writePushPromise(ctx, promise, streamId, promisedStreamId, headers,
padding);
} catch (Http2Exception e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
protected ChannelFuture writeAltSvc(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, long maxAge, int port, ByteBuf protocolId, String host, String origin)
throws Http2Exception {
int streamId, long maxAge, int port, ByteBuf protocolId, String host, String origin) {
if (!connection.isServer()) {
throw protocolError("Client sending ALT_SVC frame");
return promise.setFailure(protocolError("Client sending ALT_SVC frame"));
}
return frameWriter.writeAltSvc(ctx, promise, streamId, maxAge, port, protocolId, host,
origin);
@ -392,7 +386,7 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
* preface strings can only be received by servers, returns true immediately for client
* endpoints.
*/
private boolean readClientPrefaceString(ChannelHandlerContext ctx, ByteBuf in) throws Http2Exception {
private boolean readClientPrefaceString(ChannelHandlerContext ctx, ByteBuf in) {
if (clientPrefaceString == null) {
return true;
}
@ -918,59 +912,55 @@ public abstract class AbstractHttp2ConnectionHandler extends ByteToMessageDecode
@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);
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()));
}
}
});
// Close the local side of the stream if this is the last frame
if (endStream) {
Http2Stream stream = connection.stream(streamId);

View File

@ -22,7 +22,6 @@ 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;
@ -146,10 +145,10 @@ public class DefaultHttp2Connection implements Http2Connection {
public Http2Stream openForPush() throws Http2Exception {
switch (state) {
case RESERVED_LOCAL:
state = State.HALF_CLOSED_REMOTE;
state = HALF_CLOSED_REMOTE;
break;
case RESERVED_REMOTE:
state = State.HALF_CLOSED_LOCAL;
state = HALF_CLOSED_LOCAL;
break;
default:
throw protocolError("Attempting to open non-reserved stream for push");
@ -173,7 +172,7 @@ public class DefaultHttp2Connection implements Http2Connection {
public Http2Stream closeLocalSide() {
switch (state) {
case OPEN:
state = State.HALF_CLOSED_LOCAL;
state = HALF_CLOSED_LOCAL;
break;
case HALF_CLOSED_LOCAL:
break;
@ -188,7 +187,7 @@ public class DefaultHttp2Connection implements Http2Connection {
public Http2Stream closeRemoteSide() {
switch (state) {
case OPEN:
state = State.HALF_CLOSED_REMOTE;
state = HALF_CLOSED_REMOTE;
break;
case HALF_CLOSED_REMOTE:
break;
@ -243,9 +242,9 @@ public class DefaultHttp2Connection implements Http2Connection {
// Create and initialize the stream.
DefaultStream stream = new DefaultStream(streamId);
if (halfClosed) {
stream.state = isLocal() ? State.HALF_CLOSED_LOCAL : State.HALF_CLOSED_REMOTE;
stream.state = isLocal() ? HALF_CLOSED_LOCAL : HALF_CLOSED_REMOTE;
} else {
stream.state = State.OPEN;
stream.state = OPEN;
}
// Update the next and last stream IDs.
@ -278,7 +277,7 @@ public class DefaultHttp2Connection implements Http2Connection {
// Create and initialize the stream.
DefaultStream stream = new DefaultStream(streamId);
stream.state = isLocal() ? State.RESERVED_LOCAL : State.RESERVED_REMOTE;
stream.state = isLocal() ? RESERVED_LOCAL : RESERVED_REMOTE;
// Update the next and last stream IDs.
nextStreamId = streamId + 2;

View File

@ -575,7 +575,7 @@ public class DefaultHttp2FrameReader implements Http2FrameReader {
}
}
private void readGoAwayFrame(ChannelHandlerContext ctx, ByteBuf payload,
private static void readGoAwayFrame(ChannelHandlerContext ctx, ByteBuf payload,
Http2FrameObserver observer) throws Http2Exception {
int lastStreamId = readUnsignedInt(payload);
long errorCode = payload.readUnsignedInt();
@ -620,7 +620,7 @@ public class DefaultHttp2FrameReader implements Http2FrameReader {
/**
* 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.
* {@link Http2FrameObserver} once the end of headers is reached.
*/
private abstract class HeadersContinuation {
private final HeadersBuilder builder = new HeadersBuilder();
@ -635,7 +635,7 @@ public class DefaultHttp2FrameReader implements Http2FrameReader {
*
* @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}
* @param padding the amount of padding to be supplied to the {@link Http2FrameObserver}
* callback.
* @param observer the observer to be notified if the header block is completed.
*/

View File

@ -30,7 +30,6 @@ import static io.netty.handler.codec.http2.Http2CodecUtil.SETTINGS_HEADER_TABLE_
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;
@ -97,7 +96,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
out.writeZero(padding);
return ctx.writeAndFlush(out, promise);
} catch (RuntimeException e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
} finally {
data.release();
}
@ -105,8 +104,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
@Override
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment)
throws Http2Exception {
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment) {
return writeHeadersInternal(ctx, promise, streamId, headers, padding, endStream,
endSegment, false, 0, (short) 0, false);
}
@ -114,8 +112,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
@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 {
boolean exclusive, int padding, boolean endStream, boolean endSegment) {
return writeHeadersInternal(ctx, promise, streamId, headers, padding, endStream,
endSegment, true, streamDependency, weight, exclusive);
}
@ -138,7 +135,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
frame.writeByte(weight - 1);
return ctx.writeAndFlush(frame, promise);
} catch (RuntimeException e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
@ -155,7 +152,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
writeUnsignedInt(errorCode, frame);
return ctx.writeAndFlush(frame, promise);
} catch (RuntimeException e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
@ -197,7 +194,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
}
return ctx.writeAndFlush(frame, promise);
} catch (RuntimeException e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
@ -208,7 +205,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
writeFrameHeader(frame, 0, Http2FrameType.SETTINGS, Http2Flags.ACK_ONLY, 0);
return ctx.writeAndFlush(frame, promise);
} catch (RuntimeException e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
@ -224,7 +221,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
frame.writeBytes(data, data.readerIndex(), data.readableBytes());
return ctx.writeAndFlush(frame, promise);
} catch (RuntimeException e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
} finally {
data.release();
}
@ -232,8 +229,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
@Override
public ChannelFuture writePushPromise(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, int promisedStreamId, Http2Headers headers, int padding)
throws Http2Exception {
int streamId, int promisedStreamId, Http2Headers headers, int padding) {
ByteBuf headerBlock = null;
try {
verifyStreamId(streamId, "Stream ID");
@ -279,8 +275,8 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
// 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);
} catch (Exception e) {
return promise.setFailure(e);
} finally {
if (headerBlock != null) {
headerBlock.release();
@ -303,7 +299,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
frame.writeBytes(debugData, debugData.readerIndex(), debugData.readableBytes());
return ctx.writeAndFlush(frame, promise);
} catch (RuntimeException e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
} finally {
debugData.release();
}
@ -322,7 +318,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
frame.writeInt(windowSizeIncrement);
return ctx.writeAndFlush(frame, promise);
} catch (RuntimeException e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
@ -356,7 +352,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
}
return ctx.writeAndFlush(frame, promise);
} catch (RuntimeException e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
} finally {
protocolId.release();
}
@ -372,7 +368,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
writeFrameHeader(frame, 0, Http2FrameType.BLOCKED, Http2Flags.EMPTY, streamId);
return ctx.writeAndFlush(frame, promise);
} catch (RuntimeException e) {
throw failAndThrow(promise, e);
return promise.setFailure(e);
}
}
@ -387,8 +383,7 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
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 {
boolean hasPriority, int streamDependency, short weight, boolean exclusive) {
ByteBuf headerBlock = null;
try {
verifyStreamId(streamId, "Stream ID");
@ -445,8 +440,8 @@ public class DefaultHttp2FrameWriter implements Http2FrameWriter {
// 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);
} catch (Exception e) {
return promise.setFailure(e);
} finally {
if (headerBlock != null) {
headerBlock.release();

View File

@ -37,6 +37,11 @@ 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 static final long GARBAGE_COLLECTION_INTERVAL = TimeUnit.SECONDS.toNanos(2);
/**
* A comparators that sorts priority nodes in ascending order by the amount
* of priority data available for its subtree.
@ -577,7 +582,7 @@ public class DefaultHttp2OutboundFlowController implements Http2OutboundFlowCont
*/
private void incrementPendingBytes(int numBytes) {
int previouslyStreamable = streamableBytes();
this.pendingBytes += numBytes;
pendingBytes += numBytes;
incrementPriorityBytes(streamableBytes() - previouslyStreamable);
}
@ -688,10 +693,6 @@ public class DefaultHttp2OutboundFlowController implements Http2OutboundFlowCont
* Controls garbage collection for priorities that have been marked for removal.
*/
private final class GarbageCollector implements Runnable {
/**
* The interval (in ns) at which the removed priority garbage collector runs.
*/
private final long GARBAGE_COLLECTION_INTERVAL = TimeUnit.SECONDS.toNanos(2);
private final Queue<Priority<FlowState>> garbage;
private long lastGarbageCollection;

View File

@ -59,32 +59,28 @@ public class DelegatingHttp2ConnectionHandler extends AbstractHttp2ConnectionHan
@Override
public ChannelFuture writeData(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
ByteBuf data, int padding, boolean endStream, boolean endSegment, boolean compressed)
throws Http2Exception {
ByteBuf data, int padding, boolean endStream, boolean endSegment, boolean compressed) {
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 {
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment) {
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 {
boolean exclusive, int padding, boolean endStream, boolean endSegment) {
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 {
int streamId, int streamDependency, short weight, boolean exclusive) {
return super.writePriority(ctx, promise, streamId, streamDependency, weight, exclusive);
}
@ -96,27 +92,24 @@ public class DelegatingHttp2ConnectionHandler extends AbstractHttp2ConnectionHan
@Override
public ChannelFuture writeSettings(ChannelHandlerContext ctx, ChannelPromise promise,
Http2Settings settings) throws Http2Exception {
Http2Settings settings) {
return super.writeSettings(ctx, promise, settings);
}
@Override
public ChannelFuture writePing(ChannelHandlerContext ctx, ChannelPromise promise, ByteBuf data)
throws Http2Exception {
public ChannelFuture writePing(ChannelHandlerContext ctx, ChannelPromise promise, ByteBuf data) {
return super.writePing(ctx, promise, data);
}
@Override
public ChannelFuture writePushPromise(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, int promisedStreamId, Http2Headers headers, int padding)
throws Http2Exception {
int streamId, int promisedStreamId, Http2Headers headers, int padding) {
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 {
int streamId, long maxAge, int port, ByteBuf protocolId, String host, String origin) {
return super.writeAltSvc(ctx, promise, streamId, maxAge, port, protocolId, host, origin);
}

View File

@ -133,7 +133,7 @@ public interface Http2FrameObserver {
* @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.
* @param padding the number of padding bytes found at the end of the frame.
*/
void onPushPromiseRead(ChannelHandlerContext ctx, int streamId, int promisedStreamId,
Http2Headers headers, int padding) throws Http2Exception;

View File

@ -49,7 +49,7 @@ public enum Http2FrameType {
private final short code;
private Http2FrameType(short code) {
Http2FrameType(short code) {
this.code = code;
}

View File

@ -41,8 +41,7 @@ public interface Http2FrameWriter extends Closeable {
* @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;
ByteBuf data, int padding, boolean endStream, boolean endSegment, boolean compressed);
/**
* Writes a HEADERS frame to the remote endpoint.
@ -57,8 +56,7 @@ public interface Http2FrameWriter extends Closeable {
* @return the future for the write.
*/
ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
Http2Headers headers, int padding, boolean endStream, boolean endSegment)
throws Http2Exception;
Http2Headers headers, int padding, boolean endStream, boolean endSegment);
/**
* Writes a HEADERS frame with priority specified to the remote endpoint.
@ -78,7 +76,7 @@ public interface Http2FrameWriter extends Closeable {
*/
ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
Http2Headers headers, int streamDependency, short weight, boolean exclusive,
int padding, boolean endStream, boolean endSegment) throws Http2Exception;
int padding, boolean endStream, boolean endSegment);
/**
* Writes a PRIORITY frame to the remote endpoint.
@ -93,7 +91,7 @@ public interface Http2FrameWriter extends Closeable {
* @return the future for the write.
*/
ChannelFuture writePriority(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
int streamDependency, short weight, boolean exclusive) throws Http2Exception;
int streamDependency, short weight, boolean exclusive);
/**
* Writes a RST_STREAM frame to the remote endpoint.
@ -116,7 +114,7 @@ public interface Http2FrameWriter extends Closeable {
* @return the future for the write.
*/
ChannelFuture writeSettings(ChannelHandlerContext ctx, ChannelPromise promise,
Http2Settings settings) throws Http2Exception;
Http2Settings settings);
/**
* Writes a SETTINGS acknowledgment to the remote endpoint.
@ -125,8 +123,7 @@ public interface Http2FrameWriter extends Closeable {
* @param promise the promise for the write.
* @return the future for the write.
*/
ChannelFuture writeSettingsAck(ChannelHandlerContext ctx, ChannelPromise promise)
throws Http2Exception;
ChannelFuture writeSettingsAck(ChannelHandlerContext ctx, ChannelPromise promise);
/**
* Writes a PING frame to the remote endpoint.
@ -139,7 +136,7 @@ public interface Http2FrameWriter extends Closeable {
* @return the future for the write.
*/
ChannelFuture writePing(ChannelHandlerContext ctx, ChannelPromise promise, boolean ack,
ByteBuf data) throws Http2Exception;
ByteBuf data);
/**
* Writes a PUSH_PROMISE frame to the remote endpoint.
@ -153,7 +150,7 @@ public interface Http2FrameWriter extends Closeable {
* @return the future for the write.
*/
ChannelFuture writePushPromise(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
int promisedStreamId, Http2Headers headers, int padding) throws Http2Exception;
int promisedStreamId, Http2Headers headers, int padding);
/**
* Writes a GO_AWAY frame to the remote endpoint.
@ -179,7 +176,7 @@ public interface Http2FrameWriter extends Closeable {
* @return the future for the write.
*/
ChannelFuture writeWindowUpdate(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, int windowSizeIncrement) throws Http2Exception;
int streamId, int windowSizeIncrement);
/**
* Writes a ALT_SVC frame to the remote endpoint.
@ -196,8 +193,7 @@ public interface Http2FrameWriter extends Closeable {
* @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;
long maxAge, int port, ByteBuf protocolId, String host, String origin);
/**
* Writes a BLOCKED frame to the remote endpoint.
@ -207,8 +203,7 @@ public interface Http2FrameWriter extends Closeable {
* @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;
ChannelFuture writeBlocked(ChannelHandlerContext ctx, ChannelPromise promise, int streamId);
/**
* Closes this writer and frees any allocated resources.

View File

@ -23,7 +23,7 @@ import io.netty.buffer.ByteBuf;
public interface Http2OutboundFlowController {
/**
* Interface that abstracts the writing of {@link Http2Frame} objects to the remote endpoint.
* Interface that abstracts the writing of frames to the remote endpoint.
*/
interface FrameWriter {

View File

@ -43,8 +43,7 @@ public class Http2OutboundFrameLogger implements Http2FrameWriter {
@Override
public ChannelFuture writeData(ChannelHandlerContext ctx, ChannelPromise promise, int streamId,
ByteBuf data, int padding, boolean endStream, boolean endSegment, boolean compressed)
throws Http2Exception {
ByteBuf data, int padding, boolean endStream, boolean endSegment, boolean compressed) {
logger.logData(OUTBOUND, streamId, data, padding, endStream, endSegment, compressed);
return writer.writeData(ctx, promise, streamId, data, padding, endStream, endSegment,
compressed);
@ -52,8 +51,7 @@ public class Http2OutboundFrameLogger implements Http2FrameWriter {
@Override
public ChannelFuture writeHeaders(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment)
throws Http2Exception {
int streamId, Http2Headers headers, int padding, boolean endStream, boolean endSegment) {
logger.logHeaders(OUTBOUND, streamId, headers, padding, endStream, endSegment);
return writer.writeHeaders(ctx, promise, streamId, headers, padding, endStream, endSegment);
}
@ -61,8 +59,7 @@ public class Http2OutboundFrameLogger implements Http2FrameWriter {
@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 {
boolean exclusive, int padding, boolean endStream, boolean endSegment) {
logger.logHeaders(OUTBOUND, streamId, headers, streamDependency, weight, exclusive,
padding, endStream, endSegment);
return writer.writeHeaders(ctx, promise, streamId, headers, streamDependency, weight,
@ -71,8 +68,7 @@ public class Http2OutboundFrameLogger implements Http2FrameWriter {
@Override
public ChannelFuture writePriority(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, int streamDependency, short weight, boolean exclusive)
throws Http2Exception {
int streamId, int streamDependency, short weight, boolean exclusive) {
logger.logPriority(OUTBOUND, streamId, streamDependency, weight, exclusive);
return writer.writePriority(ctx, promise, streamId, streamDependency, weight, exclusive);
}
@ -85,29 +81,27 @@ public class Http2OutboundFrameLogger implements Http2FrameWriter {
@Override
public ChannelFuture writeSettings(ChannelHandlerContext ctx, ChannelPromise promise,
Http2Settings settings) throws Http2Exception {
Http2Settings settings) {
logger.logSettings(OUTBOUND, settings);
return writer.writeSettings(ctx, promise, settings);
}
@Override
public ChannelFuture writeSettingsAck(ChannelHandlerContext ctx, ChannelPromise promise)
throws Http2Exception {
public ChannelFuture writeSettingsAck(ChannelHandlerContext ctx, ChannelPromise promise) {
logger.logSettingsAck(OUTBOUND);
return writer.writeSettingsAck(ctx, promise);
}
@Override
public ChannelFuture writePing(ChannelHandlerContext ctx, ChannelPromise promise, boolean ack,
ByteBuf data) throws Http2Exception {
ByteBuf data) {
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 {
int streamId, int promisedStreamId, Http2Headers headers, int padding) {
logger.logPushPromise(OUTBOUND, streamId, promisedStreamId, headers, padding);
return writer.writePushPromise(ctx, promise, streamId, promisedStreamId, headers, padding);
}
@ -121,22 +115,21 @@ public class Http2OutboundFrameLogger implements Http2FrameWriter {
@Override
public ChannelFuture writeWindowUpdate(ChannelHandlerContext ctx, ChannelPromise promise,
int streamId, int windowSizeIncrement) throws Http2Exception {
int streamId, int windowSizeIncrement) {
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 {
int streamId, long maxAge, int port, ByteBuf protocolId, String host, String origin) {
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 {
int streamId) {
logger.logBlocked(OUTBOUND, streamId);
return writer.writeBlocked(ctx, promise, streamId);
}

View File

@ -101,7 +101,6 @@ public interface Http2PriorityTree<T> extends Iterable<Http2PriorityTree.Priorit
* 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);

View File

@ -62,7 +62,7 @@ public class Http2Settings {
}
enable(MAX_HEADER_TABLE_SIZE_MASK);
this.maxHeaderTableSize = headerTableSize;
maxHeaderTableSize = headerTableSize;
return this;
}

View File

@ -50,6 +50,7 @@ import io.netty.channel.ChannelPromise;
import java.util.Arrays;
import java.util.Collections;
import io.netty.channel.DefaultChannelPromise;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
@ -88,7 +89,6 @@ public class DelegatingHttp2ConnectionHandlerTest {
@Mock
private Channel channel;
@Mock
private ChannelPromise promise;
@Mock
@ -112,6 +112,9 @@ public class DelegatingHttp2ConnectionHandlerTest {
@Before
public void setup() throws Exception {
MockitoAnnotations.initMocks(this);
promise = new DefaultChannelPromise(channel);
when(channel.isActive()).thenReturn(true);
when(stream.id()).thenReturn(STREAM_ID);
when(stream.state()).thenReturn(OPEN);
@ -492,16 +495,18 @@ public class DelegatingHttp2ConnectionHandlerTest {
eq(host), eq(origin));
}
@Test(expected = Http2Exception.class)
@Test
public void dataWriteAfterGoAwayShouldFail() throws Exception {
when(connection.isGoAway()).thenReturn(true);
handler.writeData(ctx, promise, STREAM_ID, dummyData(), 0, false, false, false);
ChannelFuture future = handler.writeData(ctx, promise, STREAM_ID, dummyData(), 0, false, false, false);
assertTrue(future.awaitUninterruptibly().cause() instanceof Http2Exception);
}
@Test(expected = Http2Exception.class)
@Test
public void dataWriteShouldDisallowCompression() throws Exception {
when(local.allowCompressedData()).thenReturn(false);
handler.writeData(ctx, promise, STREAM_ID, dummyData(), 0, false, false, true);
ChannelFuture future = handler.writeData(ctx, promise, STREAM_ID, dummyData(), 0, false, false, true);
assertTrue(future.awaitUninterruptibly().cause() instanceof Http2Exception);
}
@Test
@ -519,14 +524,16 @@ public class DelegatingHttp2ConnectionHandlerTest {
eq(false), eq(false), any(Http2OutboundFlowController.FrameWriter.class));
}
@Test(expected = Http2Exception.class)
@Test
public void headersWriteAfterGoAwayShouldFail() throws Exception {
when(connection.isGoAway()).thenReturn(true);
handler.writeHeaders(ctx, promise, 5, EMPTY_HEADERS, 0, (short) 255, false, 0, false, false);
ChannelFuture future = handler.writeHeaders(
ctx, promise, 5, EMPTY_HEADERS, 0, (short) 255, false, 0, false, false);
verify(writer, never()).writeHeaders(eq(ctx), eq(promise), anyInt(),
any(Http2Headers.class), anyInt(), anyBoolean(), anyBoolean());
verify(outboundFlow, never()).addStream(anyInt(), anyInt(), anyShort(), anyBoolean());
verify(inboundFlow, never()).addStream(anyInt());
assertTrue(future.awaitUninterruptibly().cause() instanceof Http2Exception);
}
@Test
@ -569,10 +576,11 @@ public class DelegatingHttp2ConnectionHandlerTest {
eq(DEFAULT_PRIORITY_WEIGHT), eq(false), eq(0), eq(true), eq(false));
}
@Test(expected = Http2Exception.class)
@Test
public void pushPromiseWriteAfterGoAwayShouldFail() throws Exception {
when(connection.isGoAway()).thenReturn(true);
handler.writePushPromise(ctx, promise, STREAM_ID, PUSH_STREAM_ID, EMPTY_HEADERS, 0);
ChannelFuture future = handler.writePushPromise(ctx, promise, STREAM_ID, PUSH_STREAM_ID, EMPTY_HEADERS, 0);
assertTrue(future.awaitUninterruptibly().cause() instanceof Http2Exception);
}
@Test
@ -583,10 +591,11 @@ public class DelegatingHttp2ConnectionHandlerTest {
eq(EMPTY_HEADERS), eq(0));
}
@Test(expected = Http2Exception.class)
@Test
public void priorityWriteAfterGoAwayShouldFail() throws Exception {
when(connection.isGoAway()).thenReturn(true);
handler.writePriority(ctx, promise, STREAM_ID, 0, (short) 255, true);
ChannelFuture future = handler.writePriority(ctx, promise, STREAM_ID, 0, (short) 255, true);
assertTrue(future.awaitUninterruptibly().cause() instanceof Http2Exception);
}
@Test
@ -611,10 +620,11 @@ public class DelegatingHttp2ConnectionHandlerTest {
eq((long) PROTOCOL_ERROR.code()));
}
@Test(expected = Http2Exception.class)
@Test
public void pingWriteAfterGoAwayShouldFail() throws Exception {
when(connection.isGoAway()).thenReturn(true);
handler.writePing(ctx, promise, emptyPingBuf());
ChannelFuture future = handler.writePing(ctx, promise, emptyPingBuf());
assertTrue(future.awaitUninterruptibly().cause() instanceof Http2Exception);
}
@Test
@ -623,10 +633,11 @@ public class DelegatingHttp2ConnectionHandlerTest {
verify(writer).writePing(eq(ctx), eq(promise), eq(false), eq(emptyPingBuf()));
}
@Test(expected = Http2Exception.class)
@Test
public void settingsWriteAfterGoAwayShouldFail() throws Exception {
when(connection.isGoAway()).thenReturn(true);
handler.writeSettings(ctx, promise, new Http2Settings());
ChannelFuture future = handler.writeSettings(ctx, promise, new Http2Settings());
assertTrue(future.awaitUninterruptibly().cause() instanceof Http2Exception);
}
@Test
@ -654,11 +665,12 @@ public class DelegatingHttp2ConnectionHandlerTest {
verify(reader).maxHeaderTableSize(eq(2000));
}
@Test(expected = Http2Exception.class)
@Test
public void clientWriteAltSvcShouldThrow() throws Exception {
when(connection.isServer()).thenReturn(false);
handler.writeAltSvc(ctx, promise, STREAM_ID, 1, 2, Unpooled.EMPTY_BUFFER,
ChannelFuture future = handler.writeAltSvc(ctx, promise, STREAM_ID, 1, 2, Unpooled.EMPTY_BUFFER,
"www.example.com", null);
assertTrue(future.awaitUninterruptibly().cause() instanceof Http2Exception);
}
@Test

View File

@ -120,14 +120,10 @@ public class Http2ConnectionRoundtripTest {
clientChannel.eventLoop().execute(new Runnable() {
@Override
public void run() {
try {
http2Client.writeHeaders(ctx(), newPromise(), streamId, headers, 0,
(short) 16, false, 0, false, false);
http2Client.writeData(ctx(), newPromise(), streamId,
Unpooled.copiedBuffer(text.getBytes()), 0, true, true, false);
} catch (Http2Exception e) {
throw new RuntimeException(e);
}
http2Client.writeHeaders(ctx(), newPromise(), streamId, headers, 0,
(short) 16, false, 0, false, false);
http2Client.writeData(ctx(), newPromise(), streamId,
Unpooled.copiedBuffer(text.getBytes()), 0, true, true, false);
}
});
}