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

stack commented on HBASE-10476:
-------------------------------

I could commit this though it is special handling of a single exception only in 
a single handler type.  Should it therefore be inside that handler type only 
rather than general?  In EventHandler, change the catching of the exception so 
it can be overridden -- make a protected method handleException? -- and then in 
an override added to MetaServerShutdownHandler, add this handling?

Maybe don't make this static:

 private static AtomicInteger eventExceptionCount = new AtomicInteger(0);

There should be one instance only anyways?  Statics have a nice way of coming 
back and biting especially in ugly contexts like minihbasecluster where we have 
multiple regions and many threads all running concurrently making it hard to 
debug.

Good on you [~nidmhbase]

> HBase Master log grows very fast after stopped hadoop (due to connection 
> exception)
> -----------------------------------------------------------------------------------
>
>                 Key: HBASE-10476
>                 URL: https://issues.apache.org/jira/browse/HBASE-10476
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 0.98.0, 0.96.0
>            Reporter: Demai Ni
>            Assignee: Demai Ni
>             Fix For: 0.96.2, 0.98.1, 0.99.0
>
>         Attachments: HBASE-10476-trunk-v0.patch, HBASE-10476-trunk-v0.patch
>
>
> hbase 96.0(probably the same issue on 94.x) on single node cluster. At some 
> point, we stopped Hadoop, but keep hbase running. As expected, hbase began to 
> throw connection errors.  
> P.S., later testing shows that this problem doesn't limit to single node 
> cluster. 
> For the first hour, the regionserver log grows by ~10MB, and master log 
> doesn't grow much,  which is ok. 
> {code:title=log size after one hour}
> -rw-rw-r-- 1 biadmin biadmin  497959 Feb  5 10:36 
> hbase-biadmin-master-hdtest014.svl.ibm.com.log
> ...
> -rw-rw-r-- 1 biadmin biadmin 8865371 Feb  5 10:37 
> hbase-biadmin-regionserver-hdtest014.svl.ibm.com.log
> {code}
> However, within 4 hours, the Master log grows to 13GB. And it only stops due 
> to out of disk space. 
> {code:title=log size after 4 hour}
> -rw-rw-r-- 1 biadmin biadmin  3521880064 Feb  5 14:10 
> hbase-biadmin-master-hdtest014.svl.ibm.com.log
> -rw-rw-r-- 1 biadmin biadmin 10737418582 Feb  5 11:25 
> hbase-biadmin-master-hdtest014.svl.ibm.com.log.1
> ...
> -rw-rw-r-- 1 biadmin biadmin    11222365 Feb  5 10:49 
> hbase-biadmin-regionserver-hdtest014.svl.ibm.com.log
> {code}
> The exception/error message filled out Master log is 
> {code:title=Error message filling up Master log}
> 2014-02-05 11:37:48,688 INFO 
> org.apache.hadoop.hbase.master.handler.MetaServerShutdownHandler: Splitting 
> hbase:meta logs for hdtest014.svl.ibm.com,60020,1391622549030
> 2014-02-05 11:37:48,689 ERROR org.apache.hadoop.hbase.executor.EventHandler: 
> Caught throwable while processing event M_META_SERVER_SHUTDOWN
> java.io.IOException: failed log splitting for 
> hdtest014.svl.ibm.com,60020,1391622549030, will retry
>     at 
> org.apache.hadoop.hbase.master.handler.MetaServerShutdownHandler.process(MetaServerShutdownHandler.java:70)
>     at 
> org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128)
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:906)
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:929)
>     at java.lang.Thread.run(Thread.java:738)
> Caused by: java.net.ConnectException: Call From 
> hdtest014.svl.ibm.com/9.30.194.23 to hdtest014.svl.ibm.com:9000 failed on 
> connection exception: java.net.ConnectException: Connection refused; For more 
> details see:  http://wiki.apache.org/hadoop/ConnectionRefused
>     at sun.reflect.GeneratedConstructorAccessor5.newInstance(Unknown Source)
>     at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:39)
>     at java.lang.reflect.Constructor.newInstance(Constructor.java:527)
>     at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:783)
>     at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:730)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1351)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1300)
>     at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
>     at com.sun.proxy.$Proxy8.getFileInfo(Unknown Source)
>     at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
>     at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:37)
>     at java.lang.reflect.Method.invoke(Method.java:611)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:186)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
>     at com.sun.proxy.$Proxy8.getFileInfo(Unknown Source)
>     at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:651)
>     at sun.reflect.GeneratedMethodAccessor9.invoke(Unknown Source)
>     at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:37)
>     at java.lang.reflect.Method.invoke(Method.java:611)
>     at org.apache.hadoop.hbase.fs.HFileSystem$1.invoke(HFileSystem.java:266)
>     at com.sun.proxy.$Proxy11.getFileInfo(Unknown Source)
>     at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1679)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem$17.doCall(DistributedFileSystem.java:1106)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem$17.doCall(DistributedFileSystem.java:1102)
>     at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1102)
>     at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1406)
>     at 
> org.apache.hadoop.hbase.master.MasterFileSystem.getLogDirs(MasterFileSystem.java:317)
>     at 
> org.apache.hadoop.hbase.master.MasterFileSystem.splitLog(MasterFileSystem.java:405)
>     at 
> org.apache.hadoop.hbase.master.MasterFileSystem.splitMetaLog(MasterFileSystem.java:301)
>     at 
> org.apache.hadoop.hbase.master.MasterFileSystem.splitMetaLog(MasterFileSystem.java:292)
>     at 
> org.apache.hadoop.hbase.master.handler.MetaServerShutdownHandler.process(MetaServerShutdownHandler.java:63)
>     ... 4 more
> Caused by: java.net.ConnectException: Connection refused
>     at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>     at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:614)
>     at 
> org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
>     at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:529)
>     at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:493)
>     at 
> org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:547)
>     at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:642)
>     at org.apache.hadoop.ipc.Client$Connection.access$2600(Client.java:314)
>     at org.apache.hadoop.ipc.Client.getConnection(Client.java:1399)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1318)
> {code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to