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

Viraj Jasani commented on HBASE-26411:
--------------------------------------

Thanks for sharing this [~bbeaudreault]. I have the exact similar traces to 
share, on the same version 2.4.6.

Ever since I have been able to consistently reproduce this issue, I have 
switched back to FSHLog and now I see some DirectMemory leaks HBASE-26708, due 
to which Netty Rpc server stays stuck, keeping all the incoming requests on 
hold. I found these leaks to be more critical and hence I delayed taking thread 
dumps and commenting on this Jira reg the issue with AsyncFSWAL but I just 
checked after your above comment and I see the exact same logs.

Sequence of errors starting with sync failure:
{code:java}
2022-01-25 16:48:42,965 WARN  [hoenix-hbase4a/hbase] wal.AsyncFSWAL - sync 
failed
java.io.IOException: Bad response ERROR_CHECKSUM for block 
BP-899691376-10.118.169.39-1642167885215:blk_1073874435_134258 from datanode 
dn-us-west-2a.datanode-us-west-2a-cluster1.local/10.118.164.153:50010
    at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.lambda$channelRead0$0(FanOutOneBlockAsyncDFSOutput.java:266)
    at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.failed(FanOutOneBlockAsyncDFSOutput.java:233)
    at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.access$300(FanOutOneBlockAsyncDFSOutput.java:98)
    at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.channelRead0(FanOutOneBlockAsyncDFSOutput.java:265)
    at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.channelRead0(FanOutOneBlockAsyncDFSOutput.java:252)
    at 
org.apache.hbase.thirdparty.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
    at 
org.apache.hbase.thirdparty.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
    at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:324)
    at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:296)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
    at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:286)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
    at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
    at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper$DecryptHandler.channelRead0(FanOutOneBlockAsyncDFSOutputSaslHelper.java:703)
    at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper$DecryptHandler.channelRead0(FanOutOneBlockAsyncDFSOutputSaslHelper.java:674)
 {code}
 

Followed by usual stream broken errors:
{code:java}
2022-01-25 16:48:42,985 WARN  [Close-WAL-Writer-7] 
wal.AbstractProtobufLogWriter - Failed to write trailer, non-fatal, 
continuing...
java.io.IOException: stream already broken
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.flush0(FanOutOneBlockAsyncDFSOutput.java:420)
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.flush(FanOutOneBlockAsyncDFSOutput.java:509)
        at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.lambda$writeWALTrailerAndMagic$3(AsyncProtobufLogWriter.java:231)
        at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(AsyncProtobufLogWriter.java:187)
        at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeWALTrailerAndMagic(AsyncProtobufLogWriter.java:222)
        at 
org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.writeWALTrailer(AbstractProtobufLogWriter.java:261)
        at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.close(AsyncProtobufLogWriter.java:157)
        at 
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.lambda$closeWriter$5(AsyncFSWAL.java:698)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748) {code}
And this obvious error:
{code:java}
2022-01-25 16:48:42,985 WARN  [Close-WAL-Writer-7] wal.AsyncProtobufLogWriter - 
normal close failed, try recover
java.lang.IllegalStateException: should call flush first before calling close
        at 
org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkState(Preconditions.java:510)
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:514)
        at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.close(FanOutOneBlockAsyncDFSOutput.java:565)
        at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.close(AsyncProtobufLogWriter.java:158)
        at 
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.lambda$closeWriter$5(AsyncFSWAL.java:698)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748) {code}
Eventually leading to abort as you mentioned.

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

Reply via email to