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 2015/09/17 02:25:42 UTC

hbase git commit: HBASE-14278 Fix NPE that is showing up since HBASE-14274 went in

Repository: hbase
Updated Branches:
  refs/heads/branch-1.2 20f272cb7 -> a229ac91f


HBASE-14278 Fix NPE that is showing up since HBASE-14274 went in


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a229ac91
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a229ac91
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a229ac91

Branch: refs/heads/branch-1.2
Commit: a229ac91fbab2608ae89bbe44b1dd05e5aef1183
Parents: 20f272c
Author: Elliott Clark <ec...@apache.org>
Authored: Mon Aug 24 10:09:10 2015 -0700
Committer: Elliott Clark <ec...@apache.org>
Committed: Wed Sep 16 17:25:36 2015 -0700

----------------------------------------------------------------------
 .../MetricsRegionAggregateSourceImpl.java       | 18 +++++--
 .../MetricsRegionServerSourceImpl.java          | 12 ++---
 .../regionserver/MetricsRegionSourceImpl.java   | 50 ++++++++++++--------
 .../hadoop/metrics2/impl/JmxCacheBuster.java    | 13 ++---
 .../TestMetricsRegionSourceImpl.java            |  8 ++--
 5 files changed, 61 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a229ac91/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java
index 009fa9c..1835f6b 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionAggregateSourceImpl.java
@@ -23,6 +23,8 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
 import org.apache.hadoop.metrics2.MetricsCollector;
@@ -35,6 +37,8 @@ import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
 public class MetricsRegionAggregateSourceImpl extends BaseSourceImpl
     implements MetricsRegionAggregateSource {
 
+  private static final Log LOG = LogFactory.getLog(MetricsRegionAggregateSourceImpl.class);
+
   private final MetricsExecutorImpl executor = new MetricsExecutorImpl();
 
   private final Set<MetricsRegionSource> regionSources =
@@ -54,7 +58,7 @@ public class MetricsRegionAggregateSourceImpl extends BaseSourceImpl
     // Every few mins clean the JMX cache.
     executor.getExecutor().scheduleWithFixedDelay(new Runnable() {
       public void run() {
-        JmxCacheBuster.clearJmxCache(true);
+        JmxCacheBuster.clearJmxCache();
       }
     }, 5, 5, TimeUnit.MINUTES);
   }
@@ -67,12 +71,20 @@ public class MetricsRegionAggregateSourceImpl extends BaseSourceImpl
 
   @Override
   public void deregister(MetricsRegionSource toRemove) {
-    regionSources.remove(toRemove);
+    try {
+      regionSources.remove(toRemove);
+    } catch (Exception e) {
+      // Ignored. If this errors out it means that someone is double
+      // closing the region source and the region is already nulled out.
+      LOG.info(
+          "Error trying to remove " + toRemove + " from " + this.getClass().getSimpleName(),
+          e);
+    }
     clearCache();
   }
 
   private synchronized void clearCache() {
-    JmxCacheBuster.clearJmxCache(true);
+    JmxCacheBuster.clearJmxCache();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a229ac91/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index 3d2e0f1..d76d25c 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@ -68,19 +68,19 @@ public class MetricsRegionServerSourceImpl
     this.rsWrap = rsWrap;
 
     putHisto = getMetricsRegistry().newHistogram(MUTATE_KEY);
-    slowPut = getMetricsRegistry().newCounter(SLOW_MUTATE_KEY, SLOW_MUTATE_DESC, 0l);
+    slowPut = getMetricsRegistry().newCounter(SLOW_MUTATE_KEY, SLOW_MUTATE_DESC, 0L);
 
     deleteHisto = getMetricsRegistry().newHistogram(DELETE_KEY);
-    slowDelete = getMetricsRegistry().newCounter(SLOW_DELETE_KEY, SLOW_DELETE_DESC, 0l);
+    slowDelete = getMetricsRegistry().newCounter(SLOW_DELETE_KEY, SLOW_DELETE_DESC, 0L);
 
     getHisto = getMetricsRegistry().newHistogram(GET_KEY);
-    slowGet = getMetricsRegistry().newCounter(SLOW_GET_KEY, SLOW_GET_DESC, 0l);
+    slowGet = getMetricsRegistry().newCounter(SLOW_GET_KEY, SLOW_GET_DESC, 0L);
 
     incrementHisto = getMetricsRegistry().newHistogram(INCREMENT_KEY);
     slowIncrement = getMetricsRegistry().newCounter(SLOW_INCREMENT_KEY, SLOW_INCREMENT_DESC, 0L);
 
     appendHisto = getMetricsRegistry().newHistogram(APPEND_KEY);
-    slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0l);
+    slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0L);
     
     replayHisto = getMetricsRegistry().newHistogram(REPLAY_KEY);
     scanNextHisto = getMetricsRegistry().newHistogram(SCAN_NEXT_KEY);
