Re: Spark Streaming + Kudu

2018-03-06 Thread Ravi Kanth
Mike- I actually got a hold of the pid's for the spark executors but facing
issues to run the jstack. There are some VM exceptions. I will figure it
out and will attach the jstack. Thanks for your patience.

On 6 March 2018 at 20:42, Mike Percy  wrote:

> Hmm, could you try in spark local mode? i.e. https://jaceklaskowski.gi
> tbooks.io/mastering-apache-spark/content/spark-local.html
>
> Mike
>
> On Tue, Mar 6, 2018 at 7:14 PM, Ravi Kanth 
> wrote:
>
>> Mike,
>>
>> Can you clarify a bit on grabbing the jstack for the process? I launched
>> my Spark application and tried to get the pid using which I thought I can
>> grab jstack trace during hang. Unfortunately, I am not able to figure out
>> grabbing pid for Spark application.
>>
>> Thanks,
>> Ravi
>>
>> On 6 March 2018 at 18:36, Mike Percy  wrote:
>>
>>> Thanks Ravi. Would you mind attaching the output of jstack on the
>>> process during this hang? That would show what the Kudu client threads are
>>> doing, as what we are seeing here is just the netty boss thread.
>>>
>>> Mike
>>>
>>> On Tue, Mar 6, 2018 at 8:52 AM, Ravi Kanth 
>>> wrote:
>>>

 Yes, I have debugged to find the root cause. Every logger before "table
 = client.openTable(tableName);" is executing fine and exactly at the
 point of opening the table, it is throwing the below exception and nothing
 is being executed after that. Still the Spark batches are being processed
 and at opening the table is failing. I tried catching it with no luck.
 Please find below the exception.

 8/02/23 00:16:30 ERROR client.TabletClient: [Peer
 bd91f34d456a4eccaae50003c90f0fb2] Unexpected exception from downstream
 on [id: 0x6e13b01f]
 java.net.ConnectException: Connection refused:
 kudu102.dev.sac.int.threatmetrix.com/10.112.3.12:7050
 at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
 at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl
 .java:717)
 at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
 .nio.NioClientBoss.connect(NioClientBoss.java:152)
 at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
 .nio.NioClientBoss.processSelectedKeys(NioClientBoss.java:105)
 at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
 .nio.NioClientBoss.process(NioClientBoss.java:79)
 at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
 .nio.AbstractNioSelector.run(AbstractNioSelector.java:337)
 at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
 .nio.NioClientBoss.run(NioClientBoss.java:42)
 at org.apache.kudu.client.shaded.org.jboss.netty.util.ThreadRen
 amingRunnable.run(ThreadRenamingRunnable.java:108)
 at org.apache.kudu.client.shaded.org.jboss.netty.util.internal.
 DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPool
 Executor.java:1142)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoo
 lExecutor.java:617)
 at java.lang.Thread.run(Thread.java:745)


 Thanks,
 Ravi

 On 5 March 2018 at 23:52, Mike Percy  wrote:

> Have you considered checking your session error count or pending
> errors in your while loop every so often? Can you identify where your code
> is hanging when the connection is lost (what line)?
>
> Mike
>
> On Mon, Mar 5, 2018 at 9:08 PM, Ravi Kanth 
> wrote:
>
>> In addition to my previous comment, I raised a support ticket for
>> this issue with Cloudera and one of the support person mentioned below,
>>
>> *"Thank you for clarifying, The exceptions are logged but not
>> re-thrown to an upper layer, so that explains why the Spark application 
>> is
>> not aware of the underlying error."*
>>
>> On 5 March 2018 at 21:02, Ravi Kanth  wrote:
>>
>>> Mike,
>>>
>>> Thanks for the information. But, once the connection to any of the
>>> Kudu servers is lost then there is no way I can have a control on the
>>> KuduSession object and so with getPendingErrors(). The KuduClient in 
>>> this
>>> case is becoming a zombie and never returned back till the connection is
>>> properly established. I tried doing all that you have suggested with no
>>> luck. Attaching my KuduClient code.
>>>
>>> package org.dwh.streaming.kudu.sparkkudustreaming;
>>>
>>> import java.util.HashMap;
>>> import java.util.Iterator;
>>> import java.util.Map;
>>> import org.apache.hadoop.util.ShutdownHookManager;
>>> import org.apache.kudu.client.*;
>>> import org.apache.spark.api.java.JavaRDD;
>>> import org.slf4j.Logger;
>>> import org.slf4j.LoggerFactory;
>>> import 

