diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java index a6e846e..0641a5c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java @@ -188,6 +188,55 @@ public class RegionLoad { } /** + * @return the number of cache hit count for the region. + */ + public long getCacheHitCount() { + return regionLoadPB.getCacheHitCount(); + } + + /** + * @return the number of cache miss count for the region. + */ + public long getCacheMissCount() { + return regionLoadPB.getCacheMissCount(); + } + + /** + * @return the number of blocks evicted for the region. + */ + public long getCacheEvictedBlockCount() { + return regionLoadPB.getCacheEvictedBlockCount(); + } + + /** + * @return the number of cached block count for the region. + */ + public long getCacheBlockCount() { + return regionLoadPB.getCacheBlockCount(); + } + + /** + * @return the size of cache for the region. + */ + public long getCacheSize() { + return regionLoadPB.getCacheSize(); + } + + /** + * @return the current cache hit ratio for the region. + */ + public float getCacheHitRatio() { + return regionLoadPB.getCacheHitRatio(); + } + + /** + * @return the cache hit ratio of latest N periods for the region. + */ + public String getCacheHitRatioLatestNPeriods() { + return regionLoadPB.getCacheHitRatioLatestNPeriods(); + } + + /** * @see java.lang.Object#toString() */ @Override @@ -236,6 +285,20 @@ public class RegionLoad { this.getCompleteSequenceId()); sb = Strings.appendKeyValue(sb, "dataLocality", this.getDataLocality()); + sb = Strings.appendKeyValue(sb, "cacheHitCount", + this.getCacheHitCount()); + sb = Strings.appendKeyValue(sb, "cacheMissCount", + this.getCacheMissCount()); + sb = Strings.appendKeyValue(sb, "cacheEvictedBlockCount", + this.getCacheEvictedBlockCount()); + sb = Strings.appendKeyValue(sb, "cacheBlockCount", + this.getCacheBlockCount()); + sb = Strings.appendKeyValue(sb, "cacheSize", + this.getCacheSize()); + sb = Strings.appendKeyValue(sb, "cacheHitRatio", + this.getCacheHitRatio()); + sb = Strings.appendKeyValue(sb, "cacheHitRatioLastNPeriods", + this.getCacheHitRatioLatestNPeriods()); return sb.toString(); } } diff --git a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java index f820193..1c1038c 100644 --- a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java +++ b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java @@ -72,10 +72,11 @@ public class MemcachedBlockCache implements BlockCache { private final MemcachedClient client; private final HFileBlockTranscoder tc = new HFileBlockTranscoder(); - private final CacheStats cacheStats = new CacheStats("MemcachedBlockCache"); + private final CacheStats cacheStats; public MemcachedBlockCache(Configuration c) throws IOException { LOG.info("Creating MemcachedBlockCache"); + cacheStats = new CacheStats(c, "MemcachedBlockCache"); long opTimeout = c.getLong(MEMCACHED_OPTIMEOUT_KEY, MEMCACHED_DEFAULT_TIMEOUT); long queueTimeout = c.getLong(MEMCACHED_TIMEOUT_KEY, opTimeout + MEMCACHED_DEFAULT_TIMEOUT); @@ -149,9 +150,9 @@ public class MemcachedBlockCache implements BlockCache { // Update stats if this request doesn't have it turned off 100% of the time if (updateCacheMetrics) { if (result == null) { - cacheStats.miss(caching, cacheKey.isPrimary()); + cacheStats.miss(cacheKey.getRegion(), caching, cacheKey.isPrimary()); } else { - cacheStats.hit(caching, cacheKey.isPrimary()); + cacheStats.hit(cacheKey.getRegion(), caching, cacheKey.isPrimary()); } } } diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java index 5fd4e18..e6e6d0f 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java @@ -3613,6 +3613,141 @@ public final class ClusterStatusProtos { */ org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder getStoreCompleteSequenceIdOrBuilder( int index); + + // optional uint64 cache_hit_count = 19; + /** + * optional uint64 cache_hit_count = 19; + * + *
+     ** the number of cache hit count for the region 
+     * 
+ */ + boolean hasCacheHitCount(); + /** + * optional uint64 cache_hit_count = 19; + * + *
+     ** the number of cache hit count for the region 
+     * 
+ */ + long getCacheHitCount(); + + // optional uint64 cache_miss_count = 20; + /** + * optional uint64 cache_miss_count = 20; + * + *
+     ** the number of cache miss count for the region 
+     * 
+ */ + boolean hasCacheMissCount(); + /** + * optional uint64 cache_miss_count = 20; + * + *
+     ** the number of cache miss count for the region 
+     * 
+ */ + long getCacheMissCount(); + + // optional uint64 cache_evicted_block_count = 21; + /** + * optional uint64 cache_evicted_block_count = 21; + * + *
+     ** the number of blocks evicted for the region 
+     * 
+ */ + boolean hasCacheEvictedBlockCount(); + /** + * optional uint64 cache_evicted_block_count = 21; + * + *
+     ** the number of blocks evicted for the region 
+     * 
+ */ + long getCacheEvictedBlockCount(); + + // optional uint64 cache_size = 22; + /** + * optional uint64 cache_size = 22; + * + *
+     ** the size of cache for the region 
+     * 
+ */ + boolean hasCacheSize(); + /** + * optional uint64 cache_size = 22; + * + *
+     ** the size of cache for the region 
+     * 
+ */ + long getCacheSize(); + + // optional uint64 cache_block_count = 23; + /** + * optional uint64 cache_block_count = 23; + * + *
+     ** the number of cached block count for the region 
+     * 
+ */ + boolean hasCacheBlockCount(); + /** + * optional uint64 cache_block_count = 23; + * + *
+     ** the number of cached block count for the region 
+     * 
+ */ + long getCacheBlockCount(); + + // optional float cache_hit_ratio = 24; + /** + * optional float cache_hit_ratio = 24; + * + *
+     ** the current cache hit ratio for the region 
+     * 
+ */ + boolean hasCacheHitRatio(); + /** + * optional float cache_hit_ratio = 24; + * + *
+     ** the current cache hit ratio for the region 
+     * 
+ */ + float getCacheHitRatio(); + + // optional string cache_hit_ratio_latest_N_periods = 25; + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+     ** the cache hit ratio of latest N peroids for the region 
+     * 
+ */ + boolean hasCacheHitRatioLatestNPeriods(); + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+     ** the cache hit ratio of latest N peroids for the region 
+     * 
+ */ + java.lang.String getCacheHitRatioLatestNPeriods(); + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+     ** the cache hit ratio of latest N peroids for the region 
+     * 
+ */ + com.google.protobuf.ByteString + getCacheHitRatioLatestNPeriodsBytes(); } /** * Protobuf type {@code hbase.pb.RegionLoad} @@ -3766,6 +3901,41 @@ public final class ClusterStatusProtos { storeCompleteSequenceId_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId.PARSER, extensionRegistry)); break; } + case 152: { + bitField0_ |= 0x00020000; + cacheHitCount_ = input.readUInt64(); + break; + } + case 160: { + bitField0_ |= 0x00040000; + cacheMissCount_ = input.readUInt64(); + break; + } + case 168: { + bitField0_ |= 0x00080000; + cacheEvictedBlockCount_ = input.readUInt64(); + break; + } + case 176: { + bitField0_ |= 0x00100000; + cacheSize_ = input.readUInt64(); + break; + } + case 184: { + bitField0_ |= 0x00200000; + cacheBlockCount_ = input.readUInt64(); + break; + } + case 197: { + bitField0_ |= 0x00400000; + cacheHitRatio_ = input.readFloat(); + break; + } + case 202: { + bitField0_ |= 0x00800000; + cacheHitRatioLatestNPeriods_ = input.readBytes(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -4283,6 +4453,205 @@ public final class ClusterStatusProtos { return storeCompleteSequenceId_.get(index); } + // optional uint64 cache_hit_count = 19; + public static final int CACHE_HIT_COUNT_FIELD_NUMBER = 19; + private long cacheHitCount_; + /** + * optional uint64 cache_hit_count = 19; + * + *
+     ** the number of cache hit count for the region 
+     * 
+ */ + public boolean hasCacheHitCount() { + return ((bitField0_ & 0x00020000) == 0x00020000); + } + /** + * optional uint64 cache_hit_count = 19; + * + *
+     ** the number of cache hit count for the region 
+     * 
+ */ + public long getCacheHitCount() { + return cacheHitCount_; + } + + // optional uint64 cache_miss_count = 20; + public static final int CACHE_MISS_COUNT_FIELD_NUMBER = 20; + private long cacheMissCount_; + /** + * optional uint64 cache_miss_count = 20; + * + *
+     ** the number of cache miss count for the region 
+     * 
+ */ + public boolean hasCacheMissCount() { + return ((bitField0_ & 0x00040000) == 0x00040000); + } + /** + * optional uint64 cache_miss_count = 20; + * + *
+     ** the number of cache miss count for the region 
+     * 
+ */ + public long getCacheMissCount() { + return cacheMissCount_; + } + + // optional uint64 cache_evicted_block_count = 21; + public static final int CACHE_EVICTED_BLOCK_COUNT_FIELD_NUMBER = 21; + private long cacheEvictedBlockCount_; + /** + * optional uint64 cache_evicted_block_count = 21; + * + *
+     ** the number of blocks evicted for the region 
+     * 
+ */ + public boolean hasCacheEvictedBlockCount() { + return ((bitField0_ & 0x00080000) == 0x00080000); + } + /** + * optional uint64 cache_evicted_block_count = 21; + * + *
+     ** the number of blocks evicted for the region 
+     * 
+ */ + public long getCacheEvictedBlockCount() { + return cacheEvictedBlockCount_; + } + + // optional uint64 cache_size = 22; + public static final int CACHE_SIZE_FIELD_NUMBER = 22; + private long cacheSize_; + /** + * optional uint64 cache_size = 22; + * + *
+     ** the size of cache for the region 
+     * 
+ */ + public boolean hasCacheSize() { + return ((bitField0_ & 0x00100000) == 0x00100000); + } + /** + * optional uint64 cache_size = 22; + * + *
+     ** the size of cache for the region 
+     * 
+ */ + public long getCacheSize() { + return cacheSize_; + } + + // optional uint64 cache_block_count = 23; + public static final int CACHE_BLOCK_COUNT_FIELD_NUMBER = 23; + private long cacheBlockCount_; + /** + * optional uint64 cache_block_count = 23; + * + *
+     ** the number of cached block count for the region 
+     * 
+ */ + public boolean hasCacheBlockCount() { + return ((bitField0_ & 0x00200000) == 0x00200000); + } + /** + * optional uint64 cache_block_count = 23; + * + *
+     ** the number of cached block count for the region 
+     * 
+ */ + public long getCacheBlockCount() { + return cacheBlockCount_; + } + + // optional float cache_hit_ratio = 24; + public static final int CACHE_HIT_RATIO_FIELD_NUMBER = 24; + private float cacheHitRatio_; + /** + * optional float cache_hit_ratio = 24; + * + *
+     ** the current cache hit ratio for the region 
+     * 
+ */ + public boolean hasCacheHitRatio() { + return ((bitField0_ & 0x00400000) == 0x00400000); + } + /** + * optional float cache_hit_ratio = 24; + * + *
+     ** the current cache hit ratio for the region 
+     * 
+ */ + public float getCacheHitRatio() { + return cacheHitRatio_; + } + + // optional string cache_hit_ratio_latest_N_periods = 25; + public static final int CACHE_HIT_RATIO_LATEST_N_PERIODS_FIELD_NUMBER = 25; + private java.lang.Object cacheHitRatioLatestNPeriods_; + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+     ** the cache hit ratio of latest N peroids for the region 
+     * 
+ */ + public boolean hasCacheHitRatioLatestNPeriods() { + return ((bitField0_ & 0x00800000) == 0x00800000); + } + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+     ** the cache hit ratio of latest N peroids for the region 
+     * 
+ */ + public java.lang.String getCacheHitRatioLatestNPeriods() { + java.lang.Object ref = cacheHitRatioLatestNPeriods_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + cacheHitRatioLatestNPeriods_ = s; + } + return s; + } + } + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+     ** the cache hit ratio of latest N peroids for the region 
+     * 
+ */ + public com.google.protobuf.ByteString + getCacheHitRatioLatestNPeriodsBytes() { + java.lang.Object ref = cacheHitRatioLatestNPeriods_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + cacheHitRatioLatestNPeriods_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + private void initFields() { regionSpecifier_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); stores_ = 0; @@ -4302,6 +4671,13 @@ public final class ClusterStatusProtos { dataLocality_ = 0F; lastMajorCompactionTs_ = 0L; storeCompleteSequenceId_ = java.util.Collections.emptyList(); + cacheHitCount_ = 0L; + cacheMissCount_ = 0L; + cacheEvictedBlockCount_ = 0L; + cacheSize_ = 0L; + cacheBlockCount_ = 0L; + cacheHitRatio_ = 0F; + cacheHitRatioLatestNPeriods_ = ""; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -4383,6 +4759,27 @@ public final class ClusterStatusProtos { for (int i = 0; i < storeCompleteSequenceId_.size(); i++) { output.writeMessage(18, storeCompleteSequenceId_.get(i)); } + if (((bitField0_ & 0x00020000) == 0x00020000)) { + output.writeUInt64(19, cacheHitCount_); + } + if (((bitField0_ & 0x00040000) == 0x00040000)) { + output.writeUInt64(20, cacheMissCount_); + } + if (((bitField0_ & 0x00080000) == 0x00080000)) { + output.writeUInt64(21, cacheEvictedBlockCount_); + } + if (((bitField0_ & 0x00100000) == 0x00100000)) { + output.writeUInt64(22, cacheSize_); + } + if (((bitField0_ & 0x00200000) == 0x00200000)) { + output.writeUInt64(23, cacheBlockCount_); + } + if (((bitField0_ & 0x00400000) == 0x00400000)) { + output.writeFloat(24, cacheHitRatio_); + } + if (((bitField0_ & 0x00800000) == 0x00800000)) { + output.writeBytes(25, getCacheHitRatioLatestNPeriodsBytes()); + } getUnknownFields().writeTo(output); } @@ -4464,6 +4861,34 @@ public final class ClusterStatusProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(18, storeCompleteSequenceId_.get(i)); } + if (((bitField0_ & 0x00020000) == 0x00020000)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(19, cacheHitCount_); + } + if (((bitField0_ & 0x00040000) == 0x00040000)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(20, cacheMissCount_); + } + if (((bitField0_ & 0x00080000) == 0x00080000)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(21, cacheEvictedBlockCount_); + } + if (((bitField0_ & 0x00100000) == 0x00100000)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(22, cacheSize_); + } + if (((bitField0_ & 0x00200000) == 0x00200000)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(23, cacheBlockCount_); + } + if (((bitField0_ & 0x00400000) == 0x00400000)) { + size += com.google.protobuf.CodedOutputStream + .computeFloatSize(24, cacheHitRatio_); + } + if (((bitField0_ & 0x00800000) == 0x00800000)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(25, getCacheHitRatioLatestNPeriodsBytes()); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -4573,6 +4998,40 @@ public final class ClusterStatusProtos { } result = result && getStoreCompleteSequenceIdList() .equals(other.getStoreCompleteSequenceIdList()); + result = result && (hasCacheHitCount() == other.hasCacheHitCount()); + if (hasCacheHitCount()) { + result = result && (getCacheHitCount() + == other.getCacheHitCount()); + } + result = result && (hasCacheMissCount() == other.hasCacheMissCount()); + if (hasCacheMissCount()) { + result = result && (getCacheMissCount() + == other.getCacheMissCount()); + } + result = result && (hasCacheEvictedBlockCount() == other.hasCacheEvictedBlockCount()); + if (hasCacheEvictedBlockCount()) { + result = result && (getCacheEvictedBlockCount() + == other.getCacheEvictedBlockCount()); + } + result = result && (hasCacheSize() == other.hasCacheSize()); + if (hasCacheSize()) { + result = result && (getCacheSize() + == other.getCacheSize()); + } + result = result && (hasCacheBlockCount() == other.hasCacheBlockCount()); + if (hasCacheBlockCount()) { + result = result && (getCacheBlockCount() + == other.getCacheBlockCount()); + } + result = result && (hasCacheHitRatio() == other.hasCacheHitRatio()); + if (hasCacheHitRatio()) { + result = result && (Float.floatToIntBits(getCacheHitRatio()) == Float.floatToIntBits(other.getCacheHitRatio())); + } + result = result && (hasCacheHitRatioLatestNPeriods() == other.hasCacheHitRatioLatestNPeriods()); + if (hasCacheHitRatioLatestNPeriods()) { + result = result && getCacheHitRatioLatestNPeriods() + .equals(other.getCacheHitRatioLatestNPeriods()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -4659,6 +5118,35 @@ public final class ClusterStatusProtos { hash = (37 * hash) + STORE_COMPLETE_SEQUENCE_ID_FIELD_NUMBER; hash = (53 * hash) + getStoreCompleteSequenceIdList().hashCode(); } + if (hasCacheHitCount()) { + hash = (37 * hash) + CACHE_HIT_COUNT_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getCacheHitCount()); + } + if (hasCacheMissCount()) { + hash = (37 * hash) + CACHE_MISS_COUNT_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getCacheMissCount()); + } + if (hasCacheEvictedBlockCount()) { + hash = (37 * hash) + CACHE_EVICTED_BLOCK_COUNT_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getCacheEvictedBlockCount()); + } + if (hasCacheSize()) { + hash = (37 * hash) + CACHE_SIZE_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getCacheSize()); + } + if (hasCacheBlockCount()) { + hash = (37 * hash) + CACHE_BLOCK_COUNT_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getCacheBlockCount()); + } + if (hasCacheHitRatio()) { + hash = (37 * hash) + CACHE_HIT_RATIO_FIELD_NUMBER; + hash = (53 * hash) + Float.floatToIntBits( + getCacheHitRatio()); + } + if (hasCacheHitRatioLatestNPeriods()) { + hash = (37 * hash) + CACHE_HIT_RATIO_LATEST_N_PERIODS_FIELD_NUMBER; + hash = (53 * hash) + getCacheHitRatioLatestNPeriods().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -4814,6 +5302,20 @@ public final class ClusterStatusProtos { } else { storeCompleteSequenceIdBuilder_.clear(); } + cacheHitCount_ = 0L; + bitField0_ = (bitField0_ & ~0x00040000); + cacheMissCount_ = 0L; + bitField0_ = (bitField0_ & ~0x00080000); + cacheEvictedBlockCount_ = 0L; + bitField0_ = (bitField0_ & ~0x00100000); + cacheSize_ = 0L; + bitField0_ = (bitField0_ & ~0x00200000); + cacheBlockCount_ = 0L; + bitField0_ = (bitField0_ & ~0x00400000); + cacheHitRatio_ = 0F; + bitField0_ = (bitField0_ & ~0x00800000); + cacheHitRatioLatestNPeriods_ = ""; + bitField0_ = (bitField0_ & ~0x01000000); return this; } @@ -4923,6 +5425,34 @@ public final class ClusterStatusProtos { } else { result.storeCompleteSequenceId_ = storeCompleteSequenceIdBuilder_.build(); } + if (((from_bitField0_ & 0x00040000) == 0x00040000)) { + to_bitField0_ |= 0x00020000; + } + result.cacheHitCount_ = cacheHitCount_; + if (((from_bitField0_ & 0x00080000) == 0x00080000)) { + to_bitField0_ |= 0x00040000; + } + result.cacheMissCount_ = cacheMissCount_; + if (((from_bitField0_ & 0x00100000) == 0x00100000)) { + to_bitField0_ |= 0x00080000; + } + result.cacheEvictedBlockCount_ = cacheEvictedBlockCount_; + if (((from_bitField0_ & 0x00200000) == 0x00200000)) { + to_bitField0_ |= 0x00100000; + } + result.cacheSize_ = cacheSize_; + if (((from_bitField0_ & 0x00400000) == 0x00400000)) { + to_bitField0_ |= 0x00200000; + } + result.cacheBlockCount_ = cacheBlockCount_; + if (((from_bitField0_ & 0x00800000) == 0x00800000)) { + to_bitField0_ |= 0x00400000; + } + result.cacheHitRatio_ = cacheHitRatio_; + if (((from_bitField0_ & 0x01000000) == 0x01000000)) { + to_bitField0_ |= 0x00800000; + } + result.cacheHitRatioLatestNPeriods_ = cacheHitRatioLatestNPeriods_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -5016,6 +5546,29 @@ public final class ClusterStatusProtos { } } } + if (other.hasCacheHitCount()) { + setCacheHitCount(other.getCacheHitCount()); + } + if (other.hasCacheMissCount()) { + setCacheMissCount(other.getCacheMissCount()); + } + if (other.hasCacheEvictedBlockCount()) { + setCacheEvictedBlockCount(other.getCacheEvictedBlockCount()); + } + if (other.hasCacheSize()) { + setCacheSize(other.getCacheSize()); + } + if (other.hasCacheBlockCount()) { + setCacheBlockCount(other.getCacheBlockCount()); + } + if (other.hasCacheHitRatio()) { + setCacheHitRatio(other.getCacheHitRatio()); + } + if (other.hasCacheHitRatioLatestNPeriods()) { + bitField0_ |= 0x01000000; + cacheHitRatioLatestNPeriods_ = other.cacheHitRatioLatestNPeriods_; + onChanged(); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -6306,6 +6859,398 @@ public final class ClusterStatusProtos { return storeCompleteSequenceIdBuilder_; } + // optional uint64 cache_hit_count = 19; + private long cacheHitCount_ ; + /** + * optional uint64 cache_hit_count = 19; + * + *
+       ** the number of cache hit count for the region 
+       * 
+ */ + public boolean hasCacheHitCount() { + return ((bitField0_ & 0x00040000) == 0x00040000); + } + /** + * optional uint64 cache_hit_count = 19; + * + *
+       ** the number of cache hit count for the region 
+       * 
+ */ + public long getCacheHitCount() { + return cacheHitCount_; + } + /** + * optional uint64 cache_hit_count = 19; + * + *
+       ** the number of cache hit count for the region 
+       * 
+ */ + public Builder setCacheHitCount(long value) { + bitField0_ |= 0x00040000; + cacheHitCount_ = value; + onChanged(); + return this; + } + /** + * optional uint64 cache_hit_count = 19; + * + *
+       ** the number of cache hit count for the region 
+       * 
+ */ + public Builder clearCacheHitCount() { + bitField0_ = (bitField0_ & ~0x00040000); + cacheHitCount_ = 0L; + onChanged(); + return this; + } + + // optional uint64 cache_miss_count = 20; + private long cacheMissCount_ ; + /** + * optional uint64 cache_miss_count = 20; + * + *
+       ** the number of cache miss count for the region 
+       * 
+ */ + public boolean hasCacheMissCount() { + return ((bitField0_ & 0x00080000) == 0x00080000); + } + /** + * optional uint64 cache_miss_count = 20; + * + *
+       ** the number of cache miss count for the region 
+       * 
+ */ + public long getCacheMissCount() { + return cacheMissCount_; + } + /** + * optional uint64 cache_miss_count = 20; + * + *
+       ** the number of cache miss count for the region 
+       * 
+ */ + public Builder setCacheMissCount(long value) { + bitField0_ |= 0x00080000; + cacheMissCount_ = value; + onChanged(); + return this; + } + /** + * optional uint64 cache_miss_count = 20; + * + *
+       ** the number of cache miss count for the region 
+       * 
+ */ + public Builder clearCacheMissCount() { + bitField0_ = (bitField0_ & ~0x00080000); + cacheMissCount_ = 0L; + onChanged(); + return this; + } + + // optional uint64 cache_evicted_block_count = 21; + private long cacheEvictedBlockCount_ ; + /** + * optional uint64 cache_evicted_block_count = 21; + * + *
+       ** the number of blocks evicted for the region 
+       * 
+ */ + public boolean hasCacheEvictedBlockCount() { + return ((bitField0_ & 0x00100000) == 0x00100000); + } + /** + * optional uint64 cache_evicted_block_count = 21; + * + *
+       ** the number of blocks evicted for the region 
+       * 
+ */ + public long getCacheEvictedBlockCount() { + return cacheEvictedBlockCount_; + } + /** + * optional uint64 cache_evicted_block_count = 21; + * + *
+       ** the number of blocks evicted for the region 
+       * 
+ */ + public Builder setCacheEvictedBlockCount(long value) { + bitField0_ |= 0x00100000; + cacheEvictedBlockCount_ = value; + onChanged(); + return this; + } + /** + * optional uint64 cache_evicted_block_count = 21; + * + *
+       ** the number of blocks evicted for the region 
+       * 
+ */ + public Builder clearCacheEvictedBlockCount() { + bitField0_ = (bitField0_ & ~0x00100000); + cacheEvictedBlockCount_ = 0L; + onChanged(); + return this; + } + + // optional uint64 cache_size = 22; + private long cacheSize_ ; + /** + * optional uint64 cache_size = 22; + * + *
+       ** the size of cache for the region 
+       * 
+ */ + public boolean hasCacheSize() { + return ((bitField0_ & 0x00200000) == 0x00200000); + } + /** + * optional uint64 cache_size = 22; + * + *
+       ** the size of cache for the region 
+       * 
+ */ + public long getCacheSize() { + return cacheSize_; + } + /** + * optional uint64 cache_size = 22; + * + *
+       ** the size of cache for the region 
+       * 
+ */ + public Builder setCacheSize(long value) { + bitField0_ |= 0x00200000; + cacheSize_ = value; + onChanged(); + return this; + } + /** + * optional uint64 cache_size = 22; + * + *
+       ** the size of cache for the region 
+       * 
+ */ + public Builder clearCacheSize() { + bitField0_ = (bitField0_ & ~0x00200000); + cacheSize_ = 0L; + onChanged(); + return this; + } + + // optional uint64 cache_block_count = 23; + private long cacheBlockCount_ ; + /** + * optional uint64 cache_block_count = 23; + * + *
+       ** the number of cached block count for the region 
+       * 
+ */ + public boolean hasCacheBlockCount() { + return ((bitField0_ & 0x00400000) == 0x00400000); + } + /** + * optional uint64 cache_block_count = 23; + * + *
+       ** the number of cached block count for the region 
+       * 
+ */ + public long getCacheBlockCount() { + return cacheBlockCount_; + } + /** + * optional uint64 cache_block_count = 23; + * + *
+       ** the number of cached block count for the region 
+       * 
+ */ + public Builder setCacheBlockCount(long value) { + bitField0_ |= 0x00400000; + cacheBlockCount_ = value; + onChanged(); + return this; + } + /** + * optional uint64 cache_block_count = 23; + * + *
+       ** the number of cached block count for the region 
+       * 
+ */ + public Builder clearCacheBlockCount() { + bitField0_ = (bitField0_ & ~0x00400000); + cacheBlockCount_ = 0L; + onChanged(); + return this; + } + + // optional float cache_hit_ratio = 24; + private float cacheHitRatio_ ; + /** + * optional float cache_hit_ratio = 24; + * + *
+       ** the current cache hit ratio for the region 
+       * 
+ */ + public boolean hasCacheHitRatio() { + return ((bitField0_ & 0x00800000) == 0x00800000); + } + /** + * optional float cache_hit_ratio = 24; + * + *
+       ** the current cache hit ratio for the region 
+       * 
+ */ + public float getCacheHitRatio() { + return cacheHitRatio_; + } + /** + * optional float cache_hit_ratio = 24; + * + *
+       ** the current cache hit ratio for the region 
+       * 
+ */ + public Builder setCacheHitRatio(float value) { + bitField0_ |= 0x00800000; + cacheHitRatio_ = value; + onChanged(); + return this; + } + /** + * optional float cache_hit_ratio = 24; + * + *
+       ** the current cache hit ratio for the region 
+       * 
+ */ + public Builder clearCacheHitRatio() { + bitField0_ = (bitField0_ & ~0x00800000); + cacheHitRatio_ = 0F; + onChanged(); + return this; + } + + // optional string cache_hit_ratio_latest_N_periods = 25; + private java.lang.Object cacheHitRatioLatestNPeriods_ = ""; + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+       ** the cache hit ratio of latest N peroids for the region 
+       * 
+ */ + public boolean hasCacheHitRatioLatestNPeriods() { + return ((bitField0_ & 0x01000000) == 0x01000000); + } + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+       ** the cache hit ratio of latest N peroids for the region 
+       * 
+ */ + public java.lang.String getCacheHitRatioLatestNPeriods() { + java.lang.Object ref = cacheHitRatioLatestNPeriods_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); + cacheHitRatioLatestNPeriods_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+       ** the cache hit ratio of latest N peroids for the region 
+       * 
+ */ + public com.google.protobuf.ByteString + getCacheHitRatioLatestNPeriodsBytes() { + java.lang.Object ref = cacheHitRatioLatestNPeriods_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + cacheHitRatioLatestNPeriods_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+       ** the cache hit ratio of latest N peroids for the region 
+       * 
+ */ + public Builder setCacheHitRatioLatestNPeriods( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x01000000; + cacheHitRatioLatestNPeriods_ = value; + onChanged(); + return this; + } + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+       ** the cache hit ratio of latest N peroids for the region 
+       * 
+ */ + public Builder clearCacheHitRatioLatestNPeriods() { + bitField0_ = (bitField0_ & ~0x01000000); + cacheHitRatioLatestNPeriods_ = getDefaultInstance().getCacheHitRatioLatestNPeriods(); + onChanged(); + return this; + } + /** + * optional string cache_hit_ratio_latest_N_periods = 25; + * + *
+       ** the cache hit ratio of latest N peroids for the region 
+       * 
+ */ + public Builder setCacheHitRatioLatestNPeriodsBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x01000000; + cacheHitRatioLatestNPeriods_ = value; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.RegionLoad) } @@ -14723,7 +15668,7 @@ public final class ClusterStatusProtos { "e\030\001 \002(\014\022\023\n\013sequence_id\030\002 \002(\004\"p\n\026RegionSt" + "oreSequenceIds\022 \n\030last_flushed_sequence_" + "id\030\001 \002(\004\0224\n\021store_sequence_id\030\002 \003(\0132\031.hb" + - "ase.pb.StoreSequenceId\"\324\004\n\nRegionLoad\0223\n" + + "ase.pb.StoreSequenceId\"\234\006\n\nRegionLoad\0223\n" + "\020region_specifier\030\001 \002(\0132\031.hbase.pb.Regio" + "nSpecifier\022\016\n\006stores\030\002 \001(\r\022\022\n\nstorefiles", "\030\003 \001(\r\022\"\n\032store_uncompressed_size_MB\030\004 \001" + @@ -14738,34 +15683,39 @@ public final class ClusterStatusProtos { "_sequence_id\030\017 \001(\004\022\025\n\rdata_locality\030\020 \001(", "\002\022#\n\030last_major_compaction_ts\030\021 \001(\004:\0010\022=" + "\n\032store_complete_sequence_id\030\022 \003(\0132\031.hba" + - "se.pb.StoreSequenceId\"T\n\023ReplicationLoad" + + "se.pb.StoreSequenceId\022\027\n\017cache_hit_count" + + "\030\023 \001(\004\022\030\n\020cache_miss_count\030\024 \001(\004\022!\n\031cach" + + "e_evicted_block_count\030\025 \001(\004\022\022\n\ncache_siz" + + "e\030\026 \001(\004\022\031\n\021cache_block_count\030\027 \001(\004\022\027\n\017ca" + + "che_hit_ratio\030\030 \001(\002\022(\n cache_hit_ratio_l" + + "atest_N_periods\030\031 \001(\t\"T\n\023ReplicationLoad" + "Sink\022\032\n\022ageOfLastAppliedOp\030\001 \002(\004\022!\n\031time" + - "StampsOfLastAppliedOp\030\002 \002(\004\"\225\001\n\025Replicat" + + "StampsOfLastAppliedOp\030\002 \002(\004\"\225\001\n\025Replicat", "ionLoadSource\022\016\n\006peerID\030\001 \002(\t\022\032\n\022ageOfLa" + "stShippedOp\030\002 \002(\004\022\026\n\016sizeOfLogQueue\030\003 \002(" + "\r\022 \n\030timeStampOfLastShippedOp\030\004 \002(\004\022\026\n\016r" + "eplicationLag\030\005 \002(\004\"\212\003\n\nServerLoad\022\032\n\022nu" + - "mber_of_requests\030\001 \001(\004\022 \n\030total_number_o", + "mber_of_requests\030\001 \001(\004\022 \n\030total_number_o" + "f_requests\030\002 \001(\004\022\024\n\014used_heap_MB\030\003 \001(\r\022\023" + "\n\013max_heap_MB\030\004 \001(\r\022*\n\014region_loads\030\005 \003(" + "\0132\024.hbase.pb.RegionLoad\022+\n\014coprocessors\030" + "\006 \003(\0132\025.hbase.pb.Coprocessor\022\031\n\021report_s" + - "tart_time\030\007 \001(\004\022\027\n\017report_end_time\030\010 \001(\004" + + "tart_time\030\007 \001(\004\022\027\n\017report_end_time\030\010 \001(\004", "\022\030\n\020info_server_port\030\t \001(\r\0227\n\016replLoadSo" + "urce\030\n \003(\0132\037.hbase.pb.ReplicationLoadSou" + "rce\0223\n\014replLoadSink\030\013 \001(\0132\035.hbase.pb.Rep" + "licationLoadSink\"a\n\016LiveServerInfo\022$\n\006se" + - "rver\030\001 \002(\0132\024.hbase.pb.ServerName\022)\n\013serv", + "rver\030\001 \002(\0132\024.hbase.pb.ServerName\022)\n\013serv" + "er_load\030\002 \002(\0132\024.hbase.pb.ServerLoad\"\250\003\n\r" + "ClusterStatus\0228\n\rhbase_version\030\001 \001(\0132!.h" + "base.pb.HBaseVersionFileContent\022.\n\014live_" + "servers\030\002 \003(\0132\030.hbase.pb.LiveServerInfo\022" + - "*\n\014dead_servers\030\003 \003(\0132\024.hbase.pb.ServerN" + + "*\n\014dead_servers\030\003 \003(\0132\024.hbase.pb.ServerN", "ame\022;\n\025regions_in_transition\030\004 \003(\0132\034.hba" + "se.pb.RegionInTransition\022\'\n\ncluster_id\030\005" + " \001(\0132\023.hbase.pb.ClusterId\0222\n\023master_copr" + "ocessors\030\006 \003(\0132\025.hbase.pb.Coprocessor\022$\n" + - "\006master\030\007 \001(\0132\024.hbase.pb.ServerName\022,\n\016b", + "\006master\030\007 \001(\0132\024.hbase.pb.ServerName\022,\n\016b" + "ackup_masters\030\010 \003(\0132\024.hbase.pb.ServerNam" + "e\022\023\n\013balancer_on\030\t \001(\010BF\n*org.apache.had" + "oop.hbase.protobuf.generatedB\023ClusterSta" + @@ -14805,7 +15755,7 @@ public final class ClusterStatusProtos { internal_static_hbase_pb_RegionLoad_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_RegionLoad_descriptor, - new java.lang.String[] { "RegionSpecifier", "Stores", "Storefiles", "StoreUncompressedSizeMB", "StorefileSizeMB", "MemstoreSizeMB", "StorefileIndexSizeMB", "ReadRequestsCount", "WriteRequestsCount", "TotalCompactingKVs", "CurrentCompactedKVs", "RootIndexSizeKB", "TotalStaticIndexSizeKB", "TotalStaticBloomSizeKB", "CompleteSequenceId", "DataLocality", "LastMajorCompactionTs", "StoreCompleteSequenceId", }); + new java.lang.String[] { "RegionSpecifier", "Stores", "Storefiles", "StoreUncompressedSizeMB", "StorefileSizeMB", "MemstoreSizeMB", "StorefileIndexSizeMB", "ReadRequestsCount", "WriteRequestsCount", "TotalCompactingKVs", "CurrentCompactedKVs", "RootIndexSizeKB", "TotalStaticIndexSizeKB", "TotalStaticBloomSizeKB", "CompleteSequenceId", "DataLocality", "LastMajorCompactionTs", "StoreCompleteSequenceId", "CacheHitCount", "CacheMissCount", "CacheEvictedBlockCount", "CacheSize", "CacheBlockCount", "CacheHitRatio", "CacheHitRatioLatestNPeriods", }); internal_static_hbase_pb_ReplicationLoadSink_descriptor = getDescriptor().getMessageTypes().get(5); internal_static_hbase_pb_ReplicationLoadSink_fieldAccessorTable = new diff --git a/hbase-protocol/src/main/protobuf/ClusterStatus.proto b/hbase-protocol/src/main/protobuf/ClusterStatus.proto index 228be7e..7c2f770 100644 --- a/hbase-protocol/src/main/protobuf/ClusterStatus.proto +++ b/hbase-protocol/src/main/protobuf/ClusterStatus.proto @@ -136,6 +136,27 @@ message RegionLoad { /** the most recent sequence Id of store from cache flush */ repeated StoreSequenceId store_complete_sequence_id = 18; + + /** the number of cache hit count for the region */ + optional uint64 cache_hit_count = 19; + + /** the number of cache miss count for the region */ + optional uint64 cache_miss_count = 20; + + /** the number of blocks evicted for the region */ + optional uint64 cache_evicted_block_count = 21; + + /** the size of cache for the region */ + optional uint64 cache_size = 22; + + /** the number of cached block count for the region */ + optional uint64 cache_block_count = 23; + + /** the current cache hit ratio for the region */ + optional float cache_hit_ratio = 24; + + /** the cache hit ratio of latest N peroids for the region */ + optional string cache_hit_ratio_latest_N_periods = 25; } /* Server-level protobufs */ diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon index 33cdf00..2775c95 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon @@ -33,6 +33,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad; org.apache.hadoop.hbase.client.RegionReplicaUtil; org.apache.hadoop.hbase.regionserver.MetricsRegionWrapper; + org.apache.hadoop.util.StringUtils; <%if (onlineRegions != null && onlineRegions.size() > 0) %> @@ -47,6 +48,9 @@
  • Storefile Metrics
  • Memstore Metrics
  • Compaction Metrics
  • +
  • + Block Cache Metrics +
  • @@ -64,6 +68,9 @@
    <& compactStats; onlineRegions = onlineRegions; &>
    +
    + <& cacheStats; onlineRegions = onlineRegions; &> +

    Region names are made of the containing table's name, a comma, @@ -234,4 +241,42 @@ + + +<%def cacheStats> +<%args> + List onlineRegions; + + + + + + + + + + + + + + <%for HRegionInfo r: onlineRegions %> + + + <%java> + RegionLoad load = regionServer.createRegionLoad(r.getEncodedName()); + + + <%if load != null %> + + + + + + + + + + +
    Region NameSizeCountHitsMissesEvictedHit RatioHit Ratios of Latest Periods
    <% HRegionInfo.getRegionNameAsStringForDisplay(r, + regionServer.getConfiguration()) %><% StringUtils.humanReadableInt(load.getCacheSize()) %><% String.format("%,d", load.getCacheBlockCount()) %><% String.format("%,d", load.getCacheHitCount()) %><% String.format("%,d", load.getCacheMissCount()) %><% String.format("%,d", load.getCacheEvictedBlockCount()) %><% String.format("%.2f", 100 * load.getCacheHitRatio()) + "%" %><% load.getCacheHitRatioLatestNPeriods() %>
    \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheKey.java index 180cbb4..b2387a4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheKey.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheKey.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.io.hfile; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.util.Bytes; @@ -31,6 +32,7 @@ public class BlockCacheKey implements HeapSize, java.io.Serializable { private final String hfileName; private final long offset; private final boolean isPrimaryReplicaBlock; + private final String region; /** * Construct a new BlockCacheKey @@ -38,13 +40,28 @@ public class BlockCacheKey implements HeapSize, java.io.Serializable { * @param offset Offset of the block into the file */ public BlockCacheKey(String hfileName, long offset) { - this(hfileName, offset, true); + this(null, hfileName, offset, true); } - public BlockCacheKey(String hfileName, long offset, boolean isPrimaryReplica) { + /** + * Construct a new BlockCacheKey + * @param region The name of the region this block belongs to. can be null + * @param hfileName The name of the HFile this block belongs to. + * @param offset Offset of the block into the file + */ + public BlockCacheKey(String region, String hfileName, long offset) { + this(region, hfileName, offset, true); + } + + public BlockCacheKey(String region, String hfileName, long offset, boolean isPrimaryReplica) { this.isPrimaryReplicaBlock = isPrimaryReplica; this.hfileName = hfileName; this.offset = offset; + this.region = region; + } + + public BlockCacheKey(String hfileName, long offset, boolean isPrimaryReplica) { + this(null, hfileName, offset, isPrimaryReplica); } @Override @@ -66,11 +83,13 @@ public class BlockCacheKey implements HeapSize, java.io.Serializable { @Override public String toString() { - return String.format("%s_%d", hfileName, offset); + return region == null ? String.format("%s_%d", hfileName, offset) : + String.format("%s_%d_%s", hfileName, offset, region); } public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT +Bytes.SIZEOF_BOOLEAN + ClassSize.REFERENCE + // this.hfileName + ClassSize.REFERENCE + // this.region Bytes.SIZEOF_LONG); // this.offset /** @@ -80,7 +99,7 @@ public class BlockCacheKey implements HeapSize, java.io.Serializable { @Override public long heapSize() { return ClassSize.align(FIXED_OVERHEAD + ClassSize.STRING + - 2 * hfileName.length()); + 2 * hfileName.length() + (region == null ? 0 : 2 * region.length())); } // can't avoid this unfortunately @@ -98,4 +117,18 @@ public class BlockCacheKey implements HeapSize, java.io.Serializable { public long getOffset() { return offset; } + + /** + * Parse region name from file path + * + * @param path The full path of the file + * @return region name + */ + static String parseRegion(Path path) { + return path.depth() > 2 ? path.getParent().getParent().getName() : ""; + } + + public String getRegion() { + return region; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java index d6bdec0..16556b7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java @@ -634,7 +634,7 @@ public class CacheConfig { "hbase.bucketcache.ioengine.errors.tolerated.duration", BucketCache.DEFAULT_ERROR_TOLERATION_DURATION); // Bucket cache logs its stats on creation internal to the constructor. - bucketCache = new BucketCache(bucketCacheIOEngineName, + bucketCache = new BucketCache(c, bucketCacheIOEngineName, bucketCacheSize, blockSize, bucketSizes, writerThreads, writerQueueLen, persistentPath, ioErrorsTolerationDuration); } catch (IOException ioex) { @@ -665,10 +665,10 @@ public class CacheConfig { boolean combinedWithLru = conf.getBoolean(BUCKET_CACHE_COMBINED_KEY, DEFAULT_BUCKET_CACHE_COMBINED); if (useExternal) { - GLOBAL_BLOCK_CACHE_INSTANCE = new InclusiveCombinedBlockCache(l1, l2); + GLOBAL_BLOCK_CACHE_INSTANCE = new InclusiveCombinedBlockCache(conf, l1, l2); } else { if (combinedWithLru) { - GLOBAL_BLOCK_CACHE_INSTANCE = new CombinedBlockCache(l1, l2); + GLOBAL_BLOCK_CACHE_INSTANCE = new CombinedBlockCache(conf, l1, l2); } else { // L1 and L2 are not 'combined'. They are connected via the LruBlockCache victimhandler // mechanism. It is a little ugly but works according to the following: when the diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java index 9301de2..f60e429 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java @@ -18,12 +18,19 @@ */ package org.apache.hadoop.hbase.io.hfile; +import java.util.*; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import com.google.common.util.concurrent.AtomicLongMap; +import org.apache.commons.collections.buffer.CircularFifoBuffer; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.classification.InterfaceAudience; import com.yammer.metrics.core.Histogram; import com.yammer.metrics.core.MetricsRegistry; +import org.apache.hadoop.util.StringUtils; /** * Class that implements cache metrics. @@ -40,6 +47,13 @@ public class CacheStats { */ static final int DEFAULT_WINDOW_PERIODS = 5; + /** The key of period of chore */ + public static final String CACHE_STATS_CHORE_PERIOD_KEY = + "hbase.regionserver.blockcache.stat.chore.period"; + + /** The default of period of chore */ + public static final int CACHE_STATS_CHORE_PERIOD_DEFAULT = 5 * 60 * 1000; + /** The number of getBlock requests that were cache hits */ private final AtomicLong hitCount = new AtomicLong(0); @@ -74,6 +88,29 @@ public class CacheStats { /** The total number of blocks for primary replica that have been evicted */ private final AtomicLong primaryEvictedBlockCount = new AtomicLong(0); + /** The map of cache hits for regions */ + private final AtomicLongMap regionHitCountMap = AtomicLongMap.create(); + + /** The map of cache misses for regions */ + private final AtomicLongMap regionMissCountMap = AtomicLongMap.create(); + + /** The map of evicted cache blocks for regions */ + private final AtomicLongMap regionEvictedBlockCountMap = AtomicLongMap.create(); + + /** The map of cache size for regions */ + private final AtomicLongMap regionSizeMap = AtomicLongMap.create(); + + /** The map of cached blocks for regions */ + private final AtomicLongMap regionBlockCountMap = AtomicLongMap.create(); + + /** The map of latest N period hit count for regions */ + private final Map regionHitCountHistoryMap; + + /** The map of latest N period request count for regions */ + private final Map regionRequestCountHistoryMap; + + static final String STAT_NOT_AVAILABLE = String.valueOf(Float.NaN); + /** The number of metrics periods to include in window */ private final int numPeriodsInWindow; /** Hit counts for each period in window */ @@ -99,18 +136,22 @@ public class CacheStats { */ private Histogram ageAtEviction; private long startTime = System.nanoTime(); + private final Configuration conf; - public CacheStats(final String name) { - this(name, DEFAULT_WINDOW_PERIODS); + public CacheStats(Configuration conf, final String name) { + this(conf, name, DEFAULT_WINDOW_PERIODS); } - public CacheStats(final String name, int numPeriodsInWindow) { + public CacheStats(Configuration conf, final String name, int numPeriodsInWindow) { this.numPeriodsInWindow = numPeriodsInWindow; this.hitCounts = initializeZeros(numPeriodsInWindow); this.hitCachingCounts = initializeZeros(numPeriodsInWindow); this.requestCounts = initializeZeros(numPeriodsInWindow); this.requestCachingCounts = initializeZeros(numPeriodsInWindow); this.ageAtEviction = METRICS.newHistogram(CacheStats.class, name + ".ageAtEviction"); + this.regionHitCountHistoryMap = new ConcurrentHashMap<>(); + this.regionRequestCountHistoryMap = new ConcurrentHashMap<>(); + this.conf = conf; } @Override @@ -123,41 +164,82 @@ public class CacheStats { ", primaryMissCount=" + getPrimaryMissCount() + ", primaryHitCount=" + getPrimaryHitCount() + ", evictedAgeMean=" + snapshot.getMean() + - ", evictedAgeStdDev=" + snapshot.getStdDev(); + ", evictedAgeStdDev=" + snapshot.getStdDev() + + ", " + getRegionStatString(); + } + + public String getRegionStatString() { + return "numPeriodsInWindow=" + numPeriodsInWindow + + ", regionHitCountMap=" + regionHitCountMap.toString() + + ", regionMissCountMap=" + regionMissCountMap.toString() + + ", regionSizeMap=" + regionSizeMap.toString() + + ", regionEvictedBlockCountMap=" + regionEvictedBlockCountMap.toString() + + ", regionBlockCountMap=" + regionBlockCountMap.toString(); } - public void miss(boolean caching, boolean primary) { + public void miss(String region, boolean caching, boolean primary) { missCount.incrementAndGet(); if (primary) primaryMissCount.incrementAndGet(); if (caching) missCachingCount.incrementAndGet(); + if (region != null) { + regionMissCountMap.incrementAndGet(region); + } } - public void hit(boolean caching) { - hit(caching, true); + public void hit(String region, boolean caching) { + hit(region, caching, true); } - public void hit(boolean caching, boolean primary) { + public void hit(String region, boolean caching, boolean primary) { hitCount.incrementAndGet(); if (primary) primaryHitCount.incrementAndGet(); if (caching) hitCachingCount.incrementAndGet(); + if (region != null) { + regionHitCountMap.incrementAndGet(region); + } } public void evict() { evictionCount.incrementAndGet(); } - public void evicted(final long t, boolean primary) { + public void evicted(String region, final long t, boolean primary) { if (t > this.startTime) this.ageAtEviction.update(t - this.startTime); this.evictedBlockCount.incrementAndGet(); if (primary) { primaryEvictedBlockCount.incrementAndGet(); } + if (region != null) { + regionEvictedBlockCountMap.incrementAndGet(region); + } + } + + public void setSize(String region, long size) { + if (region != null) { + regionSizeMap.addAndGet(region, size); + } + } + + public void incrementBlockCount(String region) { + if (region != null) { + regionBlockCountMap.incrementAndGet(region); + } + } + + public void decrementBlockCount(String region) { + if (region != null) { + regionBlockCountMap.decrementAndGet(region); + } } public long getRequestCount() { return getHitCount() + getMissCount(); } + public long getRequestCount(String region) { + return getHitCount(region) + getMissCount(region); + } + public long getRequestCachingCount() { return getHitCachingCount() + getMissCachingCount(); } @@ -166,6 +248,10 @@ public class CacheStats { return missCount.get(); } + public long getMissCount(String region) { + return region == null ? getMissCount() : regionMissCountMap.get(region); + } + public long getPrimaryMissCount() { return primaryMissCount.get(); } @@ -178,6 +264,10 @@ public class CacheStats { return hitCount.get(); } + public long getHitCount(String region) { + return region == null ? getHitCount() : regionHitCountMap.get(region); + } + public long getPrimaryHitCount() { return primaryHitCount.get(); } @@ -194,6 +284,10 @@ public class CacheStats { return this.evictedBlockCount.get(); } + public long getEvictedCount(String region) { + return region == null ? getEvictedCount() : regionEvictedBlockCountMap.get(region); + } + public long getPrimaryEvictedCount() { return primaryEvictedBlockCount.get(); } @@ -202,6 +296,139 @@ public class CacheStats { return ((float)getHitCount()/(float)getRequestCount()); } + public float getHitRatio(String region) { + return ((float)getHitCount(region)/(float)getRequestCount(region)); + } + + /** + * Roll and clean the stats for regions. + */ + public void rollAndCleanRegionMetrics(Set onlineRegions) { + rollRegionMetrics(onlineRegions); + + /** + * FIXME: CacheStats should be cleaned in distributed mode because a block cache is a + * singleton. The below line can be removed when HBASE-14704 is resolved. + */ + if (!conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false)) return; + + clean(onlineRegions); + } + + /** + * Roll the stats for regions. + */ + private void rollRegionMetrics(Set onlineRegions) { + for (String region : onlineRegions) { + CircularFifoBuffer hitCounts = regionHitCountHistoryMap.get(region); + if (hitCounts == null) { + hitCounts = new CircularFifoBuffer(numPeriodsInWindow + 1); + regionHitCountHistoryMap.put(region, hitCounts); + } + hitCounts.add(getHitCount(region)); + + CircularFifoBuffer requestCounts = regionRequestCountHistoryMap.get(region); + if (requestCounts == null) { + requestCounts = new CircularFifoBuffer(numPeriodsInWindow + 1); + regionRequestCountHistoryMap.put(region, requestCounts); + } + requestCounts.add(getRequestCount(region)); + } + } + + /** + * Clean entries related to regions that are not online. + * + * @param onlineRegions the set of online regions + */ + private void clean(Set onlineRegions) { + clean(onlineRegions, regionHitCountMap); + clean(onlineRegions, regionMissCountMap); + clean(onlineRegions, regionEvictedBlockCountMap); + clean(onlineRegions, regionSizeMap); + clean(onlineRegions, regionBlockCountMap); + clean(onlineRegions, regionHitCountHistoryMap); + clean(onlineRegions, regionRequestCountHistoryMap); + } + + private void clean(Set onlineRegions, AtomicLongMap atomicLongMap) { + Set toRemove = new HashSet<>(); + for (String region : atomicLongMap.asMap().keySet()) { + if (!onlineRegions.contains(region)) + toRemove.add(region); + } + for (String region : toRemove) { + atomicLongMap.remove(region); + } + } + + private void clean(Set onlineRegions, Map map) { + Set toRemove = new HashSet<>(); + for (String region : map.keySet()) { + if (!onlineRegions.contains(region)) + toRemove.add(region); + } + for (String region : toRemove) { + map.remove(region); + } + } + + /** + * Returns cache hit ratio as String for latest periods. + * Stats are based on the differences of each contiguous periods. + * Output format: [interval] ratio1 ratio2 ratio3 ... + * + * @param region a region name + * @return cache hit ratio string for latest periods + */ + public String getLatestPeriodsHitRatioString(String region) { + CircularFifoBuffer hitCounts = regionHitCountHistoryMap.get(region); + CircularFifoBuffer requestCounts = regionRequestCountHistoryMap.get(region); + if (hitCounts != null && requestCounts != null && + hitCounts.size() == requestCounts.size() && hitCounts.size() > 0) { + Object[] hitCountArray = hitCounts.toArray(); + Object[] requestCountArray = requestCounts.toArray(); + + long hitCount, requestCount; + StringBuilder sb = new StringBuilder(); + sb.append("Every ") + .append(conf.getInt(CACHE_STATS_CHORE_PERIOD_KEY, + CACHE_STATS_CHORE_PERIOD_DEFAULT) / 1000).append("sec: "); + + if (hitCounts.size() == 1) { + hitCount = (long) hitCountArray[0]; + requestCount = (long) requestCountArray[0]; + sb.append(currentPeriodHitRatio(region, hitCount, requestCount)); + } else { + for (int i = 0; i < hitCounts.size() - 1; i++) { + hitCount = (long) hitCountArray[i + 1] - (long) hitCountArray[i]; + requestCount = (long) requestCountArray[i + 1] - (long) requestCountArray[i]; + + if (i > 0) sb.append(", "); + + if (requestCount == 0) + sb.append(STAT_NOT_AVAILABLE); + else + sb.append(StringUtils.formatPercent(((float) hitCount / (float) requestCount), 2)); + } + + // append hit ratio of current period + hitCount = (long) hitCountArray[hitCounts.size() - 1]; + requestCount = (long) requestCountArray[hitCounts.size() - 1]; + sb.append(", ").append(currentPeriodHitRatio(region, hitCount, requestCount)); + } + return sb.toString(); + } else + return STAT_NOT_AVAILABLE; + } + + private String currentPeriodHitRatio(String region, long hitCountPrev, long requestCountPrev) { + long request = getRequestCount(region) - requestCountPrev; + long hit = getHitCount(region) - hitCountPrev; + return request == 0 ? + String.valueOf(Float.NaN) : StringUtils.formatPercent(((float) hit / (float) request), 2); + } + public double getHitCachingRatio() { return ((float)getHitCachingCount()/(float)getRequestCachingCount()); } @@ -218,6 +445,14 @@ public class CacheStats { return ((float)getEvictedCount()/(float)getEvictionCount()); } + public long getSize(String region) { + return regionSizeMap.get(region); + } + + public long getBlockCount(String region) { + return region == null ? 0 : regionBlockCountMap.get(region); + } + public void rollMetricsPeriod() { hitCounts[windowIndex] = getHitCount() - lastHitCount; lastHitCount = getHitCount(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java index 3f65823..d2e11eb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.io.hfile; import java.util.Iterator; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory; @@ -44,10 +45,10 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize { protected final BlockCache l2Cache; protected final CombinedCacheStats combinedCacheStats; - public CombinedBlockCache(LruBlockCache lruCache, BlockCache l2Cache) { + public CombinedBlockCache(Configuration conf, LruBlockCache lruCache, BlockCache l2Cache) { this.lruCache = lruCache; this.l2Cache = l2Cache; - this.combinedCacheStats = new CombinedCacheStats(lruCache.getStats(), + this.combinedCacheStats = new CombinedCacheStats(conf, lruCache.getStats(), l2Cache.getStats()); } @@ -135,8 +136,8 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize { private final CacheStats lruCacheStats; private final CacheStats bucketCacheStats; - CombinedCacheStats(CacheStats lbcStats, CacheStats fcStats) { - super("CombinedBlockCache"); + CombinedCacheStats(Configuration conf, CacheStats lbcStats, CacheStats fcStats) { + super(conf, "CombinedBlockCache"); this.lruCacheStats = lbcStats; this.bucketCacheStats = fcStats; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java index 930f42a..c2ae644 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java @@ -352,8 +352,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { public void returnBlock(HFileBlock block) { BlockCache blockCache = this.cacheConf.getBlockCache(); if (blockCache != null && block != null) { - BlockCacheKey cacheKey = new BlockCacheKey(this.getFileContext().getHFileName(), - block.getOffset(), this.isPrimaryReplicaReader()); + BlockCacheKey cacheKey = new BlockCacheKey(BlockCacheKey.parseRegion(path), + this.getFileContext().getHFileName(), block.getOffset(), this.isPrimaryReplicaReader()); blockCache.returnBlock(cacheKey, block); } } @@ -1410,7 +1410,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { .getRootBlockKey(block)) { // Check cache for block. If found return. long metaBlockOffset = metaBlockIndexReader.getRootBlockOffset(block); - BlockCacheKey cacheKey = new BlockCacheKey(name, metaBlockOffset, + BlockCacheKey cacheKey = new BlockCacheKey(BlockCacheKey.parseRegion(path), + name, metaBlockOffset, this.isPrimaryReplicaReader()); cacheBlock &= cacheConf.shouldCacheDataOnRead(); @@ -1458,7 +1459,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { // the other choice is to duplicate work (which the cache would prevent you // from doing). - BlockCacheKey cacheKey = new BlockCacheKey(name, dataBlockOffset, + BlockCacheKey cacheKey = new BlockCacheKey(BlockCacheKey.parseRegion(path), + name, dataBlockOffset, this.isPrimaryReplicaReader()); boolean useLock = false; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index 66c7f1d..c2e6570 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -482,7 +482,8 @@ public class HFileWriterImpl implements HFile.Writer { */ private void doCacheOnWrite(long offset) { HFileBlock cacheFormatBlock = fsBlockWriter.getBlockForCaching(cacheConf); - cacheConf.getBlockCache().cacheBlock(new BlockCacheKey(name, offset), cacheFormatBlock); + cacheConf.getBlockCache().cacheBlock( + new BlockCacheKey(BlockCacheKey.parseRegion(path), name, offset), cacheFormatBlock); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/InclusiveCombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/InclusiveCombinedBlockCache.java index 667e7b4..cdc5f0c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/InclusiveCombinedBlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/InclusiveCombinedBlockCache.java @@ -19,13 +19,14 @@ package org.apache.hadoop.hbase.io.hfile; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience; @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) public class InclusiveCombinedBlockCache extends CombinedBlockCache implements BlockCache { - public InclusiveCombinedBlockCache(LruBlockCache l1, BlockCache l2) { - super(l1,l2); + public InclusiveCombinedBlockCache(Configuration conf, LruBlockCache l1, BlockCache l2) { + super(conf,l1,l2); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java index 04983f6..e46b30d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java @@ -199,6 +199,8 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { /** Where to send victims (blocks evicted/missing from the cache) */ private BlockCache victimHandler = null; + private final Configuration conf; + /** * Default constructor. Specify maximum size and expected average block * size (approximation is fine). @@ -208,15 +210,15 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { * @param maxSize maximum size of cache, in bytes * @param blockSize approximate size of each block, in bytes */ - public LruBlockCache(long maxSize, long blockSize) { - this(maxSize, blockSize, true); + public LruBlockCache(Configuration conf, long maxSize, long blockSize) { + this(conf, maxSize, blockSize, true); } /** * Constructor used for testing. Allows disabling of the eviction thread. */ - public LruBlockCache(long maxSize, long blockSize, boolean evictionThread) { - this(maxSize, blockSize, evictionThread, + public LruBlockCache(Configuration conf, long maxSize, long blockSize, boolean evictionThread) { + this(conf, maxSize, blockSize, evictionThread, (int)Math.ceil(1.2*maxSize/blockSize), DEFAULT_LOAD_FACTOR, DEFAULT_CONCURRENCY_LEVEL, DEFAULT_MIN_FACTOR, DEFAULT_ACCEPTABLE_FACTOR, @@ -228,7 +230,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { } public LruBlockCache(long maxSize, long blockSize, boolean evictionThread, Configuration conf) { - this(maxSize, blockSize, evictionThread, + this(conf, maxSize, blockSize, evictionThread, (int)Math.ceil(1.2*maxSize/blockSize), DEFAULT_LOAD_FACTOR, DEFAULT_CONCURRENCY_LEVEL, @@ -247,6 +249,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { /** * Configurable constructor. Use this constructor if not using defaults. + * @param conf configuration * @param maxSize maximum size of this cache, in bytes * @param blockSize expected average size of blocks, in bytes * @param evictionThread whether to run evictions in a bg thread or not @@ -259,7 +262,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { * @param multiFactor percentage of total size for multiple-access blocks * @param memoryFactor percentage of total size for in-memory blocks */ - public LruBlockCache(long maxSize, long blockSize, boolean evictionThread, + public LruBlockCache(Configuration conf, long maxSize, long blockSize, boolean evictionThread, int mapInitialSize, float mapLoadFactor, int mapConcurrencyLevel, float minFactor, float acceptableFactor, float singleFactor, float multiFactor, float memoryFactor, boolean forceInMemory) { @@ -284,7 +287,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { this.singleFactor = singleFactor; this.multiFactor = multiFactor; this.memoryFactor = memoryFactor; - this.stats = new CacheStats(this.getClass().getSimpleName()); + this.stats = new CacheStats(conf, this.getClass().getSimpleName()); this.count = new AtomicLong(0); this.elements = new AtomicLong(0); this.overhead = calculateOverhead(maxSize, blockSize, mapConcurrencyLevel); @@ -299,6 +302,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { // every five minutes. this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this), statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS); + this.conf = conf; } @Override @@ -397,9 +401,14 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { */ protected long updateSizeMetrics(LruCachedBlock cb, boolean evict) { long heapsize = cb.heapSize(); + String region = cb.getCacheKey().getRegion(); if (evict) { heapsize *= -1; + stats.decrementBlockCount(region); + } else { + stats.incrementBlockCount(region); } + stats.setSize(region, heapsize); return size.addAndGet(heapsize); } @@ -416,8 +425,9 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat, boolean updateCacheMetrics) { LruCachedBlock cb = map.get(cacheKey); + String region = cacheKey.getRegion(); if (cb == null) { - if (!repeat && updateCacheMetrics) stats.miss(caching, cacheKey.isPrimary()); + if (!repeat && updateCacheMetrics) stats.miss(region, caching, cacheKey.isPrimary()); // If there is another block cache then try and read there. // However if this is a retry ( second time in double checked locking ) // And it's already a miss then the l2 will also be a miss. @@ -432,7 +442,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { } return null; } - if (updateCacheMetrics) stats.hit(caching, cacheKey.isPrimary()); + if (updateCacheMetrics) stats.hit(region, caching, cacheKey.isPrimary()); cb.access(count.incrementAndGet()); return cb.getBuffer(); } @@ -495,7 +505,8 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { long size = map.size(); assertCounterSanity(size, val); } - stats.evicted(block.getCachedTime(), block.getCacheKey().isPrimary()); + stats.evicted(block.getCacheKey().getRegion(), block.getCachedTime(), + block.getCacheKey().isPrimary()); if (evictedByEvictionProcess && victimHandler != null) { if (victimHandler instanceof BucketCache) { boolean wait = getCurrentSize() < acceptableSize(); @@ -867,7 +878,8 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { "0,": (StringUtils.formatPercent(stats.getHitCachingRatio(), 2) + ", ")) + "evictions=" + stats.getEvictionCount() + ", " + "evicted=" + stats.getEvictedCount() + ", " + - "evictedPerRun=" + stats.evictedPerEviction()); + "evictedPerRun=" + stats.evictedPerEviction() + ", " + + stats.getRegionStatString()); } /** @@ -881,7 +893,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize { } public final static long CACHE_FIXED_OVERHEAD = ClassSize.align( - (3 * Bytes.SIZEOF_LONG) + (9 * ClassSize.REFERENCE) + + (3 * Bytes.SIZEOF_LONG) + (10 * ClassSize.REFERENCE) + (5 * Bytes.SIZEOF_FLOAT) + Bytes.SIZEOF_BOOLEAN + ClassSize.OBJECT); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index 5eb6f8f..762a92f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -50,6 +50,7 @@ import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.BlockCache; @@ -163,7 +164,7 @@ public class BucketCache implements BlockCache, HeapSize { // will wait blocks flushed to IOEngine for some time when caching boolean wait_when_cache = false; - private final BucketCacheStats cacheStats = new BucketCacheStats(); + private final BucketCacheStats cacheStats; private final String persistencePath; private final long cacheCapacity; @@ -209,16 +210,19 @@ public class BucketCache implements BlockCache, HeapSize { // Allocate or free space for the block private BucketAllocator bucketAllocator; - public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes, + public BucketCache(Configuration conf, String ioEngineName, long capacity, int blockSize, + int[] bucketSizes, int writerThreadNum, int writerQLen, String persistencePath) throws FileNotFoundException, IOException { - this(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen, + this(conf, ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen, persistencePath, DEFAULT_ERROR_TOLERATION_DURATION); } - public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes, + public BucketCache(Configuration conf, String ioEngineName, long capacity, int blockSize, + int[] bucketSizes, int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration) throws FileNotFoundException, IOException { + this.cacheStats = new BucketCacheStats(conf); this.ioEngine = getIOEngineFromName(ioEngineName, capacity); this.writerThreads = new WriterThread[writerThreadNum]; long blockNumCapacity = capacity / blockSize; @@ -404,7 +408,7 @@ public class BucketCache implements BlockCache, HeapSize { RAMQueueEntry re = ramCache.get(key); if (re != null) { if (updateCacheMetrics) { - cacheStats.hit(caching, key.isPrimary()); + cacheStats.hit(key.getRegion(), caching, key.isPrimary()); } re.access(accessCount.incrementAndGet()); return re.getData(); @@ -426,7 +430,7 @@ public class BucketCache implements BlockCache, HeapSize { bucketEntry.deserializerReference(this.deserialiserMap)); long timeTaken = System.nanoTime() - start; if (updateCacheMetrics) { - cacheStats.hit(caching, key.isPrimary()); + cacheStats.hit(key.getRegion(), caching, key.isPrimary()); cacheStats.ioHit(timeTaken); } if (cachedBlock.getMemoryType() == MemoryType.SHARED) { @@ -448,7 +452,7 @@ public class BucketCache implements BlockCache, HeapSize { } } if (!repeat && updateCacheMetrics) { - cacheStats.miss(caching, key.isPrimary()); + cacheStats.miss(key.getRegion(), caching, key.isPrimary()); } return null; } @@ -478,7 +482,7 @@ public class BucketCache implements BlockCache, HeapSize { BucketEntry bucketEntry = backingMap.get(cacheKey); if (bucketEntry == null) { if (removedBlock != null) { - cacheStats.evicted(0, cacheKey.isPrimary()); + cacheStats.evicted(cacheKey.getRegion(), 0, cacheKey.isPrimary()); return true; } else { return false; @@ -500,7 +504,7 @@ public class BucketCache implements BlockCache, HeapSize { offsetLock.releaseLockEntry(lockEntry); } } - cacheStats.evicted(bucketEntry.getCachedTime(), cacheKey.isPrimary()); + cacheStats.evicted(cacheKey.getRegion(), bucketEntry.getCachedTime(), cacheKey.isPrimary()); return true; } @@ -521,7 +525,7 @@ public class BucketCache implements BlockCache, HeapSize { BucketEntry bucketEntry = backingMap.get(cacheKey); if (bucketEntry == null) { if (removedBlock != null) { - cacheStats.evicted(0, cacheKey.isPrimary()); + cacheStats.evicted(cacheKey.getRegion(), 0, cacheKey.isPrimary()); return true; } else { return false; @@ -561,7 +565,7 @@ public class BucketCache implements BlockCache, HeapSize { offsetLock.releaseLockEntry(lockEntry); } } - cacheStats.evicted(bucketEntry.getCachedTime(), cacheKey.isPrimary()); + cacheStats.evicted(cacheKey.getRegion(), bucketEntry.getCachedTime(), cacheKey.isPrimary()); return true; } @@ -605,7 +609,8 @@ public class BucketCache implements BlockCache, HeapSize { (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2)+ ", ")) + "evictions=" + cacheStats.getEvictionCount() + ", " + "evicted=" + cacheStats.getEvictedCount() + ", " + - "evictedPerRun=" + cacheStats.evictedPerEviction()); + "evictedPerRun=" + cacheStats.evictedPerEviction() + ", " + + cacheStats.getRegionStatString()); cacheStats.reset(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java index 51e6268..d0c31bd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.io.hfile.bucket; import java.util.concurrent.atomic.AtomicLong; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.CacheStats; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -34,8 +35,8 @@ public class BucketCacheStats extends CacheStats { private final static int nanoTime = 1000000; private long lastLogTime = EnvironmentEdgeManager.currentTime(); - BucketCacheStats() { - super("BucketCache"); + BucketCacheStats(Configuration conf) { + super(conf, "BucketCache"); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 2ce2193..d0bef72 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -93,6 +93,7 @@ import org.apache.hadoop.hbase.executor.ExecutorType; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.http.InfoServer; import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.CacheStats; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.ipc.RpcClient; import org.apache.hadoop.hbase.ipc.RpcClientFactory; @@ -440,6 +441,11 @@ public class HRegionServer extends HasThread implements */ private MovedRegionsCleaner movedRegionsCleaner; + /** + * Chore for CacheStats + */ + CacheStatsChore cacheStatsChore; + // chore for refreshing store files for secondary regions private StorefileRefresherChore storefileRefresher; @@ -853,6 +859,9 @@ public class HRegionServer extends HasThread implements // Create the thread to clean the moved regions list movedRegionsCleaner = MovedRegionsCleaner.create(this); + // Create the thread to clean CacheStats + cacheStatsChore = CacheStatsChore.create(this); + if (this.nonceManager != null) { // Create the scheduled chore that cleans up nonces. nonceManagerChore = this.nonceManager.createCleanupScheduledChore(this); @@ -1007,6 +1016,10 @@ public class HRegionServer extends HasThread implements movedRegionsCleaner.stop("Region Server stopping"); } + if (cacheStatsChore != null) { + cacheStatsChore.stop("Region Server stopping"); + } + // Send interrupts to wake up threads if sleeping so they notice shutdown. // TODO: Should we check they are alive? If OOME could have exited already if (this.hMemManager != null) this.hMemManager.stop(); @@ -1503,6 +1516,20 @@ public class HRegionServer extends HasThread implements .setLastMajorCompactionTs(r.getOldestHfileTs(true)); ((HRegion)r).setCompleteSequenceId(regionLoadBldr); + if (cacheConfig != null && cacheConfig.getBlockCache() != null) { + CacheStats stats = cacheConfig.getBlockCache().getStats(); + if (stats != null) { + String region = r.getRegionInfo().getEncodedName(); + regionLoadBldr.setCacheHitCount(stats.getHitCount(region)) + .setCacheMissCount(stats.getMissCount(region)) + .setCacheEvictedBlockCount(stats.getEvictedCount(region)) + .setCacheSize(stats.getSize(region)) + .setCacheBlockCount(stats.getBlockCount(region)) + .setCacheHitRatio(stats.getHitRatio(region)) + .setCacheHitRatioLatestNPeriods(stats.getLatestPeriodsHitRatioString(region)); + } + } + return regionLoadBldr.build(); } @@ -1733,6 +1760,7 @@ public class HRegionServer extends HasThread implements if (this.nonceManagerChore != null) choreService.scheduleChore(nonceManagerChore); if (this.storefileRefresher != null) choreService.scheduleChore(storefileRefresher); if (this.movedRegionsCleaner != null) choreService.scheduleChore(movedRegionsCleaner); + if (this.cacheStatsChore != null) choreService.scheduleChore(cacheStatsChore); // Leases is not a Thread. Internally it runs a daemon thread. If it gets // an unhandled exception, it will just exit. @@ -2158,6 +2186,7 @@ public class HRegionServer extends HasThread implements if (this.healthCheckChore != null) healthCheckChore.cancel(true); if (this.storefileRefresher != null) storefileRefresher.cancel(true); if (this.movedRegionsCleaner != null) movedRegionsCleaner.cancel(true); + if (this.cacheStatsChore != null) cacheStatsChore.cancel(true); if (this.cacheFlusher != null) { this.cacheFlusher.join(); @@ -3143,6 +3172,48 @@ public class HRegionServer extends HasThread implements } } + /** + * Creates a Chore thread for CacheStats. + */ + protected final static class CacheStatsChore extends ScheduledChore implements Stoppable { + private HRegionServer regionServer; + Stoppable stoppable; + + private CacheStatsChore(HRegionServer regionServer, Stoppable stoppable){ + super("CacheStatsCleaner for region " + regionServer, stoppable, + regionServer.conf.getInt(CacheStats.CACHE_STATS_CHORE_PERIOD_KEY, + CacheStats.CACHE_STATS_CHORE_PERIOD_DEFAULT)); + this.regionServer = regionServer; + this.stoppable = stoppable; + } + + static CacheStatsChore create(HRegionServer rs){ + Stoppable stoppable = new Stoppable() { + private volatile boolean isStopped = false; + @Override public void stop(String why) { isStopped = true;} + @Override public boolean isStopped() {return isStopped;} + }; + + return new CacheStatsChore(rs, stoppable); + } + + @Override + protected void chore() { + regionServer.cacheConfig.getBlockCache().getStats().rollAndCleanRegionMetrics( + regionServer.onlineRegions.keySet()); + } + + @Override + public void stop(String why) { + stoppable.stop(why); + } + + @Override + public boolean isStopped() { + return stoppable.isStopped(); + } + } + private String getMyEphemeralNodePath() { return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java index 2c957ef..b8a1ce7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java @@ -166,7 +166,8 @@ public class TestCacheOnWrite { int[] bucketSizes = { INDEX_BLOCK_SIZE, DATA_BLOCK_SIZE, BLOOM_BLOCK_SIZE, 64 * 1024, 128 * 1024 }; BlockCache bucketcache = - new BucketCache("offheap", 128 * 1024 * 1024, 64 * 1024, bucketSizes, 5, 64 * 100, null); + new BucketCache(conf, "offheap", 128 * 1024 * 1024, 64 * 1024, bucketSizes, 5, 64 * 100, + null); blockcaches.add(bucketcache); return blockcaches; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheStats.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheStats.java new file mode 100644 index 0000000..d9b44e7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheStats.java @@ -0,0 +1,172 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io.hfile; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.HashSet; +import java.util.Set; + +import static org.junit.Assert.assertEquals; + +@Category({SmallTests.class}) +public class TestCacheStats { + private String period; + + @Test + public void testRollAndCleanRegionMetrics() throws Exception { + int periods = CacheStats.DEFAULT_WINDOW_PERIODS; + String regionName1 = "region1", regionName2 = "region2", regionName3 = "region3"; + Set onlineRegions = new HashSet<>(); + onlineRegions.add(regionName1); + onlineRegions.add(regionName2); + onlineRegions.add(regionName3); + + Configuration conf = HBaseConfiguration.create(); + /** + * FIXME: CacheStats should be cleaned in distributed mode because a block cache is a + * singleton. The below line can be removed when HBASE-14704 is resolved. + */ + conf.setBoolean(HConstants.CLUSTER_DISTRIBUTED, true); + CacheStats stats = new CacheStats(conf, this.getClass().getName(), periods); + period = String.valueOf(conf.getInt(CacheStats.CACHE_STATS_CHORE_PERIOD_KEY, + CacheStats.CACHE_STATS_CHORE_PERIOD_DEFAULT) / 1000); + + // update region cache stats except region3 + stats.setSize(regionName1, 1); + stats.incrementBlockCount(regionName1); + stats.setSize(regionName2, 2); + stats.incrementBlockCount(regionName2); + stats.incrementBlockCount(regionName2); + assertStats(stats, regionName1, 1, 1, 0, 0, 0, 0, Float.NaN, CacheStats.STAT_NOT_AVAILABLE); + assertStats(stats, regionName2, 2, 2, 0, 0, 0, 0, Float.NaN, CacheStats.STAT_NOT_AVAILABLE); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, CacheStats.STAT_NOT_AVAILABLE); + + // roll and clean stats + stats.rollAndCleanRegionMetrics(onlineRegions); + assertStats(stats, regionName1, 1, 1, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN")); + assertStats(stats, regionName2, 2, 2, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN")); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN")); + + // update region cache stats + stats.miss(regionName1, true, true); + assertStats(stats, regionName1, 1, 1, 0, 0, 1, 1, 0.0, hitRatioString("0.00%")); + assertStats(stats, regionName2, 2, 2, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN")); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN")); + stats.hit(regionName1, true, true); + assertStats(stats, regionName1, 1, 1, 0, 1, 1, 2, 0.5, hitRatioString("50.00%")); + assertStats(stats, regionName2, 2, 2, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN")); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN")); + stats.evicted(regionName1, System.currentTimeMillis(), true); + stats.setSize(regionName1, -1); + stats.decrementBlockCount(regionName1); + assertStats(stats, regionName1, 0, 0, 1, 1, 1, 2, 0.5, hitRatioString("50.00%")); + assertStats(stats, regionName2, 2, 2, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN")); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN")); + + // roll and clean stats + stats.rollAndCleanRegionMetrics(onlineRegions); + assertStats(stats, regionName1, 0, 0, 1, 1, 1, 2, 0.5, hitRatioString("50.00%, NaN")); + assertStats(stats, regionName2, 2, 2, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN, NaN")); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN, NaN")); + + // roll and clean stats + stats.rollAndCleanRegionMetrics(onlineRegions); + assertStats(stats, regionName1, 0, 0, 1, 1, 1, 2, 0.5, hitRatioString("50.00%, NaN, NaN")); + assertStats(stats, regionName2, 2, 2, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN, NaN, NaN")); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN, NaN, NaN")); + + // update region cache stats + stats.miss(regionName2, true, true); + assertStats(stats, regionName1, 0, 0, 1, 1, 1, 2, 0.5, hitRatioString("50.00%, NaN, NaN")); + assertStats(stats, regionName2, 2, 2, 0, 0, 1, 1, 0.0, hitRatioString("NaN, NaN, 0.00%")); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, hitRatioString("NaN, NaN, NaN")); + + // roll and clean stats + stats.rollAndCleanRegionMetrics(onlineRegions); + assertStats(stats, regionName1, 0, 0, 1, 1, 1, 2, 0.5, hitRatioString("50.00%, NaN, NaN, NaN")); + assertStats(stats, regionName2, 2, 2, 0, 0, 1, 1, 0.0, hitRatioString("NaN, NaN, 0.00%, NaN")); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, + hitRatioString("NaN, NaN, NaN, NaN")); + + // region2 is offlined + onlineRegions.remove(regionName2); + + // roll and clean stats + stats.rollAndCleanRegionMetrics(onlineRegions); + assertStats(stats, regionName1, 0, 0, 1, 1, 1, 2, 0.5, + hitRatioString("50.00%, NaN, NaN, NaN, NaN")); + assertStats(stats, regionName2, 0, 0, 0, 0, 0, 0, Float.NaN, "NaN"); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, + hitRatioString("NaN, NaN, NaN, NaN, NaN")); + + // roll and clean stats + stats.rollAndCleanRegionMetrics(onlineRegions); + assertStats(stats, regionName1, 0, 0, 1, 1, 1, 2, 0.5, + hitRatioString("50.00%, NaN, NaN, NaN, NaN, NaN")); + assertStats(stats, regionName2, 0, 0, 0, 0, 0, 0, Float.NaN, "NaN"); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, + hitRatioString("NaN, NaN, NaN, NaN, NaN, NaN")); + + // roll and clean stats. The first hit ratio should be removed. + stats.rollAndCleanRegionMetrics(onlineRegions); + assertStats(stats, regionName1, 0, 0, 1, 1, 1, 2, 0.5, + hitRatioString("NaN, NaN, NaN, NaN, NaN, NaN")); + assertStats(stats, regionName2, 0, 0, 0, 0, 0, 0, Float.NaN, "NaN"); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, + hitRatioString("NaN, NaN, NaN, NaN, NaN, NaN")); + + // update region cache stats + stats.miss(regionName1, true, true); + assertStats(stats, regionName1, 0, 0, 1, 1, 2, 3, (float)1/(float)3, + hitRatioString("NaN, NaN, NaN, NaN, NaN, 0.00%")); + assertStats(stats, regionName2, 0, 0, 0, 0, 0, 0, Float.NaN, "NaN"); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, + hitRatioString("NaN, NaN, NaN, NaN, NaN, NaN")); + + // roll and clean stats + stats.rollAndCleanRegionMetrics(onlineRegions); + assertStats(stats, regionName1, 0, 0, 1, 1, 2, 3, (float)1/(float)3, + hitRatioString("NaN, NaN, NaN, NaN, 0.00%, NaN")); + assertStats(stats, regionName2, 0, 0, 0, 0, 0, 0, Float.NaN, "NaN"); + assertStats(stats, regionName3, 0, 0, 0, 0, 0, 0, Float.NaN, + hitRatioString("NaN, NaN, NaN, NaN, NaN, NaN")); + } + + private String hitRatioString(String hitRatios) { + return "Every " + period + "sec: " + hitRatios; + } + + private void assertStats(CacheStats stats, String region, long size, long blocks, + long evicted, long hit, long miss, long request, double hitRatio, String hitRatioStr) { + assertEquals(size, stats.getSize(region)); + assertEquals(blocks, stats.getBlockCount(region)); + assertEquals(evicted, stats.getEvictedCount(region)); + assertEquals(hit, stats.getHitCount(region)); + assertEquals(miss, stats.getMissCount(region)); + assertEquals(request, stats.getRequestCount(region)); + assertEquals(hitRatio, stats.getHitRatio(region), 0.0); + assertEquals(hitRatioStr, stats.getLatestPeriodsHitRatioString(region)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java index 3cdc92b..6d4931b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java @@ -27,6 +27,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.HeapSize; @@ -51,6 +53,7 @@ public class TestHFileDataBlockEncoder { private HFileDataBlockEncoder blockEncoder; private RedundantKVGenerator generator = new RedundantKVGenerator(); private boolean includesMemstoreTS; + private final Configuration conf = HBaseConfiguration.create(); /** * Create test for given data block encoding configuration. @@ -80,7 +83,7 @@ public class TestHFileDataBlockEncoder { HFileBlock cacheBlock = createBlockOnDisk(kvs, block, useTag); LruBlockCache blockCache = - new LruBlockCache(8 * 1024 * 1024, 32 * 1024); + new LruBlockCache(conf, 8 * 1024 * 1024, 32 * 1024); BlockCacheKey cacheKey = new BlockCacheKey("test", 0); blockCache.cacheBlock(cacheKey, cacheBlock); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java index 9a548f5..c25a32b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java @@ -24,6 +24,8 @@ import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; import java.util.Random; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.io.HeapSize; @@ -41,6 +43,7 @@ import org.junit.experimental.categories.Category; */ @Category({IOTests.class, SmallTests.class}) public class TestLruBlockCache { + private final Configuration conf = HBaseConfiguration.create(); @Test @@ -50,7 +53,7 @@ public class TestLruBlockCache { long blockSize = calculateBlockSizeDefault(maxSize, numBlocks); assertTrue("calculateBlockSize appears broken.", blockSize * numBlocks <= maxSize); - LruBlockCache cache = new LruBlockCache(maxSize,blockSize); + LruBlockCache cache = new LruBlockCache(conf, maxSize,blockSize); EvictionThread evictionThread = cache.getEvictionThread(); assertTrue(evictionThread != null); @@ -99,7 +102,7 @@ public class TestLruBlockCache { long maxSize = 1000000; long blockSize = calculateBlockSizeDefault(maxSize, 101); - LruBlockCache cache = new LruBlockCache(maxSize, blockSize); + LruBlockCache cache = new LruBlockCache(conf, maxSize, blockSize); CachedItem [] blocks = generateRandomBlocks(100, blockSize); @@ -154,24 +157,45 @@ public class TestLruBlockCache { @Test public void testCacheEvictionSimple() throws Exception { + long regionCacheSizePrev, regionCacheSizeCur; + String region0 = "region0"; long maxSize = 100000; - long blockSize = calculateBlockSizeDefault(maxSize, 10); + int numBlocks = 11; + long blockSize = calculateBlockSizeDefault(maxSize, numBlocks - 1); - LruBlockCache cache = new LruBlockCache(maxSize,blockSize,false); + LruBlockCache cache = new LruBlockCache(conf, maxSize,blockSize,false); - CachedItem [] blocks = generateFixedBlocks(10, blockSize, "block"); + CachedItem [] blocks = generateFixedBlocks(numBlocks, blockSize, "block"); long expectedCacheSize = cache.heapSize(); - // Add all the blocks - for (CachedItem block : blocks) { + // Add all the blocks except the last two blocks + for (int i = 0; i < blocks.length - 2; i++) { + CachedItem block = blocks[i]; cache.cacheBlock(block.cacheKey, block); expectedCacheSize += block.cacheBlockHeapSize(); } + // No eviction yet + assertEquals(0, cache.getStats().getEvictionCount()); + assertEquals(0, cache.getStats().getEvictedCount(region0)); + assertEquals(2, cache.getStats().getBlockCount(region0)); + regionCacheSizeCur = cache.getStats().getSize(region0); + assertTrue(regionCacheSizeCur > 0); + + // Add the (numBlocks-1)th block + CachedItem block = blocks[numBlocks - 2]; + cache.cacheBlock(block.cacheKey, block); + expectedCacheSize += block.cacheBlockHeapSize(); + // A single eviction run should have occurred assertEquals(1, cache.getStats().getEvictionCount()); + assertEquals(1, cache.getStats().getEvictedCount(region0)); + assertEquals(1, cache.getStats().getBlockCount(region0)); + regionCacheSizePrev = regionCacheSizeCur; + regionCacheSizeCur = cache.getStats().getSize(region0); + assertTrue(regionCacheSizeCur < regionCacheSizePrev); // Our expected size overruns acceptable limit assertTrue(expectedCacheSize > @@ -186,10 +210,24 @@ public class TestLruBlockCache { // All blocks except block 0 should be in the cache assertTrue(cache.getBlock(blocks[0].cacheKey, true, false, true) == null); - for(int i=1;i regionCacheSizePrev); } @Test @@ -198,7 +236,7 @@ public class TestLruBlockCache { long maxSize = 100000; long blockSize = calculateBlockSizeDefault(maxSize, 10); - LruBlockCache cache = new LruBlockCache(maxSize,blockSize,false); + LruBlockCache cache = new LruBlockCache(conf, maxSize,blockSize,false); CachedItem [] singleBlocks = generateFixedBlocks(5, 10000, "single"); CachedItem [] multiBlocks = generateFixedBlocks(5, 10000, "multi"); @@ -257,7 +295,7 @@ public class TestLruBlockCache { long maxSize = 100000; long blockSize = calculateBlockSize(maxSize, 10); - LruBlockCache cache = new LruBlockCache(maxSize, blockSize, false, + LruBlockCache cache = new LruBlockCache(conf, maxSize, blockSize, false, (int)Math.ceil(1.2*maxSize/blockSize), LruBlockCache.DEFAULT_LOAD_FACTOR, LruBlockCache.DEFAULT_CONCURRENCY_LEVEL, @@ -377,7 +415,7 @@ public class TestLruBlockCache { long maxSize = 100000; long blockSize = calculateBlockSize(maxSize, 10); - LruBlockCache cache = new LruBlockCache(maxSize, blockSize, false, + LruBlockCache cache = new LruBlockCache(conf, maxSize, blockSize, false, (int)Math.ceil(1.2*maxSize/blockSize), LruBlockCache.DEFAULT_LOAD_FACTOR, LruBlockCache.DEFAULT_CONCURRENCY_LEVEL, @@ -482,7 +520,7 @@ public class TestLruBlockCache { long maxSize = 100000; long blockSize = calculateBlockSize(maxSize, 10); - LruBlockCache cache = new LruBlockCache(maxSize, blockSize, false, + LruBlockCache cache = new LruBlockCache(conf, maxSize, blockSize, false, (int)Math.ceil(1.2*maxSize/blockSize), LruBlockCache.DEFAULT_LOAD_FACTOR, LruBlockCache.DEFAULT_CONCURRENCY_LEVEL, @@ -546,7 +584,7 @@ public class TestLruBlockCache { long maxSize = 300000; long blockSize = calculateBlockSize(maxSize, 31); - LruBlockCache cache = new LruBlockCache(maxSize, blockSize, false, + LruBlockCache cache = new LruBlockCache(conf, maxSize, blockSize, false, (int)Math.ceil(1.2*maxSize/blockSize), LruBlockCache.DEFAULT_LOAD_FACTOR, LruBlockCache.DEFAULT_CONCURRENCY_LEVEL, @@ -608,7 +646,7 @@ public class TestLruBlockCache { double delta = 0.01; // 3 total periods - CacheStats stats = new CacheStats("test", 3); + CacheStats stats = new CacheStats(conf, "test", 3); // No accesses, should be 0 stats.rollMetricsPeriod(); @@ -617,48 +655,48 @@ public class TestLruBlockCache { // period 1, 1 hit caching, 1 hit non-caching, 2 miss non-caching // should be (2/4)=0.5 and (1/1)=1 - stats.hit(false); - stats.hit(true); - stats.miss(false, false); - stats.miss(false, false); + stats.hit(null, false); + stats.hit(null, true); + stats.miss(null, false, false); + stats.miss(null, false, false); stats.rollMetricsPeriod(); assertEquals(0.5, stats.getHitRatioPastNPeriods(), delta); assertEquals(1.0, stats.getHitCachingRatioPastNPeriods(), delta); // period 2, 1 miss caching, 3 miss non-caching // should be (2/8)=0.25 and (1/2)=0.5 - stats.miss(true, false); - stats.miss(false, false); - stats.miss(false, false); - stats.miss(false, false); + stats.miss(null, true, false); + stats.miss(null, false, false); + stats.miss(null, false, false); + stats.miss(null, false, false); stats.rollMetricsPeriod(); assertEquals(0.25, stats.getHitRatioPastNPeriods(), delta); assertEquals(0.5, stats.getHitCachingRatioPastNPeriods(), delta); // period 3, 2 hits of each type // should be (6/12)=0.5 and (3/4)=0.75 - stats.hit(false); - stats.hit(true); - stats.hit(false); - stats.hit(true); + stats.hit(null, false); + stats.hit(null, true); + stats.hit(null, false); + stats.hit(null, true); stats.rollMetricsPeriod(); assertEquals(0.5, stats.getHitRatioPastNPeriods(), delta); assertEquals(0.75, stats.getHitCachingRatioPastNPeriods(), delta); // period 4, evict period 1, two caching misses // should be (4/10)=0.4 and (2/5)=0.4 - stats.miss(true, false); - stats.miss(true, false); + stats.miss(null, true, false); + stats.miss(null, true, false); stats.rollMetricsPeriod(); assertEquals(0.4, stats.getHitRatioPastNPeriods(), delta); assertEquals(0.4, stats.getHitCachingRatioPastNPeriods(), delta); // period 5, evict period 2, 2 caching misses, 2 non-caching hit // should be (6/10)=0.6 and (2/6)=1/3 - stats.miss(true, false); - stats.miss(true, false); - stats.hit(false); - stats.hit(false); + stats.miss(null, true, false); + stats.miss(null, true, false); + stats.hit(null, false); + stats.hit(null, false); stats.rollMetricsPeriod(); assertEquals(0.6, stats.getHitRatioPastNPeriods(), delta); assertEquals((double)1/3, stats.getHitCachingRatioPastNPeriods(), delta); @@ -683,10 +721,10 @@ public class TestLruBlockCache { // period 9, one of each // should be (2/4)=0.5 and (1/2)=0.5 - stats.miss(true, false); - stats.miss(false, false); - stats.hit(true); - stats.hit(false); + stats.miss(null, true, false); + stats.miss(null, false, false); + stats.hit(null, true); + stats.hit(null, false); stats.rollMetricsPeriod(); assertEquals(0.5, stats.getHitRatioPastNPeriods(), delta); assertEquals(0.5, stats.getHitCachingRatioPastNPeriods(), delta); @@ -695,7 +733,7 @@ public class TestLruBlockCache { private CachedItem [] generateFixedBlocks(int numBlocks, int size, String pfx) { CachedItem [] blocks = new CachedItem[numBlocks]; for(int i=0;i data() { @@ -89,7 +92,7 @@ public class TestBucketCache { public MockedBucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes, int writerThreads, int writerQLen, String persistencePath) throws FileNotFoundException, IOException { - super(ioEngineName, capacity, blockSize, bucketSizes, writerThreads, writerQLen, + super(conf, ioEngineName, capacity, blockSize, bucketSizes, writerThreads, writerQLen, persistencePath); super.wait_when_cache = true; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java index 4d3f550..cb7b598 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java @@ -18,6 +18,8 @@ */ package org.apache.hadoop.hbase.io.hfile.bucket; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; @@ -49,6 +51,7 @@ public class TestBucketWriterThread { private BlockingQueue q; private Cacheable plainCacheable; private BlockCacheKey plainKey; + private static final Configuration conf = HBaseConfiguration.create(); /** A BucketCache that does not start its writer threads. */ private static class MockBucketCache extends BucketCache { @@ -56,7 +59,7 @@ public class TestBucketWriterThread { public MockBucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes, int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration) throws FileNotFoundException, IOException { - super(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen, + super(conf, ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen, persistencePath, ioErrorsTolerationDuration); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java index 6f35fc8..56f7915 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java @@ -61,7 +61,7 @@ public class TestHeapMemoryManager { conf.setFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_KEY, 0.02f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.75f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MIN_RANGE_KEY, 0.03f); - HeapMemoryManager manager = new HeapMemoryManager(new BlockCacheStub(0), + HeapMemoryManager manager = new HeapMemoryManager(new BlockCacheStub(conf, 0), new MemstoreFlusherStub(0), new RegionServerStub(conf), new RegionServerAccountingStub()); assertFalse(manager.isTunerOn()); } @@ -72,16 +72,16 @@ public class TestHeapMemoryManager { conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.02f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.75f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.03f); - HeapMemoryManager manager = new HeapMemoryManager(new BlockCacheStub(0), + HeapMemoryManager manager = new HeapMemoryManager(new BlockCacheStub(conf, 0), new MemstoreFlusherStub(0), new RegionServerStub(conf), new RegionServerAccountingStub()); assertFalse(manager.isTunerOn()); } @Test public void testWhenMemstoreAndBlockCacheMaxMinChecksFails() throws Exception { - BlockCacheStub blockCache = new BlockCacheStub(0); - MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub(0); Configuration conf = HBaseConfiguration.create(); + BlockCacheStub blockCache = new BlockCacheStub(conf, 0); + MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub(0); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.75f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MIN_RANGE_KEY, 0.06f); try { @@ -103,13 +103,13 @@ public class TestHeapMemoryManager { @Test public void testWhenClusterIsWriteHeavyWithEmptyMemstore() throws Exception { - BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4)); + Configuration conf = HBaseConfiguration.create(); + BlockCacheStub blockCache = new BlockCacheStub(conf, (long) (maxHeapSize * 0.4)); MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); RegionServerAccountingStub regionServerAccounting = new RegionServerAccountingStub(); // Empty block cache and memstore blockCache.setTestBlockSize(0); regionServerAccounting.setTestMemstoreSize(0); - Configuration conf = HBaseConfiguration.create(); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.75f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.10f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f); @@ -137,13 +137,13 @@ public class TestHeapMemoryManager { @Test public void testWhenClusterIsReadHeavyWithEmptyBlockCache() throws Exception { - BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4)); + Configuration conf = HBaseConfiguration.create(); + BlockCacheStub blockCache = new BlockCacheStub(conf, (long) (maxHeapSize * 0.4)); MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); RegionServerAccountingStub regionServerAccounting = new RegionServerAccountingStub(); // Empty block cache and memstore blockCache.setTestBlockSize(0); regionServerAccounting.setTestMemstoreSize(0); - Configuration conf = HBaseConfiguration.create(); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.75f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.10f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f); @@ -168,13 +168,13 @@ public class TestHeapMemoryManager { @Test public void testWhenClusterIsWriteHeavy() throws Exception { - BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4)); + Configuration conf = HBaseConfiguration.create(); + BlockCacheStub blockCache = new BlockCacheStub(conf, (long) (maxHeapSize * 0.4)); MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); RegionServerAccountingStub regionServerAccounting = new RegionServerAccountingStub(); // Empty block cache and but nearly filled memstore blockCache.setTestBlockSize(0); regionServerAccounting.setTestMemstoreSize((long) (maxHeapSize * 0.4 * 0.8)); - Configuration conf = HBaseConfiguration.create(); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.75f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.10f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f); @@ -213,13 +213,13 @@ public class TestHeapMemoryManager { @Test public void testWhenClusterIsReadHeavy() throws Exception { - BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4)); + Configuration conf = HBaseConfiguration.create(); + BlockCacheStub blockCache = new BlockCacheStub(conf, (long) (maxHeapSize * 0.4)); MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); RegionServerAccountingStub regionServerAccounting = new RegionServerAccountingStub(); // Empty memstore and but nearly filled block cache blockCache.setTestBlockSize((long) (maxHeapSize * 0.4 * 0.8)); regionServerAccounting.setTestMemstoreSize(0); - Configuration conf = HBaseConfiguration.create(); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.75f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.10f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f); @@ -254,14 +254,14 @@ public class TestHeapMemoryManager { @Test public void testWhenClusterIsHavingMoreWritesThanReads() throws Exception { - BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4)); + Configuration conf = HBaseConfiguration.create(); + BlockCacheStub blockCache = new BlockCacheStub(conf, (long) (maxHeapSize * 0.4)); MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); RegionServerAccountingStub regionServerAccounting = new RegionServerAccountingStub(); // Both memstore and block cache are nearly filled blockCache.setTestBlockSize(0); regionServerAccounting.setTestMemstoreSize((long) (maxHeapSize * 0.4 * 0.8)); blockCache.setTestBlockSize((long) (maxHeapSize * 0.4 * 0.8)); - Configuration conf = HBaseConfiguration.create(); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.75f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.10f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f); @@ -298,14 +298,14 @@ public class TestHeapMemoryManager { @Test public void testBlockedFlushesIncreaseMemstoreInSteadyState() throws Exception { - BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4)); + Configuration conf = HBaseConfiguration.create(); + BlockCacheStub blockCache = new BlockCacheStub(conf, (long) (maxHeapSize * 0.4)); MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); RegionServerAccountingStub regionServerAccounting = new RegionServerAccountingStub(); // Both memstore and block cache are nearly filled blockCache.setTestBlockSize(0); regionServerAccounting.setTestMemstoreSize((long) (maxHeapSize * 0.4 * 0.8)); blockCache.setTestBlockSize((long) (maxHeapSize * 0.4 * 0.8)); - Configuration conf = HBaseConfiguration.create(); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.75f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.10f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f); @@ -345,9 +345,9 @@ public class TestHeapMemoryManager { @Test public void testPluggingInHeapMemoryTuner() throws Exception { - BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4)); - MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); Configuration conf = HBaseConfiguration.create(); + BlockCacheStub blockCache = new BlockCacheStub(conf, (long) (maxHeapSize * 0.4)); + MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.78f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.05f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.75f); @@ -377,9 +377,9 @@ public class TestHeapMemoryManager { @Test public void testWhenSizeGivenByHeapTunerGoesOutsideRange() throws Exception { - BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4)); - MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); Configuration conf = HBaseConfiguration.create(); + BlockCacheStub blockCache = new BlockCacheStub(conf, (long) (maxHeapSize * 0.4)); + MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.7f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.1f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f); @@ -403,9 +403,9 @@ public class TestHeapMemoryManager { @Test public void testWhenCombinedHeapSizesFromTunerGoesOutSideMaxLimit() throws Exception { - BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4)); - MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); Configuration conf = HBaseConfiguration.create(); + BlockCacheStub blockCache = new BlockCacheStub(conf, (long) (maxHeapSize * 0.4)); + MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.4)); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.7f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.1f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f); @@ -429,10 +429,10 @@ public class TestHeapMemoryManager { @Test public void testWhenL2BlockCacheIsOnHeap() throws Exception { + Configuration conf = HBaseConfiguration.create(); HeapMemoryManager heapMemoryManager = null; - BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4)); + BlockCacheStub blockCache = new BlockCacheStub(conf, (long) (maxHeapSize * 0.4)); MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.3)); - Configuration conf = HBaseConfiguration.create(); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.7f); conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.1f); conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f); @@ -496,12 +496,13 @@ public class TestHeapMemoryManager { } private static class BlockCacheStub implements ResizableBlockCache { - CacheStats stats = new CacheStats("test"); + final CacheStats stats; long maxSize = 0; private long testBlockSize = 0; - public BlockCacheStub(long size){ + public BlockCacheStub(Configuration conf, long size){ this.maxSize = size; + this.stats = new CacheStats(conf, "test"); } @Override @@ -523,13 +524,14 @@ public class TestHeapMemoryManager { @Override public boolean evictBlock(BlockCacheKey cacheKey) { - stats.evicted(0, cacheKey != null ? cacheKey.isPrimary() : true); + stats.evicted(cacheKey == null ? null : cacheKey.getRegion(), 0, + cacheKey != null ? cacheKey.isPrimary() : true); return false; } @Override public int evictBlocksByHfileName(String hfileName) { - stats.evicted(0, true); // Just assuming only one block for file here. + stats.evicted(null, 0, true); // Just assuming only one block for file here. return 0; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionLoad.java new file mode 100644 index 0000000..afc51da --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionLoad.java @@ -0,0 +1,160 @@ +/** + * Copyright The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import com.google.protobuf.ByteString; +import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.List; + +import static org.junit.Assert.*; + +@Category({MediumTests.class}) +public class TestRegionLoad { + @Test + public void testAllMethods() throws Exception { + final String regionName = "TEST"; + + HBaseProtos.RegionSpecifier regionSpecifier = + HBaseProtos.RegionSpecifier.newBuilder() + .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME) + .setValue(ByteString.copyFromUtf8(regionName)).build(); + + ClusterStatusProtos.RegionLoad regionLoadPB = + ClusterStatusProtos.RegionLoad.newBuilder() + .setRegionSpecifier(regionSpecifier) + .setCacheBlockCount(1) + .setCacheEvictedBlockCount(2) + .setCacheHitCount(3) + .setCacheHitRatio(4) + .setCacheHitRatioLatestNPeriods("5") + .setCacheMissCount(6) + .setCacheSize(7) + .setCompleteSequenceId(8) + .setCurrentCompactedKVs(9) + .setDataLocality(10) + .setLastMajorCompactionTs(11) + .setMemstoreSizeMB(12) + .setReadRequestsCount(13) + .setRootIndexSizeKB(14) + .setStorefileIndexSizeMB(15) + .setStorefiles(16) + .setStorefileSizeMB(17) + .setStores(18) + .setStoreUncompressedSizeMB(19) + .setTotalCompactingKVs(20) + .setTotalStaticBloomSizeKB(21) + .setTotalStaticIndexSizeKB(22) + .setWriteRequestsCount(23) + .build(); + + RegionLoad regionLoad = new RegionLoad(regionLoadPB); + assertEquals(regionName, regionLoad.getNameAsString()); + assertArrayEquals(regionName.getBytes(), regionLoad.getName()); + assertEquals(1, regionLoad.getCacheBlockCount()); + assertEquals(2, regionLoad.getCacheEvictedBlockCount()); + assertEquals(3, regionLoad.getCacheHitCount()); + assertEquals(4, regionLoad.getCacheHitRatio(), 0.0); + assertEquals("5", regionLoad.getCacheHitRatioLatestNPeriods()); + assertEquals(6, regionLoad.getCacheMissCount()); + assertEquals(7, regionLoad.getCacheSize()); + assertEquals(8, regionLoad.getCompleteSequenceId()); + assertEquals(9, regionLoad.getCurrentCompactedKVs()); + assertEquals(10, regionLoad.getDataLocality(), 0.0); + assertEquals(11, regionLoad.getLastMajorCompactionTs()); + assertEquals(12, regionLoad.getMemStoreSizeMB()); + assertEquals(13, regionLoad.getReadRequestsCount()); + assertEquals(14, regionLoad.getRootIndexSizeKB()); + assertEquals(15, regionLoad.getStorefileIndexSizeMB()); + assertEquals(16, regionLoad.getStorefiles()); + assertEquals(17, regionLoad.getStorefileSizeMB()); + assertEquals(18, regionLoad.getStores()); + assertEquals(19, regionLoad.getStoreUncompressedSizeMB()); + assertEquals(20, regionLoad.getTotalCompactingKVs()); + assertEquals(21, regionLoad.getTotalStaticBloomSizeKB()); + assertEquals(22, regionLoad.getTotalStaticIndexSizeKB()); + assertEquals(23, regionLoad.getWriteRequestsCount()); + + // getRequestsCount() = getReadRequestsCount() + getWriteRequestsCount() + assertEquals(36, regionLoad.getRequestsCount()); + + assertEquals(0, regionLoad.getStoreCompleteSequenceId().size()); + } + + @Test + public void testRegionLoadFromHBaseAdmin() throws Exception { + int numSlaves = 1; + HBaseTestingUtility hbase = new HBaseTestingUtility(); + try { + hbase.startMiniCluster(numSlaves); + + HRegionServer regionServer = hbase.getHBaseCluster().getRegionServer(0); + + try (HBaseAdmin admin = hbase.getHBaseAdmin()) { + // create table + TableName tableName = TableName.valueOf("test"); + HTableDescriptor td = new HTableDescriptor(tableName); + HColumnDescriptor cd = new HColumnDescriptor("d"); + td.addFamily(cd); + admin.createTable(td); + + // find RegionInfo + List tableRegions = admin.getTableRegions(tableName); + assertEquals(1, tableRegions.size()); + HRegionInfo hRegionInfo = tableRegions.get(0); + + ClusterStatusProtos.RegionLoad regionLoad = + regionServer.createRegionLoad(hRegionInfo.getEncodedName()); + assertTrue(regionLoad.hasCacheBlockCount()); + assertTrue(regionLoad.hasCacheEvictedBlockCount()); + assertTrue(regionLoad.hasCacheHitCount()); + assertTrue(regionLoad.hasCacheHitRatio()); + assertTrue(regionLoad.hasCacheHitRatioLatestNPeriods()); + assertTrue(regionLoad.hasCacheMissCount()); + assertTrue(regionLoad.hasCacheSize()); + assertTrue(regionLoad.hasCompleteSequenceId()); + assertTrue(regionLoad.hasCurrentCompactedKVs()); + assertTrue(regionLoad.hasDataLocality()); + assertTrue(regionLoad.hasLastMajorCompactionTs()); + assertTrue(regionLoad.hasMemstoreSizeMB()); + assertTrue(regionLoad.hasReadRequestsCount()); + assertTrue(regionLoad.hasRegionSpecifier()); + assertTrue(regionLoad.hasRootIndexSizeKB()); + assertTrue(regionLoad.hasStorefileIndexSizeMB()); + assertTrue(regionLoad.hasStorefiles()); + assertTrue(regionLoad.hasStorefileSizeMB()); + assertTrue(regionLoad.hasStores()); + assertTrue(regionLoad.hasStoreUncompressedSizeMB()); + assertTrue(regionLoad.hasTotalCompactingKVs()); + assertTrue(regionLoad.hasTotalStaticBloomSizeKB()); + assertTrue(regionLoad.hasTotalStaticIndexSizeKB()); + assertTrue(regionLoad.hasWriteRequestsCount()); + } + } finally { + hbase.shutdownMiniCluster(); + } + } +}