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

stack commented on HBASE-12684:
-------------------------------

Looking at the stack traces, the async one has a bit more going on:

{code}
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:502)
        at 
io.netty.util.concurrent.DefaultPromise.await(DefaultPromise.java:254)
        - locked <0x00000000fe9cc8b8> (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:158)
        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:30860)
        at org.apache.hadoop.hbase.client.HTable$4.call(HTable.java:873)
        at org.apache.hadoop.hbase.client.HTable$4.call(HTable.java:864)
        at 
org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:126)
        at org.apache.hadoop.hbase.client.HTable.get(HTable.java:881)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation$RandomReadTest.testRow(PerformanceEvaluation.java:1253)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation$Test.testTimed(PerformanceEvaluation.java:1039)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation$Test.test(PerformanceEvaluation.java:1021)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation.runOneClient(PerformanceEvaluation.java:1515)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation$1.call(PerformanceEvaluation.java:408)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation$1.call(PerformanceEvaluation.java:403)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
{code}

vs

{code}
"TestClient-2" #35 prio=5 os_prio=0 tid=0x00007fe16d81d000 nid=0x3a66 in 
Object.wait() [0x00007fe14936a000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at 
org.apache.hadoop.hbase.ipc.RpcClientImpl.call(RpcClientImpl.java:1188)
        - locked <0x00000000ff91cba0> (a org.apache.hadoop.hbase.ipc.Call)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient.callBlockingMethod(AbstractRpcClient.java:216)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.callBlockingMethod(AbstractRpcClient.java:300)
        at 
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.get(ClientProtos.java:30860)
        at org.apache.hadoop.hbase.client.HTable$4.call(HTable.java:873)
        at org.apache.hadoop.hbase.client.HTable$4.call(HTable.java:864)
        at 
org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:126)
        at org.apache.hadoop.hbase.client.HTable.get(HTable.java:881)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation$RandomReadTest.testRow(PerformanceEvaluation.java:1253)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation$Test.testTimed(PerformanceEvaluation.java:1039)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation$Test.test(PerformanceEvaluation.java:1021)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation.runOneClient(PerformanceEvaluation.java:1515)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation$1.call(PerformanceEvaluation.java:408)
        at 
org.apache.hadoop.hbase.PerformanceEvaluation$1.call(PerformanceEvaluation.java:403)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
{code}

But both are pretty clean (These are random reads -- that ok?)

The async has those double Object.wait (a native and a java one) where old 
client just has one native (but you probably can't change that since netty 
internals) and DefaultPromise.await calling a DefaultPromise.await but.. has we 
have just the one lock in both cases...



> Add new AsyncRpcClient
> ----------------------
>
>                 Key: HBASE-12684
>                 URL: https://issues.apache.org/jira/browse/HBASE-12684
>             Project: HBase
>          Issue Type: Improvement
>          Components: Client
>            Reporter: Jurriaan Mous
>            Assignee: Jurriaan Mous
>         Attachments: HBASE-12684-DEBUG2.patch, HBASE-12684-DEBUG3.patch, 
> HBASE-12684-v1.patch, HBASE-12684-v10.patch, HBASE-12684-v11.patch, 
> HBASE-12684-v12.patch, HBASE-12684-v13.patch, HBASE-12684-v14.patch, 
> HBASE-12684-v15.patch, HBASE-12684-v16.patch, HBASE-12684-v17.patch, 
> HBASE-12684-v17.patch, HBASE-12684-v18.patch, HBASE-12684-v19.1.patch, 
> HBASE-12684-v19.patch, HBASE-12684-v19.patch, HBASE-12684-v2.patch, 
> HBASE-12684-v3.patch, HBASE-12684-v4.patch, HBASE-12684-v5.patch, 
> HBASE-12684-v6.patch, HBASE-12684-v7.patch, HBASE-12684-v8.patch, 
> HBASE-12684-v9.patch, HBASE-12684.patch
>
>
> With the changes in HBASE-12597 it is possible to add new RpcClients. This 
> issue is about adding a new Async RpcClient which would enable HBase to do 
> non blocking protobuf service communication.
> Besides delivering a new AsyncRpcClient I would also like to ask the question 
> what it would take to replace the current RpcClient? This would enable to 
> simplify async code in some next issues.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to