Return-Path: X-Original-To: apmail-hbase-user-archive@www.apache.org Delivered-To: apmail-hbase-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7E81C10F for ; Wed, 20 Apr 2011 04:33:40 +0000 (UTC) Received: (qmail 1769 invoked by uid 500); 20 Apr 2011 04:33:39 -0000 Delivered-To: apmail-hbase-user-archive@hbase.apache.org Received: (qmail 1742 invoked by uid 500); 20 Apr 2011 04:33:38 -0000 Mailing-List: contact user-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@hbase.apache.org Delivered-To: mailing list user@hbase.apache.org Received: (qmail 1734 invoked by uid 99); 20 Apr 2011 04:33:36 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 20 Apr 2011 04:33:36 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=FREEMAIL_FROM,HTML_MESSAGE,NORMAL_HTTP_TO_IP,RCVD_IN_DNSWL_LOW,RFC_ABUSE_POST,SPF_PASS,T_TO_NO_BRKTS_FREEMAIL,WEIRD_PORT X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of yuzhihong@gmail.com designates 209.85.160.41 as permitted sender) Received: from [209.85.160.41] (HELO mail-pw0-f41.google.com) (209.85.160.41) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 20 Apr 2011 04:33:30 +0000 Received: by pwi10 with SMTP id 10so295918pwi.14 for ; Tue, 19 Apr 2011 21:33:10 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=domainkey-signature:mime-version:in-reply-to:references:date :message-id:subject:from:to:content-type; bh=6K5dQ39UrDV5lCeKnQfsjegVNDLy+XMpnl/dZ/ZbQDM=; b=OqtiqhU69eZAXYJVynBcFMh6HTcbHPOvmBUUiGhYu5l9tJcFtIeJ2JnY5xZ4MbzwfM OXyY575m6uiNVRvmZQHJOvV7BW5loQYrL5Al5tkq12T3k29nI9sfpEBYI4SEkOGytuEX VebZi7nxImTIbq/4EX9eRfzvPyGejXAlVfPS4= DomainKey-Signature: a=rsa-sha1; c=nofws; d=gmail.com; s=gamma; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; b=XKq2XuDZRW4zjMSQn1MrwfBXiQ3RGoQLqkVAIBQsQPcHh+kjZ6Ajp2A/Jp/I/JYMuX kgNf6/nJHejOcr3nQ0HQhBpkvGxh6guNJGQ/Yy6gwhAm4Ai/nBorl/4+Tuq63oMfw12y UioJ7HeHz81YsJiAha9xXQOWfySU73UMUpZl8= MIME-Version: 1.0 Received: by 10.68.22.68 with SMTP id b4mr10302465pbf.413.1303273990042; Tue, 19 Apr 2011 21:33:10 -0700 (PDT) Received: by 10.68.56.70 with HTTP; Tue, 19 Apr 2011 21:33:09 -0700 (PDT) In-Reply-To: <87AB0DC3526542588AA8572DC4FAFCE2@china.huawei.com> References: <87AB0DC3526542588AA8572DC4FAFCE2@china.huawei.com> Date: Tue, 19 Apr 2011 21:33:09 -0700 Message-ID: Subject: Re: Possible dead lock From: Ted Yu To: user@hbase.apache.org, ramakrishnas@huawei.com Content-Type: multipart/alternative; boundary=bcaec5215bd138b4a804a1521f4d --bcaec5215bd138b4a804a1521f4d Content-Type: text/plain; charset=ISO-8859-1 Which HBase version are you using ? I don't see lock object in MemStoreFlusher.flushOneForGlobalPressure(). In the future, please make sure stack trace lines aren't wrapped in the middle. On Tue, Apr 19, 2011 at 9:16 PM, Ramkrishna S Vasudevan < ramakrishnas@huawei.com> wrote: > Possible dead lock in hBase. > > > > We are using a client to do multiple write operations parallely. > > > > Here the problem could be like as part of flushforGlobalPressure the lock > object is obtained and the flow goes into requestCompaction in compact > split > thread. > > From there there is a an addition to the PriorityCompactionQueue. Here the > toString of CompactionRequest is getting used. > > > > Inside this a Data object is used, further digging leads to ResourceBundle, > where the currentThread lock is tried to obtain. > > > > By the time the other other write operation comes and acquires a lock on > Memstoreflusher thro recliamStoreMemory api.(synchronized). > > > > Inside the recliamStoreMemory the lock is again tried to obtain but it is > already obtained as part of flushforGlobalPressure. > > So one is waiting for getting the lock object where the one inside the > ResourceBundle is trying to acquire the current thread which is nothing but > the memstoreflusher. > > > > Pls chk and verify. > > > > 2011-04-19 14:29:37 > > Full thread dump Java HotSpot(TM) 64-Bit Server VM (14.3-b01 mixed mode): > > > > "IPC Client (-1912063050) connection to /10.18.52.108:9000 from root" > daemon > prio=10 tid=0x00007fe7b8372800 nid=0x64f9 in Object.wait() > [0x00007fe7be5e4000] > > java.lang.Thread.State: TIMED_WAITING (on object monitor) > > at java.lang.Object.wait(Native Method) > > - waiting on <0x00007fe7ca1369b0> (a > org.apache.hadoop.ipc.Client$Connection) > > at > org.apache.hadoop.ipc.Client$Connection.waitForWork(Client.java:545) > > - locked <0x00007fe7ca1369b0> (a > org.apache.hadoop.ipc.Client$Connection) > > at org.apache.hadoop.ipc.Client$Connection.run(Client.java:593) > > > > "Attach Listener" daemon prio=10 tid=0x0000000040a01800 nid=0x64ec waiting > on condition [0x0000000000000000] > > java.lang.Thread.State: RUNNABLE > > > > "1470390985@qtp-171206900-11" prio=10 tid=0x0000000041201800 nid=0x6466 in > Object.wait() [0x00007fe7c4316000] > > java.lang.Thread.State: TIMED_WAITING (on object monitor) > > at java.lang.Object.wait(Native Method) > > - waiting on <0x00007fe7cbaf6550> (a > org.mortbay.thread.QueuedThreadPool$PoolThread) > > at > > org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:626 > ) > > - locked <0x00007fe7cbaf6550> (a > org.mortbay.thread.QueuedThreadPool$PoolThread) > > > > "RS_OPEN_REGION-linux108,60020,1303201486948-0" daemon prio=10 > tid=0x00000000407e5800 nid=0x63d4 waiting on condition [0x00007fe7be6e5000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7ccaf1c88> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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 > > java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:947) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 07) > > at java.lang.Thread.run(Thread.java:619) > > > > "ResponseProcessor for block blk_1303201331094_1008" daemon prio=10 > tid=0x00000000407db000 nid=0x63bf runnable [0x00007fe7be7e6000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7ccbbce50> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7ccbbce38> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7ccbbcad8> (a sun.nio.ch.EPollSelectorImpl) > > at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) > > at > > org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTi > meout.java:333) > > at > > org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:158) > > 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.protocol.DataTransferProtocol$PipelineAck.readFields( > DataTransferProtocol.java:133) > > at > > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSCl > ient.java:3236) > > > > "LRU Statistics #0" prio=10 tid=0x00007fe7b862a800 nid=0x63bc waiting on > condition [0x00007fe7be8e7000] > > java.lang.Thread.State: TIMED_WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbac2248> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > at > java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198) > > at > > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitN > anos(AbstractQueuedSynchronizer.java:1963) > > at java.util.concurrent.DelayQueue.take(DelayQueue.java:164) > > at > > java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(Sched > uledThreadPoolExecutor.java:583) > > at > > java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(Sched > uledThreadPoolExecutor.java:576) > > at > > java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:947) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 07) > > at java.lang.Thread.run(Thread.java:619) > > > > "LruBlockCache.EvictionThread" daemon prio=10 tid=0x00007fe7b862a000 > nid=0x63bb in Object.wait() [0x00007fe7be9e8000] > > java.lang.Thread.State: WAITING (on object monitor) > > at java.lang.Object.wait(Native Method) > > - waiting on <0x00007fe7cbb08268> (a > org.apache.hadoop.hbase.io.hfile.LruBlockCache$EvictionThread) > > at java.lang.Object.wait(Object.java:485) > > at > > org.apache.hadoop.hbase.io.hfile.LruBlockCache$EvictionThread.run(LruBlockCa > che.java:519) > > - locked <0x00007fe7cbb08268> (a > org.apache.hadoop.hbase.io.hfile.LruBlockCache$EvictionThread) > > > > "PRI IPC Server handler 9 on 60020" daemon prio=10 tid=0x00007fe7b85fd800 > nid=0x63ba waiting on condition [0x00007fe7beae9000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbad3d10> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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) > > > > "PRI IPC Server handler 8 on 60020" daemon prio=10 tid=0x00007fe7b85fb800 > nid=0x63b9 waiting on condition [0x00007fe7bebea000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbad3d10> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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) > > > > "PRI IPC Server handler 7 on 60020" daemon prio=10 tid=0x00007fe7b85f9800 > nid=0x63b8 waiting on condition [0x00007fe7beceb000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbad3d10> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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) > > > > "PRI IPC Server handler 6 on 60020" daemon prio=10 tid=0x00007fe7b85f8000 > nid=0x63b7 waiting on condition [0x00007fe7bedec000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbad3d10> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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) > > > > "PRI IPC Server handler 5 on 60020" daemon prio=10 tid=0x00007fe7b85f6000 > nid=0x63b6 waiting on condition [0x00007fe7beeed000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbad3d10> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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) > > > > "PRI IPC Server handler 4 on 60020" daemon prio=10 tid=0x00007fe7b85f4000 > nid=0x63b5 waiting on condition [0x00007fe7befee000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbad3d10> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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) > > > > "PRI IPC Server handler 3 on 60020" daemon prio=10 tid=0x00007fe7b85f2000 > nid=0x63b4 waiting on condition [0x00007fe7bf0ef000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbad3d10> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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) > > > > "PRI IPC Server handler 2 on 60020" daemon prio=10 tid=0x00007fe7b85ef000 > nid=0x63b3 waiting on condition [0x00007fe7bf1f0000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbad3d10> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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) > > > > "PRI IPC Server handler 1 on 60020" daemon prio=10 tid=0x00007fe7b85ee800 > nid=0x63b2 waiting on condition [0x00007fe7bf2f1000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbad3d10> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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) > > > > "PRI IPC Server handler 0 on 60020" daemon prio=10 tid=0x00007fe7b85ed800 > nid=0x63b1 waiting on condition [0x00007fe7bf3f2000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbad3d10> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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 9 on 60020" daemon prio=10 tid=0x00007fe7b85ed000 > nid=0x63b0 waiting for monitor entry [0x00007fe7bf4f3000] > > java.lang.Thread.State: BLOCKED (on object monitor) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > > > "IPC Server handler 8 on 60020" daemon prio=10 tid=0x00007fe7b85ec800 > nid=0x63af waiting for monitor entry [0x00007fe7bf5f4000] > > java.lang.Thread.State: BLOCKED (on object monitor) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > > > "IPC Server handler 7 on 60020" daemon prio=10 tid=0x00007fe7b85eb800 > nid=0x63ae waiting on condition [0x00007fe7bf6f5000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbb06228> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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(Abstract > QueuedSynchronizer.java:778) > > at > > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueued > Synchronizer.java:1114) > > at > > java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java > :186) > > at > java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:262) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java:429) > > - locked <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > > > "IPC Server handler 6 on 60020" daemon prio=10 tid=0x00007fe7b85eb000 > nid=0x63ad waiting for monitor entry [0x00007fe7bf7f6000] > > java.lang.Thread.State: BLOCKED (on object monitor) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > > > "IPC Server handler 5 on 60020" daemon prio=10 tid=0x00007fe7b85ea000 > nid=0x63ac waiting for monitor entry [0x00007fe7bf8f7000] > > java.lang.Thread.State: BLOCKED (on object monitor) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > > > "IPC Server handler 4 on 60020" daemon prio=10 tid=0x00007fe7b85e9800 > nid=0x63ab waiting for monitor entry [0x00007fe7bf9f8000] > > java.lang.Thread.State: BLOCKED (on object monitor) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > > > "IPC Server handler 3 on 60020" daemon prio=10 tid=0x00007fe7b854a800 > nid=0x63aa waiting for monitor entry [0x00007fe7bfaf9000] > > java.lang.Thread.State: BLOCKED (on object monitor) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > > > "IPC Server handler 2 on 60020" daemon prio=10 tid=0x00007fe7b854a000 > nid=0x63a9 waiting for monitor entry [0x00007fe7bfbfa000] > > java.lang.Thread.State: BLOCKED (on object monitor) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > > > "IPC Server handler 1 on 60020" daemon prio=10 tid=0x00007fe7b83ba000 > nid=0x63a8 waiting for monitor entry [0x00007fe7bfcfb000] > > java.lang.Thread.State: BLOCKED (on object monitor) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > > > "IPC Server handler 0 on 60020" daemon prio=10 tid=0x00007fe7b8542000 > nid=0x63a7 waiting for monitor entry [0x00007fe7bfdfc000] > > java.lang.Thread.State: BLOCKED (on object monitor) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > > > "IPC Server listener on 60020" daemon prio=10 tid=0x00007fe7b84d2000 > nid=0x63a6 runnable [0x00007fe7bfefd000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbb01528> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbb01510> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf648> (a sun.nio.ch.EPollSelectorImpl) > > 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:414) > > > > "IPC Server Responder" daemon prio=10 tid=0x00007fe7b8549000 nid=0x63a5 > runnable [0x00007fe7bfffe000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbab2d68> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbab2d50> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbaba240> (a sun.nio.ch.EPollSelectorImpl) > > at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:588) > > > > "Timer-0" daemon prio=10 tid=0x00007fe7b8544800 nid=0x63a4 in Object.wait() > [0x00007fe7c4114000] > > java.lang.Thread.State: TIMED_WAITING (on object monitor) > > at java.lang.Object.wait(Native Method) > > - waiting on <0x00007fe7cbacf138> (a java.util.TaskQueue) > > at java.util.TimerThread.mainLoop(Timer.java:509) > > - locked <0x00007fe7cbacf138> (a java.util.TaskQueue) > > at java.util.TimerThread.run(Timer.java:462) > > > > "251172046@qtp-171206900-1 - Acceptor0 > SelectChannelConnector@0.0.0.0:60030" > prio=10 tid=0x00007fe7b84e1800 nid=0x63a3 runnable [0x00007fe7c4215000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cc0536c8> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cc0536b0> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7ccb05208> (a sun.nio.ch.EPollSelectorImpl) > > at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) > > at > > org.mortbay.io.nio.SelectorManager$SelectSet.doSelect(SelectorManager.java:4 > 98) > > at > org.mortbay.io.nio.SelectorManager.doSelect(SelectorManager.java:192) > > at > > org.mortbay.jetty.nio.SelectChannelConnector.accept(SelectChannelConnector.j > ava:124) > > at > > org.mortbay.jetty.AbstractConnector$Acceptor.run(AbstractConnector.java:708) > > at > > org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:582 > ) > > > > "regionserver60020.leaseChecker" daemon prio=10 tid=0x00007fe7b8349000 > nid=0x63a1 waiting on condition [0x00007fe7c4417000] > > java.lang.Thread.State: TIMED_WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbb05168> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > at > java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198) > > at > > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitN > anos(AbstractQueuedSynchronizer.java:1963) > > at java.util.concurrent.DelayQueue.poll(DelayQueue.java:201) > > at > org.apache.hadoop.hbase.regionserver.Leases.run(Leases.java:82) > > > > "regionserver60020.majorCompactionChecker" daemon prio=10 > tid=0x00007fe7b831e000 nid=0x63a0 in Object.wait() [0x00007fe7c4518000] > > java.lang.Thread.State: TIMED_WAITING (on object monitor) > > at java.lang.Object.wait(Native Method) > > - waiting on <0x00007fe7ccabcdb0> (a java.lang.Object) > > at org.apache.hadoop.hbase.util.Sleeper.sleep(Sleeper.java:91) > > - locked <0x00007fe7ccabcdb0> (a java.lang.Object) > > at org.apache.hadoop.hbase.Chore.run(Chore.java:74) > > > > "regionserver60020.compactor" daemon prio=10 tid=0x00007fe7b8351000 > nid=0x639f waiting on condition [0x00007fe7c4619000] > > java.lang.Thread.State: TIMED_WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbb051e0> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > at > java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198) > > at > > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitN > anos(AbstractQueuedSynchronizer.java:1963) > > at > > java.util.concurrent.PriorityBlockingQueue.poll(PriorityBlockingQueue.java:2 > 45) > > at > > org.apache.hadoop.hbase.regionserver.PriorityCompactionQueue.poll(PriorityCo > mpactionQueue.java:248) > > at > > org.apache.hadoop.hbase.regionserver.CompactSplitThread.run(CompactSplitThre > ad.java:75) > > > > "regionserver60020.cacheFlusher" daemon prio=10 tid=0x00007fe7b831b800 > nid=0x639e waiting for monitor entry [0x00007fe7c4719000] > > java.lang.Thread.State: BLOCKED (on object monitor) > > at java.util.ResourceBundle.endLoading(ResourceBundle.java:1506) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at java.util.ResourceBundle.findBundle(ResourceBundle.java:1379) > > at java.util.ResourceBundle.findBundle(ResourceBundle.java:1292) > > at > java.util.ResourceBundle.getBundleImpl(ResourceBundle.java:1234) > > at java.util.ResourceBundle.getBundle(ResourceBundle.java:832) > > at sun.util.resources.LocaleData$1.run(LocaleData.java:127) > > at java.security.AccessController.$$YJP$$doPrivileged(Native > Method) > > at > java.security.AccessController.doPrivileged(AccessController.java) > > at sun.util.resources.LocaleData.getBundle(LocaleData.java:125) > > at > sun.util.resources.LocaleData.getTimeZoneNames(LocaleData.java:97) > > at > sun.util.TimeZoneNameUtility.getBundle(TimeZoneNameUtility.java:115) > > at > > sun.util.TimeZoneNameUtility.retrieveDisplayNames(TimeZoneNameUtility.java:8 > 0) > > at java.util.TimeZone.getDisplayNames(TimeZone.java:399) > > at java.util.TimeZone.getDisplayName(TimeZone.java:350) > > at java.util.Date.toString(Date.java:1025) > > at java.lang.String.valueOf(String.java:2826) > > at java.lang.StringBuilder.append(StringBuilder.java:115) > > at > > org.apache.hadoop.hbase.regionserver.PriorityCompactionQueue$CompactionReque > st.toString(PriorityCompactionQueue.java:114) > > at java.lang.String.valueOf(String.java:2826) > > at java.lang.StringBuilder.append(StringBuilder.java:115) > > at > > org.apache.hadoop.hbase.regionserver.PriorityCompactionQueue.addToRegionsInQ > ueue(PriorityCompactionQueue.java:145) > > - locked <0x00007fe7ccabd258> (a java.util.HashMap) > > at > > org.apache.hadoop.hbase.regionserver.PriorityCompactionQueue.add(PriorityCom > pactionQueue.java:188) > > at > > org.apache.hadoop.hbase.regionserver.CompactSplitThread.requestCompaction(Co > mpactSplitThread.java:140) > > - locked <0x00007fe7cbaf08c8> (a > org.apache.hadoop.hbase.regionserver.CompactSplitThread) > > at > > org.apache.hadoop.hbase.regionserver.CompactSplitThread.requestCompaction(Co > mpactSplitThread.java:118) > > - locked <0x00007fe7cbaf08c8> (a > org.apache.hadoop.hbase.regionserver.CompactSplitThread) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlu > sher.java:387) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushOneForGlobalPressu > re(MemStoreFlusher.java:194) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.run(MemStoreFlusher.jav > a:214) > > > > "regionserver60020.logRoller" daemon prio=10 tid=0x00007fe7b836d800 > nid=0x639d in Object.wait() [0x00007fe7c481b000] > > java.lang.Thread.State: TIMED_WAITING (on object monitor) > > at java.lang.Object.wait(Native Method) > > - waiting on <0x00007fe7cbacdbb0> (a > java.util.concurrent.atomic.AtomicBoolean) > > at > org.apache.hadoop.hbase.regionserver.LogRoller.run(LogRoller.java:78) > > - locked <0x00007fe7cbacdbb0> (a > java.util.concurrent.atomic.AtomicBoolean) > > > > "regionserver60020.logSyncer" daemon prio=10 tid=0x00007fe7b832e800 > nid=0x639c waiting on condition [0x00007fe7c491c000] > > java.lang.Thread.State: TIMED_WAITING (sleeping) > > at java.lang.Thread.$$YJP$$sleep(Native Method) > > at java.lang.Thread.sleep(Thread.java) > > at > org.apache.hadoop.hbase.regionserver.wal.HLog$LogSyncer.run(HLog.java:963) > > > > "LeaseChecker" daemon prio=10 tid=0x00007fe7b831d000 nid=0x639b waiting on > condition [0x00007fe7c4a1d000] > > java.lang.Thread.State: TIMED_WAITING (sleeping) > > at java.lang.Thread.$$YJP$$sleep(Native Method) > > at java.lang.Thread.sleep(Thread.java) > > at > org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:1458) > > at java.lang.Thread.run(Thread.java:619) > > > > "DataStreamer for file > /hbase/.logs/linux108,60020,1303201486948/linux108%3A60020.1303201488470 > block blk_1303201331094_1008" daemon prio=10 tid=0x00007fe7b832c800 > nid=0x639a in Object.wait() [0x00007fe7c4b1e000] > > java.lang.Thread.State: TIMED_WAITING (on object monitor) > > at java.lang.Object.wait(Native Method) > > - waiting on <0x00007fe7cbacdf10> (a java.util.LinkedList) > > at > > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient. > java:3036) > > - locked <0x00007fe7cbacdf10> (a java.util.LinkedList) > > > > "sendParams-0" daemon prio=10 tid=0x00007fe7b832d800 nid=0x6399 waiting on > condition [0x00007fe7c4c1f000] > > java.lang.Thread.State: TIMED_WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbafc3a8> (a > java.util.concurrent.SynchronousQueue$TransferStack) > > at sun.misc.Unsafe.park(Unsafe.java) > > at > java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198) > > at > > java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(Synchronous > Queue.java:424) > > at > > java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueu > e.java:323) > > at > java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:874) > > at > > java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:945) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 07) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Client (-1912063050) connection to linux253/10.18.52.253:60000 from > an > unknown user" daemon prio=10 tid=0x00007fe7b82aa000 nid=0x6394 in > Object.wait() [0x00007fe7c4d20000] > > java.lang.Thread.State: TIMED_WAITING (on object monitor) > > at java.lang.Object.wait(Native Method) > > - waiting on <0x00007fe7cc597be8> (a > org.apache.hadoop.hbase.ipc.HBaseClient$Connection) > > at > > org.apache.hadoop.hbase.ipc.HBaseClient$Connection.waitForWork(HBaseClient.j > ava:431) > > - locked <0x00007fe7cc597be8> (a > org.apache.hadoop.hbase.ipc.HBaseClient$Connection) > > at > > org.apache.hadoop.hbase.ipc.HBaseClient$Connection.run(HBaseClient.java:476) > > > > "DestroyJavaVM" prio=10 tid=0x00007fe7c016b000 nid=0x6361 waiting on > condition [0x0000000000000000] > > java.lang.Thread.State: RUNNABLE > > > > "regionserver60020-EventThread" daemon prio=10 tid=0x0000000040520000 > nid=0x6381 waiting on condition [0x00007fe7c4f22000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbb053c0> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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:502) > > > > "regionserver60020-SendThread(10.18.52.96:2191)" daemon prio=10 > tid=0x0000000040425800 nid=0x6380 runnable [0x00007fe7c5023000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7ccabcf30> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7ccabcf18> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7ccac12c8> (a sun.nio.ch.EPollSelectorImpl) > > at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) > > at > org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1107) > > > > "regionserver60020" prio=10 tid=0x00007fe7b826e800 nid=0x637f waiting on > condition [0x00007fe7c5124000] > > java.lang.Thread.State: TIMED_WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cc5ba120> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > at > java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:198) > > at > > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitN > anos(AbstractQueuedSynchronizer.java:1963) > > at > java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:395) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:61 > 6) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Reader 9 on port 60020" prio=10 tid=0x00007fe7b8075800 nid=0x637e > runnable [0x00007fe7c5225000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbab0890> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbab0878> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf198> (a sun.nio.ch.EPollSelectorImpl) > > 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$Reader.run(HBaseServer.java > :305) > > - locked <0x00007fe7cbad3bf8> (a > org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Reader 8 on port 60020" prio=10 tid=0x00007fe7b8074800 nid=0x637d > runnable [0x00007fe7c5326000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbadb4f0> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbadb4d8> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf3f0> (a sun.nio.ch.EPollSelectorImpl) > > 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$Reader.run(HBaseServer.java > :305) > > - locked <0x00007fe7cbb00d68> (a > org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Reader 7 on port 60020" prio=10 tid=0x00007fe7b8074000 nid=0x637c > runnable [0x00007fe7c5427000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbadc7d0> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbadc7b8> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf5d0> (a sun.nio.ch.EPollSelectorImpl) > > 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$Reader.run(HBaseServer.java > :305) > > - locked <0x00007fe7cbb00d40> (a > org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Reader 6 on port 60020" prio=10 tid=0x00007fe7b8073800 nid=0x637b > runnable [0x00007fe7c5528000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbadb610> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbadb5f8> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf468> (a sun.nio.ch.EPollSelectorImpl) > > 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$Reader.run(HBaseServer.java > :305) > > - locked <0x00007fe7cbad3f40> (a > org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Reader 5 on port 60020" prio=10 tid=0x00007fe7b806f800 nid=0x637a > runnable [0x00007fe7c5629000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbab0b60> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbab0b48> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf288> (a sun.nio.ch.EPollSelectorImpl) > > 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$Reader.run(HBaseServer.java > :305) > > - locked <0x00007fe7cbad3f18> (a > org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Reader 4 on port 60020" prio=10 tid=0x00007fe7b8072000 nid=0x6379 > runnable [0x00007fe7c572a000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbab0c80> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbab0c68> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf300> (a sun.nio.ch.EPollSelectorImpl) > > 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$Reader.run(HBaseServer.java > :305) > > - locked <0x00007fe7cbad3ef0> (a > org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Reader 3 on port 60020" prio=10 tid=0x00007fe7b8083000 nid=0x6378 > runnable [0x00007fe7c582b000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbadc6b0> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbadc698> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf558> (a sun.nio.ch.EPollSelectorImpl) > > 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$Reader.run(HBaseServer.java > :305) > > - locked <0x00007fe7cbad3ec8> (a > org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Reader 2 on port 60020" prio=10 tid=0x00007fe7b8056800 nid=0x6377 > runnable [0x00007fe7c592c000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbab0a40> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbab0a28> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf210> (a sun.nio.ch.EPollSelectorImpl) > > 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$Reader.run(HBaseServer.java > :305) > > - locked <0x00007fe7cbad3ea0> (a > org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Reader 1 on port 60020" prio=10 tid=0x00007fe7b806d800 nid=0x6376 > runnable [0x00007fe7c5a2d000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbadb730> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbadb718> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf4e0> (a sun.nio.ch.EPollSelectorImpl) > > 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$Reader.run(HBaseServer.java > :305) > > - locked <0x00007fe7cbad3e78> (a > org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:619) > > > > "IPC Reader 0 on port 60020" prio=10 tid=0x00007fe7b806a800 nid=0x6375 > runnable [0x00007fe7c5b2e000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbadb3d0> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbadb3b8> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf378> (a sun.nio.ch.EPollSelectorImpl) > > 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$Reader.run(HBaseServer.java > :305) > > - locked <0x00007fe7cbad3e50> (a > org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:619) > > > > "main-EventThread" daemon prio=10 tid=0x00007fe7b8080800 nid=0x6374 waiting > on condition [0x00007fe7c6084000] > > java.lang.Thread.State: WAITING (parking) > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbb00e80> (a > java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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:502) > > > > "main-SendThread(10.18.52.109:2191)" daemon prio=10 tid=0x00007fe7b8063000 > nid=0x6373 runnable [0x00007fe7c6185000] > > java.lang.Thread.State: RUNNABLE > > at sun.nio.ch.EPollArrayWrapper.$$YJP$$epollWait(Native Method) > > at > sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java) > > 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 <0x00007fe7cbb01738> (a sun.nio.ch.Util$1) > > - locked <0x00007fe7cbb01720> (a > java.util.Collections$UnmodifiableSet) > > - locked <0x00007fe7cbacf6c0> (a sun.nio.ch.EPollSelectorImpl) > > at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) > > at > org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1107) > > > > "Low Memory Detector" daemon prio=10 tid=0x0000000040301000 nid=0x6370 > runnable [0x0000000000000000] > > java.lang.Thread.State: RUNNABLE > > > > "CompilerThread1" daemon prio=10 tid=0x00000000402e4800 nid=0x636f waiting > on condition [0x0000000000000000] > > java.lang.Thread.State: RUNNABLE > > > > "CompilerThread0" daemon prio=10 tid=0x000000004029b800 nid=0x636e waiting > on condition [0x0000000000000000] > > java.lang.Thread.State: RUNNABLE > > > > "YJPAgent-OOMESnapshotDetector" daemon prio=10 tid=0x00000000402f4000 > nid=0x636d runnable [0x0000000000000000] > > java.lang.Thread.State: RUNNABLE > > > > "YJPAgent-CPUSampler" daemon prio=10 tid=0x00000000402eb800 nid=0x636c > runnable [0x0000000000000000] > > java.lang.Thread.State: RUNNABLE > > > > "YJPAgent-RequestListener" daemon prio=10 tid=0x00000000402e7800 nid=0x636b > runnable [0x00007fe7c6b78000] > > java.lang.Thread.State: RUNNABLE > > at java.net.PlainSocketImpl.$$YJP$$socketAccept(Native Method) > > at java.net.PlainSocketImpl.socketAccept(PlainSocketImpl.java) > > at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:390) > > - locked <0x00007fe7cbb01b30> (a java.net.SocksSocketImpl) > > at java.net.ServerSocket.implAccept(ServerSocket.java:453) > > at java.net.ServerSocket.accept(ServerSocket.java:421) > > at com.yourkit.runtime.Core$3.run(Core.java:561) > > at java.lang.Thread.run(Thread.java:619) > > > > "YJPAgent-Telemetry" daemon prio=10 tid=0x00007fe7c0159800 nid=0x636a > waiting on condition [0x00007fe7c6ef8000] > > java.lang.Thread.State: TIMED_WAITING (sleeping) > > at java.lang.Thread.$$YJP$$sleep(Native Method) > > at java.lang.Thread.sleep(Thread.java) > > at > com.yourkit.runtime.TelemetryThreadJVMTI.run(TelemetryThreadJVMTI.java:298) > > > > "Signal Dispatcher" daemon prio=10 tid=0x00007fe7c0090000 nid=0x6369 > runnable [0x0000000000000000] > > java.lang.Thread.State: RUNNABLE > > > > "Surrogate Locker Thread (CMS)" daemon prio=10 tid=0x00007fe7c00b0000 > nid=0x6368 waiting on condition [0x0000000000000000] > > java.lang.Thread.State: RUNNABLE > > > > "Finalizer" daemon prio=10 tid=0x00007fe7c0067000 nid=0x6367 in > Object.wait() [0x00007fe7c71fb000] > > java.lang.Thread.State: WAITING (on object monitor) > > at java.lang.Object.wait(Native Method) > > - waiting on <0x00007fe7cc00ad28> (a > java.lang.ref.ReferenceQueue$Lock) > > at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:118) > > - locked <0x00007fe7cc00ad28> (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) > > > > "Reference Handler" daemon prio=10 tid=0x00007fe7c0040000 nid=0x6366 in > Object.wait() [0x00007fe7c72fc000] > > java.lang.Thread.State: WAITING (on object monitor) > > at java.lang.Object.wait(Native Method) > > - waiting on <0x00007fe7cc00ad58> (a > java.lang.ref.Reference$Lock) > > at java.lang.Object.wait(Object.java:485) > > at > java.lang.ref.Reference$ReferenceHandler.run(Reference.java:116) > > - locked <0x00007fe7cc00ad58> (a java.lang.ref.Reference$Lock) > > > > "VM Thread" prio=10 tid=0x00007fe7c0050000 nid=0x6365 runnable > > > > "Gang worker#0 (Parallel GC Threads)" prio=10 tid=0x0000000040173000 > nid=0x6362 runnable > > > > "Gang worker#1 (Parallel GC Threads)" prio=10 tid=0x0000000040175000 > nid=0x6363 runnable > > > > "Concurrent Mark-Sweep GC Thread" prio=10 tid=0x0000000040248000 nid=0x6364 > runnable > > "VM Periodic Task Thread" prio=10 tid=0x00007fe7c00f3000 nid=0x6371 waiting > on condition > > > > JNI global references: 1304 > > > > > > Found one Java-level deadlock: > > ============================= > > "IPC Server handler 9 on 60020": > > waiting to lock monitor 0x00000000409f3908 (object 0x00007fe7cbacbd48, a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher), > > which is held by "IPC Server handler 7 on 60020" > > "IPC Server handler 7 on 60020": > > waiting for ownable synchronizer 0x00007fe7cbb06228, (a > java.util.concurrent.locks.ReentrantLock$NonfairSync), > > which is held by "regionserver60020.cacheFlusher" > > "regionserver60020.cacheFlusher": > > waiting to lock monitor 0x00000000409f3908 (object 0x00007fe7cbacbd48, a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher), > > which is held by "IPC Server handler 7 on 60020" > > > > Java stack information for the threads listed above: > > =================================================== > > "IPC Server handler 9 on 60020": > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > "IPC Server handler 7 on 60020": > > at sun.misc.Unsafe.$$YJP$$park(Native Method) > > - parking to wait for <0x00007fe7cbb06228> (a > java.util.concurrent.locks.ReentrantLock$NonfairSync) > > at sun.misc.Unsafe.park(Unsafe.java) > > 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(Abstract > QueuedSynchronizer.java:778) > > at > > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueued > Synchronizer.java:1114) > > at > > java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java > :186) > > at > java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:262) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.reclaimMemStoreMemory(M > emStoreFlusher.java:429) > > - locked <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at > > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java: > 2558) > > at sun.reflect.GeneratedMethodAccessor4.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:570) > > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > > "regionserver60020.cacheFlusher": > > at java.util.ResourceBundle.endLoading(ResourceBundle.java:1506) > > - waiting to lock <0x00007fe7cbacbd48> (a > org.apache.hadoop.hbase.regionserver.MemStoreFlusher) > > at java.util.ResourceBundle.findBundle(ResourceBundle.java:1379) > > at java.util.ResourceBundle.findBundle(ResourceBundle.java:1292) > > at > java.util.ResourceBundle.getBundleImpl(ResourceBundle.java:1234) > > at java.util.ResourceBundle.getBundle(ResourceBundle.java:832) > > at sun.util.resources.LocaleData$1.run(LocaleData.java:127) > > at java.security.AccessController.$$YJP$$doPrivileged(Native > Method) > > at > java.security.AccessController.doPrivileged(AccessController.java) > > at sun.util.resources.LocaleData.getBundle(LocaleData.java:125) > > at > sun.util.resources.LocaleData.getTimeZoneNames(LocaleData.java:97) > > at > sun.util.TimeZoneNameUtility.getBundle(TimeZoneNameUtility.java:115) > > at > > sun.util.TimeZoneNameUtility.retrieveDisplayNames(TimeZoneNameUtility.java:8 > 0) > > at java.util.TimeZone.getDisplayNames(TimeZone.java:399) > > at java.util.TimeZone.getDisplayName(TimeZone.java:350) > > at java.util.Date.toString(Date.java:1025) > > at java.lang.String.valueOf(String.java:2826) > > at java.lang.StringBuilder.append(StringBuilder.java:115) > > at > > org.apache.hadoop.hbase.regionserver.PriorityCompactionQueue$CompactionReque > st.toString(PriorityCompactionQueue.java:114) > > at java.lang.String.valueOf(String.java:2826) > > at java.lang.StringBuilder.append(StringBuilder.java:115) > > at > > org.apache.hadoop.hbase.regionserver.PriorityCompactionQueue.addToRegionsInQ > ueue(PriorityCompactionQueue.java:145) > > - locked <0x00007fe7ccabd258> (a java.util.HashMap) > > at > > org.apache.hadoop.hbase.regionserver.PriorityCompactionQueue.add(PriorityCom > pactionQueue.java:188) > > at > > org.apache.hadoop.hbase.regionserver.CompactSplitThread.requestCompaction(Co > mpactSplitThread.java:140) > > - locked <0x00007fe7cbaf08c8> (a > org.apache.hadoop.hbase.regionserver.CompactSplitThread) > > at > > org.apache.hadoop.hbase.regionserver.CompactSplitThread.requestCompaction(Co > mpactSplitThread.java:118) > > - locked <0x00007fe7cbaf08c8> (a > org.apache.hadoop.hbase.regionserver.CompactSplitThread) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlu > sher.java:387) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushOneForGlobalPressu > re(MemStoreFlusher.java:194) > > at > > org.apache.hadoop.hbase.regionserver.MemStoreFlusher.run(MemStoreFlusher.jav > a:214) > > > > Found 1 deadlock. > > > > --bcaec5215bd138b4a804a1521f4d--