Thanks for doing the debugging. In general for at-scale I am not sure I can recommend the indexed features. You might get better miliage with hand maintaining the indexes yourself.
Core-hbase is scalable to 500-800tb tables and more. -ryan On Thu, Oct 15, 2009 at 5:50 PM, Tatsuya Kawano <[email protected]> wrote: > Hi Kevin, > > It seems you are having a deadlock issue on your indexed region server. > > I tried to reproduce this -- 50 client threads who try to put millions > of records, autoFlush(false), flushCommits() on every 5,000 put. After > inserting about 3 million records, a deadlock occurred on a region > server who has both the table and index regions loaded. > > I have attached a full thread dump of the deadlocked region server, > and you can see IPC Server handlers are blocked in > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(). > > I found the flowing FIXME comment on updateIndex() method, and it > seems this is the deadlock I'm having. > > ===================================== > // FIXME: This call takes place in an RPC, and requires an RPC. This makes > for > // a likely deadlock if the number of RPCs we are trying to serve is >= the > // number of handler threads. > private void updateIndex(IndexSpecification indexSpec, byte[] row, > SortedMap<byte[], byte[]> columnValues) throws IOException { > ===================================== > > > I use HBase 0.20.1 and my region servers were running with 10 RPC > handler threads on each (default). > > Maybe you can workaround this by adding more RPC handlers (increase > the value of "hbase.regionserver.handler.count" in hbase-site.xml) > I'm going to run the same test again with 70 RPC handlers, and let you > know the result. > > Thanks, > Tatsuya > > -- > Tatsuya Kawano (Mr.) > Tokyo, Japan > > > > On Thu, Oct 15, 2009 at 2:30 AM, Kevin Peterson <[email protected]> wrote: >> I'm trying to load data into a table from a Hadoop map job. I have a main >> table that stores an average of about 2k per row, and I want to have two >> additional index tables, which index 10-20byte keys in the primary table. I >> have used TableIndexed and it worked beautifully on small scale testing. >> >> When I tried to use it at a larger scale, it seems to just freeze up. I see >> the Hadoop jobs get through maybe 2.5 million records at a good pace, and >> then they just hang. Eventually Hadoop kills the jobs after they haven't >> responded for 40 minutes. I don't see anything in the logs (though I >> wouldn't know what to look for). >> >> In comparison, when I remove the TableIndexed region server from >> hbase-site.xml, I'm able to easily load my full batch of 12 million records >> in an hour. >> >> Details of cluster: >> 1 node ZooKeeper and HBase Master >> 4 nodes ZooKeeper, Region Server and DataNode >> 4 hadoop datanode / tasktrackers with 3 map slots each >> 1 hadoop namenode and jobtracker >> >> All nodes are EC2 large instances, 2 cores, 8GB ram, two local 500GB disks. >> >> I have not tuned any memory or performance related settings. I turn on >> TableIndexed by setting hbase.regionserver.class to >> org.apache.hadoop.hbase.ipc.IndexedRegionInterface and >> hbase.regionserver.impl to >> org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegionServer. I'm >> using HBase 20.1 RC1, with transactional jar compiled from 0.20.0 with >> HBASE-1885, which includes my index key creator. >> >> The behavior makes me think it's something like I need to call commit, but I >> can't find anything mentioned. Any ideas? > > > > > ===================================== > > 2009-10-16 09:12:53 > Full thread dump Java HotSpot(TM) 64-Bit Server VM (14.1-b02-90 mixed mode): > > "JMX server connection timeout 377" daemon prio=9 > tid=0x00000001550b3000 nid=0x158ec6000 in Object.wait() > [0x0000000158ec5000] > java.lang.Thread.State: TIMED_WAITING (on object monitor) > at java.lang.Object.wait(Native Method) > at > com.sun.jmx.remote.internal.ServerCommunicatorAdmin$Timeout.run(ServerCommunicatorAdmin.java:150) > - locked <0x0000000107fa1728> (a [I) > at java.lang.Thread.run(Thread.java:637) > > Locked ownable synchronizers: > - None > > "RMI Scheduler(0)" daemon prio=9 tid=0x000000014f85a800 > nid=0x158dc3000 waiting on condition [0x0000000158dc2000] > java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0000000107fa00d8> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > 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:637) > > Locked ownable synchronizers: > - None > > "RMI TCP Connection(1)-172.16.80.26" daemon prio=9 > tid=0x0000000106b31000 nid=0x158bb6000 runnable [0x0000000158bb5000] > 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 <0x0000000107fa3898> (a java.io.BufferedInputStream) > 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:637) > > Locked ownable synchronizers: > - <0x0000000108035bd0> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > > "RMI TCP Accept-0" daemon prio=9 tid=0x00000001530ca000 > nid=0x156901000 runnable [0x0000000156900000] > java.lang.Thread.State: RUNNABLE > at java.net.PlainSocketImpl.socketAccept(Native Method) > at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:390) > - locked <0x0000000107fa0330> (a java.net.SocksSocketImpl) > 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:637) > > Locked ownable synchronizers: > - None > > "Attach Listener" daemon prio=9 tid=0x00000001548ca000 nid=0x157a83000 > waiting on condition [0x0000000000000000] > java.lang.Thread.State: RUNNABLE > > Locked ownable synchronizers: > - None > > "IPC Client (47) connection to /172.16.80.26:60022 from an unknown > user" daemon prio=5 tid=0x000000015620a000 nid=0x158fc9000 runnable > [0x0000000158fc8000] > java.lang.Thread.State: RUNNABLE > at sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method) > at sun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:136) > at sun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:69) > at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) > - locked <0x0000000119b56d60> (a sun.nio.ch.Util$1) > - locked <0x0000000119b56d48> (a java.util.Collections$UnmodifiableSet) > - locked <0x0000000124398148> (a sun.nio.ch.KQueueSelectorImpl) > 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.FilterInputStream.read(FilterInputStream.java:116) > at > org.apache.hadoop.hbase.ipc.HBaseClient$Connection$PingInputStream.read(HBaseClient.java:279) > at java.io.BufferedInputStream.fill(BufferedInputStream.java:218) > at java.io.BufferedInputStream.read(BufferedInputStream.java:237) > - locked <0x000000011899bf40> (a java.io.BufferedInputStream) > at java.io.DataInputStream.readInt(DataInputStream.java:370) > at > org.apache.hadoop.hbase.ipc.HBaseClient$Connection.receiveResponse(HBaseClient.java:504) > at > org.apache.hadoop.hbase.ipc.HBaseClient$Connection.run(HBaseClient.java:448) > > Locked ownable synchronizers: > - None > > "ResponseProcessor for block blk_6354822949724586880_4142" daemon > prio=5 tid=0x000000010295b000 nid=0x156c5b000 runnable > [0x0000000156c5a000] > java.lang.Thread.State: RUNNABLE > at sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method) > at sun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:136) > at sun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:69) > at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) > - locked <0x000000012411b920> (a sun.nio.ch.Util$1) > - locked <0x000000012411b908> (a java.util.Collections$UnmodifiableSet) > - locked <0x0000000126e55740> (a sun.nio.ch.KQueueSelectorImpl) > 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/172.16.80.26,60022,1255650242212/hlog.dat.1255651572580 > block blk_6354822949724586880_4142" daemon prio=5 > tid=0x0000000106b30000 nid=0x156a04000 in Object.wait() > [0x0000000156a03000] > java.lang.Thread.State: TIMED_WAITING (on object monitor) > at java.lang.Object.wait(Native Method) > at > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2247) > - locked <0x0000000126088e70> (a java.util.LinkedList) > > Locked ownable synchronizers: > - None > > "IPC Server handler 8 on 60022-EventThread" daemon prio=5 > tid=0x000000014f845000 nid=0x158cc0000 waiting on condition > [0x0000000158cbf000] > java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x00000001105843c8> (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.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:414) > > Locked ownable synchronizers: > - None > > "IPC Server handler 8 on 60022-SendThread" daemon prio=5 > tid=0x000000014f844800 nid=0x156b16000 runnable [0x0000000156b15000] > java.lang.Thread.State: RUNNABLE > at sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method) > at sun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:136) > at sun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:69) > at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) > - locked <0x00000001105957e0> (a sun.nio.ch.Util$1) > - locked <0x00000001105957f8> (a java.util.Collections$UnmodifiableSet) > - locked <0x000000011015cef0> (a sun.nio.ch.KQueueSelectorImpl) > at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) > at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:921) > > Locked ownable synchronizers: > - None > > "pool-1-thread-1" prio=5 tid=0x000000014f95a000 nid=0x158ab3000 > waiting on condition [0x0000000158ab2000] > java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0000000110552968> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > 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:637) > > Locked ownable synchronizers: > - None > > "LruBlockCache.EvictionThread" daemon prio=5 tid=0x000000014f959800 > nid=0x1589b0000 in Object.wait() [0x00000001589af000] > java.lang.Thread.State: WAITING (on object monitor) > 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 <0x00000001103316c0> (a > org.apache.hadoop.hbase.io.hfile.LruBlockCache$EvictionThread) > > Locked ownable synchronizers: > - None > > "Transactional leases" daemon prio=5 tid=0x00000001530f1800 > nid=0x1588ad000 waiting on condition [0x00000001588ac000] > java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0000000110597a40> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > 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/172.16.80.26:60022.oldTransactionCleaner" daemon prio=5 > tid=0x00000001530f0800 nid=0x1587aa000 waiting on condition > [0x00000001587a9000] > java.lang.Thread.State: TIMED_WAITING (sleeping) > 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 > > "IPC Server handler 9 on 60022" daemon prio=5 tid=0x00000001530f0000 > nid=0x1586a7000 waiting for monitor entry [0x00000001586a6000] > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hbase.client.HTable.put(HTable.java:466) > - waiting to lock <0x00000001105852a0> (a > org.apache.hadoop.hbase.client.HTable) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(IndexedRegion.java:213) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndexes(IndexedRegion.java:145) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.put(IndexedRegion.java:96) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1208) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1831) > at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:648) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915) > > Locked ownable synchronizers: > - None > > "IPC Server handler 8 on 60022" daemon prio=5 tid=0x00000001530ef000 > nid=0x1585a4000 waiting for monitor entry [0x00000001585a3000] > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hbase.client.HTable.put(HTable.java:466) > - waiting to lock <0x00000001105852a0> (a > org.apache.hadoop.hbase.client.HTable) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(IndexedRegion.java:213) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndexes(IndexedRegion.java:145) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.put(IndexedRegion.java:96) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1208) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1831) > at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:648) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915) > > Locked ownable synchronizers: > - None > > "IPC Server handler 7 on 60022" daemon prio=5 tid=0x00000001530ee800 > nid=0x1584a1000 waiting for monitor entry [0x00000001584a0000] > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hbase.client.HTable.put(HTable.java:466) > - waiting to lock <0x00000001105852a0> (a > org.apache.hadoop.hbase.client.HTable) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(IndexedRegion.java:213) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndexes(IndexedRegion.java:145) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.put(IndexedRegion.java:96) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1208) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1831) > at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:648) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915) > > Locked ownable synchronizers: > - None > > "IPC Server handler 6 on 60022" daemon prio=5 tid=0x00000001530ed800 > nid=0x15839e000 waiting for monitor entry [0x000000015839d000] > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hbase.client.HTable.put(HTable.java:466) > - waiting to lock <0x00000001105852a0> (a > org.apache.hadoop.hbase.client.HTable) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(IndexedRegion.java:213) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndexes(IndexedRegion.java:145) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.put(IndexedRegion.java:96) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1208) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1831) > at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:648) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915) > > Locked ownable synchronizers: > - None > > "IPC Server handler 5 on 60022" daemon prio=5 tid=0x00000001530ed000 > nid=0x15829b000 in Object.wait() [0x0000000158299000] > java.lang.Thread.State: WAITING (on object monitor) > at java.lang.Object.wait(Native Method) > at java.lang.Object.wait(Object.java:485) > at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:717) > - locked <0x0000000121595ad0> (a > org.apache.hadoop.hbase.ipc.HBaseClient$Call) > at > org.apache.hadoop.hbase.ipc.HBaseRPC$Invoker.invoke(HBaseRPC.java:328) > at $Proxy6.getProtocolVersion(Unknown Source) > at org.apache.hadoop.hbase.ipc.HBaseRPC.getProxy(HBaseRPC.java:484) > at org.apache.hadoop.hbase.ipc.HBaseRPC.getProxy(HBaseRPC.java:460) > at org.apache.hadoop.hbase.ipc.HBaseRPC.getProxy(HBaseRPC.java:507) > at org.apache.hadoop.hbase.ipc.HBaseRPC.waitForProxy(HBaseRPC.java:418) > at > org.apache.hadoop.hbase.client.HConnectionManager$TableServers.getHRegionConnection(HConnectionManager.java:864) > - locked <0x00000001105cc1e8> (a > java.util.concurrent.ConcurrentHashMap) > at > org.apache.hadoop.hbase.client.HConnectionManager$TableServers.getHRegionConnection(HConnectionManager.java:880) > at > org.apache.hadoop.hbase.client.ServerCallable.instantiateServer(ServerCallable.java:58) > at > org.apache.hadoop.hbase.client.HConnectionManager$TableServers.getRegionServerWithRetries(HConnectionManager.java:987) > at > org.apache.hadoop.hbase.client.HConnectionManager$TableServers$2.doCall(HConnectionManager.java:1192) > at > org.apache.hadoop.hbase.client.HConnectionManager$TableServers$Batch.process(HConnectionManager.java:1114) > at > org.apache.hadoop.hbase.client.HConnectionManager$TableServers.processBatchOfRows(HConnectionManager.java:1200) > at org.apache.hadoop.hbase.client.HTable.flushCommits(HTable.java:605) > at org.apache.hadoop.hbase.client.HTable.put(HTable.java:470) > - locked <0x00000001105852a0> (a org.apache.hadoop.hbase.client.HTable) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(IndexedRegion.java:213) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndexes(IndexedRegion.java:145) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.put(IndexedRegion.java:96) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1208) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1831) > at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:648) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915) > > Locked ownable synchronizers: > - None > > "IPC Server handler 4 on 60022" daemon prio=5 tid=0x00000001530ec000 > nid=0x158198000 waiting for monitor entry [0x0000000158197000] > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hbase.client.HTable.put(HTable.java:466) > - waiting to lock <0x00000001105852a0> (a > org.apache.hadoop.hbase.client.HTable) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(IndexedRegion.java:213) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndexes(IndexedRegion.java:145) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.put(IndexedRegion.java:96) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1208) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1831) > at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:648) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915) > > Locked ownable synchronizers: > - None > > "IPC Server handler 3 on 60022" daemon prio=5 tid=0x00000001530eb800 > nid=0x158095000 waiting for monitor entry [0x0000000158094000] > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hbase.client.HTable.put(HTable.java:466) > - waiting to lock <0x00000001105852a0> (a > org.apache.hadoop.hbase.client.HTable) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(IndexedRegion.java:213) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndexes(IndexedRegion.java:145) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.put(IndexedRegion.java:96) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1208) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1831) > at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:648) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915) > > Locked ownable synchronizers: > - None > > "IPC Server handler 2 on 60022" daemon prio=5 tid=0x00000001530ea800 > nid=0x157f92000 waiting for monitor entry [0x0000000157f91000] > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hbase.client.HTable.put(HTable.java:466) > - waiting to lock <0x00000001105852a0> (a > org.apache.hadoop.hbase.client.HTable) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(IndexedRegion.java:213) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndexes(IndexedRegion.java:145) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.put(IndexedRegion.java:96) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1208) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1831) > at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:648) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915) > > Locked ownable synchronizers: > - None > > "IPC Server handler 1 on 60022" daemon prio=5 tid=0x00000001530ea000 > nid=0x157e8f000 waiting for monitor entry [0x0000000157e8e000] > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hbase.client.HTable.put(HTable.java:466) > - waiting to lock <0x00000001105852a0> (a > org.apache.hadoop.hbase.client.HTable) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(IndexedRegion.java:213) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndexes(IndexedRegion.java:145) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.put(IndexedRegion.java:96) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1208) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1831) > at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:648) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915) > > Locked ownable synchronizers: > - None > > "IPC Server handler 0 on 60022" daemon prio=5 tid=0x00000001530e9000 > nid=0x157d8c000 waiting for monitor entry [0x0000000157d8b000] > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hbase.client.HTable.put(HTable.java:466) > - waiting to lock <0x00000001105852a0> (a > org.apache.hadoop.hbase.client.HTable) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndex(IndexedRegion.java:213) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.updateIndexes(IndexedRegion.java:145) > at > org.apache.hadoop.hbase.regionserver.tableindexed.IndexedRegion.put(IndexedRegion.java:96) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1208) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.put(HRegionServer.java:1831) > at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.HBaseRPC$Server.call(HBaseRPC.java:648) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:915) > > Locked ownable synchronizers: > - None > > "IPC Server listener on 60022" daemon prio=5 tid=0x00000001530e8800 > nid=0x157c89000 runnable [0x0000000157c88000] > java.lang.Thread.State: RUNNABLE > at sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method) > at sun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:136) > at sun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:69) > at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) > - locked <0x0000000110365750> (a sun.nio.ch.Util$1) > - locked <0x0000000110365738> (a java.util.Collections$UnmodifiableSet) > - locked <0x0000000110224398> (a sun.nio.ch.KQueueSelectorImpl) > 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" daemon prio=5 tid=0x0000000153121000 > nid=0x157b86000 runnable [0x0000000157b85000] > java.lang.Thread.State: RUNNABLE > at sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method) > at sun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:136) > at sun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:69) > at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) > - locked <0x0000000110365678> (a sun.nio.ch.Util$1) > - locked <0x0000000110365660> (a java.util.Collections$UnmodifiableSet) > - locked <0x0000000110224320> (a sun.nio.ch.KQueueSelectorImpl) > 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" daemon prio=5 tid=0x00000001530d9800 nid=0x157980000 in > Object.wait() [0x000000015797f000] > java.lang.Thread.State: TIMED_WAITING (on object monitor) > at java.lang.Object.wait(Native Method) > at java.util.TimerThread.mainLoop(Timer.java:509) > - locked <0x000000011058ecf0> (a java.util.TaskQueue) > at java.util.TimerThread.run(Timer.java:462) > > Locked ownable synchronizers: > - None > > "1034011...@qtp0-1 - Acceptor0 [email protected]:60032" > prio=5 tid=0x0000000154988000 nid=0x15787d000 runnable > [0x000000015787c000] > java.lang.Thread.State: RUNNABLE > at sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method) > at sun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:136) > at sun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:69) > at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) > - locked <0x00000001105a80d8> (a sun.nio.ch.Util$1) > - locked <0x00000001105a80f0> (a java.util.Collections$UnmodifiableSet) > - locked <0x000000011016ded0> (a sun.nio.ch.KQueueSelectorImpl) > 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 > > "449580...@qtp0-0" prio=5 tid=0x000000015486e000 nid=0x15777a000 in > Object.wait() [0x0000000157779000] > java.lang.Thread.State: TIMED_WAITING (on object monitor) > at java.lang.Object.wait(Native Method) > at > org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:565) > - locked <0x00000001104823f0> (a > org.mortbay.thread.QueuedThreadPool$PoolThread) > > Locked ownable synchronizers: > - None > > "regionserver/172.16.80.26:60022.leaseChecker" prio=5 > tid=0x000000015309b000 nid=0x1574ea000 waiting on condition > [0x00000001574e9000] > java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0000000110139fe0> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > 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/172.16.80.26:60022.majorCompactionChecker" daemon prio=5 > tid=0x000000015313e000 nid=0x1573e7000 waiting on condition > [0x00000001573e6000] > java.lang.Thread.State: TIMED_WAITING (sleeping) > 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/172.16.80.26:60022.worker" daemon prio=5 > tid=0x000000015313d000 nid=0x1572e4000 waiting on condition > [0x00000001572e3000] > java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x00000001101386e0> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > 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:1481) > at java.lang.Thread.run(Thread.java:637) > > Locked ownable synchronizers: > - None > > "regionserver/172.16.80.26:60022.compactor" daemon prio=5 > tid=0x000000015313c800 nid=0x1571e1000 waiting on condition > [0x00000001571e0000] > java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0000000110379960> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > 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:96) > > Locked ownable synchronizers: > - None > > "regionserver/172.16.80.26:60022.cacheFlusher" daemon prio=5 > tid=0x000000015313b800 nid=0x1570de000 waiting on condition > [0x00000001570dd000] > java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0000000110379870> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > 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/172.16.80.26:60022.logFlusher" daemon prio=5 > tid=0x000000015313b000 nid=0x156fdb000 waiting on condition > [0x0000000156fda000] > java.lang.Thread.State: TIMED_WAITING (sleeping) > 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/172.16.80.26:60022.logRoller" daemon prio=5 > tid=0x000000015313a000 nid=0x156ed8000 in Object.wait() > [0x0000000156ed7000] > java.lang.Thread.State: TIMED_WAITING (on object monitor) > at java.lang.Object.wait(Native Method) > at > org.apache.hadoop.hbase.regionserver.LogRoller.run(LogRoller.java:65) > - locked <0x0000000110139b30> (a > java.util.concurrent.atomic.AtomicBoolean) > > Locked ownable synchronizers: > - None > > "LeaseChecker" daemon prio=5 tid=0x000000014f846000 nid=0x156dd5000 > waiting on condition [0x0000000156dd4000] > java.lang.Thread.State: TIMED_WAITING (sleeping) > 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:637) > > Locked ownable synchronizers: > - None > > "IPC Client (47) connection to /172.16.80.26:60000 from an unknown > user" daemon prio=5 tid=0x000000014f82d000 nid=0x153f5f000 in > Object.wait() [0x0000000153f5e000] > java.lang.Thread.State: TIMED_WAITING (on object monitor) > at java.lang.Object.wait(Native Method) > at > org.apache.hadoop.hbase.ipc.HBaseClient$Connection.waitForWork(HBaseClient.java:404) > - locked <0x00000001101c0ee0> (a > org.apache.hadoop.hbase.ipc.HBaseClient$Connection) > at > org.apache.hadoop.hbase.ipc.HBaseClient$Connection.run(HBaseClient.java:447) > > Locked ownable synchronizers: > - None > > "DestroyJavaVM" prio=5 tid=0x0000000154866800 nid=0x100501000 waiting > on condition [0x0000000000000000] > java.lang.Thread.State: RUNNABLE > > Locked ownable synchronizers: > - None > > "regionserver/172.16.80.26:60022" prio=5 tid=0x0000000154866000 > nid=0x153e07000 waiting on condition [0x0000000153e06000] > java.lang.Thread.State: TIMED_WAITING (sleeping) > 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:591) > at java.lang.Thread.run(Thread.java:637) > > Locked ownable synchronizers: > - None > > "main-EventThread" daemon prio=5 tid=0x000000015502c800 > nid=0x153d04000 waiting on condition [0x0000000153d03000] > java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0000000110139388> (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.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:414) > > Locked ownable synchronizers: > - None > > "main-SendThread" daemon prio=5 tid=0x000000015502b800 nid=0x153c01000 > runnable [0x0000000153c00000] > java.lang.Thread.State: RUNNABLE > at sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method) > at sun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:136) > at sun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:69) > at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) > - locked <0x000000011016d2a0> (a sun.nio.ch.Util$1) > - locked <0x000000011016d2b8> (a java.util.Collections$UnmodifiableSet) > - locked <0x000000011015c2c0> (a sun.nio.ch.KQueueSelectorImpl) > at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) > at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:921) > > Locked ownable synchronizers: > - None > > "Low Memory Detector" daemon prio=5 tid=0x000000014f859000 > nid=0x151b04000 runnable [0x0000000000000000] > java.lang.Thread.State: RUNNABLE > > Locked ownable synchronizers: > - None > > "CompilerThread1" daemon prio=9 tid=0x000000014f858000 nid=0x151a01000 > waiting on condition [0x0000000000000000] > java.lang.Thread.State: RUNNABLE > > Locked ownable synchronizers: > - None > > "CompilerThread0" daemon prio=9 tid=0x000000014f857800 nid=0x150f07000 > waiting on condition [0x0000000000000000] > java.lang.Thread.State: RUNNABLE > > Locked ownable synchronizers: > - None > > "Signal Dispatcher" daemon prio=9 tid=0x000000014f856800 > nid=0x150e04000 runnable [0x0000000000000000] > java.lang.Thread.State: RUNNABLE > > Locked ownable synchronizers: > - None > > "Surrogate Locker Thread (CMS)" daemon prio=5 tid=0x000000014f856000 > nid=0x150d01000 waiting on condition [0x0000000000000000] > java.lang.Thread.State: RUNNABLE > > Locked ownable synchronizers: > - None > > "Finalizer" daemon prio=8 tid=0x000000014f828000 nid=0x150a01000 in > Object.wait() [0x0000000150a00000] > java.lang.Thread.State: WAITING (on object monitor) > at java.lang.Object.wait(Native Method) > at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:118) > - locked <0x0000000110122138> (a java.lang.ref.ReferenceQueue$Lock) > 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" daemon prio=10 tid=0x000000014f827800 > nid=0x14ef04000 in Object.wait() [0x000000014ef03000] > java.lang.Thread.State: WAITING (on object monitor) > 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 <0x0000000110138eb0> (a java.lang.ref.Reference$Lock) > > Locked ownable synchronizers: > - None > > "VM Thread" prio=9 tid=0x000000014f820000 nid=0x14ee01000 runnable > > "Gang worker#0 (Parallel GC Threads)" prio=9 tid=0x0000000102802800 > nid=0x101e01000 runnable > > "Gang worker#1 (Parallel GC Threads)" prio=9 tid=0x0000000102803000 > nid=0x101f04000 runnable > > "Gang worker#2 (Parallel GC Threads)" prio=9 tid=0x0000000102804000 > nid=0x106102000 runnable > > "Gang worker#3 (Parallel GC Threads)" prio=9 tid=0x0000000106800000 > nid=0x106205000 runnable > > "Gang worker#4 (Parallel GC Threads)" prio=9 tid=0x0000000106801000 > nid=0x106308000 runnable > > "Gang worker#5 (Parallel GC Threads)" prio=9 tid=0x0000000106801800 > nid=0x10640b000 runnable > > "Gang worker#6 (Parallel GC Threads)" prio=9 tid=0x0000000106802000 > nid=0x10650e000 runnable > > "Gang worker#7 (Parallel GC Threads)" prio=9 tid=0x0000000106802800 > nid=0x106611000 runnable > > "Concurrent Mark-Sweep GC Thread" prio=9 tid=0x00000001068bd000 > nid=0x14e823000 runnable > "Gang worker#0 (Parallel CMS Threads)" prio=9 tid=0x00000001068bc000 > nid=0x14e61d000 runnable > > "Gang worker#1 (Parallel CMS Threads)" prio=9 tid=0x00000001068bc800 > nid=0x14e720000 runnable > > "VM Periodic Task Thread" prio=10 tid=0x000000014f859800 > nid=0x151c07000 waiting on condition > > "Exception Catcher Thread" prio=10 tid=0x0000000102800000 > nid=0x101b01000 runnable > JNI global references: 1121 > ===================================== >
