From 98cf3e27ee5fa30b962984629cc4195e5bb38276 Mon Sep 17 00:00:00 2001 From: Michael Stack Date: Thu, 13 Mar 2014 22:11:52 +0000 Subject: [PATCH] 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 --- .../hadoop/hbase/executor/EventHandler.java | 10 +++++- .../handler/MetaServerShutdownHandler.java | 31 +++++++++++++++++-- 2 files changed, 38 insertions(+), 3 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java index 79115be6fed..5452b32789a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java @@ -128,7 +128,7 @@ public abstract class EventHandler implements Runnable, Comparable { 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 { 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); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java index 44a933fc2d6..03acccfd6cf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java @@ -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 hbase:meta */ @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."); + } + } +} \ No newline at end of file