HBASE-12496 A blockedRequestsCount metric (Yi Deng)

This commit is contained in:
Andrew Purtell 2014-11-18 14:14:31 -08:00
parent d3c9425c9b
commit eb34c956a1
6 changed files with 39 additions and 2 deletions

View File

@ -246,6 +246,10 @@ public interface MetricsRegionServerSource extends BaseSource {
String MAJOR_COMPACTED_CELLS_SIZE_DESC =
"The total amount of data processed during major compactions, in bytes";
String BLOCKED_REQUESTS_COUNT = "blockedRequestCount";
String BLOCKED_REQUESTS_COUNT_DESC = "The number of blocked requests because of memstore size is "
+ "larger than blockingMemStoreSize";
String SPLIT_KEY = "splitTime";
String FLUSH_KEY = "flushTime";
}

View File

@ -246,4 +246,9 @@ public interface MetricsRegionServerWrapper {
* Get the total amount of data processed during major compactions, in bytes.
*/
long getMajorCompactedCellsSize();
/**
* @return Count of requests blocked because the memstore size is larger than blockingMemStoreSize
*/
public long getBlockedRequestsCount();
}

View File

@ -232,6 +232,10 @@ public class MetricsRegionServerSourceImpl
rsWrap.getCompactedCellsSize())
.addCounter(Interns.info(MAJOR_COMPACTED_CELLS_SIZE, MAJOR_COMPACTED_CELLS_SIZE_DESC),
rsWrap.getMajorCompactedCellsSize())
.addCounter(Interns.info(BLOCKED_REQUESTS_COUNT, BLOCKED_REQUESTS_COUNT_DESC),
rsWrap.getBlockedRequestsCount())
.tag(Interns.info(ZOOKEEPER_QUORUM_NAME, ZOOKEEPER_QUORUM_DESC),
rsWrap.getZookeeperQuorum())
.tag(Interns.info(SERVER_NAME_NAME, SERVER_NAME_DESC), rsWrap.getServerName())

View File

@ -282,6 +282,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
final Counter readRequestsCount = new Counter();
final Counter writeRequestsCount = new Counter();
// Number of requests blocked by memstore size.
private final Counter blockedRequestsCount = new Counter();
/**
* @return the number of blocked requests count.
*/
public long getBlockedRequestsCount() {
return this.blockedRequestsCount.get();
}
// Compaction counters
final AtomicLong compactionsFinished = new AtomicLong(0L);
final AtomicLong compactionNumFilesCompacted = new AtomicLong(0L);
@ -3123,6 +3133,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
if (this.getRegionInfo().isMetaRegion()) return;
if (this.memstoreSize.get() > this.blockingMemStoreSize) {
blockedRequestsCount.increment();
requestFlush();
throw new RegionTooBusyException("Above memstore limit, " +
"regionName=" + (this.getRegionInfo() == null ? "unknown" :
@ -5766,7 +5777,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
public static final long FIXED_OVERHEAD = ClassSize.align(
ClassSize.OBJECT +
ClassSize.ARRAY +
41 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
42 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
(12 * Bytes.SIZEOF_LONG) +
4 * Bytes.SIZEOF_BOOLEAN);

View File

@ -72,6 +72,7 @@ class MetricsRegionServerWrapperImpl
private volatile long flushedCellsSize = 0;
private volatile long compactedCellsSize = 0;
private volatile long majorCompactedCellsSize = 0;
private volatile long blockedRequestsCount = 0L;
private CacheStats cacheStats;
private ScheduledExecutorService executor;
@ -427,6 +428,7 @@ class MetricsRegionServerWrapperImpl
long tempFlushedCellsSize = 0;
long tempCompactedCellsSize = 0;
long tempMajorCompactedCellsSize = 0;
long tempBlockedRequestsCount = 0L;
for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
tempNumMutationsWithoutWAL += r.numMutationsWithoutWAL.get();
@ -435,6 +437,7 @@ class MetricsRegionServerWrapperImpl
tempWriteRequestsCount += r.writeRequestsCount.get();
tempCheckAndMutateChecksFailed += r.checkAndMutateChecksFailed.get();
tempCheckAndMutateChecksPassed += r.checkAndMutateChecksPassed.get();
tempBlockedRequestsCount += r.getBlockedRequestsCount();
tempNumStores += r.stores.size();
for (Store store : r.stores.values()) {
tempNumStoreFiles += store.getStorefilesCount();
@ -511,6 +514,12 @@ class MetricsRegionServerWrapperImpl
flushedCellsSize = tempFlushedCellsSize;
compactedCellsSize = tempCompactedCellsSize;
majorCompactedCellsSize = tempMajorCompactedCellsSize;
blockedRequestsCount = tempBlockedRequestsCount;
}
}
@Override
public long getBlockedRequestsCount() {
return blockedRequestsCount;
}
}

View File

@ -241,4 +241,8 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe
return 10240000;
}
@Override
public long getBlockedRequestsCount() {
return 0;
}
}