Re: Spark Streaming + Kudu

2018-03-06 Thread Mike Percy
Hmm, could you try in spark local mode? i.e. https://jaceklaskowski.
gitbooks.io/mastering-apache-spark/content/spark-local.html

Mike

On Tue, Mar 6, 2018 at 7:14 PM, Ravi Kanth  wrote:

> Mike,
>
> Can you clarify a bit on grabbing the jstack for the process? I launched
> my Spark application and tried to get the pid using which I thought I can
> grab jstack trace during hang. Unfortunately, I am not able to figure out
> grabbing pid for Spark application.
>
> Thanks,
> Ravi
>
> On 6 March 2018 at 18:36, Mike Percy  wrote:
>
>> Thanks Ravi. Would you mind attaching the output of jstack on the process
>> during this hang? That would show what the Kudu client threads are doing,
>> as what we are seeing here is just the netty boss thread.
>>
>> Mike
>>
>> On Tue, Mar 6, 2018 at 8:52 AM, Ravi Kanth 
>> wrote:
>>
>>>
>>> Yes, I have debugged to find the root cause. Every logger before "table
>>> = client.openTable(tableName);" is executing fine and exactly at the
>>> point of opening the table, it is throwing the below exception and nothing
>>> is being executed after that. Still the Spark batches are being processed
>>> and at opening the table is failing. I tried catching it with no luck.
>>> Please find below the exception.
>>>
>>> 8/02/23 00:16:30 ERROR client.TabletClient: [Peer
>>> bd91f34d456a4eccaae50003c90f0fb2] Unexpected exception from downstream
>>> on [id: 0x6e13b01f]
>>> java.net.ConnectException: Connection refused:
>>> kudu102.dev.sac.int.threatmetrix.com/10.112.3.12:7050
>>> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>>> at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl
>>> .java:717)
>>> at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
>>> .nio.NioClientBoss.connect(NioClientBoss.java:152)
>>> at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
>>> .nio.NioClientBoss.processSelectedKeys(NioClientBoss.java:105)
>>> at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
>>> .nio.NioClientBoss.process(NioClientBoss.java:79)
>>> at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
>>> .nio.AbstractNioSelector.run(AbstractNioSelector.java:337)
>>> at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
>>> .nio.NioClientBoss.run(NioClientBoss.java:42)
>>> at org.apache.kudu.client.shaded.org.jboss.netty.util.ThreadRen
>>> amingRunnable.run(ThreadRenamingRunnable.java:108)
>>> at org.apache.kudu.client.shaded.org.jboss.netty.util.internal.
>>> DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPool
>>> Executor.java:1142)
>>> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoo
>>> lExecutor.java:617)
>>> at java.lang.Thread.run(Thread.java:745)
>>>
>>>
>>> Thanks,
>>> Ravi
>>>
>>> On 5 March 2018 at 23:52, Mike Percy  wrote:
>>>
 Have you considered checking your session error count or pending errors
 in your while loop every so often? Can you identify where your code is
 hanging when the connection is lost (what line)?

 Mike

 On Mon, Mar 5, 2018 at 9:08 PM, Ravi Kanth 
 wrote:

> In addition to my previous comment, I raised a support ticket for this
> issue with Cloudera and one of the support person mentioned below,
>
> *"Thank you for clarifying, The exceptions are logged but not
> re-thrown to an upper layer, so that explains why the Spark application is
> not aware of the underlying error."*
>
> On 5 March 2018 at 21:02, Ravi Kanth  wrote:
>
>> Mike,
>>
>> Thanks for the information. But, once the connection to any of the
>> Kudu servers is lost then there is no way I can have a control on the
>> KuduSession object and so with getPendingErrors(). The KuduClient in this
>> case is becoming a zombie and never returned back till the connection is
>> properly established. I tried doing all that you have suggested with no
>> luck. Attaching my KuduClient code.
>>
>> package org.dwh.streaming.kudu.sparkkudustreaming;
>>
>> import java.util.HashMap;
>> import java.util.Iterator;
>> import java.util.Map;
>> import org.apache.hadoop.util.ShutdownHookManager;
>> import org.apache.kudu.client.*;
>> import org.apache.spark.api.java.JavaRDD;
>> import org.slf4j.Logger;
>> import org.slf4j.LoggerFactory;
>> import org.dwh.streaming.kudu.sparkkudustreaming.constants.SpecialN
>> ullConstants;
>>
>> public class KuduProcess {
>> private static Logger logger = LoggerFactory.getLogger(KuduPr
>> ocess.class);
>> private KuduTable table;
>> private KuduSession session;
>>
>> public static void upsertKudu(JavaRDD> rdd,
>> String host, String tableName) {

Re: Spark Streaming + Kudu

2018-03-06 Thread Ravi Kanth
Mike,

Can you clarify a bit on grabbing the jstack for the process? I launched my
Spark application and tried to get the pid using which I thought I can grab
jstack trace during hang. Unfortunately, I am not able to figure out
grabbing pid for Spark application.

Thanks,
Ravi

On 6 March 2018 at 18:36, Mike Percy  wrote:

> Thanks Ravi. Would you mind attaching the output of jstack on the process
> during this hang? That would show what the Kudu client threads are doing,
> as what we are seeing here is just the netty boss thread.
>
> Mike
>
> On Tue, Mar 6, 2018 at 8:52 AM, Ravi Kanth 
> wrote:
>
>>
>> Yes, I have debugged to find the root cause. Every logger before "table
>> = client.openTable(tableName);" is executing fine and exactly at the
>> point of opening the table, it is throwing the below exception and nothing
>> is being executed after that. Still the Spark batches are being processed
>> and at opening the table is failing. I tried catching it with no luck.
>> Please find below the exception.
>>
>> 8/02/23 00:16:30 ERROR client.TabletClient: [Peer
>> bd91f34d456a4eccaae50003c90f0fb2] Unexpected exception from downstream
>> on [id: 0x6e13b01f]
>> java.net.ConnectException: Connection refused:
>> kudu102.dev.sac.int.threatmetrix.com/10.112.3.12:7050
>> at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>> at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl
>> .java:717)
>> at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
>> .nio.NioClientBoss.connect(NioClientBoss.java:152)
>> at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
>> .nio.NioClientBoss.processSelectedKeys(NioClientBoss.java:105)
>> at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
>> .nio.NioClientBoss.process(NioClientBoss.java:79)
>> at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
>> .nio.AbstractNioSelector.run(AbstractNioSelector.java:337)
>> at org.apache.kudu.client.shaded.org.jboss.netty.channel.socket
>> .nio.NioClientBoss.run(NioClientBoss.java:42)
>> at org.apache.kudu.client.shaded.org.jboss.netty.util.ThreadRen
>> amingRunnable.run(ThreadRenamingRunnable.java:108)
>> at org.apache.kudu.client.shaded.org.jboss.netty.util.internal.
>> DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPool
>> Executor.java:1142)
>> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoo
>> lExecutor.java:617)
>> at java.lang.Thread.run(Thread.java:745)
>>
>>
>> Thanks,
>> Ravi
>>
>> On 5 March 2018 at 23:52, Mike Percy  wrote:
>>
>>> Have you considered checking your session error count or pending errors
>>> in your while loop every so often? Can you identify where your code is
>>> hanging when the connection is lost (what line)?
>>>
>>> Mike
>>>
>>> On Mon, Mar 5, 2018 at 9:08 PM, Ravi Kanth 
>>> wrote:
>>>
 In addition to my previous comment, I raised a support ticket for this
 issue with Cloudera and one of the support person mentioned below,

 *"Thank you for clarifying, The exceptions are logged but not re-thrown
 to an upper layer, so that explains why the Spark application is not aware
 of the underlying error."*

 On 5 March 2018 at 21:02, Ravi Kanth  wrote:

