You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2012/11/29 02:12:56 UTC

svn commit: r1415016 - in /hbase/trunk: hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/impl/ hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/

Author: eclark
Date: Thu Nov 29 01:12:55 2012
New Revision: 1415016

URL: http://svn.apache.org/viewvc?rev=1415016&view=rev
Log:
HBASE-7220 Creating a table with 3000 regions on 2 nodes fails after 1 hour

Modified:
    hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
    hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java

Modified: hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java?rev=1415016&r1=1415015&r2=1415016&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java (original)
+++ hbase/trunk/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java Thu Nov 29 01:12:55 2012
@@ -20,7 +20,13 @@ package org.apache.hadoop.metrics2.impl;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.metrics2.MetricsExecutor;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
+
+
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 
 /**
  * JMX caches the beans that have been exported; even after the values are removed from hadoop's
@@ -32,21 +38,38 @@ import org.apache.hadoop.metrics2.lib.De
  */
 public class JmxCacheBuster {
   private static final Log LOG = LogFactory.getLog(JmxCacheBuster.class);
+  private static Object lock = new Object();
+  private static ScheduledFuture fut = null;
+  private static MetricsExecutor executor = new MetricsExecutorImpl();
 
   /**
    * For JMX to forget about all previously exported metrics.
    */
   public static void clearJmxCache() {
-    LOG.trace("Clearing JMX mbean cache.");
 
-    // This is pretty extreme but it's the best way that
-    // I could find to get metrics to be removed.
+    //If there are more then 100 ms before the executor will run then everything should be merged.
+    if (fut == null || (!fut.isDone()  && fut.getDelay(TimeUnit.MILLISECONDS) > 100)) return;
+
+    synchronized (lock) {
+      fut = executor.getExecutor().schedule(new JmxCacheBusterRunnable(), 5, TimeUnit.SECONDS);
+    }
+  }
+
+  static class JmxCacheBusterRunnable implements Runnable {
+
+    @Override
+    public void run() {
+      LOG.trace("Clearing JMX mbean cache.");
+
+      // This is pretty extreme but it's the best way that
+      // I could find to get metrics to be removed.
 
-    try {
-      DefaultMetricsSystem.INSTANCE.stop();
-      DefaultMetricsSystem.INSTANCE.start();
-    }  catch (Exception exception )  {
-      LOG.debug("error clearing the jmx it appears the metrics system hasn't been started", exception);
+      try {
+        DefaultMetricsSystem.INSTANCE.stop();
+        DefaultMetricsSystem.INSTANCE.start();
+      }  catch (Exception exception )  {
+        LOG.debug("error clearing the jmx it appears the metrics system hasn't been started", exception);
+      }
     }
   }
 }

Modified: hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java?rev=1415016&r1=1415015&r2=1415016&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java (original)
+++ hbase/trunk/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java Thu Nov 29 01:12:55 2012
@@ -20,7 +20,12 @@ package org.apache.hadoop.metrics2.impl;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.metrics2.MetricsExecutor;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
+
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 
 /**
  * JMX caches the beans that have been exported; even after the values are removed from hadoop's
@@ -32,23 +37,40 @@ import org.apache.hadoop.metrics2.lib.De
  */
 public class JmxCacheBuster {
   private static final Log LOG = LogFactory.getLog(JmxCacheBuster.class);
+  private static Object lock = new Object();
+  private static ScheduledFuture fut = null;
+  private static MetricsExecutor executor = new MetricsExecutorImpl();
 
   /**
    * For JMX to forget about all previously exported metrics.
    */
   public static void clearJmxCache() {
-    LOG.trace("Clearing JMX mbean cache.");
 
-    // This is pretty extreme but it's the best way that
-    // I could find to get metrics to be removed.
-    try {
-      if (DefaultMetricsSystem.instance() != null ) {
-        DefaultMetricsSystem.instance().stop();
-        DefaultMetricsSystem.instance().start();
-      }
+    //If there are more then 100 ms before the executor will run then everything should be merged.
+    if (fut == null || (!fut.isDone()  && fut.getDelay(TimeUnit.MILLISECONDS) > 100)) return;
+
+    synchronized (lock) {
+      fut = executor.getExecutor().schedule(new JmxCacheBusterRunnable(), 5, TimeUnit.SECONDS);
+    }
+  }
 
-    }  catch (Exception exception )  {
-      LOG.debug("error clearing the jmx it appears the metrics system hasn't been started", exception);
+  static class JmxCacheBusterRunnable implements Runnable {
+
+    @Override
+    public void run() {
+      LOG.trace("Clearing JMX mbean cache.");
+
+      // This is pretty extreme but it's the best way that
+      // I could find to get metrics to be removed.
+      try {
+        if (DefaultMetricsSystem.instance() != null ) {
+          DefaultMetricsSystem.instance().stop();
+          DefaultMetricsSystem.instance().start();
+        }
+
+      }  catch (Exception exception )  {
+        LOG.debug("error clearing the jmx it appears the metrics system hasn't been started", exception);
+      }
     }
   }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1415016&r1=1415015&r2=1415016&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Nov 29 01:12:55 2012
@@ -451,8 +451,6 @@ public class HRegion implements HeapSize
     this.regiondir = getRegionDir(this.tableDir, encodedNameStr);
     this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
 
-    this.metricsRegion = new MetricsRegion(new MetricsRegionWrapperImpl(this));
-
     /*
      * timestamp.slop provides a server-side constraint on the timestamp. This
      * assumes that you base your TS around currentTimeMillis(). In this case,
@@ -475,6 +473,9 @@ public class HRegion implements HeapSize
       // don't initialize coprocessors if not running within a regionserver
       // TODO: revisit if coprocessors should load in other cases
       this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
+      this.metricsRegion = new MetricsRegion(new MetricsRegionWrapperImpl(this));
+    } else {
+      this.metricsRegion = null;
     }
     if (LOG.isDebugEnabled()) {
       // Write out region name as string and its encoded name.
@@ -1024,7 +1025,9 @@ public class HRegion implements HeapSize
         status.setStatus("Running coprocessor post-close hooks");
         this.coprocessorHost.postClose(abort);
       }
-      this.metricsRegion.close();
+      if ( this.metricsRegion != null) {
+        this.metricsRegion.close();
+      }
       status.markComplete("Closed");
       LOG.info("Closed " + this);
       return result;
@@ -2331,11 +2334,15 @@ public class HRegion implements HeapSize
       if (noOfPuts > 0) {
         // There were some Puts in the batch.
         double noOfMutations = noOfPuts + noOfDeletes;
-        this.metricsRegion.updatePut();
+        if (this.metricsRegion != null) {
+          this.metricsRegion.updatePut();
+        }
       }
       if (noOfDeletes > 0) {
         // There were some Deletes in the batch.
-        this.metricsRegion.updateDelete();
+        if (this.metricsRegion != null) {
+          this.metricsRegion.updateDelete();
+        }
       }
       if (!success) {
         for (int i = firstIndex; i < lastIndexExclusive; i++) {
@@ -4269,8 +4276,9 @@ public class HRegion implements HeapSize
     }
 
     // do after lock
-
-    this.metricsRegion.updateGet();
+    if (this.metricsRegion != null) {
+      this.metricsRegion.updateGet();
+    }
 
     return results;
   }
@@ -4657,8 +4665,9 @@ public class HRegion implements HeapSize
       closeRegionOperation();
     }
 
-    this.metricsRegion.updateAppend();
-
+    if (this.metricsRegion != null) {
+      this.metricsRegion.updateAppend();
+    }
 
     if (flush) {
       // Request a cache flush. Do it outside update lock.
@@ -4795,7 +4804,9 @@ public class HRegion implements HeapSize
         mvcc.completeMemstoreInsert(w);
       }
       closeRegionOperation();
-      this.metricsRegion.updateIncrement();
+      if (this.metricsRegion != null) {
+        this.metricsRegion.updateIncrement();
+      }
     }
 
     if (flush) {