@@ -88,8 +88,8 @@ public class MetricsRegionServerSourceImpl
     splitTimeHisto = getMetricsRegistry().newHistogram(SPLIT_KEY);
     flushTimeHisto = getMetricsRegistry().newHistogram(FLUSH_KEY);
 
-    splitRequest = getMetricsRegistry().newCounter(SPLIT_REQUEST_KEY, SPLIT_REQUEST_DESC, 0l);
-    splitSuccess = getMetricsRegistry().newCounter(SPLIT_SUCCESS_KEY, SPLIT_SUCCESS_DESC, 0l);
+    splitRequest = getMetricsRegistry().newCounter(SPLIT_REQUEST_KEY, SPLIT_REQUEST_DESC, 0L);
+    splitSuccess = getMetricsRegistry().newCounter(SPLIT_SUCCESS_KEY, SPLIT_SUCCESS_DESC, 0L);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a229ac91/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
index 0da6bf8..0ecf2b2 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
@@ -20,16 +20,12 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
-import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
 import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
 import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
@@ -65,6 +61,7 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
   private final MutableCounterLong regionAppend;
   private final MutableHistogram regionGet;
   private final MutableHistogram regionScanNext;
+  private final int hashCode;
 
   public MetricsRegionSourceImpl(MetricsRegionWrapper regionWrapper,
                                  MetricsRegionAggregateSourceImpl aggregate) {
@@ -101,6 +98,8 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
 
     regionScanNextKey = regionNamePrefix + MetricsRegionServerSource.SCAN_NEXT_KEY;
     regionScanNext = registry.newHistogram(regionScanNextKey);
+
+    hashCode = regionWrapper.getRegionHashCode();
   }
 
   @Override
@@ -173,12 +172,16 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
 
   @Override
   public int compareTo(MetricsRegionSource source) {
-    if (!(source instanceof MetricsRegionSourceImpl))
+    if (!(source instanceof MetricsRegionSourceImpl)) {
       return -1;
+    }
 
     MetricsRegionSourceImpl impl = (MetricsRegionSourceImpl) source;
-    return this.regionWrapper.getRegionName()
-        .compareTo(impl.regionWrapper.getRegionName());
+    if (impl == null) {
+      return -1;
+    }
+
+    return Long.compare(hashCode, impl.hashCode);
   }
 
   void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