> Mike,
>
> Thanks for the information. But, once the connection to any of the
> Kudu servers is lost then there is no way I can have a control on the
> KuduSession object and so with getPendingErrors(). The KuduClient in this
> case is becoming a zombie and never returned back till the connection is
> properly established. I tried doing all that you have suggested with no
> luck. Attaching my KuduClient code.
>
> package org.dwh.streaming.kudu.sparkkudustreaming;
>
> import java.util.HashMap;
> import java.util.Iterator;
> import java.util.Map;
> import org.apache.hadoop.util.ShutdownHookManager;
> import org.apache.kudu.client.*;
> import org.apache.spark.api.java.JavaRDD;
> import org.slf4j.Logger;
> import org.slf4j.LoggerFactory;
> import org.dwh.streaming.kudu.sparkkudustreaming.constants.SpecialN
> ullConstants;
>
> public class KuduProcess {
> private static Logger logger = LoggerFactory.getLogger(KuduPr
> ocess.class);
> private KuduTable table;
> private KuduSession session;
>
> public static void upsertKudu(JavaRDD> rdd, String
> host, String tableName) {
> rdd.foreachPartition(iterator -> {
> RowErrorsAndOverflowStatus errors = upsertOpIterator(iterator,
> tableName, host);
> int errorCount = errors.getRowErrors().length;
> if(errorCount > 0){
> throw new RuntimeException("Failed to write " + errorCount + "
> messages into Kudu");
> }
> });
> }

Re: Spark Streaming + Kudu

2018-03-06 Thread Ravi Kanth
Yes, I have debugged to find the root cause. Every logger before "table =
client.openTable(tableName);" is executing fine and exactly at the point of
opening the table, it is throwing the below exception and nothing is being
executed after that. Still the Spark batches are being processed and at
opening the table is failing. I tried catching it with no luck. Please find
below the exception.

8/02/23 00:16:30 ERROR client.TabletClient: [Peer
bd91f34d456a4eccaae50003c90f0fb2] Unexpected exception from downstream on
[id: 0x6e13b01f]
java.net.ConnectException: Connection refused:
kudu102.dev.sac.int.threatmetrix.com/10.112.3.12:7050
at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
at
sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
at
org.apache.kudu.client.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.connect(NioClientBoss.java:152)
at
org.apache.kudu.client.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.processSelectedKeys(NioClientBoss.java:105)
at
org.apache.kudu.client.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.process(NioClientBoss.java:79)
at
org.apache.kudu.client.shaded.org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:337)
at
org.apache.kudu.client.shaded.org.jboss.netty.channel.socket.nio.NioClientBoss.run(NioClientBoss.java:42)
at
org.apache.kudu.client.shaded.org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
at
org.apache.kudu.client.shaded.org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)


Thanks,
Ravi

On 5 March 2018 at 23:52, Mike Percy  wrote:

