flume-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Brock Noland <br...@cloudera.com>
Subject Re: Lock contention in FileChannel
Date Wed, 14 Aug 2013 02:54:51 GMT
Gotcha. When you run tge test what is tye disk utilization percentage?
Iostat can be used for this.
On Aug 13, 2013 9:47 PM, "Pankaj Gupta" <pankaj@brightroll.com> wrote:

> Those are the boxes we want to collect data from. They run flume and send
> data through their avro sinks to the avro source on this box. We are
> getting data at a pretty good rate and the problem is in fact that the
> events don't drain from the FileChannel fast enough and the channel fill
> percentage keeps getting higher.
>
>
> On Tue, Aug 13, 2013 at 7:41 PM, Brock Noland <brock@cloudera.com> wrote:
>
>> What is sending the events to the avro source?
>> On Aug 13, 2013 9:34 PM, "Pankaj Gupta" <pankaj@brightroll.com> wrote:
>>
>>> Here's the config:
>>> # define channels, one for each disk
>>>
>>>
>>>
>>>
>>> agent1.channels.ch1.type = FILE
>>> agent1.channels.ch1.checkpointDir = /flume1/checkpoint
>>> agent1.channels.ch1.dataDirs = /flume1/data
>>> agent1.channels.ch1.maxFileSize = 375809638400
>>> agent1.channels.ch1.capacity = 75000000
>>> agent1.channels.ch1.transactionCapacity = 4000
>>>
>>> agent1.channels.ch2.type = FILE
>>> agent1.channels.ch2.checkpointDir = /flume2/checkpoint
>>> agent1.channels.ch2.dataDirs = /flume2/data
>>> agent1.channels.ch2.maxFileSize = 375809638400
>>> agent1.channels.ch2.capacity = 75000000
>>> agent1.channels.ch2.transactionCapacity = 4000
>>>
>>>
>>>
>>> # Define an Avro source named avroSource1
>>> # Each sink can connect to only one channel.
>>> # Connect it to channel ch1. Load balance it to 2 avroSinks
>>>
>>>
>>> agent1.sources.avroSource1.channels = ch1
>>> agent1.sources.avroSource1.type = avro
>>> agent1.sources.avroSource1.bind = 0.0.0.0
>>> agent1.sources.avroSource1.port = <port>
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink1-1-1.type = avro
>>> agent1.sinks.avroSink1-1-1.channel = ch1
>>> agent1.sinks.avroSink1-1-1.hostname = <hostname>
>>> agent1.sinks.avroSink1-1-1.port = <port>
>>> agent1.sinks.avroSink1-1-1.connect-timeout = 300000
>>> agent1.sinks.avroSink1-1-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink1-2-1.type = avro
>>> agent1.sinks.avroSink1-2-1.channel = ch1
>>> agent1.sinks.avroSink1-2-1.hostname = <hostname>
>>> agent1.sinks.avroSink1-2-1.port = <port>
>>> agent1.sinks.avroSink1-2-1.connect-timeout = 300000
>>> agent1.sinks.avroSink1-2-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink1-3-1.type = avro
>>> agent1.sinks.avroSink1-3-1.channel = ch1
>>> agent1.sinks.avroSink1-3-1.hostname = <hostname>
>>> agent1.sinks.avroSink1-3-1.port = <port>
>>> agent1.sinks.avroSink1-3-1.connect-timeout = 300000
>>> agent1.sinks.avroSink1-3-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink1-4-1.type = avro
>>> agent1.sinks.avroSink1-4-1.channel = ch1
>>> agent1.sinks.avroSink1-4-1.hostname = <hostname>
>>> agent1.sinks.avroSink1-4-1.port = <port>
>>> agent1.sinks.avroSink1-4-1.connect-timeout = 300000
>>> agent1.sinks.avroSink1-4-1.batch-size = 4000
>>>
>>>
>>>
>>> #Add the sink groups; load-balance between each group of sinks which
>>> round robin between different hops
>>> agent1.sinkgroups.group1.sinks = avroSink1-1-1 avroSink1-2-1
>>> avroSink1-3-1 avroSink1-4-1
>>> agent1.sinkgroups.group1.processor.type = load_balance
>>> agent1.sinkgroups.group1.processor.selector = ROUND_ROBIN
>>> agent1.sinkgroups.group1.processor.backoff = true
>>>
>>>
>>> #End of set
>>>
>>> # Define an Avro source named avroSource2
>>> # Each sink can connect to only one channel.
>>> # Connect it to channel ch2. Load balance it to 2 avroSinks
>>>
>>>
>>> agent1.sources.avroSource2.channels = ch2
>>> agent1.sources.avroSource2.type = avro
>>> agent1.sources.avroSource2.bind = 0.0.0.0
>>> agent1.sources.avroSource2.port = <port>
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink2-1-1.type = avro
>>> agent1.sinks.avroSink2-1-1.channel = ch2
>>> agent1.sinks.avroSink2-1-1.hostname = <hostname>
>>> agent1.sinks.avroSink2-1-1.port = <port>
>>> agent1.sinks.avroSink2-1-1.connect-timeout = 300000
>>> agent1.sinks.avroSink2-1-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink2-2-1.type = avro
>>> agent1.sinks.avroSink2-2-1.channel = ch2
>>> agent1.sinks.avroSink2-2-1.hostname = <hostname>
>>> agent1.sinks.avroSink2-2-1.port = <port>
>>> agent1.sinks.avroSink2-2-1.connect-timeout = 300000
>>> agent1.sinks.avroSink2-2-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink2-3-1.type = avro
>>> agent1.sinks.avroSink2-3-1.channel = ch2
>>> agent1.sinks.avroSink2-3-1.hostname = <hostname>
>>> agent1.sinks.avroSink2-3-1.port = <port>
>>> agent1.sinks.avroSink2-3-1.connect-timeout = 300000
>>> agent1.sinks.avroSink2-3-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink2-4-1.type = avro
>>> agent1.sinks.avroSink2-4-1.channel = ch2
>>> agent1.sinks.avroSink2-4-1.hostname = <hostname>
>>> agent1.sinks.avroSink2-4-1.port = <port>
>>> agent1.sinks.avroSink2-4-1.connect-timeout = 300000
>>> agent1.sinks.avroSink2-4-1.batch-size = 4000
>>>
>>>
>>>
>>> #Add the sink groups; load-balance between each group of sinks which
>>> round robin between different hops
>>> agent1.sinkgroups.group2.sinks = avroSink2-1-1 avroSink2-2-1
>>> avroSink2-3-1 avroSink2-4-1
>>> agent1.sinkgroups.group2.processor.type = load_balance
>>> agent1.sinkgroups.group2.processor.selector = ROUND_ROBIN
>>> agent1.sinkgroups.group2.processor.backoff = true
>>>
>>>
>>> #End of set
>>>
>>> # Define an Avro source named avroSource3
>>> # Each sink can connect to only one channel.
>>> # Connect it to channel ch1. Load balance it to 2 avroSinks
>>>
>>>
>>> agent1.sources.avroSource3.channels = ch1
>>> agent1.sources.avroSource3.type = avro
>>> agent1.sources.avroSource3.bind = 0.0.0.0
>>> agent1.sources.avroSource3.port = <port>
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink3-1-1.type = avro
>>> agent1.sinks.avroSink3-1-1.channel = ch1
>>> agent1.sinks.avroSink3-1-1.hostname = <hostname>
>>> agent1.sinks.avroSink3-1-1.port = <port>
>>> agent1.sinks.avroSink3-1-1.connect-timeout = 300000
>>> agent1.sinks.avroSink3-1-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink3-2-1.type = avro
>>> agent1.sinks.avroSink3-2-1.channel = ch1
>>> agent1.sinks.avroSink3-2-1.hostname = <hostname>
>>> agent1.sinks.avroSink3-2-1.port = <port>
>>> agent1.sinks.avroSink3-2-1.connect-timeout = 300000
>>> agent1.sinks.avroSink3-2-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink3-3-1.type = avro
>>> agent1.sinks.avroSink3-3-1.channel = ch1
>>> agent1.sinks.avroSink3-3-1.hostname = <hostname>
>>> agent1.sinks.avroSink3-3-1.port = <port>
>>> agent1.sinks.avroSink3-3-1.connect-timeout = 300000
>>> agent1.sinks.avroSink3-3-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink3-4-1.type = avro
>>> agent1.sinks.avroSink3-4-1.channel = ch1
>>> agent1.sinks.avroSink3-4-1.hostname = <hostname>
>>> agent1.sinks.avroSink3-4-1.port = <port>
>>> agent1.sinks.avroSink3-4-1.connect-timeout = 300000
>>> agent1.sinks.avroSink3-4-1.batch-size = 4000
>>>
>>>
>>>
>>> #Add the sink groups; load-balance between each group of sinks which
>>> round robin between different hops
>>> agent1.sinkgroups.group3.sinks = avroSink3-1-1 avroSink3-2-1
>>> avroSink3-3-1 avroSink3-4-1
>>> agent1.sinkgroups.group3.processor.type = load_balance
>>> agent1.sinkgroups.group3.processor.selector = ROUND_ROBIN
>>> agent1.sinkgroups.group3.processor.backoff = true
>>>
>>>
>>> #End of set
>>>
>>> # Define an Avro source named avroSource4
>>> # Each sink can connect to only one channel.
>>> # Connect it to channel ch2. Load balance it to 2 avroSinks
>>>
>>>
>>> agent1.sources.avroSource4.channels = ch2
>>> agent1.sources.avroSource4.type = avro
>>> agent1.sources.avroSource4.bind = 0.0.0.0
>>> agent1.sources.avroSource4.port = <port>
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink4-1-1.type = avro
>>> agent1.sinks.avroSink4-1-1.channel = ch2
>>> agent1.sinks.avroSink4-1-1.hostname = <hostname>
>>> agent1.sinks.avroSink4-1-1.port = <port>
>>> agent1.sinks.avroSink4-1-1.connect-timeout = 300000
>>> agent1.sinks.avroSink4-1-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink4-2-1.type = avro
>>> agent1.sinks.avroSink4-2-1.channel = ch2
>>> agent1.sinks.avroSink4-2-1.hostname = <hostname>
>>> agent1.sinks.avroSink4-2-1.port = <port>
>>> agent1.sinks.avroSink4-2-1.connect-timeout = 300000
>>> agent1.sinks.avroSink4-2-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink4-3-1.type = avro
>>> agent1.sinks.avroSink4-3-1.channel = ch2
>>> agent1.sinks.avroSink4-3-1.hostname = <hostname>
>>> agent1.sinks.avroSink4-3-1.port = <port>
>>> agent1.sinks.avroSink4-3-1.connect-timeout = 300000
>>> agent1.sinks.avroSink4-3-1.batch-size = 4000
>>>
>>>
>>>
>>>
>>> agent1.sinks.avroSink4-4-1.type = avro
>>> agent1.sinks.avroSink4-4-1.channel = ch2
>>> agent1.sinks.avroSink4-4-1.hostname = <hostname>
>>> agent1.sinks.avroSink4-4-1.port = <port>
>>> agent1.sinks.avroSink4-4-1.connect-timeout = 300000
>>> agent1.sinks.avroSink4-4-1.batch-size = 4000
>>>
>>>
>>>
>>> #Add the sink groups; load-balance between each group of sinks which
>>> round robin between different hops
>>> agent1.sinkgroups.group4.sinks = avroSink4-1-1 avroSink4-2-1
>>> avroSink4-3-1 avroSink4-4-1
>>> agent1.sinkgroups.group4.processor.type = load_balance
>>> agent1.sinkgroups.group4.processor.selector = ROUND_ROBIN
>>> agent1.sinkgroups.group4.processor.backoff = true
>>>
>>>
>>> #End of set
>>>
>>>
>>>
>>> # Finally, now that we've defined all of our components, tell
>>> # agent1 which ones we want to activate.
>>> agent1.channels = ch1 ch2
>>> agent1.sources = avroSource1 avroSource2 avroSource3 avroSource4
>>> agent1.sinks = avroSink1-1-1 avroSink1-2-1 avroSink1-3-1 avroSink1-4-1
>>> avroSink2-1-1 avroSink2-2-1 avroSink2-3-1 avroSink2-4-1 avroSink3-1-1
>>> avroSink3-2-1 avroSink3-3-1 avroSink3-4-1 avroSink4-1-1 avroSink4-2-1
>>> avroSink4-3-1 avroSink4-4-1
>>> agent1.sinkgroups = group1 group2 group3 group4
>>>
>>> There are two disks on the box /flume1 and /flume2
>>> 32 cores of CPU, 32 GB RAM, 1GbPS ethernet.
>>>
>>>
>>> On Tue, Aug 13, 2013 at 7:22 PM, Brock Noland <brock@cloudera.com>wrote:
>>>
>>>> Can you share your conf file?
>>>> On Aug 13, 2013 9:19 PM, "Hari Shreedharan" <hshreedharan@cloudera.com>
>>>> wrote:
>>>>
>>>>>  Even though the writes are done per batch, they don't go to disk
>>>>> rightaway - commits are the only ones which actually cause an fsync - which
>>>>> is when writes actually go to disk.
>>>>>
>>>>>
>>>>> Thanks,
>>>>> Hari
>>>>>
>>>>> On Tuesday, August 13, 2013 at 7:06 PM, Pankaj Gupta wrote:
>>>>>
>>>>> Looking at the code it seems like the lock and the i/o is done per
>>>>> event and not for a batch. Is that correct? If that is the case then it
>>>>> seems like there is a lot of overhead per event. The throughput I'm seeing
>>>>> is 1 - 1.5 MBps per disk which is way below the sequential read/write
>>>>> capacity of the disk which is easily over 50MBps. Adding more sinks doesn't
>>>>> help, they just block waiting for the queue to become free. CPU usage is
>>>>> 20%, there is enough RAM for page cache so that no read is going to disk.
>>>>> The queue seems to be the bottleneck. What is the throughput I should
>>>>> expect per disk?
>>>>>
>>>>>
>>>>> On Tue, Aug 13, 2013 at 5:51 PM, Brock Noland <brock@cloudera.com>wrote:
>>>>>
>>>>> The lock is per file. Adding more directories to the channel will
>>>>> cause more files to be created. Of course you'll need additional disks
>>>>> behind those directories to see any performance increase.
>>>>>
>>>>>
>>>>> On Tue, Aug 13, 2013 at 7:14 PM, Hari Shreedharan <
>>>>> hshreedharan@cloudera.com> wrote:
>>>>>
>>>>>  Yes, IO is done inside locks to avoid multiple takes and puts getting
>>>>> written out at the same time. Even though Java makes sure the writes are
>>>>> serialized, Flume still needs to keep track of some counters etc, so the
>>>>> lock is required. Note that the lock you are talking about is  in the
>>>>> LogFile class, which represents a single file - so even if the write is
>>>>> inside that lock (which is also inside that class itself) that  does not
>>>>> cause any contention - because the lock is just preventing two IO ops to
>>>>> happen at the same time.
>>>>>
>>>>>
>>>>> Thanks,
>>>>> Hari
>>>>>
>>>>> On Tuesday, August 13, 2013 at 5:01 PM, Pankaj Gupta wrote:
>>>>>
>>>>> It seems like some i/o is done inside the lock, which means that time
>>>>> for taking a lock is proportional to the time for i/o and thus it becomes a
>>>>> problem. I apologize in advance if I am wrong but the call stack and
>>>>> behavior I'm seeing seems to suggest that. Specifically, it seems that we
>>>>> do a write while inside take:
>>>>> "SinkRunner-PollingRunner-LoadBalancingSinkProcessor" prio=10
>>>>> tid=0x00007f857338c800 nid=0x404a runnable [0x00007f821b2f1000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>>         at sun.nio.ch.NativeThread.current(Native Method)
>>>>>         at sun.nio.ch.NativeThreadSet.add(NativeThreadSet.java:27)
>>>>>         at sun.nio.ch.FileChannelImpl.write(FileChannelImpl.java:194)
>>>>>         - locked <0x00000005190ec998> (a java.lang.Object)
>>>>>         at
>>>>> org.apache.flume.channel.file.LogFile$Writer.write(LogFile.java:247)
>>>>>         at
>>>>> org.apache.flume.channel.file.LogFile$Writer.take(LogFile.java:212)
>>>>>         - locked <0x0000000519111590> (a
>>>>> org.apache.flume.channel.file.LogFileV3$Writer)
>>>>>         at org.apache.flume.channel.file.Log.take(Log.java:550)
>>>>>         at
>>>>> org.apache.flume.channel.file.FileChannel$FileBackedTransaction.doTake(FileChannel.java:499)
>>>>>         at
>>>>> org.apache.flume.channel.BasicTransactionSemantics.take(BasicTransactionSemantics.java:113)
>>>>>         at
>>>>> org.apache.flume.channel.BasicChannelSemantics.take(BasicChannelSemantics.java:95)
>>>>>         at
>>>>> org.apache.flume.sink.AbstractRpcSink.process(AbstractRpcSink.java:330)
>>>>>         at
>>>>> org.apache.flume.sink.LoadBalancingSinkProcessor.process(LoadBalancingSinkProcessor.java:154)
>>>>>         at
>>>>> org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
>>>>>         at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>>
>>>>>
>>>>> On Tue, Aug 13, 2013 at 4:39 PM, Hari Shreedharan <
>>>>> hshreedharan@cloudera.com> wrote:
>>>>>
>>>>> Since the channel is designed to make sure that events are not
>>>>> duplicated to multiple sinks, and to protect against corruption due to
>>>>> concurrency issues, we do not need the locking in the channel's flume event
>>>>> queue. It is unlikely that locking is what is causing performance issues
>>>>> because the channel is heavily I/O bound. If you take a series of thread
>>>>> dumps, you will probably see that those threads are moving forward and the
>>>>> ones reading/writing from/to disk are the ones which are slower. These
>>>>> locks are unlikely to hit performance much.
>>>>>
>>>>> Thanks,
>>>>> Hari
>>>>>
>>>>> On Tuesday, August 13, 2013 at 4:13 PM, Pankaj Gupta wrote:
>>>>>
>>>>> Hi,
>>>>>
>>>>> Spent some more time debugging issues with FileChannel. The problem
>>>>> seems to lock contention reading from FlumeEventQueue:
>>>>>
>>>>> I see a lot of threads like this:
>>>>> "SinkRunner-PollingRunner-LoadBalancingSinkProcessor" prio=10
>>>>> tid=0x00007f857b378800 nid=0x404d waiting for monitor entry
>>>>> [0x00007f821afee000]
>>>>>    java.lang.Thread.State: BLOCKED (on object monitor)
>>>>>         at
>>>>> org.apache.flume.channel.file.FlumeEventQueue.removeHead(FlumeEventQueue.java:117)
>>>>>         - waiting to lock <0x0000000518ee4c90> (a
>>>>> org.apache.flume.channel.file.FlumeEventQueue)
>>>>>         at
>>>>> org.apache.flume.channel.file.FileChannel$FileBackedTransaction.doTake(FileChannel.java:492)
>>>>>         at
>>>>> org.apache.flume.channel.BasicTransactionSemantics.take(BasicTransactionSemantics.java:113)
>>>>>         at
>>>>> org.apache.flume.channel.BasicChannelSemantics.take(BasicChannelSemantics.java:95)
>>>>>         at
>>>>> org.apache.flume.sink.AbstractRpcSink.process(AbstractRpcSink.java:330)
>>>>>         at
>>>>> org.apache.flume.sink.LoadBalancingSinkProcessor.process(LoadBalancingSinkProcessor.java:154)
>>>>>         at
>>>>> org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
>>>>>         at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>>
>>>>> I have two file channels and 8 Avro Sinks per file channel. I added
>>>>> more sinks because they weren't draining fast enough. It seems like they
>>>>> send the batch then wait for an ack before sending again, thus sends are
>>>>> not pipelined and having more sinks seemed like a good way of getting some
>>>>> parallelism.
>>>>>
>>>>> Here's the full stack trace:
>>>>> 2013-08-13 15:30:32
>>>>> Full thread dump Java HotSpot(TM) 64-Bit Server VM (20.13-b02 mixed
>>>>> mode):
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f834c02c000 nid=0x41a2 waiting on condition [0x00007f8210341000]
>>>>>    java.lang.Thread.State: WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x000000056d932120> (a
>>>>> java.util.concurrent.CountDownLatch$Sync)
>>>>>  at java.util.concurrent.locks.LockSupport.park(LockSupport.java:156)
>>>>> at
>>>>> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:811)
>>>>>  at
>>>>> java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:969)
>>>>> at
>>>>> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1281)
>>>>>  at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:207)
>>>>> at org.apache.avro.ipc.CallFuture.await(CallFuture.java:141)
>>>>>  at org.apache.avro.ipc.Requestor.request(Requestor.java:150)
>>>>> at org.apache.avro.ipc.Requestor.request(Requestor.java:129)
>>>>>  at
>>>>> org.apache.avro.ipc.specific.SpecificRequestor.invoke(SpecificRequestor.java:68)
>>>>> at $Proxy7.appendBatch(Unknown Source)
>>>>>  at
>>>>> org.apache.flume.api.NettyAvroRpcClient$2.call(NettyAvroRpcClient.java:314)
>>>>> at
>>>>> org.apache.flume.api.NettyAvroRpcClient$2.call(NettyAvroRpcClient.java:310)
>>>>>  at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>>>>> at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Attach Listener" daemon prio=10 tid=0x00007f85fc016000 nid=0x41a1
>>>>> waiting on condition [0x0000000000000000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f834c02a800 nid=0x4173 waiting on condition [0x00007f8210544000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>>  at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x00000005194c3340> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f836000f800 nid=0x4172 waiting on condition [0x00007f8210645000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x00000005191a16d8> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f82ec01f000 nid=0x4164 waiting on condition [0x00007f8210746000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x0000000519174e08> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f8374023000 nid=0x4163 waiting on condition [0x00007f8210847000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x0000000519174e98> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f82ec030000 nid=0x415d waiting on condition [0x00007f8210948000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x00000005195f9670> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f8374021800 nid=0x415b waiting on condition [0x00007f8210a49000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x0000000519174ce8> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f836000f000 nid=0x415a waiting on condition [0x00007f8210b4a000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x00000005191a1918> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f834c024000 nid=0x4154 waiting on condition [0x00007f8210c4b000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x00000005191e7f60> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-144-thread-41" prio=10 tid=0x00007f85d8466800 nid=0x4153
>>>>> runnable [0x00007f8210e4d000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x000000054a291760> (a sun.nio.ch.Util$2)
>>>>> - locked <0x000000054a2917a8> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x000000054a2c5348> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f82ec029800 nid=0x4147 waiting on condition [0x00007f8210d4c000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x0000000519174f58> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-142-thread-41" prio=10 tid=0x00007f8584469800 nid=0x4146
>>>>> runnable [0x00007f8210f4e000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x0000000546e5dd40> (a sun.nio.ch.Util$2)
>>>>> - locked <0x0000000546e5dd28> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x0000000546ec1ea0> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-140-thread-41" prio=10 tid=0x00007f8588411000 nid=0x4118
>>>>> runnable [0x00007f821104f000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x0000000544f763d8> (a sun.nio.ch.Util$2)
>>>>> - locked <0x0000000544f763c0> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x0000000544ee55f8> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-139-thread-41" prio=10 tid=0x00007f85803ff000 nid=0x4111
>>>>> runnable [0x00007f8211150000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x000000054473d5c0> (a sun.nio.ch.Util$2)
>>>>> - locked <0x000000054473d5a8> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x000000054474d088> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-142-thread-40" prio=10 tid=0x00007f85843c7800 nid=0x4110
>>>>> runnable [0x00007f8211251000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x00000005437f4f88> (a sun.nio.ch.Util$2)
>>>>> - locked <0x00000005437f4f70> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x0000000543803478> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-140-thread-40" prio=10 tid=0x00007f858840f800 nid=0x410f
>>>>> runnable [0x00007f8211352000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x00000005422a9030> (a sun.nio.ch.Util$2)
>>>>> - locked <0x00000005422a9018> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x00000005422ca310> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-142-thread-39" prio=10 tid=0x00007f85843c6000 nid=0x410e
>>>>> runnable [0x00007f8211453000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x0000000541063d68> (a sun.nio.ch.Util$2)
>>>>> - locked <0x0000000541063d50> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x00000005410c8240> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-144-thread-40" prio=10 tid=0x00007f85d83c6000 nid=0x410d
>>>>> runnable [0x00007f8211554000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x0000000540dc1128> (a sun.nio.ch.Util$2)
>>>>> - locked <0x0000000540dc1110> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x0000000540c94928> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-144-thread-39" prio=10 tid=0x00007f85d83c4800 nid=0x410a
>>>>> runnable [0x00007f8211655000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x000000053fbbc3c8> (a sun.nio.ch.Util$2)
>>>>> - locked <0x000000053fbbc3b0> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x000000053f897d00> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-142-thread-38" prio=10 tid=0x00007f85843c4800 nid=0x4109
>>>>> runnable [0x00007f8211756000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x000000053f812878> (a sun.nio.ch.Util$2)
>>>>> - locked <0x000000053f812860> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x000000053f897cb8> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-144-thread-38" prio=10 tid=0x00007f85d83c3000 nid=0x40fd
>>>>> runnable [0x00007f8211857000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x000000053e6e5588> (a sun.nio.ch.Util$2)
>>>>> - locked <0x000000053e6e5570> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x000000053e714e70> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f8374020800 nid=0x40f3 waiting on condition [0x00007f8211958000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x0000000519175138> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f834c022000 nid=0x40f2 waiting on condition [0x00007f8211a59000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x0000000519174c28> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f82ec032000 nid=0x40f1 waiting on condition [0x00007f8211b5a000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x00000005191751c8> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-140-thread-39" prio=10 tid=0x00007f85883ad800 nid=0x40f0
>>>>> runnable [0x00007f8211c5b000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x0000000536579dc8> (a sun.nio.ch.Util$2)
>>>>> - locked <0x0000000536579db0> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x0000000536585830> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f836000d800 nid=0x40ef waiting on condition [0x00007f8211d5c000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x00000005191750a8> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-140-thread-38" prio=10 tid=0x00007f85883ac000 nid=0x40e1
>>>>> runnable [0x00007f8211e5d000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x00000005313943c0> (a sun.nio.ch.Util$2)
>>>>> - locked <0x00000005313943a8> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x00000005314e0fe0> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f837401f000 nid=0x40e0 waiting on condition [0x00007f8211f5e000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x000000051936f460> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f82ec028000 nid=0x40df waiting on condition [0x00007f821205f000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x000000051936dfa8> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-140-thread-37" prio=10 tid=0x00007f85883aa800 nid=0x40de
>>>>> runnable [0x00007f8212160000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x000000052dad4480> (a sun.nio.ch.Util$2)
>>>>> - locked <0x000000052dad4468> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x000000052da31528> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-139-thread-40" prio=10 tid=0x00007f85803fd800 nid=0x40dd
>>>>> runnable [0x00007f8212261000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x000000052c73c608> (a sun.nio.ch.Util$2)
>>>>> - locked <0x000000052c73c5f0> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x000000052c652608> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-140-thread-36" prio=10 tid=0x00007f85883a9000 nid=0x40dc
>>>>> runnable [0x00007f8212362000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x000000052b8da9b0> (a sun.nio.ch.Util$2)
>>>>> - locked <0x000000052b8da998> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x000000052b8fc590> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-139-thread-39" prio=10 tid=0x00007f85803fc000 nid=0x40db waiting
>>>>> for monitor entry [0x00007f8212462000]
>>>>>    java.lang.Thread.State: BLOCKED (on object monitor)
>>>>> at
>>>>> org.apache.flume.channel.file.FlumeEventQueue.addWithoutCommit(FlumeEventQueue.java:184)
>>>>> - waiting to lock <0x0000000518ee4c90> (a
>>>>> org.apache.flume.channel.file.FlumeEventQueue)
>>>>>  at
>>>>> org.apache.flume.channel.file.FileChannel$FileBackedTransaction.doPut(FileChannel.java:460)
>>>>> at
>>>>> org.apache.flume.channel.BasicTransactionSemantics.put(BasicTransactionSemantics.java:93)
>>>>>  at
>>>>> org.apache.flume.channel.BasicChannelSemantics.put(BasicChannelSemantics.java:80)
>>>>> at
>>>>> org.apache.flume.channel.ChannelProcessor.processEventBatch(ChannelProcessor.java:189)
>>>>>  at
>>>>> org.apache.flume.source.AvroSource.appendBatch(AvroSource.java:330)
>>>>> at sun.reflect.GeneratedMethodAccessor6.invoke(Unknown Source)
>>>>>  at
>>>>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>>>> at java.lang.reflect.Method.invoke(Method.java:597)
>>>>>  at
>>>>> org.apache.avro.ipc.specific.SpecificResponder.respond(SpecificResponder.java:88)
>>>>> at org.apache.avro.ipc.Responder.respond(Responder.java:149)
>>>>>  at
>>>>> org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.messageReceived(NettyServer.java:188)
>>>>> at
>>>>> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:75)
>>>>>  at
>>>>> org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.handleUpstream(NettyServer.java:173)
>>>>> at
>>>>> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
>>>>>  at
>>>>> org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:792)
>>>>> at
>>>>> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296)
>>>>>  at
>>>>> org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:321)
>>>>> at
>>>>> org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:303)
>>>>>  at
>>>>> org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:220)
>>>>> at
>>>>> org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:75)
>>>>>  at
>>>>> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
>>>>> at
>>>>> org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:559)
>>>>>  at
>>>>> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268)
>>>>> at
>>>>> org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:94)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.processSelectedKeys(AbstractNioWorker.java:364)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:238)
>>>>> at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>>  at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>> at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f836000b000 nid=0x40da waiting on condition [0x00007f8212564000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>>  at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x000000051936df18> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "Flume Avro RPC Client Call Invoker 1" daemon prio=10
>>>>> tid=0x00007f834c020000 nid=0x40d9 waiting on condition [0x00007f8212665000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x0000000518fe7ae8> (a
>>>>> java.util.concurrent.SynchronousQueue$TransferStack)
>>>>>  at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:196)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:424)
>>>>>  at
>>>>> java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:323)
>>>>> at
>>>>> java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-142-thread-37" prio=10 tid=0x00007f85843c3000 nid=0x40d8
>>>>> runnable [0x00007f8212766000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x00000005263d6bc0> (a sun.nio.ch.Util$2)
>>>>> - locked <0x00000005263d6ba8> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x00000005264b1a48> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-144-thread-37" prio=10 tid=0x00007f85d83c1800 nid=0x40d7
>>>>> runnable [0x00007f8212867000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x0000000525d1d0f8> (a sun.nio.ch.Util$2)
>>>>> - locked <0x0000000525d1d0e0> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x0000000525d2cdf8> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-144-thread-36" prio=10 tid=0x00007f85d83c0000 nid=0x40d6
>>>>> runnable [0x00007f8212968000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x0000000525a689f0> (a sun.nio.ch.Util$2)
>>>>> - locked <0x0000000525a689d8> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x0000000525a24570> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-140-thread-35" prio=10 tid=0x00007f85883a8000 nid=0x40d5
>>>>> runnable [0x00007f8212a69000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x0000000524e1e7f0> (a sun.nio.ch.Util$2)
>>>>> - locked <0x0000000524e1e7d8> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x0000000524cfd670> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-139-thread-38" prio=10 tid=0x00007f85803fa800 nid=0x40d4
>>>>> runnable [0x00007f8212b6a000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x0000000524d437c0> (a sun.nio.ch.Util$2)
>>>>> - locked <0x0000000524d437a8> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x0000000524ab3330> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>  at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>>> at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>>>  at java.lang.Thread.run(Thread.java:662)
>>>>>
>>>>> "pool-142-thread-36" prio=10 tid=0x00007f85843ad800 nid=0x40d3
>>>>> runnable [0x00007f8212c6b000]
>>>>>    java.lang.Thread.State: RUNNABLE
>>>>> at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>>>> at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:210)
>>>>>  at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>>> at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>>>  - locked <0x0000000524cf4140> (a sun.nio.ch.Util$2)
>>>>> - locked <0x0000000524cf4128> (a java.util.Collections$UnmodifiableSet)
>>>>>  - locked <0x0000000524ab32e8> (a sun.nio.ch.EPollSelectorImpl)
>>>>> at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.SelectorUtil.select(SelectorUtil.java:52)
>>>>> at
>>>>> org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:200)
>>>>>  at
>>>>> org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:38)
>>>>> at
>>>>> org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>>>>>
>>>>> ...

Mime
View raw message