[
https://issues.apache.org/jira/browse/HBASE-12951?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14300855#comment-14300855
]
zhangduo commented on HBASE-12951:
----------------------------------
I think there is a race condition in AsyncRpcChannel. We check
shouldCloseConnection several times in callMethod, but not enough(and even
shouldCloseConnection is not volatile...).
In AsyncRpcClient.cancelConnections, we find a AsyncRpcChannel and close it.
This is under a lock protection, but the actual close operation is in another
thread. It closes the channel is the follow order
1. set shouldCloseConnection to true.
2. clean up pending calls.
3. close the netty channel.
And in AsyncRpcChannel.callMethod, we does as follow
1. check shouldCloseConnection
2. create call and put it into pending calls
3. check is channel is alive and check shouldCloseConnection again(but does
nothing if check failed!)
4. write out request
The problem is at callMethod step 3. Things can happen like this
1. 'callMethod' check shouldCloseConnection, pass
2. 'close' set shouldCloseConnection, and cleanup all pending calls, and close
netty channel.
3. 'callMethod' create a pending call and put it into pending calls, then check
channel or shouldCloseConnection, failed, but just return the call.
4. client will wait forever because no one will wake him up.
There is a cleanupTimer but we also shutdown it when close...
> TestHCM.testConnectionClose is flakey when using AsyncRpcClient as client
> implementation
> ----------------------------------------------------------------------------------------
>
> Key: HBASE-12951
> URL: https://issues.apache.org/jira/browse/HBASE-12951
> Project: HBase
> Issue Type: Bug
> Components: IPC/RPC
> Reporter: zhangduo
>
> Sometimes the background thread is stuck in table.get, here is the jstack
> result.
> "testConnectionCloseThread" prio=10 tid=0x00007fdb296f9800 nid=0x6b5c in
> Object.wait() [0x00007fdad1bf9000]
> java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> - waiting on <0x00000000f192d6e0> (a
> org.apache.hadoop.hbase.ipc.AsyncCall)
> at java.lang.Object.wait(Object.java:503)
> at
> io.netty.util.concurrent.DefaultPromise.await(DefaultPromise.java:254)
> - locked <0x00000000f192d6e0> (a
> org.apache.hadoop.hbase.ipc.AsyncCall)
> at
> io.netty.util.concurrent.DefaultPromise.await(DefaultPromise.java:32)
> at io.netty.util.concurrent.AbstractFuture.get(AbstractFuture.java:31)
> at
> org.apache.hadoop.hbase.ipc.AsyncRpcClient.call(AsyncRpcClient.java:181)
> at
> org.apache.hadoop.hbase.ipc.AbstractRpcClient.callBlockingMethod(AbstractRpcClient.java:213)
> at
> org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.callBlockingMethod(AbstractRpcClient.java:287)
> at
> org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.get(ClientProtos.java:31727)
> at org.apache.hadoop.hbase.client.HTable$4.call(HTable.java:848)
> at org.apache.hadoop.hbase.client.HTable$4.call(HTable.java:1)
> at
> org.apache.hadoop.hbase.client.RpcRetryingCallerImpl.callWithRetries(RpcRetryingCallerImpl.java:120)
> at org.apache.hadoop.hbase.client.HTable.get(HTable.java:856)
> at org.apache.hadoop.hbase.client.TestHCM$3.run(TestHCM.java:371)
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)