hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Ryan Rawson <ryano...@gmail.com>
Subject Re: Hanging regionservers
Date Thu, 15 Jul 2010 23:32:59 GMT
Are you sure you dont have any GC issues?

Also you are aware that max.tasks.maximum is _per node_ right?  How
big are these machines?

-ryan

On Thu, Jul 15, 2010 at 4:30 PM, Jean-Daniel Cryans <jdcryans@apache.org> wrote:
> Nothing particular in that dump, and I'm not aware of any deadlock in
> 0.20.3, could we see the region server log?
>
> Thx
>
> J-D
>
> On Thu, Jul 15, 2010 at 4:17 PM, Luke Forehand
> <luke.forehand@networkedinsights.com> wrote:
>> First post evar!
>>
>> I have a 3 node cluster and have set mapred.tasktracker.map.tasks.maximum=4.  I am using Cloudera's CDH2 dist (hadoop 0.20.1 and hbase 0.20.3).  I am importing a 15GB sequence file into an HTable instance straight from the Mapper, no reduce phase.  At random times during the job, the RegionServer JVMs start to completely hang.  I started up jstatd on the servers and used VisualVM to profile the processes.  When I encounter the hang, I notice absolutely no problems with heap or garbage collection, the region servers just become vegetables within minutes of eachother.  This fact, along with the randomness of the hang, leads me to believe there is a dead lock due to a race condition somewhere in the code.  I suspect something similar to the following issues:
>>
>> http://issues.apache.org/jira/browse/HADOOP-5859
>> http://issues.apache.org/jira/browse/HBASE-667
>>
>> Here is a thread dump of one of the region server procs immediately after it went silent.  Would anybody that has more knowledge about the inner workings of hadoop/hbase IO have the time to help me confirm that this is the issue I am experiencing?
>>
>> 2010-07-15 16:33:26
>> Full thread dump Java HotSpot(TM) 64-Bit Server VM (14.2-b01 mixed mode):
>>
>> "RMI TCP Connection(73)-192.168.100.113" - Thread t@62011
>>   java.lang.Thread.State: TIMED_WAITING on com.sun.jmx.remote.internal.ArrayNotificationBuffer@bd5192
>>    at java.lang.Object.wait(Native Method)
>>    at com.sun.jmx.remote.internal.ArrayNotificationBuffer.fetchNotifications(ArrayNotificationBuffer.java:417)
>>    at com.sun.jmx.remote.internal.ArrayNotificationBuffer$ShareBuffer.fetchNotifications(ArrayNotificationBuffer.java:209)
>>    at com.sun.jmx.remote.internal.ServerNotifForwarder.fetchNotifs(ServerNotifForwarder.java:258)
>>    at javax.management.remote.rmi.RMIConnectionImpl$2.run(RMIConnectionImpl.java:1227)
>>    at javax.management.remote.rmi.RMIConnectionImpl$2.run(RMIConnectionImpl.java:1225)
>>    at javax.management.remote.rmi.RMIConnectionImpl.fetchNotifications(RMIConnectionImpl.java:1231)
>>    at sun.reflect.GeneratedMethodAccessor61.invoke(Unknown Source)
>>    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>    at java.lang.reflect.Method.invoke(Method.java:597)
>>    at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:305)
>>    at sun.rmi.transport.Transport$1.run(Transport.java:159)
>>    at java.security.AccessController.doPrivileged(Native Method)
>>    at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
>>    at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
>>    at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
>>    at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
>>    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:619)
>>
>>   Locked ownable synchronizers:
>>    - locked java.util.concurrent.locks.ReentrantLock$NonfairSync@40aa3008
>>
>> "RMI TCP Connection(72)-192.168.100.113" - Thread t@62010
>>   java.lang.Thread.State: RUNNABLE
>>    at java.net.SocketInputStream.socketRead0(Native Method)
>>    at java.net.SocketInputStream.read(SocketInputStream.java:129)
>>    at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
>>    at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
>>    - locked java.io.BufferedInputStream@72bda36f
>>    at java.io.FilterInputStream.read(FilterInputStream.java:66)
>>    at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:517)
>>    at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
>>    at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
>>    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:619)
>>
>>   Locked ownable synchronizers:
>>    - locked java.util.concurrent.locks.ReentrantLock$NonfairSync@350743
>>
>> "JMX server connection timeout 62009" - Thread t@62009
>>   java.lang.Thread.State: TIMED_WAITING on [I@5eebd2c6
>>    at java.lang.Object.wait(Native Method)
>>    at com.sun.jmx.remote.internal.ServerCommunicatorAdmin$Timeout.run(ServerCommunicatorAdmin.java:150)
>>    at java.lang.Thread.run(Thread.java:619)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "ResponseProcessor for block blk_1926230463847049982_2694658" - Thread t@61160
>>   java.lang.Thread.State: RUNNABLE
>>    at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>    at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>    at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>    at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>    - locked sun.nio.ch.Util$1@196fbfd0
>>    - locked java.util.Collections$UnmodifiableSet@7799fdbb
>>    - locked sun.nio.ch.EPollSelectorImpl@1ee13d55
>>    at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>    at org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332)
>>    at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
>>    at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
>>    at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
>>    at java.io.DataInputStream.readFully(DataInputStream.java:178)
>>    at java.io.DataInputStream.readLong(DataInputStream.java:399)
>>    at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2399)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "DataStreamer for file /hbase/.logs/dn01.colo.networkedinsights.com,60020,1279222293084/hlog.dat.1279228611023 block blk_1926230463847049982_2694658" - Thread t@61158
>>   java.lang.Thread.State: TIMED_WAITING on java.util.LinkedList@475b455c
>>    at java.lang.Object.wait(Native Method)
>>    at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2247)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "RMI TCP Connection(71)-192.168.100.113" - Thread t@53904
>>   java.lang.Thread.State: RUNNABLE
>>    at sun.management.ThreadImpl.dumpThreads0(Native Method)
>>    at sun.management.ThreadImpl.dumpAllThreads(ThreadImpl.java:374)
>>    at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>    at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>>    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>    at java.lang.reflect.Method.invoke(Method.java:597)
>>    at com.sun.jmx.mbeanserver.ConvertingMethod.invokeWithOpenReturn(ConvertingMethod.java:167)
>>    at com.sun.jmx.mbeanserver.MXBeanIntrospector.invokeM2(MXBeanIntrospector.java:96)
>>    at com.sun.jmx.mbeanserver.MXBeanIntrospector.invokeM2(MXBeanIntrospector.java:33)
>>    at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:208)
>>    at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:120)
>>    at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:262)
>>    at javax.management.StandardMBean.invoke(StandardMBean.java:391)
>>    at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:836)
>>    at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:761)
>>    at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1426)
>>    at javax.management.remote.rmi.RMIConnectionImpl.access$200(RMIConnectionImpl.java:72)
>>    at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1264)
>>    at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1359)
>>    at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:788)
>>    at sun.reflect.GeneratedMethodAccessor59.invoke(Unknown Source)
>>    at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>>    at java.lang.reflect.Method.invoke(Method.java:597)
>>    at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:305)
>>    at sun.rmi.transport.Transport$1.run(Transport.java:159)
>>    at java.security.AccessController.doPrivileged(Native Method)
>>    at sun.rmi.transport.Transport.serviceCall(Transport.java:155)
>>    at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:535)
>>    at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
>>    at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
>>    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:619)
>>
>>   Locked ownable synchronizers:
>>    - locked java.util.concurrent.locks.ReentrantLock$NonfairSync@5581a749
>>
>> "JMX server connection timeout 28949" - Thread t@28949
>>   java.lang.Thread.State: TIMED_WAITING on [I@1c6e538d
>>    at java.lang.Object.wait(Native Method)
>>    at com.sun.jmx.remote.internal.ServerCommunicatorAdmin$Timeout.run(ServerCommunicatorAdmin.java:150)
>>    at java.lang.Thread.run(Thread.java:619)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "RMI TCP Connection(70)-192.168.100.113" - Thread t@28947
>>   java.lang.Thread.State: RUNNABLE
>>    at java.net.SocketInputStream.socketRead0(Native Method)
>>    at java.net.SocketInputStream.read(SocketInputStream.java:129)
>>    at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
>>    at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
>>    - locked java.io.BufferedInputStream@513a56c5
>>    at java.io.FilterInputStream.read(FilterInputStream.java:66)
>>    at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:517)
>>    at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:790)
>>    at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:649)
>>    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:619)
>>
>>   Locked ownable synchronizers:
>>    - locked java.util.concurrent.locks.ReentrantLock$NonfairSync@4d16c92c
>>
>> "regionserver/192.168.200.31:60020.compactor-EventThread" - Thread t@19245
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@35c7b446
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:414)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "regionserver/192.168.200.31:60020.compactor-SendThread" - Thread t@19244
>>   java.lang.Thread.State: RUNNABLE
>>    at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>    at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>    at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>    at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>    - locked sun.nio.ch.Util$1@30f33929
>>    - locked java.util.Collections$UnmodifiableSet@ea26dec
>>    - locked sun.nio.ch.EPollSelectorImpl@3905bf36
>>    at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>    at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:921)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "RMI Scheduler(0)" - Thread t@406
>>   java.lang.Thread.State: TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@66d9c57f
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>>    at java.util.concurrent.DelayQueue.take(DelayQueue.java:164)
>>    at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:583)
>>    at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:576)
>>    at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:947)
>>    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>    at java.lang.Thread.run(Thread.java:619)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "pool-1-thread-1" - Thread t@74
>>   java.lang.Thread.State: TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@3175e1a2
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>>    at java.util.concurrent.DelayQueue.take(DelayQueue.java:164)
>>    at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:583)
>>    at java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:576)
>>    at java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:947)
>>    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:907)
>>    at java.lang.Thread.run(Thread.java:619)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "LruBlockCache.EvictionThread" - Thread t@72
>>   java.lang.Thread.State: WAITING on org.apache.hadoop.hbase.io.hfile.LruBlockCache$EvictionThread@32306c0c
>>    at java.lang.Object.wait(Native Method)
>>    at java.lang.Object.wait(Object.java:485)
>>    at org.apache.hadoop.hbase.io.hfile.LruBlockCache$EvictionThread.run(LruBlockCache.java:512)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 24 on 60020" - Thread t@71
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 23 on 60020" - Thread t@70
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 22 on 60020" - Thread t@69
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 21 on 60020" - Thread t@68
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 20 on 60020" - Thread t@67
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 19 on 60020" - Thread t@66
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 18 on 60020" - Thread t@65
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 17 on 60020" - Thread t@64
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 16 on 60020" - Thread t@63
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 15 on 60020" - Thread t@62
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 14 on 60020" - Thread t@61
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 13 on 60020" - Thread t@60
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 12 on 60020" - Thread t@59
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 11 on 60020" - Thread t@58
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 10 on 60020" - Thread t@57
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 9 on 60020" - Thread t@56
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 8 on 60020" - Thread t@55
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 7 on 60020" - Thread t@54
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 6 on 60020" - Thread t@53
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 5 on 60020" - Thread t@52
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 4 on 60020" - Thread t@51
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 3 on 60020" - Thread t@50
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 2 on 60020" - Thread t@49
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 1 on 60020" - Thread t@48
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server handler 0 on 60020" - Thread t@47
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@905a192
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:901)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server listener on 60020" - Thread t@14
>>   java.lang.Thread.State: RUNNABLE
>>    at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>    at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>    at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>    at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>    - locked sun.nio.ch.Util$1@284c8f79
>>    - locked java.util.Collections$UnmodifiableSet@3a5f659a
>>    - locked sun.nio.ch.EPollSelectorImpl@4d077f66
>>    at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>    at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:308)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Server Responder" - Thread t@16
>>   java.lang.Thread.State: RUNNABLE
>>    at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>    at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>    at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>    at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>    - locked sun.nio.ch.Util$1@273d290c
>>    - locked java.util.Collections$UnmodifiableSet@2051e6f4
>>    - locked sun.nio.ch.EPollSelectorImpl@6111adbf
>>    at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>    at org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:477)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "Timer-0" - Thread t@46
>>   java.lang.Thread.State: TIMED_WAITING on java.util.TaskQueue@50b27cde
>>    at java.lang.Object.wait(Native Method)
>>    at java.util.TimerThread.mainLoop(Timer.java:509)
>>    at java.util.TimerThread.run(Timer.java:462)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "889074013@qtp0-1 - Acceptor0 SelectChannelConnector@0.0.0.0:60030" - Thread t@45
>>   java.lang.Thread.State: RUNNABLE
>>    at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>    at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>    at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>    at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>    - locked sun.nio.ch.Util$1@1d7082c8
>>    - locked java.util.Collections$UnmodifiableSet@b9bc44a
>>    - locked sun.nio.ch.EPollSelectorImpl@6f7f73f7
>>    at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>    at org.mortbay.io.nio.SelectorManager$SelectSet.doSelect(SelectorManager.java:429)
>>    at org.mortbay.io.nio.SelectorManager.doSelect(SelectorManager.java:185)
>>    at org.mortbay.jetty.nio.SelectChannelConnector.accept(SelectChannelConnector.java:124)
>>    at org.mortbay.jetty.AbstractConnector$Acceptor.run(AbstractConnector.java:707)
>>    at org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:522)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "1374821456@qtp0-0" - Thread t@44
>>   java.lang.Thread.State: TIMED_WAITING on org.mortbay.thread.QueuedThreadPool$PoolThread@51f21c50
>>    at java.lang.Object.wait(Native Method)
>>    at org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:565)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "regionserver/192.168.200.31:60020.leaseChecker" - Thread t@23
>>   java.lang.Thread.State: TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@7623fc35
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>>    at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201)
>>    at org.apache.hadoop.hbase.Leases.run(Leases.java:81)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "regionserver/192.168.200.31:60020.majorCompactionChecker" - Thread t@22
>>   java.lang.Thread.State: TIMED_WAITING
>>    at java.lang.Thread.sleep(Native Method)
>>    at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)
>>    at org.apache.hadoop.hbase.Chore.run(Chore.java:76)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "regionserver/192.168.200.31:60020.worker" - Thread t@17
>>   java.lang.Thread.State: TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@3483382b
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>>    at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:395)
>>    at org.apache.hadoop.hbase.regionserver.HRegionServer$Worker.run(HRegionServer.java:1425)
>>    at java.lang.Thread.run(Thread.java:619)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "regionserver/192.168.200.31:60020.compactor" - Thread t@19
>>   java.lang.Thread.State: TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@12dca9fc
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>>    at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:395)
>>    at org.apache.hadoop.hbase.regionserver.CompactSplitThread.run(CompactSplitThread.java:85)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "regionserver/192.168.200.31:60020.cacheFlusher" - Thread t@18
>>   java.lang.Thread.State: TIMED_WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@551be910
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1963)
>>    at java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:395)
>>    at org.apache.hadoop.hbase.regionserver.MemStoreFlusher.run(MemStoreFlusher.java:145)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "regionserver/192.168.200.31:60020.logFlusher" - Thread t@21
>>   java.lang.Thread.State: TIMED_WAITING
>>    at java.lang.Thread.sleep(Native Method)
>>    at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)
>>    at org.apache.hadoop.hbase.Chore.run(Chore.java:76)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "regionserver/192.168.200.31:60020.logRoller" - Thread t@20
>>   java.lang.Thread.State: TIMED_WAITING on java.util.concurrent.atomic.AtomicBoolean@1b624517
>>    at java.lang.Object.wait(Native Method)
>>    at org.apache.hadoop.hbase.regionserver.LogRoller.run(LogRoller.java:65)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "Timer thread for monitoring jvm" - Thread t@42
>>   java.lang.Thread.State: TIMED_WAITING on java.util.TaskQueue@63354999
>>    at java.lang.Object.wait(Native Method)
>>    at java.util.TimerThread.mainLoop(Timer.java:509)
>>    at java.util.TimerThread.run(Timer.java:462)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "Timer thread for monitoring hbase" - Thread t@41
>>   java.lang.Thread.State: TIMED_WAITING on java.util.TaskQueue@4c966553
>>    at java.lang.Object.wait(Native Method)
>>    at java.util.TimerThread.mainLoop(Timer.java:509)
>>    at java.util.TimerThread.run(Timer.java:462)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "LeaseChecker" - Thread t@40
>>   java.lang.Thread.State: TIMED_WAITING
>>    at java.lang.Thread.sleep(Native Method)
>>    at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:1066)
>>    at java.lang.Thread.run(Thread.java:619)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "IPC Client (47) connection to /192.168.200.30:60000 from an unknown user" - Thread t@28
>>   java.lang.Thread.State: TIMED_WAITING on org.apache.hadoop.hbase.ipc.HBaseClient$Connection@3ffe5e31
>>    at java.lang.Object.wait(Native Method)
>>    at org.apache.hadoop.hbase.ipc.HBaseClient$Connection.waitForWork(HBaseClient.java:404)
>>    at org.apache.hadoop.hbase.ipc.HBaseClient$Connection.run(HBaseClient.java:447)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "DestroyJavaVM" - Thread t@27
>>   java.lang.Thread.State: RUNNABLE
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "regionserver/192.168.200.31:60020" - Thread t@26
>>   java.lang.Thread.State: TIMED_WAITING
>>    at java.lang.Thread.sleep(Native Method)
>>    at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:74)
>>    at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:594)
>>    at java.lang.Thread.run(Thread.java:619)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "main-EventThread" - Thread t@25
>>   java.lang.Thread.State: WAITING on java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@54da03c9
>>    at sun.misc.Unsafe.park(Native Method)
>>    at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
>>    at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1925)
>>    at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:358)
>>    at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:414)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "main-SendThread" - Thread t@24
>>   java.lang.Thread.State: RUNNABLE
>>    at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>>    at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>    at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>    at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>    - locked sun.nio.ch.Util$1@18969651
>>    - locked java.util.Collections$UnmodifiableSet@4666ac73
>>    - locked sun.nio.ch.EPollSelectorImpl@7ec3d620
>>    at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>    at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:921)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "Timer thread for monitoring rpc" - Thread t@15
>>   java.lang.Thread.State: TIMED_WAITING on java.util.TaskQueue@352eaca4
>>    at java.lang.Object.wait(Native Method)
>>    at java.util.TimerThread.mainLoop(Timer.java:509)
>>    at java.util.TimerThread.run(Timer.java:462)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "RMI TCP Accept-0" - Thread t@12
>>   java.lang.Thread.State: RUNNABLE
>>    at java.net.PlainSocketImpl.socketAccept(Native Method)
>>    at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:390)
>>    - locked java.net.SocksSocketImpl@6b1f0fe4
>>    at java.net.ServerSocket.implAccept(ServerSocket.java:453)
>>    at java.net.ServerSocket.accept(ServerSocket.java:421)
>>    at sun.management.jmxremote.LocalRMIServerSocketFactory$1.accept(LocalRMIServerSocketFactory.java:34)
>>    at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:369)
>>    at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:341)
>>    at java.lang.Thread.run(Thread.java:619)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "RMI TCP Accept-10102" - Thread t@11
>>   java.lang.Thread.State: RUNNABLE
>>    at java.net.PlainSocketImpl.socketAccept(Native Method)
>>    at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:390)
>>    - locked java.net.SocksSocketImpl@444c78ad
>>    at java.net.ServerSocket.implAccept(ServerSocket.java:453)
>>    at java.net.ServerSocket.accept(ServerSocket.java:421)
>>    at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:369)
>>    at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:341)
>>    at java.lang.Thread.run(Thread.java:619)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "RMI TCP Accept-0" - Thread t@10
>>   java.lang.Thread.State: RUNNABLE
>>    at java.net.PlainSocketImpl.socketAccept(Native Method)
>>    at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:390)
>>    - locked java.net.SocksSocketImpl@7886c8e2
>>    at java.net.ServerSocket.implAccept(ServerSocket.java:453)
>>    at java.net.ServerSocket.accept(ServerSocket.java:421)
>>    at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:369)
>>    at sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:341)
>>    at java.lang.Thread.run(Thread.java:619)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "Signal Dispatcher" - Thread t@5
>>   java.lang.Thread.State: RUNNABLE
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "Finalizer" - Thread t@3
>>   java.lang.Thread.State: WAITING on java.lang.ref.ReferenceQueue$Lock@b485e43
>>    at java.lang.Object.wait(Native Method)
>>    at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:118)
>>    at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:134)
>>    at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:159)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> "Reference Handler" - Thread t@2
>>   java.lang.Thread.State: WAITING on java.lang.ref.Reference$Lock@3c24867e
>>    at java.lang.Object.wait(Native Method)
>>    at java.lang.Object.wait(Object.java:485)
>>    at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116)
>>
>>   Locked ownable synchronizers:
>>    - None
>>
>> ---
>>
>> Respectfully Yours,
>> Luke Forehand
>> Software Engineer
>> Networked Insights
>> http://www.networkedinsights.com
>>
>

Mime
View raw message