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

Andrey Elenskiy commented on HBASE-22665:
-----------------------------------------

Hitting exactly the same issue on 2.2.4. The issue was exposed when the host 
was under massive load causing various pauses and OOMs across the OS.
 All the handlers are stuck on SyncFuture.get() while AsyncFSWal thread is just 
waiting on condition. Also, regionserver didn't trigger 
AbstractFSWAL.shutdown() initially and was stuck without it. Only once I 
connected with jdb to check the state, the shutdown() was called and it was 
stuck exiting until a timeout killed the process (prior to that it was stuck 
appending). Here is how the handlers look like:
{code:java}
Thread 48 (RpcServer.default.FPBQ.Fifo.handler=7,queue=1,port=16201):
 State: TIMED_WAITING
 Blocked count: 1193588
 Waited count: 2105228
 Stack:
 java.lang.Object.wait(Native Method)
 org.apache.hadoop.hbase.regionserver.wal.SyncFuture.get(SyncFuture.java:142)
 
org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.blockOnSync(AbstractFSWAL.java:722)
 org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.sync(AsyncFSWAL.java:637)
 org.apache.hadoop.hbase.regionserver.HRegion.sync(HRegion.java:8588)
 org.apache.hadoop.hbase.regionserver.HRegion.doWALAppend(HRegion.java:7946)
 
org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutate(HRegion.java:4130)
 org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:4072)
 org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:4003)
 
org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp(RSRpcServices.java:1042)
 
org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicBatchOp(RSRpcServices.java:974)
 
org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation(RSRpcServices.java:937)
 
org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(RSRpcServices.java:2755)
 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:42290)
 org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:413)
 org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:133)
 org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:338)
 org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:318){code}
How AsynFSWal looks like:
{code:java}
Thread 164 (AsyncFSWAL-0):
 State: WAITING
 Blocked count: 6198
 Waited count: 32255423
 Waiting on 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@bf1c663
 Stack:
 sun.misc.Unsafe.park(Native Method)
 java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
 java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
 java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 java.lang.Thread.run(Thread.java:748){code}
There were lots of logs like this right before it got stuck:

 
{code:java}
2020-11-17 14:37:53,902 WARN [AsyncFSWAL-0] wal.AsyncFSWAL: sync failed
java.io.IOException: Connection to 192.168.2.23/192.168.2.23:15010 closed
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.lambda$channelInactive$2(FanOutOneBlockAsyncDFSOutput.java:289)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.failed(FanOutOneBlockAsyncDFSOutput.java:236)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.access$300(FanOutOneBlockAsyncDFSOutput.java:99)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.channelInactive(FanOutOneBlockAsyncDFSOutput.java:288)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:242)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:228)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:221)
 at 
org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:242)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:228)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:221)
 at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelInputClosed(ByteToMessageDecoder.java:390)
 at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:355)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:242)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:228)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:221)
 at 
org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
 at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelInactive(IdleStateHandler.java:277)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:242)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:228)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:221)
 at 
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelInactive(DefaultChannelPipeline.java:1403)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:242)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:228)
 at 
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:912)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannel$AbstractUnsafe$8.run(AbstractChannel.java:827)
 at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
 at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
 at 
org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:333)
 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)
...
2020-11-17 14:37:55,134 WARN [Close-WAL-Writer-533] 
wal.AbstractProtobufLogWriter: Failed to write trailer, non-fatal, continuing...
java.io.IOException: stream already broken
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.flush0(FanOutOneBlockAsyncDFSOutput.java:424)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.flush(FanOutOneBlockAsyncDFSOutput.java:513)
 at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.lambda$writeWALTrailerAndMagic$3(AsyncProtobufLogWriter.java:220)
 at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.write(AsyncProtobufLogWriter.java:176)
 at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.writeWALTrailerAndMagic(AsyncProtobufLogWriter.java:211)
 at 
org.apache.hadoop.hbase.regionserver.wal.AbstractProtobufLogWriter.writeWALTrailer(AbstractProtobufLogWriter.java:235)
 at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.close(AsyncProtobufLogWriter.java:153)
 at 
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.lambda$closeWriter$6(AsyncFSWAL.java:672)
 at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
 at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 at java.lang.Thread.run(Thread.java:748)
2020-11-17 14:37:55,152 WARN [Close-WAL-Writer-533] wal.AsyncProtobufLogWriter: 
normal close failed, try recover
java.lang.IllegalStateException: should call flush first before calling close
 at 
org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkState(Preconditions.java:508)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:518)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.close(FanOutOneBlockAsyncDFSOutput.java:569)
 at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.close(AsyncProtobufLogWriter.java:154)
 at 
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.lambda$closeWriter$6(AsyncFSWAL.java:672)
 at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
 at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 at java.lang.Thread.run(Thread.java:748)
2020-11-17 14:37:55,230 INFO [Close-WAL-Writer-533] util.FSHDFSUtils: Recover 
lease on dfs file 
/hbase/WALs/primary,16201,1605569374704/primary%2C16201%2C1605569374704.1605623807936
2020-11-17 14:37:55,252 INFO [Close-WAL-Writer-533] util.FSHDFSUtils: Failed to 
recover lease, attempt=0 on 
file=/hbase/WALs/primary,16201,1605569374704/primary%2C16201%2C1605569374704.1605623807936
 after 22ms
