> On Dec. 8, 2014, 9:03 p.m., Brock Noland wrote: > > Hey Marcelo, > > > > When I send an HTTP request to the port where RSC is listening the message > > below is printed. Thus it's doing a good job in that it's checking the max > > message size which is awesome, but I feel we need to: > > > > 1) Add a small header so that when junk data is sent to this port we can > > log a better exception than the one below. As I mentioned, we've had > > massive problems with this is in flume which also uses netty for > > communication. > > > > 2) ensure the income size is not negative. > > > > > > 2014-12-08 20:56:41,070 WARN [RPC-Handler-7]: rpc.RpcDispatcher > > (RpcDispatcher.java:exceptionCaught(154)) - [HelloDispatcher] Caught > > exception in channel pipeline. > > io.netty.handler.codec.DecoderException: > > java.lang.IllegalArgumentException: Message exceeds maximum allowed size > > (10485760 bytes). > > at > > io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:280) > > at > > io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:149) > > at > > io.netty.handler.codec.ByteToMessageCodec.channelRead(ByteToMessageCodec.java:108) > > at > > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) > > at > > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) > > at > > io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:787) > > at > > io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:130) > > at > > io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) > > at > > io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) > > at > > io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) > > at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) > > at > > io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116) > > at java.lang.Thread.run(Thread.java:745) > > Caused by: java.lang.IllegalArgumentException: Message exceeds maximum > > allowed size (10485760 bytes). > > at > > org.apache.hive.spark.client.rpc.KryoMessageCodec.checkSize(KryoMessageCodec.java:117) > > at > > org.apache.hive.spark.client.rpc.KryoMessageCodec.decode(KryoMessageCodec.java:77) > > at > > io.netty.handler.codec.ByteToMessageCodec$1.decode(ByteToMessageCodec.java:42) > > at > > io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:249) > > ... 12 more
I can add the check for negative sizes, but I still don't understand why you want a header. It doesn't serve any practical purposes. The protocol itself has a handshake that needs to be successful for the connection to be established; adding a header will add nothing to the process, just complexity. - Marcelo ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28779/#review64279 ----------------------------------------------------------- On Dec. 8, 2014, 7:47 p.m., Marcelo Vanzin wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/28779/ > ----------------------------------------------------------- > > (Updated Dec. 8, 2014, 7:47 p.m.) > > > Review request for hive, Brock Noland, chengxiang li, Szehon Ho, and Xuefu > Zhang. > > > Bugs: HIVE-9036 > https://issues.apache.org/jira/browse/HIVE-9036 > > > Repository: hive-git > > > Description > ------- > > This patch replaces akka with a simple netty-based RPC layer. It doesn't add > any features on top of the existing spark-client API, which is unchanged > (except for the need to add empty constructors in some places). > > With the new backend we can think about adding some nice features such as > future listeners (which were awkward with akka because of Scala), but those > are left for a different time. > > The full change set, with more detailed descriptions, can be seen here: > https://github.com/vanzin/hive/commits/spark-client-netty > > > Diffs > ----- > > pom.xml 630b10ce35032e4b2dee50ef3dfe5feb58223b78 > ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java > PRE-CREATION > ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java > PRE-CREATION > > ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java > PRE-CREATION > spark-client/pom.xml PRE-CREATION > spark-client/src/main/java/org/apache/hive/spark/client/ClientUtils.java > PRE-CREATION > spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java > PRE-CREATION > spark-client/src/main/java/org/apache/hive/spark/client/Protocol.java > PRE-CREATION > spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleReadMetrics.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleWriteMetrics.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/client/rpc/KryoMessageCodec.java > PRE-CREATION > spark-client/src/main/java/org/apache/hive/spark/client/rpc/README.md > PRE-CREATION > spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcDispatcher.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcException.java > PRE-CREATION > spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java > PRE-CREATION > spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounter.java > PRE-CREATION > > spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounterGroup.java > PRE-CREATION > spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounters.java > PRE-CREATION > > spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java > PRE-CREATION > > spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java > PRE-CREATION > spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java > PRE-CREATION > > Diff: https://reviews.apache.org/r/28779/diff/ > > > Testing > ------- > > spark-client unit tests, plus some qtests. > > > Thanks, > > Marcelo Vanzin > >