Duo Zhang created HBASE-21199:
---------------------------------

             Summary: Race in region opening and load balancing can cause 
region stuck in RIT
                 Key: HBASE-21199
                 URL: https://issues.apache.org/jira/browse/HBASE-21199
             Project: HBase
          Issue Type: Sub-task
          Components: amv2
            Reporter: Duo Zhang
             Fix For: 3.0.0, 2.2.0


After HBASE-20881, when region server calls reportRegionTransition with OPEN 
state, we will update the hbase:meta directly and finish the TRSP. So it is 
possible that we schedule a new TRSP immediately for this region. But at RS 
side, the region opening may still in progress, think of the rpc connection 
between master and RS is broken and RS haven't gotten the return value and 
still trying to call reportRegionTransition again... So at RS side, it is 
possible that the RS finds out that the region we want to close is still 
opening and causes problems.

I have set up a cluster to test the synchronous replication and the balancer 
for one of the clusters is a bit strange, as it keeps moving the 
hbase:namespace region and finally hit the problem described above.

We hit this error first
{noformat}
2018-09-16,11:42:11,218 WARN [RSProcedureDispatcher-pool3-t4004] 
org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher: Failed dispatch 
to server=c4-hadoop-tst-st57.bj,17200,1536907673199 try=0
org.apache.hadoop.hbase.NotServingRegionException: 
org.apache.hadoop.hbase.NotServingRegionException: The region 
7035a1c2da68172f6f9cec99e00b0ce1 was opening but not yet served. Opening is 
cancelled.
        at 
org.apache.hadoop.hbase.regionserver.HRegionServer.closeRegion(HRegionServer.java:3167)
        at 
org.apache.hadoop.hbase.regionserver.RSRpcServices.closeRegion(RSRpcServices.java:1635)
        at 
org.apache.hadoop.hbase.regionserver.RSRpcServices.executeProcedures(RSRpcServices.java:3680)
        at 
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos$AdminService$2.callBlockingMethod(AdminProtos.java:28704)
        at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:409)
        at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:130)
        at 
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:338)
        at 
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:318)

        at sun.reflect.GeneratedConstructorAccessor30.newInstance(Unknown 
Source)
        at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
        at 
org.apache.hadoop.hbase.ipc.RemoteWithExtrasException.instantiateException(RemoteWithExtrasException.java:100)
        at 
org.apache.hadoop.hbase.ipc.RemoteWithExtrasException.unwrapRemoteException(RemoteWithExtrasException.java:90)
        at 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.makeIOExceptionOfException(ProtobufUtil.java:365)
        at 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.getRemoteException(ProtobufUtil.java:342)
        at 
org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher$ExecuteProceduresRemoteCall.sendRequest(RSProcedureDispatcher.java:349)
        at 
org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher$ExecuteProceduresRemoteCall.call(RSProcedureDispatcher.java:313)
        at 
org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher$ExecuteProceduresRemoteCall.call(RSProcedureDispatcher.java:292)
        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)
Caused by: 
org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.NotServingRegionException):
 org.apache.hadoop.hbase.NotServingRegionException: The region 
7035a1c2da68172f6f9cec99e00b0ce1 was opening but not yet served. Opening is 
cancelled.
        at 
org.apache.hadoop.hbase.regionserver.HRegionServer.closeRegion(HRegionServer.java:3167)
        at 
org.apache.hadoop.hbase.regionserver.RSRpcServices.closeRegion(RSRpcServices.java:1635)
        at 
org.apache.hadoop.hbase.regionserver.RSRpcServices.executeProcedures(RSRpcServices.java:3680)
        at 
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos$AdminService$2.callBlockingMethod(AdminProtos.java:28704)
        at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:409)
        at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:130)
        at 
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:338)
        at 
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:318)

        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient.onCallFinished(AbstractRpcClient.java:395)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient.access$100(AbstractRpcClient.java:95)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient$3.run(AbstractRpcClient.java:418)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient$3.run(AbstractRpcClient.java:414)
        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 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
        at 
com.xiaomi.infra.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:310)
        at 
com.xiaomi.infra.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:284)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
        at 
com.xiaomi.infra.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:286)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1434)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:965)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:808)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:410)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:310)
        at 
com.xiaomi.infra.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:884)
        at 
com.xiaomi.infra.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        ... 1 more
{noformat}

And then
{noformat}
2018-09-16,11:42:12,375 WARN [RSProcedureDispatcher-pool3-t4005] 
org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher: Failed dispatch 
to server=c4-hadoop-tst-st57.bj,17200,1536907673199 try=0
org.apache.hadoop.hbase.NotServingRegionException: 
org.apache.hadoop.hbase.NotServingRegionException: The region 
7035a1c2da68172f6f9cec99e00b0ce1 is not online, and is not opening.
        at 
org.apache.hadoop.hbase.regionserver.HRegionServer.closeRegion(HRegionServer.java:3180)
        at 
org.apache.hadoop.hbase.regionserver.RSRpcServices.closeRegion(RSRpcServices.java:1635)
        at 
org.apache.hadoop.hbase.regionserver.RSRpcServices.executeProcedures(RSRpcServices.java:3680)
        at 
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos$AdminService$2.callBlockingMethod(AdminProtos.java:28704)
        at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:409)
        at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:130)
        at 
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:338)
        at 
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:318)

        at sun.reflect.GeneratedConstructorAccessor30.newInstance(Unknown 
Source)
        at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
        at 
org.apache.hadoop.hbase.ipc.RemoteWithExtrasException.instantiateException(RemoteWithExtrasException.java:100)
        at 
org.apache.hadoop.hbase.ipc.RemoteWithExtrasException.unwrapRemoteException(RemoteWithExtrasException.java:90)
        at 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.makeIOExceptionOfException(ProtobufUtil.java:365)
        at 
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.getRemoteException(ProtobufUtil.java:342)
        at 
org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher$ExecuteProceduresRemoteCall.sendRequest(RSProcedureDispatcher.java:349)
        at 
org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher$ExecuteProceduresRemoteCall.call(RSProcedureDispatcher.java:313)
        at 
org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher$ExecuteProceduresRemoteCall.call(RSProcedureDispatcher.java:292)
        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)
Caused by: 
org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.NotServingRegionException):
 org.apache.hadoop.hbase.NotServingRegionException: The region 
7035a1c2da68172f6f9cec99e00b0ce1 is not online, and is not opening.
        at 
org.apache.hadoop.hbase.regionserver.HRegionServer.closeRegion(HRegionServer.java:3180)
        at 
org.apache.hadoop.hbase.regionserver.RSRpcServices.closeRegion(RSRpcServices.java:1635)
        at 
org.apache.hadoop.hbase.regionserver.RSRpcServices.executeProcedures(RSRpcServices.java:3680)
        at 
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos$AdminService$2.callBlockingMethod(AdminProtos.java:28704)
        at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:409)
        at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:130)
        at 
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:338)
        at 
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:318)

        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient.onCallFinished(AbstractRpcClient.java:395)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient.access$100(AbstractRpcClient.java:95)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient$3.run(AbstractRpcClient.java:418)
        at 
org.apache.hadoop.hbase.ipc.AbstractRpcClient$3.run(AbstractRpcClient.java:414)
        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 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
        at 
com.xiaomi.infra.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:310)
        at 
com.xiaomi.infra.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:284)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
        at 
com.xiaomi.infra.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:286)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1434)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:965)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:808)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:410)
        at 
com.xiaomi.infra.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:310)
        at 
com.xiaomi.infra.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:884)
        at 
com.xiaomi.infra.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        ... 1 more
{noformat}

So we stuck there for ever...



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

Reply via email to