...

2020-11-17 14:38:24,243 WARN [AsyncFSWAL-0] wal.AsyncFSWAL: sync failed
java.io.IOException: Timeout(10000ms) waiting for response
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.lambda$userEventTriggered$4(FanOutOneBlockAsyncDFSOutput.java:303)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.failed(FanOutOneBlockAsyncDFSOutput.java:236)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.access$300(FanOutOneBlockAsyncDFSOutput.java:99)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput$AckHandler.userEventTriggered(FanOutOneBlockAsyncDFSOutput.java:302)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
 at 
org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.userEventTriggered(ChannelInboundHandlerAdapter.java:108)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
 at 
org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.userEventTriggered(ChannelInboundHandlerAdapter.java:108)
 at 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.userEventTriggered(ByteToMessageDecoder.java:366)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:326)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeUserEventTriggered(AbstractChannelHandlerContext.java:312)
 at 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireUserEventTriggered(AbstractChannelHandlerContext.java:304)
 at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelIdle(IdleStateHandler.java:371)
 at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler$ReaderIdleTimeoutTask.run(IdleStateHandler.java:494)
 at 
org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler$AbstractIdleTask.run(IdleStateHandler.java:466)
 at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.PromiseTask$RunnableAdapter.call(PromiseTask.java:38)
 at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.ScheduledFutureTask.run(ScheduledFutureTask.java:127)
 at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
 at 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
 at 
org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:333)
 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)
2020-11-17 14:38:24,381 INFO [regionserver/cvp21:16201.Chore.1] 
hbase.ChoreService: Chore [ScheduledChore: Name: CompactionChecker Period: 
10000 Unit: MILLISECONDS] is enabled.
2020-11-17 14:38:24,244 WARN [regionserver/cvp21:16201] util.Sleeper: We slept 
18138ms instead of 5000ms, this is likely due to a long garbage collecting 
pause and it's usually bad, see 
http://hbase.apache.org/book.html#trouble.rs.runtime.zkexpired
...
2020-11-17 18:47:31,364 WARN [Abort regionserver monitor] 
regionserver.HRegionServer: Aborting region server timed out, terminating 
forcibly and does not wait for any running shutdown hooks or finalizers to 
finish their work. Thread dump to stdout.
{code}

I'm attaching logs for your inspection: [^hbase.log]

> RegionServer abort failed when AbstractFSWAL.shutdown hang
> ----------------------------------------------------------
>
>                 Key: HBASE-22665
>                 URL: https://issues.apache.org/jira/browse/HBASE-22665
>             Project: HBase
>          Issue Type: Bug
>         Environment: HBase 2.1.2
> Hadoop 3.1.x
> centos 7.4
>            Reporter: Yechao Chen
>            Priority: Major
>         Attachments: HBASE-22665-UT.patch, hbase.log, 
> image-2019-07-08-16-07-37-664.png, image-2019-07-08-16-08-26-777.png, 
> image-2019-07-08-16-14-43-455.png, jstack_20190625, jstack_20190704_1, 
> jstack_20190704_2, rs.log.part1, rs.log_part2.zip
>
>
> We use hbase 2.1.2,when the rs with heavy qps and rs abort with error like 
> "Caused by: org.apache.hadoop.hbase.exceptions.TimeoutIOException: Failed to 
> get sync result after 300000 ms for txid=36380334, WAL system stuck?"
>  
> RegionServer aborted failed when AbstractFSWAL.shutdown hang
>  
> jstack info always show the regionserver hang with "AbstractFSWAL.shutdown"
> "regionserver/hbase-slave-216-99:16020" #25 daemon prio=5 os_prio=0 
> tid=0x00007f204282c600 nid=0x34aa waiting on condition [0x00007f0fe044d000]
>  java.lang.Thread.State: WAITING (parking)
>  at sun.misc.Unsafe.park(Native Method)
>  - parking to wait for <0x00007f18a49b2bb8> (a 
> java.util.concurrent.locks.ReentrantLock$FairSync)
>  at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>  at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>  at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870)
>  at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
>  at 
> java.util.concurrent.locks.ReentrantLock$FairSync.lock(ReentrantLock.java:224)
>  {color:#FF0000}at 
> java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285){color}
> {color:#FF0000} at 
> org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL.shutdown(AbstractFSWAL.java:815){color}
>  at 
> org.apache.hadoop.hbase.wal.AbstractFSWALProvider.shutdown(AbstractFSWALProvider.java:168)
>  at 
> org.apache.hadoop.hbase.wal.RegionGroupingProvider.shutdown(RegionGroupingProvider.java:221)
>  at org.apache.hadoop.hbase.wal.WALFactory.shutdown(WALFactory.java:239)
>  at 
> org.apache.hadoop.hbase.regionserver.HRegionServer.shutdownWAL(HRegionServer.java:1445)
>  {color:#FF0000}at 
> org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:1117){color}
> {color:#FF0000} at java.lang.Thread.run(Thread.java:745){color}
>  
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to