hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-13971) Flushes stuck since 6 hours on a regionserver.
Date Sat, 29 Aug 2015 18:37:45 GMT

    [ https://issues.apache.org/jira/browse/HBASE-13971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14721206#comment-14721206
] 

stack commented on HBASE-13971:
-------------------------------

The patch looks like it might timeout threads waiting on sequenceid such as this from the
attached jstack.1

{code}
"RS_CLOSE_REGION-hbase1495:16020-0" #1249 prio=5 os_prio=0 tid=0x0000000001e73000 nid=0x8ea
waiting on condition [0x00007fb3fdb55000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000001e000d3d8> (a java.util.concurrent.CountDownLatch$Sync)
	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
	at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
	at org.apache.hadoop.hbase.wal.WALKey.getSequenceId(WALKey.java:305)
	at org.apache.hadoop.hbase.regionserver.HRegion.getNextSequenceId(HRegion.java:2382)
	at org.apache.hadoop.hbase.regionserver.HRegion.internalPrepareFlushCache(HRegion.java:2144)
	at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2028)
	at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1992)
	at org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:1982)
	at org.apache.hadoop.hbase.regionserver.HRegion.doClose(HRegion.java:1397)
	at org.apache.hadoop.hbase.regionserver.HRegion.close(HRegion.java:1326)
	- locked <0x00000002c8eebd38> (a java.lang.Object)
	at org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler.process(CloseRegionHandler.java:138)
	at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)

{code}

...but other threads will just stick around because we do seem to address why the append machine
is blocked. Here is a handler that would stay stuck as I see it:

{code}
"RS_CLOSE_REGION-hbase1495:16020-2" #1300 prio=5 os_prio=0 tid=0x00007fb4551ee000 nid=0x99a
in Object.wait() [0x00007fb3fdc56000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
	at java.lang.Object.wait(Native Method)
	at org.apache.hadoop.hbase.regionserver.wal.SyncFuture.get(SyncFuture.java:167)
	- locked <0x00000002d9b7cbc8> (a org.apache.hadoop.hbase.regionserver.wal.SyncFuture)
	at org.apache.hadoop.hbase.regionserver.wal.FSHLog.blockOnSync(FSHLog.java:1466)
	at org.apache.hadoop.hbase.regionserver.wal.FSHLog.publishSyncThenBlockOnCompletion(FSHLog.java:1460)
	at org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1594)
	at org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeRegionEventMarker(WALUtil.java:97)
	at org.apache.hadoop.hbase.regionserver.HRegion.writeRegionCloseMarker(HRegion.java:978)
	at org.apache.hadoop.hbase.regionserver.HRegion.doClose(HRegion.java:1453)
	at org.apache.hadoop.hbase.regionserver.HRegion.close(HRegion.java:1326)
	- locked <0x00000001d47edc18> (a java.lang.Object)
	at org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler.process(CloseRegionHandler.java:138)
	at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)

{code}

I am not sure how this patch effects correctness either (what signal does the client get back,
what is unrolled from memstore). Lets see how HBASE-14317 goes. Am trying to reproduce the
failure in test rig (there are not tests here) so can be sure we've figured why the lockup
is happening.

> Flushes stuck since 6 hours on a regionserver.
> ----------------------------------------------
>
>                 Key: HBASE-13971
>                 URL: https://issues.apache.org/jira/browse/HBASE-13971
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver
>    Affects Versions: 1.3.0
>         Environment: Caused while running IntegrationTestLoadAndVerify for 20 M rows
on cluster with 32 region servers each with max heap size of 24GBs.
>            Reporter: Abhilash
>            Assignee: Ted Yu
>            Priority: Critical
>             Fix For: 2.0.0, 1.2.0, 1.3.0
>
>         Attachments: 13971-v1.txt, 13971-v1.txt, 13971-v1.txt, jstack.1, jstack.2, jstack.3,
jstack.4, jstack.5, rsDebugDump.txt, screenshot-1.png
>
>
> One region server stuck while flushing(possible deadlock). Its trying to flush two regions
since last 6 hours (see the screenshot).
> Caused while running IntegrationTestLoadAndVerify for 20 M rows with 600 mapper jobs
and 100 back references. ~37 Million writes on each regionserver till now but no writes happening
on any regionserver from past 6 hours  and their memstore size is zero(I dont know if this
is related). But this particular regionserver has memstore size of 9GBs from past 6 hours.
> Relevant snaps from debug dump:
> Tasks:
> ===========================================================
> Task: Flushing IntegrationTestLoadAndVerify,R\x9B\x1B\xBF\xAE\x08\xD1\xA2,1435179555993.8e2d075f94ce7699f416ec4ced9873cd.
> Status: RUNNING:Preparing to flush by snapshotting stores in 8e2d075f94ce7699f416ec4ced9873cd
> Running for 22034s
> Task: Flushing IntegrationTestLoadAndVerify,\x93\xA385\x81Z\x11\xE6,1435179555993.9f8d0e01a40405b835bf6e5a22a86390.
> Status: RUNNING:Preparing to flush by snapshotting stores in 9f8d0e01a40405b835bf6e5a22a86390
> Running for 22033s
> Executors:
> ===========================================================
> ...
> Thread 139 (MemStoreFlusher.1):
>   State: WAITING
>   Blocked count: 139711
>   Waited count: 239212
>   Waiting on java.util.concurrent.CountDownLatch$Sync@b9c094a
>   Stack:
>     sun.misc.Unsafe.park(Native Method)
>     java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>     java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
>     org.apache.hadoop.hbase.wal.WALKey.getSequenceId(WALKey.java:305)
>     org.apache.hadoop.hbase.regionserver.HRegion.getNextSequenceId(HRegion.java:2422)
>     org.apache.hadoop.hbase.regionserver.HRegion.internalPrepareFlushCache(HRegion.java:2168)
>     org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2047)
>     org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2011)
>     org.apache.hadoop.hbase.regionserver.HRegion.flushcache(HRegion.java:1902)
>     org.apache.hadoop.hbase.regionserver.HRegion.flush(HRegion.java:1828)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlusher.java:510)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlusher.java:471)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.access$900(MemStoreFlusher.java:75)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher$FlushHandler.run(MemStoreFlusher.java:259)
>     java.lang.Thread.run(Thread.java:745)
> Thread 137 (MemStoreFlusher.0):
>   State: WAITING
>   Blocked count: 138931
>   Waited count: 237448
>   Waiting on java.util.concurrent.CountDownLatch$Sync@53f41f76
>   Stack:
>     sun.misc.Unsafe.park(Native Method)
>     java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
>     java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>     java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
>     org.apache.hadoop.hbase.wal.WALKey.getSequenceId(WALKey.java:305)
>     org.apache.hadoop.hbase.regionserver.HRegion.getNextSequenceId(HRegion.java:2422)
>     org.apache.hadoop.hbase.regionserver.HRegion.internalPrepareFlushCache(HRegion.java:2168)
>     org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2047)
>     org.apache.hadoop.hbase.regionserver.HRegion.internalFlushcache(HRegion.java:2011)
>     org.apache.hadoop.hbase.regionserver.HRegion.flushcache(HRegion.java:1902)
>     org.apache.hadoop.hbase.regionserver.HRegion.flush(HRegion.java:1828)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlusher.java:510)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.flushRegion(MemStoreFlusher.java:471)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher.access$900(MemStoreFlusher.java:75)
>     org.apache.hadoop.hbase.regionserver.MemStoreFlusher$FlushHandler.run(MemStoreFlusher.java:259)
>     java.lang.Thread.run(Thread.java:745)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message