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

Xiaolin Ha edited comment on HBASE-25905 at 11/19/21, 5:23 AM:
---------------------------------------------------------------

The problem can be caused by the scenario as follows,

1. The consumer of WAL enters AsyncFSWAL#syncFailed, it will set writer broken 
and request roll writer, and also add all the unackedAppends back to the 
toWriteAppends;

2. Writer rolls and new WAL created, new consumer executed, when calling 
AsyncFSWAL#appendAndSync, add partial toWriteAppends to the writer, limited the 
writer length to the WAL_BATCH_SIZE. So after the consume, there are still some 
toWriteAppends and unackedAppends in the queue. 
{code:java}
for (Iterator<FSWALEntry> iter = toWriteAppends.iterator(); iter.hasNext();) {
      FSWALEntry entry = iter.next();
      boolean appended;
      try {
        appended = appendEntry(writer, entry);
      } catch (IOException e) {
        throw new AssertionError("should not happen", e);
      }
      newHighestProcessedAppendTxid = entry.getTxid();
      iter.remove();
      if (appended) {
        // This is possible, when we fail to sync, we will add the 
unackedAppends back to
        // toWriteAppends, so here we may get an entry which is already in the 
unackedAppends.
        if (unackedAppends.isEmpty() || unackedAppends.peekLast().getTxid() < 
entry.getTxid()) {
          unackedAppends.addLast(entry);
        }
        if (writer.getLength() - fileLengthAtLastSync >= batchSize) {
          break;
        }
      }
    }
...{code}

3. sync(writer) is called, then 'fileLengthAtLastSync = writer.getLength()', 
{code:java}
private void sync(AsyncWriter writer) {
    fileLengthAtLastSync = writer.getLength();
... {code}

4. Regionserver abort(server.isAborted() is true), shutdown the wal factory, 
enter the AsyncFSWAL#waitForSafePoint, waiting on ConditionObject 
readyForRollingCond, and current state is waiting for roll, triggers a consumer,
just return without signal the readyForRollingCond, because current 
fileLengthAtLastSync = writer.getLength() and unackedAppends is not empty,
     
{code:java}
if (waitingRoll(currentEpochAndState)) {
        if (writer.getLength() > fileLengthAtLastSync) {
          // issue a sync
          sync(writer);
        } else {
          if (unackedAppends.isEmpty()) {
            readyForRolling = true;
            readyForRollingCond.signalAll();
          }
        }
        return;
      }
... {code}

5. the thread calling sync(writer) in step 3 calls AsyncFSWAL#syncCompleted, 
when calling trySetReadyForRolling, find that unackedAppends is not empty, so 
just return without signal the readyForRollingCond,
{code:java}
private boolean trySetReadyForRolling() {
    // Check without holding lock first. Usually we will just return here.
    // waitingRoll is volatile and unacedEntries is only accessed inside event 
loop so it is safe to
    // check them outside the consumeLock.
    if (!waitingRoll(epochAndState) || !unackedAppends.isEmpty()) {
      return false;
    }
... {code}
6. Since the regionserver is aborted and last consumer has set 
AsyncFSWAL#waitingRoll be true, each consumer will just return like step 4;
7. Since the shutdown of WAL keeps the rollWriterLock, roll wal will not happen;
8. The regionserver thread stuck at AsyncFSWAL#waitForSafePoint.


was (Author: xiaolin ha):
The problem can be caused by the scenario as follows,

1. The consumer of WAL enters AsyncFSWAL#syncFailed, it will set writer broken 
and request roll writer, and also add all the unackedAppends back to the 
toWriteAppends;

2. Writer rolls and new WAL created, new consumer executed, when calling 
AsyncFSWAL#appendAndSync, add partial toWriteAppends to the writer, limited the 
writer length to the WAL_BATCH_SIZE. So after the consume, there are still some 
toWriteAppends and unackedAppends in the queue. 
for (Iterator<FSWALEntry> iter = toWriteAppends.iterator(); iter.hasNext();) {
      FSWALEntry entry = iter.next();
      boolean appended;
      try {
        appended = appendEntry(writer, entry);
      } catch (IOException e) {
        throw new AssertionError("should not happen", e);
      }
      newHighestProcessedAppendTxid = entry.getTxid();
      iter.remove();
      if (appended) {
        // This is possible, when we fail to sync, we will add the 
unackedAppends back to
        // toWriteAppends, so here we may get an entry which is already in the 
unackedAppends.
        if (unackedAppends.isEmpty() || unackedAppends.peekLast().getTxid() < 
entry.getTxid()) {
          unackedAppends.addLast(entry);
        }
        if (writer.getLength() - fileLengthAtLastSync >= batchSize) {
          break;
        }
      }
    }
    ...
3. sync(writer) is called, then 'fileLengthAtLastSync = writer.getLength()', 
private void sync(AsyncWriter writer) {
    fileLengthAtLastSync = writer.getLength();
    ...
4. Regionserver abort(server.isAborted() is true), shutdown the wal factory, 
enter the AsyncFSWAL#waitForSafePoint, waiting on ConditionObject 
readyForRollingCond, and current state is waiting for roll, triggers a consumer,
just return without signal the readyForRollingCond, because current 
fileLengthAtLastSync = writer.getLength() and unackedAppends is not empty,
      if (waitingRoll(currentEpochAndState)) {
        if (writer.getLength() > fileLengthAtLastSync) {
          // issue a sync
          sync(writer);
        } else {
          if (unackedAppends.isEmpty()) {
            readyForRolling = true;
            readyForRollingCond.signalAll();
          }
        }
        return;
      }
5. the thread calling sync(writer) in step 3 calls AsyncFSWAL#syncCompleted, 
when calling trySetReadyForRolling, find that unackedAppends is not empty, so 
just return without signal the readyForRollingCond,
private boolean trySetReadyForRolling() {
    // Check without holding lock first. Usually we will just return here.
    // waitingRoll is volatile and unacedEntries is only accessed inside event 
loop so it is safe to
    // check them outside the consumeLock.
    if (!waitingRoll(epochAndState) || !unackedAppends.isEmpty()) {
      return false;
    }
...

6. Since the regionserver is aborted and last consumer has set 
AsyncFSWAL#waitingRoll be true, each consumer will just return like step 4;
7. Since the shutdown of WAL keeps the rollWriterLock, roll wal will not happen;
8. The regionserver thread stuck at AsyncFSWAL#waitForSafePoint.

> 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.20.1#820001)

Reply via email to