> Have you considered checking your session error count or pending errors in
> your while loop every so often? Can you identify where your code is hanging
> when the connection is lost (what line)?
>
> Mike
>
> On Mon, Mar 5, 2018 at 9:08 PM, Ravi Kanth 
> wrote:
>
>> In addition to my previous comment, I raised a support ticket for this
>> issue with Cloudera and one of the support person mentioned below,
>>
>> *"Thank you for clarifying, The exceptions are logged but not re-thrown
>> to an upper layer, so that explains why the Spark application is not aware
>> of the underlying error."*
>>
>> On 5 March 2018 at 21:02, Ravi Kanth  wrote:
>>
>>> Mike,
>>>
>>> Thanks for the information. But, once the connection to any of the Kudu
>>> servers is lost then there is no way I can have a control on the
>>> KuduSession object and so with getPendingErrors(). The KuduClient in this
>>> case is becoming a zombie and never returned back till the connection is
>>> properly established. I tried doing all that you have suggested with no
>>> luck. Attaching my KuduClient code.
>>>
>>> package org.dwh.streaming.kudu.sparkkudustreaming;
>>>
>>> import java.util.HashMap;
>>> import java.util.Iterator;
>>> import java.util.Map;
>>> import org.apache.hadoop.util.ShutdownHookManager;
>>> import org.apache.kudu.client.*;
>>> import org.apache.spark.api.java.JavaRDD;
>>> import org.slf4j.Logger;
>>> import org.slf4j.LoggerFactory;
>>> import org.dwh.streaming.kudu.sparkkudustreaming.constants.SpecialN
>>> ullConstants;
>>>
>>> public class KuduProcess {
>>> private static Logger logger = LoggerFactory.getLogger(KuduPr
>>> ocess.class);
>>> private KuduTable table;
>>> private KuduSession session;
>>>
>>> public static void upsertKudu(JavaRDD> rdd, String
>>> host, String tableName) {
>>> rdd.foreachPartition(iterator -> {
>>> RowErrorsAndOverflowStatus errors = upsertOpIterator(iterator,
>>> tableName, host);
>>> int errorCount = errors.getRowErrors().length;
>>> if(errorCount > 0){
>>> throw new RuntimeException("Failed to write " + errorCount + " messages
>>> into Kudu");
>>> }
>>> });
>>> }
>>> private static RowErrorsAndOverflowStatus 
>>> upsertOpIterator(Iterator>> Object>> iter, String tableName, String host) {
>>> try {
>>> AsyncKuduClient asyncClient = KuduConnection.getAsyncClient(host);
>>> KuduClient client = asyncClient.syncClient();
>>> table = client.openTable(tableName);
>>> session = client.newSession();
>>> session.setFlushMode(SessionConfiguration.FlushMode.AUTO_FLU
>>> SH_BACKGROUND);
>>> while (iter.hasNext()) {
>>> upsertOp(iter.next());
>>> }
>>> } catch (KuduException e) {
>>> logger.error("Exception in upsertOpIterator method", e);
>>> }
>>> finally{
>>> try {
>>> session.close();
>>> } catch (KuduException e) {
>>> logger.error("Exception in Connection close", e);
>>> }
>>> }
>>> return session.getPendingErrors();-> Once,
>>> the connection is lost, this part of the code never gets called and the
>>> Spark 

Re: Problems connecting form Spark

2018-03-06 Thread William Berkeley
In each case the problem is that some part of your application can't find
the leader master of the Kudu cluster:

org.apache.kudu.client.NoLeaderFoundException: Master config (*172.17.0.43:7077
*) has no leader.
org.apache.kudu.client.NoLeaderFoundException: Master config (
*localhost:7051*) has no leader.

I think you're seeing these errors for two reasons:

1. Are you using multi-master? The first exception shows you specified one
remote master. If your cluster has multiple masters, you should specify all
of them. If you specify only one, and it's not the leader master, then
connecting to it will fail. You can check which master is the leader by
going to the /masters page on the web ui of any master.

2. In the "standalone" case, the Spark tasks are being distributed to
executors and fail there:

Lost task 1.0 in stage 0.0 (TID 1, tt-slave-2.novalocal, executor 1)

You've specified the master address as localhost. That address is passed
as-is to executors. Any task on an executor that doesn't have the leader
master locally at port 7051 will fail to connect to the leader master.
Getting the column names doesn't fail as that doesn't generate tasks sent
to remote executors.

I make this mistake all the time while playing with kudu-spark :)

-Will





On Mon, Mar 5, 2018 at 4:14 PM, Mac Noland  wrote:

> Any chance you can try spark2-shell with Kudu 1.6 and then re-try your
> tests?
>
> spark-shell --packages org.apache.kudu:kudu-spark2_2.11:1.6.0
>
> On Fri, Mar 2, 2018 at 5:02 AM, Saúl Nogueras  wrote:
>
>> I cannot properly connect to Kudu from Spark, error says “Kudu master has
>> no leader”
>>
>>- CDH 5.14
>>- Kudu 1.6
>>- Spark 1.6.0 standalone and 2.2.0
>>
>> When I use Impala in HUE to create and query kudu tables, it works
>> flawlessly.
>>
>> However, connecting from Spark throws some errors I cannot decipher.
>>
>> I have tried using both pyspark and spark-shell. With spark shell I had
>> to use spark 1.6 instead of 2.2 because some maven dependencies problems,
>> that I have localized but not been able to fix. More info here.
>> --
>> Case 1: using pyspark2 (Spark 2.2.0)
>>
>> $ pyspark2 --master yarn --jars 
>> /opt/cloudera/parcels/CDH-5.14.0-1.cdh5.14.0.p0.24/lib/kudu/kudu-spark2_2.11.jar
>>
>> > df = 
>> > sqlContext.read.format('org.apache.kudu.spark.kudu').options(**{"kudu.master":"172.17.0.43:7077",
>> >  "kudu.table":"impala::default.test"}).load()
>>
>> 18/03/02 10:23:27 WARN client.ConnectToCluster: Error receiving response 
>> from 172.17.0.43:7077
>> org.apache.kudu.client.RecoverableException: [peer master-172.17.0.43:7077] 
>> encountered a read timeout; closing the channel
>> at 
>> org.apache.kudu.client.Connection.exceptionCaught(Connection.java:412)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:112)
>> at 
>> org.apache.kudu.client.Connection.handleUpstream(Connection.java:239)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.SimpleChannelUpstreamHandler.exceptionCaught(SimpleChannelUpstreamHandler.java:153)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:112)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.Channels.fireExceptionCaught(Channels.java:536)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.handler.timeout.ReadTimeoutHandler.readTimedOut(ReadTimeoutHandler.java:236)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.handler.timeout.ReadTimeoutHandler$ReadTimeoutTask$1.run(ReadTimeoutHandler.java:276)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.socket.ChannelRunnableWrapper.run(ChannelRunnableWrapper.java:40)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.AbstractNioSelector.processTaskQueue(AbstractNioSelector.java:391)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:315)
>> at 
>> org.apache.kudu.shaded.org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:89)
>> at 
>>