Adding examples for HTTP/2 framing.

Motivation:

Provide some example code to show how to bootstrap client and server for
use with HTTP/2 framing.

Modifications:

- Fixed Http2ConnectionHandler to allow headers after stream creation.
Needed for response headers.

- Added toString() to all frame classes to help with debugging/logging

- Added example classes for HTTP/2

Result:

HTTP/2 connections now properly support response headers. Examples for
HTTP/2 provided with the distribution of examples module.

After your change, what will change.
This commit is contained in:
nmittler 2014-04-07 14:00:34 -07:00 committed by Norman Maurer
parent 25e0d9de7e
commit 69826637a8
20 changed files with 654 additions and 18 deletions

View File

@ -247,10 +247,16 @@ public class Http2ConnectionHandler extends ChannelHandlerAdapter {
connection.remote().createStream(frame.getStreamId(), frame.getPriority(), connection.remote().createStream(frame.getStreamId(), frame.getPriority(),
frame.isEndOfStream()); frame.isEndOfStream());
} else { } else {
// If the stream already exists, it must be a reserved push stream. If so, open if (stream.getState() == RESERVED_REMOTE) {
// it for push to the local endpoint. // Received headers for a reserved push stream ... open it for push to the
stream.verifyState(PROTOCOL_ERROR, RESERVED_REMOTE); // local endpoint.
stream.openForPush(); 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 the headers completes this stream, close it.
if (frame.isEndOfStream()) { if (frame.isEndOfStream()) {
@ -450,17 +456,20 @@ public class Http2ConnectionHandler extends ChannelHandlerAdapter {
stream = connection.local().createStream(frame.getStreamId(), frame.getPriority(), stream = connection.local().createStream(frame.getStreamId(), frame.getPriority(),
frame.isEndOfStream()); frame.isEndOfStream());
} else { } else {
// If the stream already exists, it must be a reserved push stream. If so, open if (stream.getState() == RESERVED_LOCAL) {
// it for push to the remote endpoint. // Sending headers on a reserved push stream ... open it for push to the remote
stream.verifyState(PROTOCOL_ERROR, RESERVED_LOCAL); // endpoint.
stream.openForPush(); 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 the headers are the end of the stream, close it now.
if (frame.isEndOfStream()) { if (frame.isEndOfStream()) {
stream.closeLocalSide(ctx, promise); stream.closeLocalSide(ctx, promise);
} }
} }
// Flush to send all of the frames. // Flush to send all of the frames.
ctx.writeAndFlush(frame, promise); ctx.writeAndFlush(frame, promise);
} }

View File

