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

Xiaolin Ha commented on HBASE-25905:
------------------------------------

So this problem offten occurs with WAL stuck issues is because there are too 
many entries in the buffer need to flush, but once sync of writer can only sync 
no more than 

FanOutOneBlockAsyncDFSOutput.maxDataLen bytes.

> 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)

Reply via email to