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

Duo Zhang commented on HBASE-22236:
-----------------------------------

OK the problem is here

{noformat}
2019-04-18 16:54:05,605 ERROR [Default-IPC-NioEventLoopGroup-8-5] 
util.FutureUtils(72): Unexpected error caught when processing CompletableFuture
java.lang.NullPointerException
        at 
org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.canUpdateOnError(AsyncRegionLocatorHelper.java:50)
        at 
org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.updateCachedLocationOnError(AsyncRegionLocatorHelper.java:62)
        at 
org.apache.hadoop.hbase.client.AsyncNonMetaRegionLocator.updateCachedLocationOnError(AsyncNonMetaRegionLocator.java:593)
        at 
org.apache.hadoop.hbase.client.AsyncRegionLocator.updateCachedLocationOnError(AsyncRegionLocator.java:148)
        at 
org.apache.hadoop.hbase.client.AsyncSingleRequestRpcRetryingCaller.lambda$null$3(AsyncSingleRequestRpcRetryingCaller.java:87)
        at 
org.apache.hadoop.hbase.client.AsyncRpcRetryingCaller.onError(AsyncRpcRetryingCaller.java:170)
        at 
org.apache.hadoop.hbase.client.AsyncSingleRequestRpcRetryingCaller.lambda$call$4(AsyncSingleRequestRpcRetryingCaller.java:84)
        at 
org.apache.hadoop.hbase.util.FutureUtils.lambda$addListener$0(FutureUtils.java:70)
        at 
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
        at 
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
        at 
java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
        at 
org.apache.hadoop.hbase.client.RawAsyncTableImpl$1.run(RawAsyncTableImpl.java:156)
        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:394)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient.access$100(AbstractRpcClient.java:97)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient$3.run(AbstractRpcClient.java:423)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient$3.run(AbstractRpcClient.java:419)
        at org.apache.hadoop.hbase.ipc.Call.callComplete(Call.java:103)
        at org.apache.hadoop.hbase.ipc.Call.setException(Call.java:118)
        at 
org.apache.hadoop.hbase.ipc.NettyRpcDuplexHandler.readResponse(NettyRpcDuplexHandler.java:162)
        at 
org.apache.hadoop.hbase.ipc.NettyRpcDuplexHandler.channelRead(NettyRpcDuplexHandler.java:192)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:359)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:345)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:337)
        at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:323)
        at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:310)
        at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:426)
        at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:278)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:359)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:345)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:337)
        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:359)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:345)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:337)
        at 
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1408)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:359)
        at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:345)
        at 
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:930)
        at 
org.apache.hbase.thirdparty.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:163)
        at 
org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:677)
        at 
org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:612)
        at 
org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:529)
        at 
org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:491)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:905)
        at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Thread.java:748)
{noformat}

Let me dig.

> TestAsyncTableGetMultiThreaded sometimes timed out
> --------------------------------------------------
>
>                 Key: HBASE-22236
>                 URL: https://issues.apache.org/jira/browse/HBASE-22236
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Duo Zhang
>            Assignee: Duo Zhang
>            Priority: Major
>         Attachments: HBASE-22236.patch
>
>
> https://builds.apache.org/job/HBase-Flaky-Tests/job/master/2992/artifact/hbase-server/target/surefire-reports/org.apache.hadoop.hbase.client.TestAsyncTableGetMultiThreaded-output.txt/*view*/
> After this line
> {noformat}
> 2019-04-14 04:44:41,736 INFO  [PEWorker-12] 
> procedure2.ProcedureExecutor(1410): Finished pid=117, state=SUCCESS, 
> hasLock=false; TransitRegionStateProcedure table=hbase:meta, 
> region=1588230740, REOPEN/MOVE in 2.0690sec
> {noformat}
> Seems we just do nothing until the test is timed out.
> And there is no main thread in the output hanging thread, which is a bit 
> strange, although all the get threads are hanging there.
> Let me add some logs for better debugging first.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to