@ -17,7 +17,6 @@ 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_FRAME_PAYLOAD_LENGTH;
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_UNSIGNED_SHORT; import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_UNSIGNED_SHORT;
import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBuf;
import io.netty.buffer.DefaultByteBufHolder; import io.netty.buffer.DefaultByteBufHolder;
import io.netty.buffer.Unpooled; import io.netty.buffer.Unpooled;
@ -121,6 +120,17 @@ public final class DefaultHttp2DataFrame extends DefaultByteBufHolder implements
return true; 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() { private Builder copyBuilder() {
return new Builder().setStreamId(streamId).setPaddingLength(paddingLength) return new Builder().setStreamId(streamId).setPaddingLength(paddingLength)
.setEndOfStream(endOfStream); .setEndOfStream(endOfStream);

View File

@ -17,7 +17,6 @@ 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_FRAME_PAYLOAD_LENGTH;
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_UNSIGNED_INT; import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.MAX_UNSIGNED_INT;
import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBuf;
import io.netty.buffer.DefaultByteBufHolder; import io.netty.buffer.DefaultByteBufHolder;
import io.netty.buffer.Unpooled; import io.netty.buffer.Unpooled;
@ -110,6 +109,15 @@ public final class DefaultHttp2GoAwayFrame extends DefaultByteBufHolder implemen
return true; 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() { private Builder copyBuilder() {
return new Builder().setErrorCode(errorCode).setLastStreamId(lastStreamId); return new Builder().setErrorCode(errorCode).setLastStreamId(lastStreamId);
} }

View File

@ -97,8 +97,13 @@ public final class DefaultHttp2HeadersFrame implements Http2HeadersFrame {
@Override @Override
public String toString() { public String toString() {
return "DefaultHttp2HeadersFrame [streamId=" + streamId + ", priority=" + priority StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
+ ", endOfStream=" + endOfStream + ", headers=" + headers + ']'; 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 { public static class Builder {

View File

@ -16,7 +16,6 @@
package io.netty.handler.codec.http2.draft10.frame; package io.netty.handler.codec.http2.draft10.frame;
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.PING_FRAME_PAYLOAD_LENGTH; import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.PING_FRAME_PAYLOAD_LENGTH;
import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBuf;
import io.netty.buffer.DefaultByteBufHolder; import io.netty.buffer.DefaultByteBufHolder;
@ -100,6 +99,14 @@ public final class DefaultHttp2PingFrame extends DefaultByteBufHolder implements
return true; 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}. * Builds instances of {@link DefaultHttp2PingFrame}.
*/ */

View File

@ -73,6 +73,14 @@ public final class DefaultHttp2PriorityFrame implements Http2PriorityFrame {
return true; 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}. * Builds instances of {@link DefaultHttp2PriorityFrame}.
*/ */

View File

@ -89,8 +89,12 @@ public final class DefaultHttp2PushPromiseFrame implements Http2PushPromiseFrame
@Override @Override
public String toString() { public String toString() {
return "DefaultHttp2PushPromiseFrame [streamId=" + streamId + ", promisedStreamId=" StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append("[");
+ promisedStreamId + ", headers=" + headers + ']'; builder.append("streamId=").append(streamId);
builder.append(", promisedStreamId=").append(promisedStreamId);
builder.append(", headers=").append(headers);
builder.append("]");
return builder.toString();
} }
public static class Builder { public static class Builder {

View File

@ -74,6 +74,15 @@ public final class DefaultHttp2RstStreamFrame implements Http2RstStreamFrame {
return true; 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}. * Builds instances of {@link DefaultHttp2RstStreamFrame}.
*/ */

View File

@ -118,6 +118,26 @@ public final class DefaultHttp2SettingsFrame implements Http2SettingsFrame {
return true; 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}. * Builds instances of {@link DefaultHttp2SettingsFrame}.
*/ */

View File

@ -73,6 +73,15 @@ public final class DefaultHttp2WindowUpdateFrame implements Http2WindowUpdateFra
return true; 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}. * Builds instances of {@link DefaultHttp2WindowUpdateFrame}.
*/ */

View File

@ -132,6 +132,35 @@ public class Http2Flags {
return true; 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) { private boolean isSet(short mask) {
return (value & mask) != 0; return (value & mask) != 0;
} }

View File

@ -16,6 +16,8 @@
package io.netty.handler.codec.http2.draft10.connection; package io.netty.handler.codec.http2.draft10.connection;
import static io.netty.handler.codec.http2.draft10.Http2Error.PROTOCOL_ERROR; import static io.netty.handler.codec.http2.draft10.Http2Error.PROTOCOL_ERROR;
import static io.netty.handler.codec.http2.draft10.connection.Http2Stream.State.RESERVED_LOCAL;
import static io.netty.handler.codec.http2.draft10.connection.Http2Stream.State.RESERVED_REMOTE;
import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.PING_FRAME_PAYLOAD_LENGTH; import static io.netty.handler.codec.http2.draft10.frame.Http2FrameCodecUtil.PING_FRAME_PAYLOAD_LENGTH;
import static io.netty.util.CharsetUtil.UTF_8; import static io.netty.util.CharsetUtil.UTF_8;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -227,7 +229,8 @@ public class Http2ConnectionHandlerTest {
} }
@Test @Test
public void inboundHeadersWithForPromisedStreamShouldHalfOpenStream() throws Exception { public void inboundHeadersForPromisedStreamShouldHalfOpenStream() throws Exception {
when(stream.getState()).thenReturn(RESERVED_REMOTE);
Http2Frame frame = Http2Frame frame =
new DefaultHttp2HeadersFrame.Builder().setStreamId(STREAM_ID).setPriority(1) new DefaultHttp2HeadersFrame.Builder().setStreamId(STREAM_ID).setPriority(1)
.setHeaders(Http2Headers.EMPTY_HEADERS).build(); .setHeaders(Http2Headers.EMPTY_HEADERS).build();
@ -237,7 +240,8 @@ public class Http2ConnectionHandlerTest {
} }
@Test @Test
public void inboundHeadersWithForPromisedStreamShouldCloseStream() throws Exception { public void inboundHeadersForPromisedStreamShouldCloseStream() throws Exception {
when(stream.getState()).thenReturn(RESERVED_REMOTE);
Http2Frame frame = Http2Frame frame =
new DefaultHttp2HeadersFrame.Builder().setStreamId(STREAM_ID).setPriority(1) new DefaultHttp2HeadersFrame.Builder().setStreamId(STREAM_ID).setPriority(1)
.setEndOfStream(true).setHeaders(Http2Headers.EMPTY_HEADERS) .setEndOfStream(true).setHeaders(Http2Headers.EMPTY_HEADERS)
@ -479,6 +483,7 @@ public class Http2ConnectionHandlerTest {
@Test @Test
public void outboundHeadersShouldOpenStreamForPush() throws Exception { public void outboundHeadersShouldOpenStreamForPush() throws Exception {
when(stream.getState()).thenReturn(RESERVED_LOCAL);
Http2Frame frame = Http2Frame frame =
new DefaultHttp2HeadersFrame.Builder().setStreamId(STREAM_ID).setPriority(1) new DefaultHttp2HeadersFrame.Builder().setStreamId(STREAM_ID).setPriority(1)
.setHeaders(Http2Headers.EMPTY_HEADERS).build(); .setHeaders(Http2Headers.EMPTY_HEADERS).build();
@ -491,6 +496,7 @@ public class Http2ConnectionHandlerTest {
@Test @Test
public void outboundHeadersShouldClosePushStream() throws Exception { public void outboundHeadersShouldClosePushStream() throws Exception {
when(stream.getState()).thenReturn(RESERVED_LOCAL);
Http2Frame frame = Http2Frame frame =
new DefaultHttp2HeadersFrame.Builder().setStreamId(STREAM_ID).setPriority(1) new DefaultHttp2HeadersFrame.Builder().setStreamId(STREAM_ID).setPriority(1)
.setEndOfStream(true).setHeaders(Http2Headers.EMPTY_HEADERS) .setEndOfStream(true).setHeaders(Http2Headers.EMPTY_HEADERS)

View File

@ -52,6 +52,11 @@
<artifactId>netty-codec-http</artifactId> <artifactId>netty-codec-http</artifactId>
<version>${project.version}</version> <version>${project.version}</version>
</dependency> </dependency>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>netty-codec-http2</artifactId>
<version>${project.version}</version>
</dependency>
<dependency> <dependency>
<groupId>${project.groupId}</groupId> <groupId>${project.groupId}</groupId>
<artifactId>netty-codec-socks</artifactId> <artifactId>netty-codec-socks</artifactId>

View File

@ -0,0 +1,131 @@
/*
* 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.example.http2.client;
import static java.util.concurrent.TimeUnit.SECONDS;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.HttpMethod;
import io.netty.handler.codec.http2.draft10.DefaultHttp2Headers;
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.Http2Frame;
import java.net.InetSocketAddress;
import java.util.concurrent.BlockingQueue;
/**
* An HTTP2 client that allows you to send HTTP2 frames to a server. Inbound and outbound frames
* are logged.
*/
public class Http2Client {
private final String host;
private final int port;
private final Http2ResponseClientHandler httpResponseHandler;
private Channel channel;
private EventLoopGroup workerGroup;
public Http2Client(String host, int port) {
this.host = host;
this.port = port;
httpResponseHandler = new Http2ResponseClientHandler();
}
public void start() {
if (channel != null) {
System.out.println("Already running!");
return;
}
workerGroup = new NioEventLoopGroup();
Bootstrap b = new Bootstrap();
b.group(workerGroup);
b.channel(NioSocketChannel.class);
b.option(ChannelOption.SO_KEEPALIVE, true);
b.remoteAddress(new InetSocketAddress(host, port));
b.handler(new Http2ClientInitializer(httpResponseHandler));
// Start the client.
channel = b.connect().syncUninterruptibly().channel();
System.out.println("Connected to [" + host + ':' + port + ']');
}
public void stop() {
try {
// Wait until the connection is closed.
channel.close().syncUninterruptibly();
} finally {
if (workerGroup != null) {
workerGroup.shutdownGracefully();
}
}
}
public ChannelFuture send(Http2Frame request) {
// Sends the HTTP request.
return channel.writeAndFlush(request);
}
public Http2Frame get() {
Http2Headers headers =
DefaultHttp2Headers.newBuilder().setAuthority(host)
.setMethod(HttpMethod.GET.name()).build();
return new DefaultHttp2HeadersFrame.Builder().setHeaders(headers).setStreamId(3)
.setEndOfStream(true).build();
}
public BlockingQueue<ChannelFuture> queue() {
return httpResponseHandler.queue();
}
public static void main(String[] args) throws Exception {
int port;
if (args.length > 0) {
port = Integer.parseInt(args[0]);
} else {
port = 8443;
}
final Http2Client client = new Http2Client("localhost", port);
try {
client.start();
ChannelFuture requestFuture = client.send(client.get()).sync();
if (!requestFuture.isSuccess()) {
requestFuture.cause().printStackTrace();
}
// Waits for the complete response
ChannelFuture responseFuture = client.queue().poll(5, SECONDS);
if (!responseFuture.isSuccess()) {
responseFuture.cause().printStackTrace();
}
System.out.println("Finished HTTP/2 request");
} finally {
client.stop();
}
}
}

View File

@ -0,0 +1,45 @@
/*
* 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.example.http2.client;
import static io.netty.util.internal.logging.InternalLogLevel.INFO;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.socket.SocketChannel;
import io.netty.handler.codec.http2.draft10.connection.Http2ConnectionHandler;
import io.netty.handler.codec.http2.draft10.frame.Http2FrameCodec;
/**
* Configures the client pipeline to support HTTP/2 frames.
*/
public class Http2ClientInitializer extends ChannelInitializer<SocketChannel> {
private final Http2ResponseClientHandler httpResponseHandler;
public Http2ClientInitializer(Http2ResponseClientHandler httpResponseHandler) {
this.httpResponseHandler = httpResponseHandler;
}
@Override
public void initChannel(SocketChannel ch) throws Exception {
ChannelPipeline pipeline = ch.pipeline();
pipeline.addLast("http2FrameCodec", new Http2FrameCodec());
pipeline.addLast("spdyFrameLogger", new Http2FrameLogger(INFO));
pipeline.addLast("http2ConnectionHandler", new Http2ConnectionHandler(false));
pipeline.addLast("httpHandler", httpResponseHandler);
}
}

View File

@ -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.example.http2.client;
import io.netty.channel.ChannelHandlerAdapter;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelPromise;
import io.netty.handler.codec.http2.draft10.frame.Http2Frame;
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 {
private enum Direction {
INBOUND, OUTBOUND
}
protected final InternalLogger logger;
private final InternalLogLevel level;
public Http2FrameLogger(InternalLogLevel level) {
if (level == null) {
throw new NullPointerException("level");
}
logger = InternalLoggerFactory.getInstance(getClass());
this.level = level;
}
@Override
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
if (acceptMessage(msg)) {
log((Http2Frame) msg, Direction.INBOUND);
}
super.channelRead(ctx, msg);
}
@Override
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception {
if (acceptMessage(msg)) {
log((Http2Frame) msg, Direction.OUTBOUND);
}
super.write(ctx, msg, promise);
}
private static boolean acceptMessage(Object msg) throws Exception {
return msg instanceof Http2Frame;
}
private void log(Http2Frame msg, Direction d) {
if (logger.isEnabled(level)) {
StringBuilder b = new StringBuilder("\n----------------");
b.append(d.name());
b.append("--------------------\n");
b.append(msg);
b.append("\n------------------------------------");
logger.log(level, b.toString());
}
}
}

View File

@ -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.example.http2.client;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.handler.codec.http2.draft10.frame.Http2DataFrame;
import io.netty.util.CharsetUtil;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
/**
* A handler that interprets response messages as text and prints it out to the console.
*/
public class Http2ResponseClientHandler extends SimpleChannelInboundHandler<Http2DataFrame> {
private final BlockingQueue<ChannelFuture> queue = new LinkedBlockingQueue<ChannelFuture>();
private ByteBuf data;
@Override
public void messageReceived(ChannelHandlerContext ctx, Http2DataFrame frame) throws Exception {
System.out.println("Received frame: " + frame);
// Copy the data into the buffer.
int available = frame.content().readableBytes();
if (data == null) {
data = ctx.alloc().buffer(available);
data.writeBytes(frame.content());
} else {
// Expand the buffer
ByteBuf newBuffer = ctx.alloc().buffer(data.readableBytes() + available);
newBuffer.writeBytes(data);
newBuffer.writeBytes(frame.content());
data.release();
data = newBuffer;
}
// If it's the last frame, print the complete message.
if (frame.isEndOfStream()) {
byte[] bytes = new byte[data.readableBytes()];
data.readBytes(bytes);
System.out.println("Received message: " + new String(bytes, CharsetUtil.UTF_8));
// Free the data buffer.
data.release();
data = null;
queue.add(ctx.channel().newSucceededFuture());
}
}
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
queue.add(ctx.channel().newFailedFuture(cause));
cause.printStackTrace();
ctx.close();
}
public BlockingQueue<ChannelFuture> queue() {
return queue;
}
}

View File

@ -0,0 +1,71 @@
/*
* 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.example.http2.server;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerAdapter;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.http2.draft10.DefaultHttp2Headers;
import io.netty.handler.codec.http2.draft10.Http2Headers;
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2DataFrame;
import io.netty.handler.codec.http2.draft10.frame.DefaultHttp2HeadersFrame;
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.Http2StreamFrame;
import io.netty.util.CharsetUtil;
import io.netty.util.ReferenceCountUtil;
/**
* A simple handler that responds with the message "Hello World!".
*/
public class HelloWorldHandler extends ChannelHandlerAdapter {
private static final byte[] RESPONSE_BYTES = "Hello World".getBytes(CharsetUtil.UTF_8);
@Override
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
if (msg instanceof Http2StreamFrame) {
Http2StreamFrame frame = (Http2StreamFrame) msg;
if (frame.isEndOfStream()) {
sendResponse(ctx, frame.getStreamId());
}
}
ReferenceCountUtil.release(msg);
}
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
cause.printStackTrace();
}
private void sendResponse(ChannelHandlerContext ctx, int streamId) {
ByteBuf content = ctx.alloc().buffer();
content.writeBytes(RESPONSE_BYTES);
// Send a frame for the response status
Http2Headers headers = DefaultHttp2Headers.newBuilder().setStatus("200").build();
Http2HeadersFrame headersFrame =
new DefaultHttp2HeadersFrame.Builder().setStreamId(streamId).setHeaders(headers)
.build();
ctx.write(headersFrame);
// Send a data frame with the response message.
Http2DataFrame data =
new DefaultHttp2DataFrame.Builder().setStreamId(streamId).setEndOfStream(true)
.setContent(content).build();
ctx.writeAndFlush(data);
}
}

View File

@ -0,0 +1,69 @@
/*
* 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.example.http2.server;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.channel.Channel;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioServerSocketChannel;
/**
* A HTTP/2 Server that responds to requests with a Hello World.
* <p>
* Once started, you can test the server with the example client.
*/
public class Http2Server {
private final int port;
public Http2Server(int port) {
this.port = port;
}
public void run() throws Exception {
// Configure the server.
EventLoopGroup bossGroup = new NioEventLoopGroup(1);
EventLoopGroup workerGroup = new NioEventLoopGroup();
try {
ServerBootstrap b = new ServerBootstrap();
b.option(ChannelOption.SO_BACKLOG, 1024);
b.group(bossGroup, workerGroup).channel(NioServerSocketChannel.class)
.childHandler(new Http2ServerInitializer());
Channel ch = b.bind(port).sync().channel();
ch.closeFuture().sync();
} finally {
bossGroup.shutdownGracefully();
workerGroup.shutdownGracefully();
}
}
public static void main(String[] args) throws Exception {
int port;
if (args.length > 0) {
port = Integer.parseInt(args[0]);
} else {
port = 8443;
}
System.out.println("HTTP2 server started at port " + port + '.');
new Http2Server(port).run();
}
}

View File

@ -0,0 +1,37 @@
/*
* 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.example.http2.server;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.socket.SocketChannel;
import io.netty.handler.codec.http2.draft10.connection.Http2ConnectionHandler;
import io.netty.handler.codec.http2.draft10.frame.Http2FrameCodec;
/**
* Sets up the Netty pipeline
*/
public class Http2ServerInitializer extends ChannelInitializer<SocketChannel> {
@Override
public void initChannel(SocketChannel ch) throws Exception {
ChannelPipeline p = ch.pipeline();
p.addLast("http2FrameCodec", new Http2FrameCodec());
p.addLast("http2ConnectionHandler", new Http2ConnectionHandler(true));
p.addLast("helloWorldHandler", new HelloWorldHandler());
}
}