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

Duo Zhang commented on HBASE-28207:
-----------------------------------

{noformat}
"MiniHBaseClusterRegionServer-EventLoopGroup-5-2" #627 daemon prio=10 os_prio=0 
tid=0x00007f7f00123000 nid=0x57ef3 in Object.wait() [0x00007f7e7458c000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:502)
        at org.apache.hadoop.hbase.util.IdLock.getLockEntry(IdLock.java:85)
        - locked <0x0000000729a70708> (a 
org.apache.hadoop.hbase.util.IdLock$Entry)
        at 
org.apache.hadoop.hbase.procedure2.ProcedureFutureUtil.lambda$0(ProcedureFutureUtil.java:82)
        at 
org.apache.hadoop.hbase.procedure2.ProcedureFutureUtil$$Lambda$551/520635809.accept(Unknown
 Source)
        at 
org.apache.hadoop.hbase.util.FutureUtils.lambda$addListener$0(FutureUtils.java:71)
        at 
org.apache.hadoop.hbase.util.FutureUtils$$Lambda$253/1730435698.accept(Unknown 
Source)
        at 
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
        at 
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at 
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at 
org.apache.hadoop.hbase.client.AsyncSingleRequestRpcRetryingCaller.lambda$2(AsyncSingleRequestRpcRetryingCaller.java:92)
        at 
org.apache.hadoop.hbase.client.AsyncSingleRequestRpcRetryingCaller$$Lambda$641/610835887.accept(Unknown
 Source)
        at 
org.apache.hadoop.hbase.util.FutureUtils.lambda$addListener$0(FutureUtils.java:71)
        at 
org.apache.hadoop.hbase.util.FutureUtils$$Lambda$253/1730435698.accept(Unknown 
Source)
        at 
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
        at 
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at 
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at 
org.apache.hadoop.hbase.client.ConnectionUtils$2.run(ConnectionUtils.java:626)
        at 
org.apache.hbase.thirdparty.com.google.protobuf.RpcUtil$1.run(RpcUtil.java:79)
        at 
org.apache.hbase.thirdparty.com.google.protobuf.RpcUtil$1.run(RpcUtil.java:70)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient.onCallFinished(AbstractRpcClient.java:399)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient.access$3(AbstractRpcClient.java:378)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient$3.run(AbstractRpcClient.java:429)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient$3.run(AbstractRpcClient.java:1)
        at org.apache.hadoop.hbase.ipc.Call.callComplete(Call.java:117)
        at org.apache.hadoop.hbase.ipc.Call.setResponse(Call.java:149)
        at 
org.apache.hadoop.hbase.ipc.NettyRpcDuplexHandler.finishCall(NettyRpcDuplexHandler.java:158)
        at 
org.apache.hadoop.hbase.ipc.NettyRpcDuplexHandler.readResponse(NettyRpcDuplexHandler.java:203)
        at 
org.apache.hadoop.hbase.ipc.NettyRpcDuplexHandler.channelRead(NettyRpcDuplexHandler.java:220)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:346)
        at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:318)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        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:442)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at 
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at 
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
        at 
org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:800)
        at 
org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:509)
        at 
org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:407)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
        at 
org.apache.hbase.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:750)
{noformat}

I think this is the problem, in the callback we need to acquiring a the 
procedure execution lock, and it blocks a netty's event loop thread, but before 
releasing the execution lock, we need to persist the procedure state, which 
need to update to master region, where we need to sync on AsyncFSWAL. And this 
could lead to a dead lock...

Let me think what is the best way to fix this...

> AsyncFSWAL may hang
> -------------------
>
>                 Key: HBASE-28207
>                 URL: https://issues.apache.org/jira/browse/HBASE-28207
>             Project: HBase
>          Issue Type: Bug
>          Components: wal
>            Reporter: Duo Zhang
>            Priority: Critical
>
> Saw this in the UT environment before when checking why 
> TestClusterScopeQuotaThrottle.
> And when implementing HBASE-28199, it is very easy to reproduce the problem 
> while executing some other UTs, like TestAdmin, and the problem disappear 
> after chaning to use FSHLog.
> Need to dig more as now AsyncFSWAL is the default implementation.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to