Deadlock in HBaseServer (JVM bug?)
----------------------------------

                 Key: HBASE-3622
                 URL: https://issues.apache.org/jira/browse/HBASE-3622
             Project: HBase
          Issue Type: Bug
    Affects Versions: 0.90.1
            Reporter: Jean-Daniel Cryans
            Priority: Critical
             Fix For: 0.92.0


On Dmitriy's cluster:

{code}

"IPC Reader 0 on port 60020" prio=10 tid=0x00002aacb4a82800 nid=0x3a72 waiting 
on condition [0x00000000429ba000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00002aaabf5fa6d0> (a 
java.util.concurrent.locks.ReentrantLock$NonfairSync)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:158)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:747)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:778)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1114)
        at 
java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:186)
        at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:262)
        at 
java.util.concurrent.LinkedBlockingQueue.signalNotEmpty(LinkedBlockingQueue.java:103)
        at 
java.util.concurrent.LinkedBlockingQueue.put(LinkedBlockingQueue.java:267)
        at 
org.apache.hadoop.hbase.ipc.HBaseServer$Connection.processData(HBaseServer.java:985)
        at 
org.apache.hadoop.hbase.ipc.HBaseServer$Connection.readAndProcess(HBaseServer.java:946)
        at 
org.apache.hadoop.hbase.ipc.HBaseServer$Listener.doRead(HBaseServer.java:522)
        at 
org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader.run(HBaseServer.java:316)
        - locked <0x00002aaabf580fb0> (a 
org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
...
"IPC Server handler 29 on 60020" daemon prio=10 tid=0x00002aacbc163800 
nid=0x3acc waiting on condition [0x00000000462f3000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00002aaabf5e3800> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:1025)
"IPC Server handler 28 on 60020" daemon prio=10 tid=0x00002aacbc161800 
nid=0x3acb waiting on condition [0x00000000461f2000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00002aaabf5e3800> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        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:1025
...
{code}

This region server stayed in this state for hours. The reader is waiting to put 
and the handlers are waiting to take, and they wait on different lock ids. It 
reminds me of the UseMembar thing about the JVM sometime missing to notify 
waiters. In any case, that RS needed to be closed in order to get out of that 
state. 

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to