HBASE-14278 Fix NPE that is showing up since HBASE-14274 went in
This commit is contained in:
parent
d4d398d942
commit
2029e85182
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue