You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2014/03/13 23:11:04 UTC

svn commit: r1577338 - in /hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase: executor/EventHandler.java master/handler/MetaServerShutdownHandler.java

Author: stack
Date: Thu Mar 13 22:11:04 2014
New Revision: 1577338

URL: http://svn.apache.org/r1577338
Log:
HBASE-10476 HBase Master log grows very fast after stopped hadoop (due to connection exception) (Demai Ni)

Modified:
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
    hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java?rev=1577338&r1=1577337&r2=1577338&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java Thu Mar 13 22:11:04 2014
@@ -128,7 +128,7 @@ public abstract class EventHandler imple
       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 imple
   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);
+  }
 }

Modified: hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java?rev=1577338&r1=1577337&r2=1577338&view=diff
==============================================================================
--- hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java (original)
+++ hbase/branches/0.96/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java Thu Mar 13 22:11:04 2014
@@ -32,14 +32,23 @@ import org.apache.hadoop.hbase.executor.
 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) {
@@ -114,8 +123,10 @@ public class MetaServerShutdownHandler e
         this.deadServers.finish(serverName);
       }     
     }
-    
+
     super.process();
+    // Clear this counter on successful handling.
+    this.eventExceptionCount.set(0);
   }
 
   @Override
@@ -194,4 +205,20 @@ public class MetaServerShutdownHandler e
     }
     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