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

Enis Soztutar commented on HBASE-10525:
---------------------------------------

I was testing this, and it seems that there is an issue when the connection is 
closed (RS killed). it maybe that the CallSender can still accept more calls 
even after Connection is closed and shouldCloseConnection is set. In case 
CallSender.cleanUp() runs first, and clears the queue, any call added to the 
CallSeender queue will not get notification, thus will hang. I think we may 
need to sync on CallSender accepting Calls with with the Connection.out. Other 
ways might also be possible. 
Here is a dump where the RPC callee threads are just hanging forever: 
{code}
"htable-pool15-t2" daemon prio=10 tid=0x0000000001f2e800 nid=0x1bc3 in 
Object.wait() [0x00007fca48f72000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1435)
        - locked <0x00000000bd6a0230> (a 
org.apache.hadoop.hbase.ipc.RpcClient$Call)
        at 
org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(RpcClient.java:1655)
        at 
org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementation.callBlockingMethod(RpcClient.java:1713)
        at 
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.multi(ClientProtos.java:29300)
        at 
org.apache.hadoop.hbase.client.MultiServerCallable.call(MultiServerCallable.java:125)
        at 
org.apache.hadoop.hbase.client.MultiServerCallable.call(MultiServerCallable.java:53)
        at 
org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:186)
        at 
org.apache.hadoop.hbase.client.AsyncProcess$AsyncRequestFutureImpl$SingleServerRequestRunnable.run(AsyncProcess.java:658)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
        at java.util.concurrent.FutureTask.run(FutureTask.java:166)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:722)

"htable-pool23-t2" daemon prio=10 tid=0x00007fca4c5db800 nid=0x1bbb in 
Object.wait() [0x00007fca49779000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1435)
        - locked <0x00000000bd6a0578> (a 
org.apache.hadoop.hbase.ipc.RpcClient$Call)
        at 
org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(RpcClient.java:1655)
        at 
org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementation.callBlockingMethod(RpcClient.java:1713)
        at 
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.multi(ClientProtos.java:29300)
        at 
org.apache.hadoop.hbase.client.MultiServerCallable.call(MultiServerCallable.java:125)
        at 
org.apache.hadoop.hbase.client.MultiServerCallable.call(MultiServerCallable.java:53)
        at 
org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:186)
        at 
org.apache.hadoop.hbase.client.AsyncProcess$AsyncRequestFutureImpl$SingleServerRequestRunnable.run(AsyncProcess.java:658)
        at 
org.apache.hadoop.hbase.client.AsyncProcess$AsyncRequestFutureImpl.sendMultiAction(AsyncProcess.java:850)
        at 
org.apache.hadoop.hbase.client.AsyncProcess$AsyncRequestFutureImpl.groupAndSendMultiAction(AsyncProcess.java:824)
        at 
org.apache.hadoop.hbase.client.AsyncProcess$AsyncRequestFutureImpl.logAndResubmit(AsyncProcess.java:998)
        at 
org.apache.hadoop.hbase.client.AsyncProcess$AsyncRequestFutureImpl.receiveGlobalFailure(AsyncProcess.java:952)
        at 
org.apache.hadoop.hbase.client.AsyncProcess$AsyncRequestFutureImpl.access$1000(AsyncProcess.java:546)
        at 
org.apache.hadoop.hbase.client.AsyncProcess$AsyncRequestFutureImpl$SingleServerRequestRunnable.run(AsyncProcess.java:662)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
        at java.util.concurrent.FutureTask.run(FutureTask.java:166)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:722)

"HBaseWriterThread_13" prio=10 tid=0x00007fca5cf68000 nid=0x1b95 in 
Object.wait() [0x00007fca4bda0000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at 
org.apache.hadoop.hbase.client.AsyncProcess.waitForMaximumCurrentTasks(AsyncProcess.java:1383)
        - locked <0x00000000cc21a7e0> (a java.util.concurrent.atomic.AtomicLong)
        at 
org.apache.hadoop.hbase.client.AsyncProcess.waitForAllPreviousOpsAndReset(AsyncProcess.java:1412)
        at 
org.apache.hadoop.hbase.client.HTable.backgroundFlushCommits(HTable.java:1027)
        at org.apache.hadoop.hbase.client.HTable.flushCommits(HTable.java:1298)
        at org.apache.hadoop.hbase.client.HTable.put(HTable.java:959)
        at 
org.apache.hadoop.hbase.util.MultiThreadedWriter$HBaseWriterThread.insert(MultiThreadedWriter.java:143)
        at 
org.apache.hadoop.hbase.util.MultiThreadedWriter$HBaseWriterThread.run(MultiThreadedWriter.java:108)
{code}

> Allow the client to use a different thread for writing to ease interrupt
> ------------------------------------------------------------------------
>
>                 Key: HBASE-10525
>                 URL: https://issues.apache.org/jira/browse/HBASE-10525
>             Project: HBase
>          Issue Type: Bug
>          Components: Client
>    Affects Versions: 0.99.0
>            Reporter: Nicolas Liochon
>            Assignee: Nicolas Liochon
>             Fix For: 0.99.0
>
>         Attachments: 10525.v1.patch, 10525.v2.patch, 10525.v3.patch, 
> HBaseclient-EventualConsistency.pdf
>
>
> This is an issue in the HBASE-10070 context, but as well more generally if 
> you want to interrupt an operation with a limited cost. 
> I will attach a doc with a more detailed explanation.
> This adds a thread per region server; so it's otional. The first patch 
> activates it by default to see how it behaves on a full hadoop-qa run. The 
> target is to be unset by default.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Reply via email to