You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "chenglei (Jira)" <ji...@apache.org> on 2021/11/01 11:28:00 UTC

[jira] [Comment Edited] (HBASE-25905) Shutdown of WAL stuck at waitForSafePoint

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

chenglei edited comment on HBASE-25905 at 11/1/21, 11:27 AM:
-------------------------------------------------------------

[~Xiaolin Ha],I have some problem about your description:
bq.I think this scenario can cause this problem:
bq.The consumer of WAL enters AsyncFSWAL#syncFailed, it will set writer broken and request roll writer;
bq.Regionserver close, shutdown the wal factory, enter the AsyncFSWAL#waitForSafePoint, waiting on ConditionObject readyForRollingCond, and current state is broken and waiting for roll;


was (Author: comnetwork):
[~Xiaolin Ha],I have some problem about your description:
bq.
I think this scenario can cause this problem:
The consumer of WAL enters AsyncFSWAL#syncFailed, it will set writer broken and request roll writer;
Regionserver close, shutdown the wal factory, enter the AsyncFSWAL#waitForSafePoint, waiting on ConditionObject readyForRollingCond, and current state is broken and waiting for roll;

> Shutdown of WAL stuck at waitForSafePoint
> -----------------------------------------
>
>                 Key: HBASE-25905
>                 URL: https://issues.apache.org/jira/browse/HBASE-25905
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver, wal
>    Affects Versions: 3.0.0-alpha-1, 2.0.0
>            Reporter: Xiaolin Ha
>            Assignee: Xiaolin Ha
>            Priority: Critical
>         Attachments: rs-jstack1, rs-jstack2, wal-stuck-error-logs.png
>
>
> We use the fan-out HDFS OutputStream and AsyncFSWAL on our clusters, but met the problem than RS can not exit completely for several hours util manual interventions.
> The two jstacks below show that the regionserver thread can waiting unlimitedly in both 
> AsyncFSWAL#waitForSafePoint()
> {code:java}
> "regionserver/gh-data-hbase-finance08.mt/10.22.179.24:16020" #29 prio=5 os_prio=0 tid=0x00007fb2feb5c000 nid=0xa92b waiting on condition [0x00007f9ccb992000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00007faea229a9d0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitUninterruptibly(AbstractQueuedSynchronizer.java:1976)
>         at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.waitForSafePoint(AsyncFSWAL.java:687)
>         at org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.doShutdown(AsyncFSWAL.java:743)
>         at org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.shutdown(AbstractFSWAL.java:900)
>         at org.apache.hadoop.hbase.wal.AbstractFSWALProvider.shutdown(AbstractFSWALProvider.java:182)
>         at org.apache.hadoop.hbase.wal.RegionGroupingProvider.shutdown(RegionGroupingProvider.java:232)
>         at org.apache.hadoop.hbase.wal.WALFactory.shutdown(WALFactory.java:271)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.shutdownWAL(HRegionServer.java:1405)
>         at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:1147)
>         at java.lang.Thread.run(Thread.java:745)
> {code}
> and the log roller stuck at waiting for lock
> {code:java}
> "regionserver/gh-data-hbase-finance08.mt/10.22.179.24:16020.logRoller" #322 daemon prio=5 os_prio=0 tid=0x00007fb2e11a4000 nid=0xa953 waiting on condition [0x00007f9cbd9f1000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00007faea1217048> (a java.util.concurrent.locks.ReentrantLock$FairSync)
>         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.acquireQueued(AbstractQueuedSynchronizer.java:870)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
>         at java.util.concurrent.locks.ReentrantLock$FairSync.lock(ReentrantLock.java:224)
>         at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285)
>         at org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(AbstractFSWAL.java:822)
>         at org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(AbstractWALRoller.java:269)
>         at org.apache.hadoop.hbase.wal.AbstractWALRoller.run(AbstractWALRoller.java:186){code}
>  
> I think this scenario can cause this problem:
>  # The consumer of WAL enters AsyncFSWAL#syncFailed, it will set writer broken and request roll writer;
>  # Regionserver close, shutdown the wal factory, enter the AsyncFSWAL#waitForSafePoint, waiting on ConditionObject readyForRollingCond, and current state is broken and waiting for roll;
>  # The AbstractFSWAL#rollWriter waiting on the lock of AbstractFSWAL.rollWriterLock, which of held by the regionserver thread when close();
>  # The consumer of WAL enters AsyncFSWAL#syncFailed, but found that writer is broken, and skipped to signal the readyForRollingCond;
>  
> {code:java}
> private void syncFailed(long epochWhenSync, Throwable error) {
>   LOG.warn("sync failed", error);
>   boolean shouldRequestLogRoll = true;
>   consumeLock.lock();
>   try {
>     int currentEpochAndState = epochAndState;
>     if (epoch(currentEpochAndState) != epochWhenSync || writerBroken(currentEpochAndState)) {
>       // this is not the previous writer which means we have already rolled the writer.
>       // or this is still the current writer, but we have already marked it as broken and request
>       // a roll.
>       return;
>     }
>     this.epochAndState = currentEpochAndState | 0b10;
>     if (waitingRoll(currentEpochAndState)) {
>       readyForRolling = true;
>       readyForRollingCond.signalAll();
>       // this means we have already in the middle of a rollWriter so just tell the roller thread
>       // that you can continue without requesting an extra log roll.
>       shouldRequestLogRoll = false;
>     }
>   } finally {
>     consumeLock.unlock();
>   }
>   for (Iterator<FSWALEntry> iter = unackedAppends.descendingIterator(); iter.hasNext();) {
>     toWriteAppends.addFirst(iter.next());
>   }
>   highestUnsyncedTxid = highestSyncedTxid.get();
>   if (shouldRequestLogRoll) {
>     // request a roll.
>     requestLogRoll(ERROR);
>   }
> }{code}
>  
> Then the regionserver thread stuck at AsyncFSWAL#waitForSafePoint, and never exit except manually kill it.
> So we should limit the shutdown time of WAL, to avoid waiting too long for the safe point.
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)