Return-Path: X-Original-To: apmail-hbase-issues-archive@www.apache.org Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 1881BCE55 for ; Wed, 12 Mar 2014 19:39:02 +0000 (UTC) Received: (qmail 38495 invoked by uid 500); 12 Mar 2014 19:39:01 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 38458 invoked by uid 500); 12 Mar 2014 19:39:00 -0000 Mailing-List: contact issues-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@hbase.apache.org Received: (qmail 38417 invoked by uid 99); 12 Mar 2014 19:39:00 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 12 Mar 2014 19:39:00 +0000 Date: Wed, 12 Mar 2014 19:39:00 +0000 (UTC) From: "stack (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HBASE-10476) HBase Master log grows very fast after stopped hadoop (due to connection exception) MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HBASE-10476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13932256#comment-13932256 ] stack commented on HBASE-10476: ------------------------------- I tried to add a test. See below. I am not sure it makes sense spinning up a cluster manufacturing the particular case where we spew logging (too much work manufacturing the scenario and then it tough catching the exception spew and measuring its rate) . The below test just exercises the MetaShutdownHandler class and its new handleException method. Again, it seems OTT checking the exception output. So maybe we should work on adding unit tests elsewhere than here? OK w/ you [~haosdent@gmail.com]? On other hand, the below little test did help and attached patch has some improvement: 1. We should show the stack trace the first time through and not have to wait till 100th instance. 2. Some cleanup of formatting, changed freq name and capitalized it because it static. 3. We should reset the counter on successful shutdown. 4. We should print out the exception message even when we are not printing stack trace. Here is the attempt at a test: {code} public class TestServerShutdownHandler { private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @Test public void testMetaServerShutdownHandlerException() { final ServerName serverName = ServerName.valueOf("a.example.org,1,1"); MasterServices mockedMasterServices = Mockito.mock(MasterServices.class); Mockito.when(mockedMasterServices.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration()); DeadServer deadServer = new DeadServer(); deadServer.add(serverName); MetaServerShutdownHandler handler = new MetaServerShutdownHandler(mockedMasterServices, mockedMasterServices, deadServer, serverName); Throwable t = new Throwable(); for (int i = 0; i <= 200; i++) handler.handleException(t); } } {code} > 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-10476-trunk-v1.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)