-
Notifications
You must be signed in to change notification settings - Fork 97
[REEF-362] Implement a Wake Transport using HTTP #1341
base: master
Are you sure you want to change the base?
Changes from 31 commits
48ec577
56d0650
4994107
d3f7a46
1ecda33
9510d61
31e5efa
bc42c3d
5b3f1b5
1943609
dc36180
abd316d
7ce5b33
14121e7
476ebe1
2befc1a
df5fea5
985ad1a
8fd67f6
37a9ada
40f335d
817fa0e
3674ffa
cdaadcc
4990972
9e9260a
29344b6
017154e
f221dbb
455464a
bd3b205
2225679
b20df37
94fb7bc
046739d
645b898
bd2030a
6541d3c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,46 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF 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 org.apache.reef.wake.remote.transport.netty; | ||
|
||
import io.netty.channel.ChannelFuture; | ||
import io.netty.channel.ChannelFutureListener; | ||
import org.apache.reef.wake.remote.transport.LinkListener; | ||
|
||
/** | ||
* Future Listener used in NettyLink. | ||
*/ | ||
public final class NettyChannelFutureListener<T> implements ChannelFutureListener { | ||
|
||
private final T message; | ||
private final LinkListener<T> listener; | ||
|
||
NettyChannelFutureListener(final T message, final LinkListener<T> listener) { | ||
this.message = message; | ||
this.listener = listener; | ||
} | ||
|
||
@Override | ||
public void operationComplete(final ChannelFuture channelFuture) throws Exception { | ||
if (channelFuture.isSuccess()) { | ||
listener.onSuccess(message); | ||
} else { | ||
listener.onException(channelFuture.cause(), channelFuture.channel().remoteAddress(), message); | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,34 +19,70 @@ | |
package org.apache.reef.wake.remote.transport.netty; | ||
|
||
import io.netty.channel.ChannelInitializer; | ||
import io.netty.channel.ChannelPipeline; | ||
import io.netty.channel.socket.SocketChannel; | ||
import io.netty.handler.codec.LengthFieldBasedFrameDecoder; | ||
import io.netty.handler.codec.LengthFieldPrepender; | ||
import io.netty.handler.codec.bytes.ByteArrayDecoder; | ||
import io.netty.handler.codec.bytes.ByteArrayEncoder; | ||
import io.netty.handler.codec.http.*; | ||
import org.apache.reef.wake.remote.transport.TransportFactory.ProtocolType; | ||
|
||
|
||
/** | ||
* Netty channel initializer for Transport. | ||
*/ | ||
class NettyChannelInitializer extends ChannelInitializer<SocketChannel> { | ||
|
||
/** | ||
* the buffer size of the frame decoder. | ||
*/ | ||
public static final int MAXFRAMELENGTH = 10 * 1024 * 1024; | ||
private static final int MAX_HTTP_MESSAGE_LENGTH = 10 * 1024 * 1024; | ||
private final NettyChannelHandlerFactory handlerFactory; | ||
|
||
NettyChannelInitializer(final NettyChannelHandlerFactory handlerFactory) { | ||
/** | ||
* Type of protocol channel use. | ||
*/ | ||
private final ProtocolType protocolType; | ||
private final boolean isServer; | ||
|
||
NettyChannelInitializer( | ||
final NettyChannelHandlerFactory handlerFactory, | ||
final ProtocolType protocol) { | ||
this(handlerFactory, protocol, false); | ||
} | ||
|
||
NettyChannelInitializer( | ||
final NettyChannelHandlerFactory handlerFactory, | ||
final ProtocolType protocol, | ||
final boolean isServer) { | ||
this.handlerFactory = handlerFactory; | ||
this.protocolType = protocol; | ||
this.isServer = isServer; | ||
} | ||
|
||
@Override | ||
protected void initChannel(final SocketChannel ch) throws Exception { | ||
ch.pipeline() | ||
.addLast("frameDecoder", new LengthFieldBasedFrameDecoder(MAXFRAMELENGTH, 0, 4, 0, 4)) | ||
.addLast("bytesDecoder", new ByteArrayDecoder()) | ||
.addLast("frameEncoder", new LengthFieldPrepender(4)) | ||
.addLast("bytesEncoder", new ByteArrayEncoder()) | ||
.addLast("chunker", new ChunkedReadWriteHandler()) | ||
.addLast("handler", handlerFactory.createChannelInboundHandler()); | ||
final ChannelPipeline pipeline = ch.pipeline(); | ||
switch (this.protocolType) { | ||
case TCP: | ||
pipeline | ||
.addLast("frameDecoder", new LengthFieldBasedFrameDecoder(MAXFRAMELENGTH, 0, 4, 0, 4)) | ||
.addLast("bytesDecoder", new ByteArrayDecoder()) | ||
.addLast("frameEncoder", new LengthFieldPrepender(4)) | ||
.addLast("bytesEncoder", new ByteArrayEncoder()) | ||
.addLast("chunker", new ChunkedReadWriteHandler()); | ||
break; | ||
case HTTP: | ||
pipeline | ||
.addLast("codec", isServer ? new HttpServerCodec() : new HttpClientCodec()) | ||
.addLast("aggregator", new HttpObjectAggregator(MAX_HTTP_MESSAGE_LENGTH)); | ||
break; | ||
default: | ||
throw new IllegalArgumentException("Invalid type of channel"); | ||
} | ||
// every channel pipeline has the same inbound handler. | ||
pipeline.addLast("handler", handlerFactory.createChannelInboundHandler()); | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How about using There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it is good idea! I'll implement it and test it. |
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,44 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF 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 org.apache.reef.wake.remote.transport.netty; | ||
|
||
import io.netty.channel.Channel; | ||
import org.apache.reef.wake.remote.Encoder; | ||
import org.apache.reef.wake.remote.transport.Link; | ||
import org.apache.reef.wake.remote.transport.LinkListener; | ||
|
||
/** | ||
* Factory that creates a NettyLink. | ||
*/ | ||
public final class NettyDefaultLinkFactory<T> implements NettyLinkFactory { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why don't you create HttpLinkFactory? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I've created NettyHttpLinkFactory. Thank you! |
||
|
||
NettyDefaultLinkFactory() {} | ||
|
||
@Override | ||
public Link newInstance(final Channel channel, final Encoder encoder) { | ||
return new NettyLink<T>(channel, encoder); | ||
} | ||
|
||
@Override | ||
public Link newInstance(final Channel channel, | ||
final Encoder encoder, | ||
final LinkListener listener) { | ||
return new NettyLink<T>(channel, encoder, listener); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,96 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF 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 org.apache.reef.wake.remote.transport.netty; | ||
|
||
import io.netty.buffer.ByteBuf; | ||
import io.netty.channel.Channel; | ||
import io.netty.channel.ChannelHandlerContext; | ||
import io.netty.handler.codec.http.FullHttpResponse; | ||
import io.netty.handler.codec.http.HttpContent; | ||
import io.netty.util.CharsetUtil; | ||
import org.apache.reef.wake.EStage; | ||
import org.apache.reef.wake.remote.exception.RemoteRuntimeException; | ||
import org.apache.reef.wake.remote.impl.TransportEvent; | ||
|
||
import java.net.SocketAddress; | ||
import java.util.concurrent.ConcurrentMap; | ||
import java.util.logging.Level; | ||
import java.util.logging.Logger; | ||
|
||
/** | ||
* A Netty event listener for client. | ||
*/ | ||
final class NettyHttpClientEventListener extends AbstractNettyEventListener { | ||
|
||
private static final Logger LOG = Logger.getLogger(NettyHttpClientEventListener.class.getName()); | ||
|
||
NettyHttpClientEventListener( | ||
final ConcurrentMap<SocketAddress, LinkReference> addrToLinkRefMap, | ||
final EStage<TransportEvent> stage) { | ||
super(addrToLinkRefMap, stage); | ||
} | ||
|
||
@Override | ||
public void channelRead(final ChannelHandlerContext ctx, final Object msg) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How about the following code? public void channelRead(final ChannelHandlerContext ctx, final FullHttpResponse response) {
byte[] content = null;
ByteBuf byteBuf = response.content();
if (byteBuf.hasArray()) {
content = byteBuf.array();
} else {
content = new byte[byteBuf.readableBytes()];
byteBuf.readBytes(content);
}
....
} There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm sorry that as method channelRead is overriding method, I cannot change type of parameter. |
||
if (msg instanceof FullHttpResponse) { | ||
if(LOG.isLoggable(Level.FINEST)) { | ||
LOG.log(Level.FINEST, "HttpResponse Received: {0}", msg); | ||
} | ||
final HttpContent httpContent = (HttpContent) msg; | ||
final ByteBuf byteBuf = httpContent.content(); | ||
final Channel channel = ctx.channel(); | ||
final byte[] content; | ||
|
||
if (byteBuf.hasArray()) { | ||
content = byteBuf.array(); | ||
} else { | ||
content = new byte[byteBuf.readableBytes()]; | ||
byteBuf.readBytes(content); | ||
} | ||
|
||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Don't we have to create a new StringBuilder here? because it is the end of the message and we need a new StringBuilder for another response? |
||
if (LOG.isLoggable(Level.FINEST)) { | ||
final StringBuilder buf = new StringBuilder(); | ||
buf.append("CONTENT: ").append(byteBuf.toString(CharsetUtil.UTF_8)).append("\r\n"); | ||
LOG.log(Level.FINEST, "MessageEvent: local: {0} remote: {1} :: {2}", new Object[]{ | ||
channel.localAddress(), channel.remoteAddress(), buf}); | ||
} | ||
|
||
// send to the dispatch stage | ||
this.stage.onNext(this.getTransportEvent(content, channel)); | ||
} else { | ||
LOG.log(Level.SEVERE, "Unknown type of message received: {0}", msg); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. throw exception? |
||
throw new RemoteRuntimeException("Unknown type of message received " + msg); | ||
} | ||
} | ||
|
||
@Override | ||
public void channelActive(final ChannelHandlerContext ctx) { | ||
// noop | ||
} | ||
|
||
@Override | ||
protected TransportEvent getTransportEvent(final byte[] message, final Channel channel) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not sure why There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think this method should not be used outside of netty implementation since the |
||
return new TransportEvent(message, channel.localAddress(), channel.remoteAddress()); | ||
} | ||
|
||
@Override | ||
protected void exceptionCleanup(final ChannelHandlerContext ctx, final Throwable cause) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Same here. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Replied above. I don't know why It says outdated... |
||
this.closeChannel(ctx.channel()); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How about defining a pipeline variable?
final ChannelPipeline pipeline = ch.pipeline();