HBASE-10476 HBase Master log grows very fast after stopped hadoop (due to connection exception) (Demai Ni)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1577340 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2014-03-13 22:11:52 +00:00
parent 889ff7cbed
commit 98cf3e27ee
2 changed files with 38 additions and 3 deletions

View File

@ -128,7 +128,7 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
process();
if (getListener() != null) getListener().afterProcess(this);
} catch(Throwable t) {
LOG.error("Caught throwable while processing event " + eventType, t);
handleException(t);
} finally {
chunk.close();
}
@ -217,4 +217,12 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
public String getInformativeName() {
return this.getClass().toString();
}
/**
* Event exception handler, may be overridden
* @param t Throwable object
*/
protected void handleException(Throwable t) {
LOG.error("Caught throwable while processing event " + eventType, t);
}
}

View File

@ -33,14 +33,23 @@ import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.DeadServer;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.zookeeper.KeeperException;
import com.google.common.annotations.VisibleForTesting;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Shutdown handler for the server hosting <code>hbase:meta</code>
*/
@InterfaceAudience.Private
public class MetaServerShutdownHandler extends ServerShutdownHandler {
private static final Log LOG = LogFactory.getLog(MetaServerShutdownHandler.class);
private AtomicInteger eventExceptionCount = new AtomicInteger(0);
@VisibleForTesting
static final int SHOW_STRACKTRACE_FREQUENCY = 100;
public MetaServerShutdownHandler(final Server server,
final MasterServices services,
final DeadServer deadServers, final ServerName serverName) {
@ -115,8 +124,10 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
this.deadServers.finish(serverName);
}
}
super.process();
// Clear this counter on successful handling.
this.eventExceptionCount.set(0);
}
@Override
@ -195,4 +206,20 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler {
}
return getClass().getSimpleName() + "-" + name + "-" + getSeqid();
}
}
@Override
protected void handleException(Throwable t) {
int count = eventExceptionCount.getAndIncrement();
if (count < 0) count = eventExceptionCount.getAndSet(0);
if (count > SHOW_STRACKTRACE_FREQUENCY) { // Too frequent, let's slow reporting
Threads.sleep(1000);
}
if (count % SHOW_STRACKTRACE_FREQUENCY == 0) {
LOG.error("Caught " + eventType + ", count=" + this.eventExceptionCount, t);
} else {
LOG.error("Caught " + eventType + ", count=" + this.eventExceptionCount +
"; " + t.getMessage() + "; stack trace shows every " + SHOW_STRACKTRACE_FREQUENCY +
"th time.");
}
}
}