[
https://issues.apache.org/jira/browse/HBASE-26411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17482614#comment-17482614
]
Bryan Beaudreault commented on HBASE-26411:
-------------------------------------------
I've hit a case of this as well I think. Not exactly the same, but I have an
aborting regionserver hung waiting on this.
A regionserver aborted, but the process stayed up indefinitely. Here's the
stacks:
{code:java}
"regionserver/test2-host:60020.logRoller" #240 daemon prio=5 os_prio=0
cpu=88.15ms elapsed=3809.72s tid=0x00007f7820225000 nid=0x550a waiting on
condition [0x00007f77f1edd000]
java.lang.Thread.State: WAITING (parking)
at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
- parking to wait for <0x00000007a0ae1578> (a
java.util.concurrent.CompletableFuture$Signaller)
at
java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:194)
at
java.util.concurrent.CompletableFuture$Signaller.block([email protected]/CompletableFuture.java:1796)
at
java.util.concurrent.ForkJoinPool.managedBlock([email protected]/ForkJoinPool.java:3128)
at
java.util.concurrent.CompletableFuture.waitingGet([email protected]/CompletableFuture.java:1823)
at
java.util.concurrent.CompletableFuture.get([email protected]/CompletableFuture.java:1998)
at
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(AsyncProtobufLogWriter.java:189)
at
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(AsyncProtobufLogWriter.java:202)
at
org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(AbstractProtobufLogWriter.java:170)
at
org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(AsyncFSWALProvider.java:113)
at
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:669)
at
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:130)
at
org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(AbstractFSWAL.java:841)
at
org.apache.hadoop.hbase.wal.AbstractWALRoller$RollController.rollWal(AbstractWALRoller.java:268)
at
org.apache.hadoop.hbase.wal.AbstractWALRoller.run(AbstractWALRoller.java:187)
{code}
This is holding the rollWriterLock, and meanwhile shutdown is trying to acquire
it:
{code:java}
"regionserver/test2-host:60020" #16 prio=5 os_prio=0 cpu=2870.36ms
elapsed=3810.32s tid=0x00007f783e8a8000 nid=0x54fd waiting on condition
[0x00007f77f41eb000]
java.lang.Thread.State: WAITING (parking)
at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
- parking to wait for <0x0000000601b10cc0> (a
java.util.concurrent.locks.ReentrantLock$FairSync)
at
java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:194)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt([email protected]/AbstractQueuedSynchronizer.java:885)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued([email protected]/AbstractQueuedSynchronizer.java:917)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire([email protected]/AbstractQueuedSynchronizer.java:1240)
at
java.util.concurrent.locks.ReentrantLock.lock([email protected]/ReentrantLock.java:267)
at
org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.shutdown(AbstractFSWAL.java:902)
at
org.apache.hadoop.hbase.wal.AbstractFSWALProvider.shutdown(AbstractFSWALProvider.java:187)
at org.apache.hadoop.hbase.wal.WALFactory.shutdown(WALFactory.java:240)
at
org.apache.hadoop.hbase.regionserver.HRegionServer.shutdownWAL(HRegionServer.java:1544)
at
org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:1188)
{code}
I'm not very familiar with this code but I have a feeling the problem is that
whatever is supposed to be serving the AsyncProtobufLogWriter.write request has
been shutdown already.
This is on version 2.4.6
> Wal do not roll and write a big wal
> ------------------------------------
>
> Key: HBASE-26411
> URL: https://issues.apache.org/jira/browse/HBASE-26411
> Project: HBase
> Issue Type: Bug
> Affects Versions: 2.4.8
> Reporter: Lijin Bin
> Priority: Major
>
> We see wal have long time to roll and write a big wal which has 3TB.
> And according to the jstack we can see the wal create hang.
> {code}
> "regionserver/11.149.48.227:60020.logRoller" #667 daemon prio=5 os_prio=0
> cpu=116916.81ms elapsed=447455.26s tid=0x00007fa35d231000 nid=0xbdd2 waiting
> on condition [0x00007f79c7407000]
> java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x00007f9f10df5158> (a
> java.util.concurrent.CompletableFuture$Signaller)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at
> java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1707)
> at
> java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
> at
> java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1742)
> at
> java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
> at
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(AsyncProtobufLogWriter.java:178)
> at
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeMagicAndWALHeader(AsyncProtobufLogWriter.java:191)
> at
> org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.init(AbstractProtobufLogWriter.java:170)
> at
> org.apache.hadoop.hbase.wal.AsyncFSWALProvider.createAsyncWriter(AsyncFSWALProvider.java:113)
> at
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:615)
> at
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.createWriterInstance(AsyncFSWAL.java:126)
> at
> org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.rollWriter(AbstractFSWAL.java:763)
> at
> org.apache.hadoop.hbase.regionserver.LogRoller.run(LogRoller.java:184)
> at java.lang.Thread.run(Thread.java:748)
> {code}
--
This message was sent by Atlassian Jira
(v8.20.1#820001)