@@ -203,31 +206,40 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
       }
 
       mrb.addGauge(
-          Interns.info(regionNamePrefix + MetricsRegionServerSource.STORE_COUNT,
+          Interns.info(
+              regionNamePrefix + MetricsRegionServerSource.STORE_COUNT,
               MetricsRegionServerSource.STORE_COUNT_DESC),
           this.regionWrapper.getNumStores());
-      mrb.addGauge(Interns.info(regionNamePrefix + MetricsRegionServerSource.STOREFILE_COUNT,
+      mrb.addGauge(Interns.info(
+              regionNamePrefix + MetricsRegionServerSource.STOREFILE_COUNT,
               MetricsRegionServerSource.STOREFILE_COUNT_DESC),
           this.regionWrapper.getNumStoreFiles());
-      mrb.addGauge(Interns.info(regionNamePrefix + MetricsRegionServerSource.MEMSTORE_SIZE,
+      mrb.addGauge(Interns.info(
+              regionNamePrefix + MetricsRegionServerSource.MEMSTORE_SIZE,
               MetricsRegionServerSource.MEMSTORE_SIZE_DESC),
           this.regionWrapper.getMemstoreSize());
-      mrb.addGauge(Interns.info(regionNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
+      mrb.addGauge(Interns.info(
+              regionNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
               MetricsRegionServerSource.STOREFILE_SIZE_DESC),
           this.regionWrapper.getStoreFileSize());
-      mrb.addCounter(Interns.info(regionNamePrefix + MetricsRegionSource.COMPACTIONS_COMPLETED_COUNT,
+      mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionSource.COMPACTIONS_COMPLETED_COUNT,
               MetricsRegionSource.COMPACTIONS_COMPLETED_DESC),
           this.regionWrapper.getNumCompactionsCompleted());
-      mrb.addCounter(Interns.info(regionNamePrefix + MetricsRegionSource.NUM_BYTES_COMPACTED_COUNT,
+      mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionSource.NUM_BYTES_COMPACTED_COUNT,
               MetricsRegionSource.NUM_BYTES_COMPACTED_DESC),
           this.regionWrapper.getNumBytesCompacted());
-      mrb.addCounter(Interns.info(regionNamePrefix + MetricsRegionSource.NUM_FILES_COMPACTED_COUNT,
+      mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionSource.NUM_FILES_COMPACTED_COUNT,
               MetricsRegionSource.NUM_FILES_COMPACTED_DESC),
           this.regionWrapper.getNumFilesCompacted());
-      mrb.addCounter(Interns.info(regionNamePrefix + MetricsRegionServerSource.READ_REQUEST_COUNT,
+      mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionServerSource.READ_REQUEST_COUNT,
               MetricsRegionServerSource.READ_REQUEST_COUNT_DESC),
           this.regionWrapper.getReadRequestCount());
-      mrb.addCounter(Interns.info(regionNamePrefix + MetricsRegionServerSource.WRITE_REQUEST_COUNT,
+      mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionServerSource.WRITE_REQUEST_COUNT,
               MetricsRegionServerSource.WRITE_REQUEST_COUNT_DESC),
           this.regionWrapper.getWriteRequestCount());
 
@@ -265,12 +277,12 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
 
   @Override
   public int hashCode() {
-    return regionWrapper.getRegionHashCode();
+    return hashCode;
   }
 
   @Override
   public boolean equals(Object obj) {
-    if (obj == this) return true;
-    return obj instanceof MetricsRegionSourceImpl && compareTo((MetricsRegionSourceImpl) obj) == 0;
+    return obj == this ||
+        (obj instanceof MetricsRegionSourceImpl && compareTo((MetricsRegionSourceImpl) obj) == 0);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a229ac91/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
index cdb202a..95734ba 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/impl/JmxCacheBuster.java
@@ -49,16 +49,10 @@ public class JmxCacheBuster {
   /**
    * For JMX to forget about all previously exported metrics.
    */
-
   public static void clearJmxCache() {
-    clearJmxCache(false);
-  }
-
-  public static synchronized void clearJmxCache(boolean force) {
     //If there are more then 100 ms before the executor will run then everything should be merged.
     ScheduledFuture future = fut.get();
-    if (!force &&
-        (future == null || (!future.isDone() && future.getDelay(TimeUnit.MILLISECONDS) > 100))) {
+    if ((future == null || (!future.isDone() && future.getDelay(TimeUnit.MILLISECONDS) > 100))) {
       // BAIL OUT
       return;
     }
@@ -66,7 +60,7 @@ public class JmxCacheBuster {
     fut.set(future);
   }
 
-  static class JmxCacheBusterRunnable implements Runnable {
+  final static class JmxCacheBusterRunnable implements Runnable {
     @Override
     public void run() {
       if (LOG.isTraceEnabled()) {
@@ -78,6 +72,9 @@ public class JmxCacheBuster {
       try {
         if (DefaultMetricsSystem.instance() != null) {
           DefaultMetricsSystem.instance().stop();
+          // Sleep some time so that the rest of the hadoop metrics
+          // system knows that things are done
+          Thread.sleep(500);
           DefaultMetricsSystem.instance().start();
         }
       }  catch (Exception exception)  {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a229ac91/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
index 28b1d62..9d60a8f 100644
--- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
@@ -42,8 +42,10 @@ public class TestMetricsRegionSourceImpl {
     assertEquals(one.hashCode(), oneClone.hashCode());
     assertNotEquals(one, two);
 
-    assertTrue( one.compareTo(two) < 0);
-    assertTrue( two.compareTo(one) > 0);
+    assertTrue( one.compareTo(two) != 0);
+    assertTrue( two.compareTo(one) != 0);
+    assertTrue( two.compareTo(one) != one.compareTo(two));
+    assertTrue( two.compareTo(two) == 0);
   }
 
 
@@ -59,8 +61,6 @@ public class TestMetricsRegionSourceImpl {
     private String regionName;
 
     public RegionWrapperStub(String regionName) {
-
-
       this.regionName = regionName;
     }