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

binlijin commented on HBASE-22370:
----------------------------------

yeh, there are many exceptions before like:

{code}

2019-05-06 02:22:02,132 WARN [regionserver/hbasedevc-6:16020.logRoller] 
asyncfs.FanOutOneBlockAsyncDFSOutputHelper: create fan-out dfs output 
/apps/hbase/data/WALs/172.22.45.142,16020,1556985880647/172.22.45.142%2C16020%2C1556985880647.meta.1557080522013.meta
 failed, retry = 1
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannel$AnnotatedConnectException:
 syscall:getsockopt(..) failed: Connection refused: /172.22.45.106:50010
 at 
org.apache.hbase.thirdparty.io.netty.channel.unix.Socket.finishConnect(..)(Unknown
 Source)
Caused by: 
org.apache.hbase.thirdparty.io.netty.channel.unix.Errors$NativeConnectException:
 syscall:getsockopt(..) failed: Connection refused
 ... 1 more
2019-05-06 02:22:02,344 INFO [regionserver/hbasedevc-6:16020.logRoller] 
wal.AbstractFSWAL: Rolled WAL 
/apps/hbase/data/WALs/172.22.45.142,16020,1556985880647/172.22.45.142%2C16020%2C1556985880647.meta.1557079318998.meta
 with entries=0, filesize=83 B; new WAL 
/apps/hbase/data/WALs/172.22.45.142,16020,1556985880647/172.22.45.142%2C16020%2C1556985880647.meta.1557080522013.meta

 

2019-05-06 02:22:02,344 WARN [Close-WAL-Writer-12] 
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:643)
 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)
2019-05-06 02:22:02,344 WARN [Close-WAL-Writer-12] wal.AsyncProtobufLogWriter: 
normal close failed, try recover
java.io.IOException: stream already broken
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.endBlock(FanOutOneBlockAsyncDFSOutput.java:521)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.close(FanOutOneBlockAsyncDFSOutput.java:565)
 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:643)
 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)

 

2019-05-06 02:22:06,349 WARN [Close-WAL-Writer-12] wal.AsyncFSWAL: close old 
writer failed
java.io.FileNotFoundException: File does not exist: 
/apps/hbase/data/WALs/172.22.45.142,16020,1556985880647/172.22.45.142%2C16020%2C1556985880647.meta.1557079318998.meta
 at org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:71)
 at org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
 at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.recoverLease(FSNamesystem.java:3086)
 at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.recoverLease(NameNodeRpcServer.java:781)
 at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.recoverLease(ClientNamenodeProtocolServerSideTranslatorPB.java:686)
 at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
 at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:640)
 at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:982)
 at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2351)
 at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2347)
 at java.security.AccessController.doPrivileged(Native Method)
 at javax.security.auth.Subject.doAs(Subject.java:422)
 at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1869)
 at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2347)
 at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
 at 
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
 at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
 at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
 at 
org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
 at 
org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
 at org.apache.hadoop.hdfs.DFSClient.recoverLease(DFSClient.java:1256)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem$2.doCall(DistributedFileSystem.java:279)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem$2.doCall(DistributedFileSystem.java:275)
 at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem.recoverLease(DistributedFileSystem.java:275)
 at org.apache.hadoop.hbase.util.FSHDFSUtils.recoverLease(FSHDFSUtils.java:283)
 at 
org.apache.hadoop.hbase.util.FSHDFSUtils.recoverDFSFileLease(FSHDFSUtils.java:216)
 at 
org.apache.hadoop.hbase.util.FSHDFSUtils.recoverFileLease(FSHDFSUtils.java:163)
 at 
org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.recoverAndClose(FanOutOneBlockAsyncDFSOutput.java:555)
 at 
org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.close(AsyncProtobufLogWriter.java:157)
 at 
org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.lambda$closeWriter$6(AsyncFSWAL.java:643)
 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)

{code}

> AsyncFSWAL ByteBuf LEAK ERROR
> -----------------------------
>
>                 Key: HBASE-22370
>                 URL: https://issues.apache.org/jira/browse/HBASE-22370
>             Project: HBase
>          Issue Type: Bug
>            Reporter: binlijin
>            Priority: Major
>
> We do failover test and throw a leak error, this is hard to reproduce.
> {code}
> 2019-05-06 02:30:27,781 ERROR [AsyncFSWAL-0] util.ResourceLeakDetector: LEAK: 
> ByteBuf.release() was not called before it's garbage-collected. See 
> http://netty.io/wiki/reference-counted-objects.html for more information.
> Recent access records:
> Created at:
>  
> org.apache.hbase.thirdparty.io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:334)
>  
> org.apache.hbase.thirdparty.io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:187)
>  
> org.apache.hbase.thirdparty.io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:178)
>  
> org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.flush0(FanOutOneBlockAsyncDFSOutput.java:494)
>  
> org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutput.flush(FanOutOneBlockAsyncDFSOutput.java:513)
>  
> org.apache.hadoop.hbase.regionserver.wal.AsyncProtobufLogWriter.sync(AsyncProtobufLogWriter.java:144)
>  org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.sync(AsyncFSWAL.java:353)
>  
> org.apache.hadoop.hbase.regionserver.wal.AsyncFSWAL.consume(AsyncFSWAL.java:536)
>  
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  java.lang.Thread.run(Thread.java:748)
> {code}
>  



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

Reply via email to