HBASE-4089 blockCache contents report

This commit is contained in:
Michael Stack 2014-06-09 16:42:30 -07:00
parent 2087489662
commit 5851a06412
41 changed files with 1742 additions and 974 deletions

View File

@ -122,9 +122,9 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
<ul class="nav navbar-nav">
<li class="active"><a href="/">Home</a></li>
<li><a href="/tablesDetailed.jsp">Table Details</a></li>
<li><a href="/logs/">Local logs</a></li>
<li><a href="/logs/">Local Logs</a></li>
<li><a href="/logLevel">Log Level</a></li>
<li><a href="/dump">Debug dump</a></li>
<li><a href="/dump">Debug Dump</a></li>
<li><a href="/jmx">Metrics Dump</a></li>
<%if HBaseConfiguration.isShowConfInServlet()%>
<li><a href="/conf">HBase Configuration</a></li>

View File

@ -0,0 +1,426 @@
<%doc>
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.
Template for rendering Block Cache tabs in RegionServer Status page.
</%doc>
<%args>
CacheConfig cacheConfig;
Configuration config;
</%args>
<%java>
BlockCache bc = cacheConfig == null? null: cacheConfig.getBlockCache();
String bcUrl = null;
String bcName = null;
if (bc != null) {
bcUrl = "http://hbase.apache.org/devapidocs/" + bc.getClass().getName().replaceAll("\\.", "/") + ".html";
bcName = bc.getClass().getSimpleName();
}
BlockCache [] bcs = cacheConfig == null? null: cacheConfig.getBlockCache() == null? null: cacheConfig.getBlockCache().getBlockCaches();
</%java>
<%import>
java.util.Map;
org.apache.hadoop.hbase.io.hfile.BlockCacheUtil.CachedBlocksByFile;
org.apache.hadoop.hbase.io.hfile.BlockCacheUtil.AgeSnapshot;
org.apache.hadoop.hbase.io.hfile.CachedBlock;
org.apache.hadoop.conf.Configuration;
org.apache.hadoop.hbase.io.hfile.CacheConfig;
org.apache.hadoop.hbase.io.hfile.BlockCache;
org.apache.hadoop.hbase.io.hfile.bucket.BucketCacheStats;
org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator;
org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.Bucket;
org.apache.hadoop.hbase.io.hfile.slab.SlabCache;
org.apache.hadoop.hbase.io.hfile.slab.SingleSizeCache;
org.apache.hadoop.util.StringUtils;
</%import>
<div class="tabbable">
<ul class="nav nav-pills">
<li class="active"><a href="#tab_bc_baseInfo" data-toggle="tab">Base Info</a></li>
<li class=""><a href="#tab_bc_config" data-toggle="tab">Config</a></li>
<li class=""><a href="#tab_bc_stats" data-toggle="tab">Stats</a></li>
<li class=""><a href="#tab_bc_l1" data-toggle="tab">L1</a></li>
<li class=""><a href="#tab_bc_l2" data-toggle="tab">L2</a></li>
</ul>
<div class="tab-content" style="padding-bottom: 9px; border-bottom: 1px solid #ddd;">
<div class="tab-pane active" id="tab_bc_baseInfo">
<& bc_baseInfo; cacheConfig = cacheConfig; bcUrl = bcUrl; bcName = bcName; &>
</div>
<div class="tab-pane" id="tab_bc_config">
<& bc_config; cacheConfig = cacheConfig &>
</div>
<div class="tab-pane" id="tab_bc_stats">
<& bc_stats; cacheConfig = cacheConfig &>
</div>
<div class="tab-pane" id="tab_bc_l1">
<& bc_l; bc = bcs == null? bc: bcs[0]; name = "L1" &>
</div>
<div class="tab-pane" id="tab_bc_l2">
<& bc_l; bc = bcs == null? null: bcs.length <= 1? null: bcs[1]; name = "L2" &>
</div>
</div>
</div>
<%def bc_baseInfo>
<%args>
CacheConfig cacheConfig;
String bcUrl;
String bcName;
</%args>
<%java>
BlockCache bc = cacheConfig == null? null: cacheConfig.getBlockCache();
BlockCache [] bcs = bc == null? null: bc.getBlockCaches();
String bcl1Url = null;
String bcl1Name = null;
String bcl2Url = null;
String bcl2Name = null;
if (bcs != null) {
BlockCache bcl1 = bcs[0];
if (bcl1 != null) {
bcl1Url = "http://hbase.apache.org/devapidocs/" + bcl1.getClass().getName().replaceAll("\\.", "/") + ".html";
bcl1Name = bcl1.getClass().getSimpleName();
}
if (bcs.length == 2) {
BlockCache bcl2 = bcs[1];
bcl2Url = "http://hbase.apache.org/devapidocs/" + bcl2.getClass().getName().replaceAll("\\.", "/") + ".html";
bcl2Name = bcl2.getClass().getSimpleName();
}
}
</%java>
<table class="table table-striped">
<tr>
<th>Attribute</th>
<th>Value</th>
<th>Description</th>
</tr>
</tr>
<tr>
<td>Implementation</td>
<td><a href="<% bcUrl %>"><% bcName %></a></td>
<td>Block Cache implementing class</td>
</tr>
</table>
<p>See <a href="http://hbase.apache.org/book.html#block.cache">Block Cache</a> in the HBase Reference Guide for help.</p>
</%def>
<%def bc_config>
<%args>
CacheConfig cacheConfig;
</%args>
<%if cacheConfig == null %>
<p>CacheConfig is null</p>
<%else>
<table class="table table-striped">
<tr>
<th>Attribute</th>
<th>Value</th>
<th>Description</th>
</tr>
<tr>
<td>Cache DATA on Read</td>
<td><% cacheConfig.shouldCacheDataOnRead() %></td>
<td>True if DATA blocks are cached on read
(INDEX & BLOOM blocks are always cached)</td>
</tr>
<tr>
<td>Cache DATA on Write</td>
<td><% cacheConfig.shouldCacheDataOnWrite() %></td>
<td>True if DATA blocks are cached on write.</td>
</tr>
<tr>
<td>Cache INDEX on Write</td>
<td><% cacheConfig.shouldCacheIndexesOnWrite() %></td>
<td>True if INDEX blocks are cached on write</td>
</tr>
<tr>
<td>Cache BLOOM on Write</td>
<td><% cacheConfig.shouldCacheBloomsOnWrite() %></td>
<td>True if BLOOM blocks are cached on write</td>
</tr>
<tr>
<td>Evict blocks on Close</td>
<td><% cacheConfig.shouldEvictOnClose() %></td>
<td>True if blocks are evicted from cache when an HFile
reader is closed</td>
</tr>
<tr>
<td>Compress blocks</td>
<td><% cacheConfig.shouldCacheCompressed() %></td>
<td>True if blocks are compressed in cache</td>
</tr>
<tr>
<td>Prefetch on Open</td>
<td><% cacheConfig.shouldPrefetchOnOpen() %></td>
<td>True if blocks are prefetched into cache on open</td>
</tr>
</table>
</%if>
</%def>
<%def bc_stats>
<%args>
CacheConfig cacheConfig;
</%args>
<%if cacheConfig == null %>
<p>CacheConfig is null</p>
<%else>
<table class="table table-striped">
<tr>
<th>Attribute</th>
<th>Value</th>
<th>Description</th>
</tr>
<tr>
<td>Size</td>
<td><% StringUtils.humanReadableInt(cacheConfig.getBlockCache().size()) %></td>
<td>Total size of Block Cache (bytes)</td>
</tr>
<tr>
<td>Free</td>
<td><% StringUtils.humanReadableInt(cacheConfig.getBlockCache().getFreeSize()) %></td>
<td>Free space in Block Cache (bytes)</td>
</tr>
<tr>
<td>Count</td>
<td><% StringUtils.humanReadableInt(cacheConfig.getBlockCache().getBlockCount()) %></td>
<td>Number of blocks in Block Cache</td>
</tr>
<tr>
<td>Evicted</td>
<td><% StringUtils.humanReadableInt(cacheConfig.getBlockCache().getStats().getEvictedCount()) %></td>
<td>Number of blocks evicted</td>
</tr>
<tr>
<td>Evictions</td>
<td><% StringUtils.humanReadableInt(cacheConfig.getBlockCache().getStats().getEvictionCount()) %></td>
<td>Number of times an eviction occurred</td>
</tr>
<tr>
<td>Hits</td>
<td><% StringUtils.humanReadableInt(cacheConfig.getBlockCache().getStats().getHitCount()) %></td>
<td>Number requests that were cache hits</td>
</tr>
<tr>
<td>Hits Caching</td>
<td><% StringUtils.humanReadableInt(cacheConfig.getBlockCache().getStats().getHitCachingCount()) %></td>
<td>Cache hit block requests but only requests set to use Block Cache</td>
</tr>
<tr>
<td>Misses</td>
<td><% StringUtils.humanReadableInt(cacheConfig.getBlockCache().getStats().getMissCount()) %></td>
<td>Number of requests that were cache misses</td>
</tr>
<tr>
<td>Misses Caching</td>
<td><% StringUtils.humanReadableInt(cacheConfig.getBlockCache().getStats().getMissCount()) %></td>
<td>Block requests that were cache misses but only requests set to use Block Cache</td>
</tr>
<tr>
<td>Hit Ratio</td>
<td><% String.format("%,.2f", cacheConfig.getBlockCache().getStats().getHitRatio() * 100) %><% "%" %></td>
<td>Hit Count divided by total requests count</td>
</tr>
</table>
<p>If Block Cache is made up of more than one cache -- i.e. a L1 and a L2 -- then the above
are combined counts. Request count is sum of hits and misses.</p>
</%if>
</%def>
<%def bc_l>
<%args>
BlockCache bc;
String name;
</%args>
<%if bc == null %>
<p>No <% name %> deployed</p>
<%else>
<& block_cache; bc = bc; name = name; &>
</%if>
</%def>
<%def block_cache>
<%args>
BlockCache bc;
String name;
</%args>
<%java>
final long nanosPerSecond = 1000000000;
String bcUrl = "http://hbase.apache.org/devapidocs/" + bc.getClass().getName().replaceAll("\\.", "/") + ".html";
String bcName = bc.getClass().getSimpleName();
org.apache.hadoop.hbase.io.hfile.BlockCacheUtil.CachedBlocksByFile cbsbf =
org.apache.hadoop.hbase.io.hfile.BlockCacheUtil.getLoadedCachedBlocksByFile(config, bc);
AgeSnapshot snapshot = cbsbf.getAgeSnapshot();
boolean slabCache = bc.getClass().getSimpleName().equals("SlabCache");
Map<Integer, SingleSizeCache> sizer = null;
boolean bucketCache = bc.getClass().getSimpleName().equals("BucketCache");
BucketCacheStats bucketCacheStats = null;
BucketAllocator bucketAllocator = null;
Bucket [] buckets = null;
if (slabCache) {
sizer = ((SlabCache)bc).getSizer();
} else if (bucketCache) {
bucketCacheStats = (BucketCacheStats)bc.getStats();
bucketAllocator = ((BucketCache)bc).getAllocator();
buckets = bucketAllocator.getBuckets();
}
</%java>
<%if cbsbf.isFull() %>
<p><b>Too many blocks!</b> Listing out the first <% snapshot.getMax() %> only (hbase.ui.blockcache.by.file.max)</p>
</%if>
<table id="blocks_summary" class="table table-striped">
<tr>
<th>Attribute</th>
<th>Value</th>
<th>Description</th>
</tr>
<tr>
<td>Implementation</td>
<td><a href="<% bcUrl %>"><% bc.getClass().getSimpleName() %></a></td>
<td>Class implementing this Block Cache Level</td>
</tr>
<tr>
<td>Count</td>
<td><% StringUtils.humanReadableInt(cbsbf.getCount()) %></td>
<td>Count of Blocks</td>
</tr>
<%if !bucketCache %>
<tr>
<td>Count</td>
<td><% StringUtils.humanReadableInt(cbsbf.getDataCount()) %></td>
<td>Count of DATA Blocks</td>
</tr>
</%if>
<tr>
<td>Size</td>
<td><% StringUtils.humanReadableInt(cbsbf.getSize()) %></td>
<td>Size of Blocks</td>
</tr>
<%if !bucketCache %>
<tr>
<td>Size</td>
<td><% StringUtils.humanReadableInt(cbsbf.getDataSize()) %></td>
<td>Size of DATA Blocks</td>
</tr>
</%if>
<%doc>Can't do age of block in cache when slab cache</%doc>
<%if !slabCache %>
<tr>
<td>Mean</td>
<td><% String.format("%,d", (long)(snapshot.getMean()/nanosPerSecond)) %></td>
<td>Mean age of Blocks in cache (seconds)</td>
</tr>
<tr>
<td>StdDev</td>
<td><% String.format("%,d", (long)(snapshot.getStdDev()/nanosPerSecond)) %></td>
<td>Age standard deviation of Blocks in cache</td>
</tr>
<tr>
<td>Min</td>
<td><% String.format("%,d", (long)(snapshot.getMin()/nanosPerSecond)) %></td>
<td>Min age of Blocks in cache (seconds)</td>
</tr>
<tr>
<td>Max</td>
<td><% String.format("%,d", (long)(snapshot.getMax()/nanosPerSecond)) %></td>
<td>Max age of Blocks in cache (seconds)</td>
</tr>
<tr>
<td>95th Percentile</td>
<td><% String.format("%,d", (long)(snapshot.get95thPercentile()/nanosPerSecond)) %></td>
<td>95th percentile of age of Blocks in cache (seconds)</td>
</tr>
<tr>
<td>99th Percentile</td>
<td><% String.format("%,d", (long)(snapshot.get99thPercentile()/nanosPerSecond)) %></td>
<td>99th percentile of age of Blocks in cache (seconds)</td>
</tr>
</%if>
<%if bucketCache %>
<tr>
<td>Hits per Second</td>
<td><% bucketCacheStats.getIOHitsPerSecond() %></td>
<td>Block gets against this cache per second</td>
</tr>
<tr>
<td>Time per Hit</td>
<td><% bucketCacheStats.getIOTimePerHit() %></td>
<td>Time per cache hit</td>
</tr>
</%if>
</table>
<%doc>Call through to Block Cache Detail rendering template</%doc>
<p>View Block Cache <a href="?format=json&bcn=<% name %>">as JSON</a> | Block Cache <a href="?format=json&bcn=<% name %>&bcv=file">as JSON by file</a></p>
<%if bucketCache %>
<p>BucketCache does not discern between DATA and META blocks so we do not show DATA counts (If deploy is using CombinedBlockCache, BucketCache is only DATA blocks</p>
<h3>BucketCache Buckets</h3>
<table class="table table-striped">
<tr>
<th>Bucket Offset</th>
<th>Allocation Size</th>
<th>Free Count</th>
<th>Used Count</th>
</tr>
<%for Bucket bucket: buckets %>
<tr>
<td><% bucket.getBaseOffset() %></td>
<td><% bucket.getItemAllocationSize() %></td>
<td><% bucket.getFreeBytes() %></td>
<td><% bucket.getUsedBytes() %></td>
</tr>
</%for>
</table>
<%elseif slabCache %>
<p>SlabCache does not keep account of block ages so can not show stats on how long blocks have been cached.</p>
<h3>SlabCache Slabs</h3>
<table class="table table-striped">
<tr>
<th>Block Size</th>
<th>Size</th>
<th>Free Size</th>
<th>Count</th>
<th>Evicted</th>
<th>Evictions</th>
<th>Hits</th>
<th>Caching</th>
<th>Misses</th>
<th>Caching</th>
<th>Hit Ratio</th>
</tr>
<%for Map.Entry<Integer, SingleSizeCache> e: sizer.entrySet() %>
<tr>
<td><% StringUtils.humanReadableInt(e.getKey()) %></td>
<td><% StringUtils.humanReadableInt(e.getValue().size()) %></td>
<td><% StringUtils.humanReadableInt(e.getValue().getFreeSize()) %></td>
<td><% StringUtils.humanReadableInt(e.getValue().getBlockCount()) %></td>
<td><% StringUtils.humanReadableInt(e.getValue().getStats().getEvictedCount()) %></td>
<td><% StringUtils.humanReadableInt(e.getValue().getStats().getEvictionCount()) %></td>
<td><% StringUtils.humanReadableInt(e.getValue().getStats().getHitCount()) %></td>
<td><% StringUtils.humanReadableInt(e.getValue().getStats().getHitCachingCount()) %></td>
<td><% StringUtils.humanReadableInt(e.getValue().getStats().getMissCount()) %></td>
<td><% StringUtils.humanReadableInt(e.getValue().getStats().getMissCachingCount()) %></td>
<td><% String.format("%,.2f", e.getValue().getStats().getHitRatio() * 100) %><% "%" %></td>
</tr>
</%for>
</table>
</%if>
<%java>
cbsbf = null;
</%java>
</%def>

View File

@ -0,0 +1,68 @@
<%doc>
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.
This template is used to give views on an individual block cache as JSON.
</%doc>
<%args>
CacheConfig cacheConfig;
Configuration conf;
String bcn;
String bcv;
</%args>
<%import>
java.util.*;
org.apache.hadoop.conf.Configuration;
org.apache.hadoop.hbase.io.hfile.BlockCacheUtil.CachedBlocksByFile;
org.apache.hadoop.hbase.io.hfile.BlockCacheUtil;
org.apache.hadoop.hbase.io.hfile.CachedBlock;
org.apache.hadoop.hbase.io.hfile.CacheConfig;
org.apache.hadoop.hbase.io.hfile.BlockCache;
org.apache.hadoop.hbase.io.hfile.bucket.BucketCacheStats;
org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator;
org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.Bucket;
org.apache.hadoop.hbase.io.hfile.slab.SlabCache;
org.apache.hadoop.hbase.io.hfile.slab.SingleSizeCache;
org.apache.hadoop.util.StringUtils;
com.yammer.metrics.stats.Snapshot;
</%import>
<%java>
BlockCache bc = cacheConfig.getBlockCache();
BlockCache [] bcs = bc.getBlockCaches();
if (bcn.equals("L1")) {
bc = bcs == null || bcs.length == 0? bc: bcs[0];
} else {
if (bcs.length < 2) {
System.out.println("There is no L2 block cache");
return;
}
bc = bcs[1];
}
CachedBlocksByFile cbsbf = BlockCacheUtil.getLoadedCachedBlocksByFile(conf, bc);
</%java>
<%if bcv.equals("file") %><& bc_by_file; cbsbf = cbsbf; &><%else><% BlockCacheUtil.toJSON(bc) %></%if>
<%java>
cbsbf = null;
</%java>
<%def bc_by_file>
<%args>
CachedBlocksByFile cbsbf;
</%args>
[<%for Map.Entry<String, NavigableSet<CachedBlock>> e: cbsbf.getCachedBlockStatsByFile().entrySet() %><% BlockCacheUtil.toJSON(e.getKey(), e.getValue()) %></%for>]
</%def>

View File

@ -20,6 +20,8 @@ limitations under the License.
HRegionServer regionServer;
String filter = "general";
String format = "html";
String bcn = "";
String bcv = "";
</%args>
<%import>
java.util.*;
@ -31,7 +33,11 @@ org.apache.hadoop.hbase.protobuf.ProtobufUtil;
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
</%import>
<%if format.equals("json") %>
<%doc>If json AND bcn is NOT an empty string presume it a block cache view request.</%doc>
<%if format.equals("json") && bcn != null && bcn.length() > 0 %>
<& BlockCacheViewTmpl; conf = regionServer.getConfiguration(); cacheConfig = regionServer.getCacheConfig(); bcn = bcn; bcv = bcv; &>
<%java return; %>
<%elseif format.equals("json") %>
<& ../common/TaskMonitorTmpl; filter = filter; format = "json" &>
<%java return; %>
</%if>
@ -75,9 +81,9 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
<div class="collapse navbar-collapse">
<ul class="nav navbar-nav">
<li class="active"><a href="/">Home</a></li>
<li><a href="/logs/">Local logs</a></li>
<li><a href="/logs/">Local Logs</a></li>
<li><a href="/logLevel">Log Level</a></li>
<li><a href="/dump">Debug dump</a></li>
<li><a href="/dump">Debug Dump</a></li>
<li><a href="/jmx">Metrics Dump</a></li>
<%if HBaseConfiguration.isShowConfInServlet()%>
<li><a href="/conf">HBase Configuration</a></li>
@ -105,11 +111,17 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
<& ../common/TaskMonitorTmpl; filter = filter &>
</section>
<section>
<h2>Block Cache</h2>
<& BlockCacheTmpl; cacheConfig = regionServer.getCacheConfig(); config = regionServer.getConfiguration() &>
</section>
<section>
<h2>Regions</h2>
<& RegionListTmpl; regionServer = regionServer; onlineRegions = onlineRegions; &>
</section>
<section>
<h2>Software Attributes</h2>
<table id="attributes_table" class="table table-striped">

View File

@ -44,7 +44,6 @@ java.lang.management.ManagementFactory;
<li class=""><a href="#tab_hlogStats" data-toggle="tab">hlogs</a></li>
<li class=""><a href="#tab_storeStats" data-toggle="tab">Storefiles</a></li>
<li class=""><a href="#tab_queueStats" data-toggle="tab">Queues</a></li>
<li class=""><a href="#tab_blockCacheStats" data-toggle="tab">Block Cache</a></li>
</ul>
<div class="tab-content" style="padding-bottom: 9px; border-bottom: 1px solid #ddd;">
<div class="tab-pane active" id="tab_baseStats">
@ -65,9 +64,6 @@ java.lang.management.ManagementFactory;
<div class="tab-pane" id="tab_queueStats">
<& queueStats; mWrap = mWrap &>
</div>
<div class="tab-pane" id="tab_blockCacheStats">
<& blockCacheStats; mWrap = mWrap &>
</div>
</div>
</div>
@ -200,31 +196,3 @@ MetricsRegionServerWrapper mWrap;
</tr>
</table>
</%def>
<%def blockCacheStats>
<%args>
MetricsRegionServerWrapper mWrap;
</%args>
<table class="table table-striped">
<tr>
<th>Cache Size (bytes)</th>
<th>Cache Free (bytes)</th>
<th>Cache Count (blocks)</th>
<th>Cache Hit Count</th>
<th>Cache Miss Count</th>
<th>Cache Hit Ratio</th>
<th>Cache Eviction Count</th>
</tr>
<tr>
<td><% StringUtils.humanReadableInt(mWrap.getBlockCacheSize()) %></td>
<td><% StringUtils.humanReadableInt(mWrap.getBlockCacheFreeSize()) %></td>
<td><% StringUtils.humanReadableInt(mWrap.getBlockCacheCount()) %></td>
<td><% StringUtils.humanReadableInt(mWrap.getBlockCacheHitCount()) %></td>
<td><% StringUtils.humanReadableInt(mWrap.getBlockCacheMissCount()) %></td>
<td><% mWrap.getBlockCacheHitPercent() %>%</td>
<td><% StringUtils.humanReadableInt(mWrap.getBlockCacheEvictedCount()) %></td>
</tr>
</table>
</%def>

View File

@ -18,18 +18,16 @@
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.util.List;
import java.util.Iterator;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
/**
* Block cache interface. Anything that implements the {@link Cacheable}
* interface can be put in the cache.
*/
@InterfaceAudience.Private
public interface BlockCache {
public interface BlockCache extends Iterable<CachedBlock> {
/**
* Add block to cache.
* @param cacheKey The block's cache key.
@ -100,12 +98,6 @@ public interface BlockCache {
*/
long getCurrentSize();
/**
* Returns the number of blocks that have been evicted.
* @return number of evicted blocks
*/
long getEvictedCount();
/**
* Returns the number of blocks currently cached in the block cache.
* @return number of blocks in the cache
@ -113,16 +105,12 @@ public interface BlockCache {
long getBlockCount();
/**
* Performs a BlockCache summary and returns a List of BlockCacheColumnFamilySummary objects.
* This method could be fairly heavyweight in that it evaluates the entire HBase file-system
* against what is in the RegionServer BlockCache.
* <br><br>
* The contract of this interface is to return the List in sorted order by Table name, then
* ColumnFamily.
*
* @param conf HBaseConfiguration
* @return List of BlockCacheColumnFamilySummary
* @throws IOException exception
* @return Iterator over the blocks in the cache.
*/
List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(Configuration conf) throws IOException;
Iterator<CachedBlock> iterator();
/**
* @return The list of sub blockcaches that make up this one; returns null if no sub caches.
*/
BlockCache [] getBlockCaches();
}

View File

@ -1,247 +0,0 @@
/**
*
* 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 java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Writable;
/**
* BlockCacheColumnFamilySummary represents a summary of the blockCache usage
* at Table/ColumnFamily granularity.
* <br><br>
* As ColumnFamilies are owned by Tables, a summary by ColumnFamily implies that
* the owning Table is included in the summarization.
*
*/
@InterfaceAudience.Private
public class BlockCacheColumnFamilySummary implements Writable, Comparable<BlockCacheColumnFamilySummary> {
private String table = "";
private String columnFamily = "";
private int blocks;
private long heapSize;
/**
* Default constructor for Writable
*/
public BlockCacheColumnFamilySummary() {
}
/**
*
* @param table table
* @param columnFamily columnFamily
*/
public BlockCacheColumnFamilySummary(String table, String columnFamily) {
this.table = table;
this.columnFamily = columnFamily;
}
/**
*
* @return table
*/
public String getTable() {
return table;
}
/**
*
* @param table (table that owns the cached block)
*/
public void setTable(String table) {
this.table = table;
}
/**
*
* @return columnFamily
*/
public String getColumnFamily() {
return columnFamily;
}
/**
*
* @param columnFamily (columnFamily that owns the cached block)
*/
public void setColumnFamily(String columnFamily) {
this.columnFamily = columnFamily;
}
/**
*
* @return blocks in the cache
*/
public int getBlocks() {
return blocks;
}
/**
*
* @param blocks in the cache
*/
public void setBlocks(int blocks) {
this.blocks = blocks;
}
/**
*
* @return heapSize in the cache
*/
public long getHeapSize() {
return heapSize;
}
/**
* Increments the number of blocks in the cache for this entry
*/
public void incrementBlocks() {
this.blocks++;
}
/**
*
* @param heapSize to increment
*/
public void incrementHeapSize(long heapSize) {
this.heapSize = this.heapSize + heapSize;
}
/**
*
* @param heapSize (total heapSize for the table/CF)
*/
public void setHeapSize(long heapSize) {
this.heapSize = heapSize;
}
@Override
public void readFields(DataInput in) throws IOException {
table = in.readUTF();
columnFamily = in.readUTF();
blocks = in.readInt();
heapSize = in.readLong();
}
@Override
public void write(DataOutput out) throws IOException {
out.writeUTF(table);
out.writeUTF(columnFamily);
out.writeInt(blocks);
out.writeLong(heapSize);
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result
+ ((columnFamily == null) ? 0 : columnFamily.hashCode());
result = prime * result + ((table == null) ? 0 : table.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
BlockCacheColumnFamilySummary other = (BlockCacheColumnFamilySummary) obj;
if (columnFamily == null) {
if (other.columnFamily != null)
return false;
} else if (!columnFamily.equals(other.columnFamily))
return false;
if (table == null) {
if (other.table != null)
return false;
} else if (!table.equals(other.table))
return false;
return true;
}
@Override
public String toString() {
return "BlockCacheSummaryEntry [table=" + table + ", columnFamily="
+ columnFamily + ", blocks=" + blocks + ", heapSize=" + heapSize + "]";
}
/**
* Construct a BlockCacheSummaryEntry from a full StoreFile Path
* <br><br>
* The path is expected to be in the format of...
* <pre>
* hdfs://localhost:51169/user/userid/-ROOT-/70236052/info/3944417774205889744
* </pre>
* ... where: <br>
* '-ROOT-' = Table <br>
* '70236052' = Region <br>
* 'info' = ColumnFamily <br>
* '3944417774205889744' = StoreFile
*
* @param path (full StoreFile Path)
* @return BlockCacheSummaryEntry
*/
public static BlockCacheColumnFamilySummary createFromStoreFilePath(Path path) {
// The full path will look something like this...
// hdfs://localhost:51169/user/doug.meil/-ROOT-/70236052/info/3944417774205889744
// tbl region cf sf
String sp = path.toString();
String s[] = sp.split("\\/");
BlockCacheColumnFamilySummary bcse = null;
if (s.length >= 4) {
// why 4? StoreFile, CF, Region, Table
String table = s[s.length - 4]; // 4th from the end
String cf = s[s.length - 2]; // 2nd from the end
bcse = new BlockCacheColumnFamilySummary(table, cf);
}
return bcse;
}
@Override
public int compareTo(BlockCacheColumnFamilySummary o) {
int i = table.compareTo(o.getTable());
if (i != 0) {
return i;
}
return columnFamily.compareTo(o.getColumnFamily());
}
/**
* Creates a new BlockCacheSummaryEntry
*
* @param e BlockCacheSummaryEntry
* @return new BlockCacheSummaryEntry
*/
public static BlockCacheColumnFamilySummary create(BlockCacheColumnFamilySummary e) {
BlockCacheColumnFamilySummary e2 = new BlockCacheColumnFamilySummary();
e2.setTable(e.getTable());
e2.setColumnFamily(e.getColumnFamily());
return e2;
}
}

View File

@ -0,0 +1,337 @@
/**
* 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 java.io.IOException;
import java.util.NavigableMap;
import java.util.NavigableSet;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.ConcurrentSkipListSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.codehaus.jackson.JsonGenerationException;
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
import org.codehaus.jackson.map.JsonMappingException;
import org.codehaus.jackson.map.ObjectMapper;
import org.codehaus.jackson.map.SerializationConfig;
import com.yammer.metrics.core.Histogram;
import com.yammer.metrics.core.MetricsRegistry;
import com.yammer.metrics.stats.Snapshot;
/**
* Utilty for aggregating counts in CachedBlocks and toString/toJSON CachedBlocks and BlockCaches.
* No attempt has been made at making this thread safe.
*/
@InterfaceAudience.Private
public class BlockCacheUtil {
/**
* Needed making histograms.
*/
private static final MetricsRegistry METRICS = new MetricsRegistry();
/**
* Needed generating JSON.
*/
private static final ObjectMapper MAPPER = new ObjectMapper();
static {
MAPPER.configure(SerializationConfig.Feature.FAIL_ON_EMPTY_BEANS, false);
MAPPER.configure(SerializationConfig.Feature.FLUSH_AFTER_WRITE_VALUE, true);
MAPPER.configure(SerializationConfig.Feature.INDENT_OUTPUT, true);
}
/**
* @param cb
* @return The block content as String.
*/
public static String toString(final CachedBlock cb, final long now) {
return "filename=" + cb.getFilename() + ", " + toStringMinusFileName(cb, now);
}
/**
* Little data structure to hold counts for a file.
* Used doing a toJSON.
*/
static class CachedBlockCountsPerFile {
private int count = 0;
private long size = 0;
private int countData = 0;
private long sizeData = 0;
private final String filename;
CachedBlockCountsPerFile(final String filename) {
this.filename = filename;
}
public int getCount() {
return count;
}
public long getSize() {
return size;
}
public int getCountData() {
return countData;
}
public long getSizeData() {
return sizeData;
}
public String getFilename() {
return filename;
}
}
/**
* @param filename
* @param blocks
* @return A JSON String of <code>filename</code> and counts of <code>blocks</code>
* @throws JsonGenerationException
* @throws JsonMappingException
* @throws IOException
*/
public static String toJSON(final String filename, final NavigableSet<CachedBlock> blocks)
throws JsonGenerationException, JsonMappingException, IOException {
CachedBlockCountsPerFile counts = new CachedBlockCountsPerFile(filename);
for (CachedBlock cb: blocks) {
counts.count++;
counts.size += cb.getSize();
BlockType bt = cb.getBlockType();
if (bt != null && bt.isData()) {
counts.countData++;
counts.sizeData += cb.getSize();
}
}
return MAPPER.writeValueAsString(counts);
}
/**
* @param cbsbf
* @return JSON string of <code>cbsf</code> aggregated
* @throws JsonGenerationException
* @throws JsonMappingException
* @throws IOException
*/
public static String toJSON(final CachedBlocksByFile cbsbf)
throws JsonGenerationException, JsonMappingException, IOException {
return MAPPER.writeValueAsString(cbsbf);
}
/**
* @param bc
* @return JSON string of <code>bc</code> content.
* @throws JsonGenerationException
* @throws JsonMappingException
* @throws IOException
*/
public static String toJSON(final BlockCache bc)
throws JsonGenerationException, JsonMappingException, IOException {
return MAPPER.writeValueAsString(bc);
}
/**
* @param cb
* @return The block content of <code>bc</code> as a String minus the filename.
*/
public static String toStringMinusFileName(final CachedBlock cb, final long now) {
return "offset=" + cb.getOffset() +
", size=" + cb.getSize() +
", age=" + (now - cb.getCachedTime()) +
", type=" + cb.getBlockType() +
", priority=" + cb.getBlockPriority();
}
/**
* Snapshot of block cache age in cache.
* This object is preferred because we can control how it is serialized out when JSON'ing.
*/
@JsonIgnoreProperties({"ageHistogram", "snapshot"})
public static class AgeSnapshot {
private final Histogram ageHistogram;
private final Snapshot snapshot;
AgeSnapshot(final Histogram ageHistogram) {
this.ageHistogram = ageHistogram;
this.snapshot = ageHistogram.getSnapshot();
}
public double get75thPercentile() {
return snapshot.get75thPercentile();
}
public double get95thPercentile() {
return snapshot.get95thPercentile();
}
public double get98thPercentile() {
return snapshot.get98thPercentile();
}
public double get999thPercentile() {
return snapshot.get999thPercentile();
}
public double get99thPercentile() {
return snapshot.get99thPercentile();
}
public double getMean() {
return this.ageHistogram.mean();
}
public double getMax() {
return ageHistogram.max();
}
public double getMin() {
return ageHistogram.min();
}
public double getStdDev() {
return ageHistogram.stdDev();
}
}
/**
* Get a {@link CachedBlocksByFile} instance and load it up by iterating content in
* {@link BlockCache}.
* @param conf Used to read configurations
* @param bc Block Cache to iterate.
* @return Laoded up instance of CachedBlocksByFile
*/
public static CachedBlocksByFile getLoadedCachedBlocksByFile(final Configuration conf,
final BlockCache bc) {
CachedBlocksByFile cbsbf = new CachedBlocksByFile(conf);
for (CachedBlock cb: bc) {
if (cbsbf.update(cb)) break;
}
return cbsbf;
}
/**
* Use one of these to keep a running account of cached blocks by file. Throw it away when done.
* This is different than metrics in that it is stats on current state of a cache.
* @see getLoadedCachedBlocksByFile
*/
@JsonIgnoreProperties({"cachedBlockStatsByFile"})
public static class CachedBlocksByFile {
private int count;
private int dataBlockCount;
private long size;
private long dataSize;
private final long now = System.nanoTime();
private final int max;
public static final int DEFAULT_MAX = 100000;
CachedBlocksByFile() {
this(null);
}
CachedBlocksByFile(final Configuration c) {
this.max = c == null? DEFAULT_MAX:
c.getInt("hbase.ui.blockcache.by.file.max", DEFAULT_MAX);
}
/**
* Map by filename. use concurent utils because we want our Map and contained blocks sorted.
*/
private NavigableMap<String, NavigableSet<CachedBlock>> cachedBlockByFile =
new ConcurrentSkipListMap<String, NavigableSet<CachedBlock>>();
Histogram age = METRICS.newHistogram(CachedBlocksByFile.class, "age");
/**
* @param cb
* @return True if full.... if we won't be adding any more.
*/
public boolean update(final CachedBlock cb) {
if (isFull()) return true;
NavigableSet<CachedBlock> set = this.cachedBlockByFile.get(cb.getFilename());
if (set == null) {
set = new ConcurrentSkipListSet<CachedBlock>();
this.cachedBlockByFile.put(cb.getFilename(), set);
}
set.add(cb);
this.size += cb.getSize();
this.count++;
BlockType bt = cb.getBlockType();
if (bt != null && bt.isData()) {
this.dataBlockCount++;
this.dataSize += cb.getSize();
}
long age = this.now - cb.getCachedTime();
this.age.update(age);
return false;
}
/**
* @return True if full; i.e. there are more items in the cache but we only loaded up
* the maximum set in configuration <code>hbase.ui.blockcache.by.file.max</code>
* (Default: DEFAULT_MAX).
*/
public boolean isFull() {
return this.count >= this.max;
}
public NavigableMap<String, NavigableSet<CachedBlock>> getCachedBlockStatsByFile() {
return this.cachedBlockByFile;
}
/**
* @return count of blocks in the cache
*/
public int getCount() {
return count;
}
public int getDataCount() {
return dataBlockCount;
}
/**
* @return size of blocks in the cache
*/
public long getSize() {
return size;
}
/**
* @return Size of data.
*/
public long getDataSize() {
return dataSize;
}
public AgeSnapshot getAgeSnapshot() {
return new AgeSnapshot(this.age);
}
@Override
public String toString() {
Snapshot snapshot = this.age.getSnapshot();
return "count=" + count + ", dataBlockCount=" + this.dataBlockCount + ", size=" + size +
", dataSize=" + getDataSize() +
", mean age=" + this.age.mean() + ", stddev age=" + this.age.stdDev() +
", min age=" + this.age.min() + ", max age=" + this.age.max() +
", 95th percentile age=" + snapshot.get95thPercentile() +
", 99th percentile age=" + snapshot.get99thPercentile();
}
}
}

View File

@ -0,0 +1,58 @@
/**
* 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.io.hfile;
import java.util.Iterator;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* Iterator over an array of BlockCache CachedBlocks.
*/
@InterfaceAudience.Private
class BlockCachesIterator implements Iterator<CachedBlock> {
int index = 0;
final BlockCache [] bcs;
Iterator<CachedBlock> current;
BlockCachesIterator(final BlockCache [] blockCaches) {
this.bcs = blockCaches;
this.current = this.bcs[this.index].iterator();
}
@Override
public boolean hasNext() {
if (current.hasNext()) return true;
this.index++;
if (this.index >= this.bcs.length) return false;
this.current = this.bcs[this.index].iterator();
return hasNext();
}
@Override
public CachedBlock next() {
return this.current.next();
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
}

View File

@ -0,0 +1,33 @@
/*
* 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;
public enum BlockPriority {
/**
* Accessed a single time (used for scan-resistance)
*/
SINGLE,
/**
* Accessed multiple times
*/
MULTI,
/**
* Block from in-memory store
*/
MEMORY
}

View File

@ -458,6 +458,8 @@ public class CacheConfig {
long lruCacheSize = (long) (mu.getMax() * cachePercentage);
int blockSize = conf.getInt("hbase.offheapcache.minblocksize", HConstants.DEFAULT_BLOCKSIZE);
long slabCacheOffHeapCacheSize =
conf.getFloat(SLAB_CACHE_OFFHEAP_PERCENTAGE_KEY, 0) == 0?
0:
(long) (conf.getFloat(SLAB_CACHE_OFFHEAP_PERCENTAGE_KEY, (float) 0) *
DirectMemoryUtils.getDirectMemorySize());
if (slabCacheOffHeapCacheSize <= 0) {

View File

@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
*/
@InterfaceAudience.Private
public class CacheStats {
/** Sliding window statistics. The number of metric periods to include in
* sliding window hit ratio calculations.
*/
@ -35,6 +34,7 @@ public class CacheStats {
/** The number of getBlock requests that were cache hits */
private final AtomicLong hitCount = new AtomicLong(0);
/**
* The number of getBlock requests that were cache hits, but only from
* requests that were set to use the block cache. This is because all reads
@ -42,15 +42,19 @@ public class CacheStats {
* into the block cache. See HBASE-2253 for more information.
*/
private final AtomicLong hitCachingCount = new AtomicLong(0);
/** The number of getBlock requests that were cache misses */
private final AtomicLong missCount = new AtomicLong(0);
/**
* The number of getBlock requests that were cache misses, but only from
* requests that were set to use the block cache.
*/
private final AtomicLong missCachingCount = new AtomicLong(0);
/** The number of times an eviction has occurred */
private final AtomicLong evictionCount = new AtomicLong(0);
/** The total number of blocks that have been evicted */
private final AtomicLong evictedBlockCount = new AtomicLong(0);
@ -89,8 +93,10 @@ public class CacheStats {
@Override
public String toString() {
return "hitCount=" + this.hitCount + ", hitCachingCount=" + this.hitCachingCount +
", missCount=" + this.missCount;
return "hitCount=" + getHitCount() + ", hitCachingCount=" + getHitCachingCount() +
", missCount=" + getMissCount() + ", missCachingCount=" + getMissCachingCount() +
", evictionCount=" + getEvictionCount() +
", evictedBlockCount=" + getEvictedCount();
}
public void miss(boolean caching) {

View File

@ -1,5 +1,4 @@
/**
*
* 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
@ -19,110 +18,13 @@
package org.apache.hadoop.hbase.io.hfile;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
/**
* Represents an entry in the {@link LruBlockCache}.
*
* <p>Makes the block memory-aware with {@link HeapSize} and Comparable
* to sort by access time for the LRU. It also takes care of priority by
* either instantiating as in-memory or handling the transition from single
* to multiple access.
*/
@InterfaceAudience.Private
public class CachedBlock implements HeapSize, Comparable<CachedBlock> {
public final static long PER_BLOCK_OVERHEAD = ClassSize.align(
ClassSize.OBJECT + (3 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) +
ClassSize.STRING + ClassSize.BYTE_BUFFER);
static enum BlockPriority {
/**
* Accessed a single time (used for scan-resistance)
*/
SINGLE,
/**
* Accessed multiple times
*/
MULTI,
/**
* Block from in-memory store
*/
MEMORY
};
private final BlockCacheKey cacheKey;
private final Cacheable buf;
private volatile long accessTime;
private long size;
private BlockPriority priority;
public CachedBlock(BlockCacheKey cacheKey, Cacheable buf, long accessTime) {
this(cacheKey, buf, accessTime, false);
}
public CachedBlock(BlockCacheKey cacheKey, Cacheable buf, long accessTime,
boolean inMemory) {
this.cacheKey = cacheKey;
this.buf = buf;
this.accessTime = accessTime;
// We approximate the size of this class by the size of its name string
// plus the size of its byte buffer plus the overhead associated with all
// the base classes. We also include the base class
// sizes in the PER_BLOCK_OVERHEAD variable rather than align()ing them with
// their buffer lengths. This variable is used elsewhere in unit tests.
this.size = ClassSize.align(cacheKey.heapSize())
+ ClassSize.align(buf.heapSize()) + PER_BLOCK_OVERHEAD;
if(inMemory) {
this.priority = BlockPriority.MEMORY;
} else {
this.priority = BlockPriority.SINGLE;
}
}
/**
* Block has been accessed. Update its local access time.
*/
public void access(long accessTime) {
this.accessTime = accessTime;
if(this.priority == BlockPriority.SINGLE) {
this.priority = BlockPriority.MULTI;
}
}
public long heapSize() {
return size;
}
@Override
public int compareTo(CachedBlock that) {
if(this.accessTime == that.accessTime) return 0;
return this.accessTime < that.accessTime ? 1 : -1;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
CachedBlock other = (CachedBlock) obj;
return compareTo(other) == 0;
}
public Cacheable getBuffer() {
return this.buf;
}
public BlockCacheKey getCacheKey() {
return this.cacheKey;
}
public BlockPriority getPriority() {
return this.priority;
}
public interface CachedBlock extends Comparable<CachedBlock> {
BlockPriority getBlockPriority();
BlockType getBlockType();
long getOffset();
long getSize();
long getCachedTime();
String getFilename();
}

View File

@ -18,11 +18,9 @@
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.util.List;
import java.util.Iterator;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
@ -118,22 +116,11 @@ public class CombinedBlockCache implements BlockCache, HeapSize {
return lruCache.getCurrentSize() + bucketCache.getCurrentSize();
}
@Override
public long getEvictedCount() {
return lruCache.getEvictedCount() + bucketCache.getEvictedCount();
}
@Override
public long getBlockCount() {
return lruCache.getBlockCount() + bucketCache.getBlockCount();
}
@Override
public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
Configuration conf) throws IOException {
throw new UnsupportedOperationException();
}
private static class CombinedCacheStats extends CacheStats {
private final CacheStats lruCacheStats;
private final CacheStats bucketCacheStats;
@ -208,4 +195,14 @@ public class CombinedBlockCache implements BlockCache, HeapSize {
return Double.isNaN(ratio) ? 0 : ratio;
}
}
@Override
public Iterator<CachedBlock> iterator() {
return new BlockCachesIterator(getBlockCaches());
}
@Override
public BlockCache[] getBlockCaches() {
return new BlockCache [] {this.lruCache, this.bucketCache};
}
}

View File

@ -18,8 +18,7 @@
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.util.List;
import java.util.Iterator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -153,10 +152,6 @@ public class DoubleBlockCache implements ResizableBlockCache, HeapSize {
return onHeapCache.getCurrentSize() + offHeapCache.getCurrentSize();
}
public long getEvictedCount() {
return onHeapCache.getEvictedCount() + offHeapCache.getEvictedCount();
}
@Override
public int evictBlocksByHfileName(String hfileName) {
onHeapCache.evictBlocksByHfileName(hfileName);
@ -164,12 +159,6 @@ public class DoubleBlockCache implements ResizableBlockCache, HeapSize {
return 0;
}
@Override
public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
Configuration conf) throws IOException {
return onHeapCache.getBlockCacheColumnFamilySummaries(conf);
}
@Override
public long getBlockCount() {
return onHeapCache.getBlockCount() + offHeapCache.getBlockCount();
@ -179,4 +168,14 @@ public class DoubleBlockCache implements ResizableBlockCache, HeapSize {
public void setMaxSize(long size) {
this.onHeapCache.setMaxSize(size);
}
@Override
public Iterator<CachedBlock> iterator() {
return new BlockCachesIterator(getBlockCaches());
}
@Override
public BlockCache[] getBlockCaches() {
return new BlockCache [] {this.onHeapCache, this.offHeapCache};
}
}

View File

@ -18,13 +18,10 @@
*/
package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.lang.ref.WeakReference;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumMap;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
@ -41,18 +38,16 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.CachedBlock.BlockPriority;
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.util.StringUtils;
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
@ -94,6 +89,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
* to the relative sizes and usage.
*/
@InterfaceAudience.Private
@JsonIgnoreProperties({"encodingCountsForTest"})
public class LruBlockCache implements ResizableBlockCache, HeapSize {
static final Log LOG = LogFactory.getLog(LruBlockCache.class);
@ -132,7 +128,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
static final int statThreadPeriod = 60 * 5;
/** Concurrent map (the cache) */
private final Map<BlockCacheKey,CachedBlock> map;
private final Map<BlockCacheKey,LruCachedBlock> map;
/** Eviction lock (locked when eviction in process) */
private final ReentrantLock evictionLock = new ReentrantLock(true);
@ -272,7 +268,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
this.maxSize = maxSize;
this.blockSize = blockSize;
this.forceInMemory = forceInMemory;
map = new ConcurrentHashMap<BlockCacheKey,CachedBlock>(mapInitialSize,
map = new ConcurrentHashMap<BlockCacheKey,LruCachedBlock>(mapInitialSize,
mapLoadFactor, mapConcurrencyLevel);
this.minFactor = minFactor;
this.acceptableFactor = acceptableFactor;
@ -315,7 +311,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
*/
@Override
public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {
CachedBlock cb = map.get(cacheKey);
LruCachedBlock cb = map.get(cacheKey);
if(cb != null) {
// compare the contents, if they are not equal, we are in big trouble
if (compare(buf, cb.getBuffer()) != 0) {
@ -327,7 +323,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
LOG.warn(msg);
return;
}
cb = new CachedBlock(cacheKey, buf, count.incrementAndGet(), inMemory);
cb = new LruCachedBlock(cacheKey, buf, count.incrementAndGet(), inMemory);
long newSize = updateSizeMetrics(cb, false);
map.put(cacheKey, cb);
elements.incrementAndGet();
@ -358,12 +354,12 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
/**
* Helper function that updates the local size counter and also updates any
* per-cf or per-blocktype metrics it can discern from given
* {@link CachedBlock}
* {@link LruCachedBlock}
*
* @param cb
* @param evict
*/
protected long updateSizeMetrics(CachedBlock cb, boolean evict) {
protected long updateSizeMetrics(LruCachedBlock cb, boolean evict) {
long heapsize = cb.heapSize();
if (evict) {
heapsize *= -1;
@ -383,7 +379,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
@Override
public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat,
boolean updateCacheMetrics) {
CachedBlock cb = map.get(cacheKey);
LruCachedBlock cb = map.get(cacheKey);
if (cb == null) {
if (!repeat && updateCacheMetrics) stats.miss(caching);
if (victimHandler != null) {
@ -407,7 +403,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
@Override
public boolean evictBlock(BlockCacheKey cacheKey) {
CachedBlock cb = map.get(cacheKey);
LruCachedBlock cb = map.get(cacheKey);
if (cb == null) return false;
evictBlock(cb, false);
return true;
@ -446,7 +442,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
* EvictionThread
* @return the heap size of evicted block
*/
protected long evictBlock(CachedBlock block, boolean evictedByEvictionProcess) {
protected long evictBlock(LruCachedBlock block, boolean evictedByEvictionProcess) {
map.remove(block.getCacheKey());
updateSizeMetrics(block, true);
elements.decrementAndGet();
@ -501,7 +497,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
memorySize());
// Scan entire map putting into appropriate buckets
for(CachedBlock cachedBlock : map.values()) {
for(LruCachedBlock cachedBlock : map.values()) {
switch(cachedBlock.getPriority()) {
case SINGLE: {
bucketSingle.add(cachedBlock);
@ -597,23 +593,23 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
*/
private class BlockBucket implements Comparable<BlockBucket> {
private CachedBlockQueue queue;
private LruCachedBlockQueue queue;
private long totalSize = 0;
private long bucketSize;
public BlockBucket(long bytesToFree, long blockSize, long bucketSize) {
this.bucketSize = bucketSize;
queue = new CachedBlockQueue(bytesToFree, blockSize);
queue = new LruCachedBlockQueue(bytesToFree, blockSize);
totalSize = 0;
}
public void add(CachedBlock block) {
public void add(LruCachedBlock block) {
totalSize += block.heapSize();
queue.add(block);
}
public long free(long toFree) {
CachedBlock cb;
LruCachedBlock cb;
long freedBytes = 0;
while ((cb = queue.pollLast()) != null) {
freedBytes += evictBlock(cb, true);
@ -668,7 +664,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
@Override
public long size() {
return this.elements.get();
return getMaxSize();
}
@Override
@ -676,18 +672,6 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
return this.elements.get();
}
/**
* Get the number of eviction runs that have occurred
*/
public long getEvictionCount() {
return this.stats.getEvictionCount();
}
@Override
public long getEvictedCount() {
return this.stats.getEvictedCount();
}
EvictionThread getEvictionThread() {
return this.evictionThread;
}
@ -812,36 +796,68 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
}
@Override
public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(Configuration conf) throws IOException {
public Iterator<CachedBlock> iterator() {
final Iterator<LruCachedBlock> iterator = map.values().iterator();
Map<String, Path> sfMap = FSUtils.getTableStoreFilePathMap(
FileSystem.get(conf),
FSUtils.getRootDir(conf));
return new Iterator<CachedBlock>() {
private final long now = System.nanoTime();
// quirky, but it's a compound key and this is a shortcut taken instead of
// creating a class that would represent only a key.
Map<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary> bcs =
new HashMap<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary>();
for (CachedBlock cb : map.values()) {
String sf = cb.getCacheKey().getHfileName();
Path path = sfMap.get(sf);
if ( path != null) {
BlockCacheColumnFamilySummary lookup =
BlockCacheColumnFamilySummary.createFromStoreFilePath(path);
BlockCacheColumnFamilySummary bcse = bcs.get(lookup);
if (bcse == null) {
bcse = BlockCacheColumnFamilySummary.create(lookup);
bcs.put(lookup,bcse);
}
bcse.incrementBlocks();
bcse.incrementHeapSize(cb.heapSize());
@Override
public boolean hasNext() {
return iterator.hasNext();
}
}
List<BlockCacheColumnFamilySummary> list =
new ArrayList<BlockCacheColumnFamilySummary>(bcs.values());
Collections.sort( list );
return list;
@Override
public CachedBlock next() {
final LruCachedBlock b = iterator.next();
return new CachedBlock() {
@Override
public String toString() {
return BlockCacheUtil.toString(this, now);
}
@Override
public BlockPriority getBlockPriority() {
return b.getPriority();
}
@Override
public BlockType getBlockType() {
return b.getBuffer().getBlockType();
}
@Override
public long getOffset() {
return b.getCacheKey().getOffset();
}
@Override
public long getSize() {
return b.getBuffer().heapSize();
}
@Override
public long getCachedTime() {
return b.getCachedTime();
}
@Override
public String getFilename() {
return b.getCacheKey().getHfileName();
}
@Override
public int compareTo(CachedBlock other) {
return (int)(other.getOffset() - this.getOffset());
}
};
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
// Simple calculators of sizes given factors and maxSize
@ -902,10 +918,11 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
return fileNames;
}
@VisibleForTesting
Map<BlockType, Integer> getBlockTypeCountsForTest() {
Map<BlockType, Integer> counts =
new EnumMap<BlockType, Integer>(BlockType.class);
for (CachedBlock cb : map.values()) {
for (LruCachedBlock cb : map.values()) {
BlockType blockType = ((HFileBlock) cb.getBuffer()).getBlockType();
Integer count = counts.get(blockType);
counts.put(blockType, (count == null ? 0 : count) + 1);
@ -916,7 +933,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
public Map<DataBlockEncoding, Integer> getEncodingCountsForTest() {
Map<DataBlockEncoding, Integer> counts =
new EnumMap<DataBlockEncoding, Integer>(DataBlockEncoding.class);
for (CachedBlock block : map.values()) {
for (LruCachedBlock block : map.values()) {
DataBlockEncoding encoding =
((HFileBlock) block.getBuffer()).getDataBlockEncoding();
Integer count = counts.get(encoding);
@ -929,4 +946,9 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
assert victimHandler == null;
victimHandler = handler;
}
@Override
public BlockCache[] getBlockCaches() {
return null;
}
}

View File

@ -0,0 +1,127 @@
/**
*
* 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.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
/**
* Represents an entry in the {@link LruBlockCache}.
*
* <p>Makes the block memory-aware with {@link HeapSize} and Comparable
* to sort by access time for the LRU. It also takes care of priority by
* either instantiating as in-memory or handling the transition from single
* to multiple access.
*/
@InterfaceAudience.Private
public class LruCachedBlock implements HeapSize, Comparable<LruCachedBlock> {
public final static long PER_BLOCK_OVERHEAD = ClassSize.align(
ClassSize.OBJECT + (3 * ClassSize.REFERENCE) + (3 * Bytes.SIZEOF_LONG) +
ClassSize.STRING + ClassSize.BYTE_BUFFER);
private final BlockCacheKey cacheKey;
private final Cacheable buf;
private volatile long accessTime;
private long size;
private BlockPriority priority;
/**
* Time this block was cached. Presumes we are created just before we are added to the cache.
*/
private final long cachedTime = System.nanoTime();
public LruCachedBlock(BlockCacheKey cacheKey, Cacheable buf, long accessTime) {
this(cacheKey, buf, accessTime, false);
}
public LruCachedBlock(BlockCacheKey cacheKey, Cacheable buf, long accessTime,
boolean inMemory) {
this.cacheKey = cacheKey;
this.buf = buf;
this.accessTime = accessTime;
// We approximate the size of this class by the size of its name string
// plus the size of its byte buffer plus the overhead associated with all
// the base classes. We also include the base class
// sizes in the PER_BLOCK_OVERHEAD variable rather than align()ing them with
// their buffer lengths. This variable is used elsewhere in unit tests.
this.size = ClassSize.align(cacheKey.heapSize())
+ ClassSize.align(buf.heapSize()) + PER_BLOCK_OVERHEAD;
if(inMemory) {
this.priority = BlockPriority.MEMORY;
} else {
this.priority = BlockPriority.SINGLE;
}
}
/**
* Block has been accessed.
* @param accessTime Last access; this is actually a incremented sequence number rather than an
* actual time.
*/
public void access(long accessTime) {
this.accessTime = accessTime;
if(this.priority == BlockPriority.SINGLE) {
this.priority = BlockPriority.MULTI;
}
}
/**
* @return Time we were cached at in nano seconds.
*/
public long getCachedTime() {
return this.cachedTime;
}
public long heapSize() {
return size;
}
@Override
public int compareTo(LruCachedBlock that) {
// Newer accessed blocks sort before older ones.
if (this.accessTime == that.accessTime) return 0;
return this.accessTime < that.accessTime ? 1 : -1;
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
LruCachedBlock other = (LruCachedBlock) obj;
return compareTo(other) == 0;
}
public Cacheable getBuffer() {
return this.buf;
}
public BlockCacheKey getCacheKey() {
return this.cacheKey;
}
public BlockPriority getPriority() {
return this.priority;
}
}

View File

@ -37,9 +37,9 @@ import org.apache.hadoop.hbase.io.HeapSize;
* {@link Comparable}.
*/
@InterfaceAudience.Private
public class CachedBlockQueue implements HeapSize {
public class LruCachedBlockQueue implements HeapSize {
private MinMaxPriorityQueue<CachedBlock> queue;
private MinMaxPriorityQueue<LruCachedBlock> queue;
private long heapSize;
private long maxSize;
@ -48,7 +48,7 @@ public class CachedBlockQueue implements HeapSize {
* @param maxSize the target size of elements in the queue
* @param blockSize expected average size of blocks
*/
public CachedBlockQueue(long maxSize, long blockSize) {
public LruCachedBlockQueue(long maxSize, long blockSize) {
int initialSize = (int)(maxSize / blockSize);
if(initialSize == 0) initialSize++;
queue = MinMaxPriorityQueue.expectedSize(initialSize).create();
@ -64,12 +64,12 @@ public class CachedBlockQueue implements HeapSize {
* added to the queue. Otherwise, there is no side effect of this call.
* @param cb block to try to add to the queue
*/
public void add(CachedBlock cb) {
public void add(LruCachedBlock cb) {
if(heapSize < maxSize) {
queue.add(cb);
heapSize += cb.heapSize();
} else {
CachedBlock head = queue.peek();
LruCachedBlock head = queue.peek();
if(cb.compareTo(head) > 0) {
heapSize += cb.heapSize();
heapSize -= head.heapSize();
@ -87,7 +87,7 @@ public class CachedBlockQueue implements HeapSize {
* @return The next element in this queue, or {@code null} if the queue is
* empty.
*/
public CachedBlock poll() {
public LruCachedBlock poll() {
return queue.poll();
}
@ -95,7 +95,7 @@ public class CachedBlockQueue implements HeapSize {
* @return The last element in this queue, or {@code null} if the queue is
* empty.
*/
public CachedBlock pollLast() {
public LruCachedBlock pollLast() {
return queue.pollLast();
}

View File

@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.BucketEntry;
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
/**
* This class is used to allocate a block with specified size and free the block
@ -42,10 +43,12 @@ import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.BucketEntry;
* This class is not thread safe.
*/
@InterfaceAudience.Private
@JsonIgnoreProperties({"indexStatistics", "freeSize", "usedSize"})
public final class BucketAllocator {
static final Log LOG = LogFactory.getLog(BucketAllocator.class);
final private static class Bucket {
@JsonIgnoreProperties({"completelyFree", "uninstantiated"})
public final static class Bucket {
private long baseOffset;
private int itemAllocationSize, sizeIndex;
private int itemCount;
@ -77,7 +80,7 @@ public final class BucketAllocator {
return sizeIndex;
}
public int itemAllocationSize() {
public int getItemAllocationSize() {
return itemAllocationSize;
}
@ -97,15 +100,15 @@ public final class BucketAllocator {
return usedCount;
}
public int freeBytes() {
public int getFreeBytes() {
return freeCount * itemAllocationSize;
}
public int usedBytes() {
public int getUsedBytes() {
return usedCount * itemAllocationSize;
}
public long baseOffset() {
public long getBaseOffset() {
return baseOffset;
}
@ -372,19 +375,18 @@ public final class BucketAllocator {
}
realCacheSize.addAndGet(foundLen);
buckets[bucketNo].addAllocation(foundOffset);
usedSize += buckets[bucketNo].itemAllocationSize();
usedSize += buckets[bucketNo].getItemAllocationSize();
bucketSizeInfos[bucketSizeIndex].blockAllocated(b);
}
}
public String getInfo() {
public String toString() {
StringBuilder sb = new StringBuilder(1024);
for (int i = 0; i < buckets.length; ++i) {
Bucket b = buckets[i];
sb.append(" Bucket ").append(i).append(": ").append(b.itemAllocationSize());
sb.append(" freeCount=").append(b.freeCount()).append(" used=")
.append(b.usedCount());
sb.append('\n');
if (i > 0) sb.append(", ");
sb.append("bucket.").append(i).append(": size=").append(b.getItemAllocationSize());
sb.append(", freeCount=").append(b.freeCount()).append(", used=").append(b.usedCount());
}
return sb.toString();
}
@ -441,8 +443,8 @@ public final class BucketAllocator {
assert bucketNo >= 0 && bucketNo < buckets.length;
Bucket targetBucket = buckets[bucketNo];
bucketSizeInfos[targetBucket.sizeIndex()].freeBlock(targetBucket, offset);
usedSize -= targetBucket.itemAllocationSize();
return targetBucket.itemAllocationSize();
usedSize -= targetBucket.getItemAllocationSize();
return targetBucket.getItemAllocationSize();
}
public int sizeIndexOfAllocation(long offset) {
@ -456,7 +458,7 @@ public final class BucketAllocator {
int bucketNo = (int) (offset / bucketCapacity);
assert bucketNo >= 0 && bucketNo < buckets.length;
Bucket targetBucket = buckets[bucketNo];
return targetBucket.itemAllocationSize();
return targetBucket.getItemAllocationSize();
}
static class IndexStatistics {
@ -506,6 +508,10 @@ public final class BucketAllocator {
}
}
public Bucket [] getBuckets() {
return this.buckets;
}
public void dumpToLog() {
logStatistics();
StringBuilder sb = new StringBuilder();

View File

@ -31,6 +31,7 @@ import java.io.Serializable;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
@ -48,15 +49,17 @@ import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.BlockPriority;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.CacheStats;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializerIdManager;
import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.io.hfile.BlockCacheUtil;
import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache;
import org.apache.hadoop.hbase.io.hfile.HFileBlock;
import org.apache.hadoop.hbase.util.ConcurrentIndex;
@ -255,7 +258,7 @@ public class BucketCache implements BlockCache, HeapSize {
", capacity=" + StringUtils.byteDesc(capacity) +
", blockSize=" + StringUtils.byteDesc(blockSize) + ", writerThreadNum=" +
writerThreadNum + ", writerQLen=" + writerQLen + ", persistencePath=" +
persistencePath);
persistencePath + ", bucketAllocator=" + this.bucketAllocator);
}
/**
@ -376,8 +379,9 @@ public class BucketCache implements BlockCache, HeapSize {
if (lenRead != len) {
throw new RuntimeException("Only " + lenRead + " bytes read, " + len + " expected");
}
Cacheable cachedBlock = bucketEntry.deserializerReference(
deserialiserMap).deserialize(bb, true);
CacheableDeserializer<Cacheable> deserializer =
bucketEntry.deserializerReference(this.deserialiserMap);
Cacheable cachedBlock = deserializer.deserialize(bb, true);
long timeTaken = System.nanoTime() - start;
if (updateCacheMetrics) {
cacheStats.hit(caching);
@ -897,7 +901,7 @@ public class BucketCache implements BlockCache, HeapSize {
return cacheStats;
}
BucketAllocator getAllocator() {
public BucketAllocator getAllocator() {
return this.bucketAllocator;
}
@ -926,11 +930,6 @@ public class BucketCache implements BlockCache, HeapSize {
return this.bucketAllocator.getUsedSize();
}
@Override
public long getEvictedCount() {
return cacheStats.getEvictedCount();
}
/**
* Evicts all blocks for a specific HFile.
* <p>
@ -957,28 +956,6 @@ public class BucketCache implements BlockCache, HeapSize {
return numEvicted;
}
@Override
public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
Configuration conf) {
throw new UnsupportedOperationException();
}
static enum BlockPriority {
/**
* Accessed a single time (used for scan-resistance)
*/
SINGLE,
/**
* Accessed multiple times
*/
MULTI,
/**
* Block from in-memory store
*/
MEMORY
};
/**
* Item in cache. We expect this to be where most memory goes. Java uses 8
* bytes just for object headers; after this, we want to use as little as
@ -995,6 +972,10 @@ public class BucketCache implements BlockCache, HeapSize {
byte deserialiserIndex;
private volatile long accessTime;
private BlockPriority priority;
/**
* Time this block was cached. Presumes we are created just before we are added to the cache.
*/
private final long cachedTime = System.nanoTime();
BucketEntry(long offset, int length, long accessTime, boolean inMemory) {
setOffset(offset);
@ -1061,6 +1042,10 @@ public class BucketCache implements BlockCache, HeapSize {
public boolean equals(Object that) {
return this == that;
}
public long getCachedTime() {
return cachedTime;
}
}
/**
@ -1198,4 +1183,76 @@ public class BucketCache implements BlockCache, HeapSize {
writerThread.join();
}
}
@Override
public Iterator<CachedBlock> iterator() {
// Don't bother with ramcache since stuff is in here only a little while.
final Iterator<Map.Entry<BlockCacheKey, BucketEntry>> i =
this.backingMap.entrySet().iterator();
return new Iterator<CachedBlock>() {
private final long now = System.nanoTime();
@Override
public boolean hasNext() {
return i.hasNext();
}
@Override
public CachedBlock next() {
final Map.Entry<BlockCacheKey, BucketEntry> e = i.next();
return new CachedBlock() {
@Override
public String toString() {
return BlockCacheUtil.toString(this, now);
}
@Override
public BlockPriority getBlockPriority() {
return e.getValue().getPriority();
}
@Override
public BlockType getBlockType() {
// Not held by BucketEntry. Could add it if wanted on BucketEntry creation.
return null;
}
@Override
public long getOffset() {
return e.getKey().getOffset();
}
@Override
public long getSize() {
return e.getValue().getLength();
}
@Override
public long getCachedTime() {
return e.getValue().getCachedTime();
}
@Override
public String getFilename() {
return e.getKey().getHfileName();
}
@Override
public int compareTo(CachedBlock other) {
return (int)(this.getOffset() - other.getOffset());
}
};
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
@Override
public BlockCache[] getBlockCaches() {
return null;
}
}

View File

@ -34,6 +34,12 @@ public class BucketCacheStats extends CacheStats {
private final static int nanoTime = 1000000;
private long lastLogTime = EnvironmentEdgeManager.currentTimeMillis();
@Override
public String toString() {
return super.toString() + ", ioHitsPerSecond=" + getIOHitsPerSecond() +
", ioTimePerHit=" + getIOTimePerHit();
}
public void ioHit(long time) {
ioHitCount.incrementAndGet();
ioHitTime.addAndGet(time);
@ -43,7 +49,7 @@ public class BucketCacheStats extends CacheStats {
long now = EnvironmentEdgeManager.currentTimeMillis();
long took = (now - lastLogTime) / 1000;
lastLogTime = now;
return ioHitCount.get() / took;
return took == 0? 0: ioHitCount.get() / took;
}
public double getIOTimePerHit() {

View File

@ -19,21 +19,20 @@
package org.apache.hadoop.hbase.io.hfile.slab;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Iterator;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.CacheStats;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.util.StringUtils;
@ -317,16 +316,6 @@ public class SingleSizeCache implements BlockCache, HeapSize {
return 0;
}
/*
* Not implemented. Extremely costly to do this from the off heap cache, you'd
* need to copy every object on heap once
*/
@Override
public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
Configuration conf) {
throw new UnsupportedOperationException();
}
/* Just a pair class, holds a reference to the parent cacheable */
private static class CacheablePair implements HeapSize {
final CacheableDeserializer<Cacheable> deserializer;
@ -351,4 +340,14 @@ public class SingleSizeCache implements BlockCache, HeapSize {
+ ClassSize.ATOMIC_LONG);
}
}
@Override
public Iterator<CachedBlock> iterator() {
return null;
}
@Override
public BlockCache[] getBlockCaches() {
return null;
}
}

View File

@ -20,7 +20,8 @@
package org.apache.hadoop.hbase.io.hfile.slab;
import java.math.BigDecimal;
import java.util.List;
import java.util.Iterator;
import java.util.Map;
import java.util.Map.Entry;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
@ -35,10 +36,13 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.BlockPriority;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.CacheStats;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.io.hfile.BlockCacheUtil;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.util.StringUtils;
@ -57,7 +61,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
@InterfaceAudience.Private
public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
private final ConcurrentHashMap<BlockCacheKey, SingleSizeCache> backingStore;
private final TreeMap<Integer, SingleSizeCache> sizer;
private final TreeMap<Integer, SingleSizeCache> slabs;
static final Log LOG = LogFactory.getLog(SlabCache.class);
static final int STAT_THREAD_PERIOD_SECS = 60 * 5;
@ -100,10 +104,13 @@ public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
this.successfullyCachedStats = new SlabStats();
backingStore = new ConcurrentHashMap<BlockCacheKey, SingleSizeCache>();
sizer = new TreeMap<Integer, SingleSizeCache>();
slabs = new TreeMap<Integer, SingleSizeCache>();
this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
STAT_THREAD_PERIOD_SECS, STAT_THREAD_PERIOD_SECS, TimeUnit.SECONDS);
}
public Map<Integer, SingleSizeCache> getSizer() {
return slabs;
}
/**
@ -179,7 +186,7 @@ public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
* object is too large, returns null.
*/
Entry<Integer, SingleSizeCache> getHigherBlock(int size) {
return sizer.higherEntry(size - 1);
return slabs.higherEntry(size - 1);
}
private BigDecimal[] stringArrayToBigDecimalArray(String[] parsee) {
@ -193,7 +200,7 @@ public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
private void addSlab(int blockSize, int numBlocks) {
LOG.info("Creating slab of blockSize " + blockSize + " with " + numBlocks
+ " blocks, " + StringUtils.byteDesc(blockSize * (long) numBlocks) + "bytes.");
sizer.put(blockSize, new SingleSizeCache(blockSize, numBlocks, this));
slabs.put(blockSize, new SingleSizeCache(blockSize, numBlocks, this));
}
/**
@ -294,7 +301,7 @@ public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
* Also terminates the scheduleThreadPool.
*/
public void shutdown() {
for (SingleSizeCache s : sizer.values()) {
for (SingleSizeCache s : slabs.values()) {
s.shutdown();
}
this.scheduleThreadPool.shutdown();
@ -302,7 +309,7 @@ public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
public long heapSize() {
long childCacheSize = 0;
for (SingleSizeCache s : sizer.values()) {
for (SingleSizeCache s : slabs.values()) {
childCacheSize += s.heapSize();
}
return SlabCache.CACHE_FIXED_OVERHEAD + childCacheSize;
@ -314,7 +321,7 @@ public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
public long getFreeSize() {
long childFreeSize = 0;
for (SingleSizeCache s : sizer.values()) {
for (SingleSizeCache s : slabs.values()) {
childFreeSize += s.getFreeSize();
}
return childFreeSize;
@ -323,7 +330,7 @@ public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
@Override
public long getBlockCount() {
long count = 0;
for (SingleSizeCache cache : sizer.values()) {
for (SingleSizeCache cache : slabs.values()) {
count += cache.getBlockCount();
}
return count;
@ -352,7 +359,7 @@ public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
@Override
public void run() {
for (SingleSizeCache s : ourcache.sizer.values()) {
for (SingleSizeCache s : ourcache.slabs.values()) {
s.logStats();
}
@ -430,14 +437,75 @@ public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
return numEvicted;
}
/*
* Not implemented. Extremely costly to do this from the off heap cache, you'd
* need to copy every object on heap once
*/
@Override
public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
Configuration conf) {
throw new UnsupportedOperationException();
public Iterator<CachedBlock> iterator() {
// Don't bother with ramcache since stuff is in here only a little while.
final Iterator<Map.Entry<BlockCacheKey, SingleSizeCache>> i =
this.backingStore.entrySet().iterator();
return new Iterator<CachedBlock>() {
private final long now = System.nanoTime();
@Override
public boolean hasNext() {
return i.hasNext();
}
@Override
public CachedBlock next() {
final Map.Entry<BlockCacheKey, SingleSizeCache> e = i.next();
final Cacheable cacheable = e.getValue().getBlock(e.getKey(), false, false, false);
return new CachedBlock() {
@Override
public String toString() {
return BlockCacheUtil.toString(this, now);
}
@Override
public BlockPriority getBlockPriority() {
return null;
}
@Override
public BlockType getBlockType() {
return cacheable.getBlockType();
}
@Override
public long getOffset() {
return e.getKey().getOffset();
}
@Override
public long getSize() {
return cacheable == null? 0: cacheable.getSerializedLength();
}
@Override
public long getCachedTime() {
return -1;
}
@Override
public String getFilename() {
return e.getKey().getHfileName();
}
@Override
public int compareTo(CachedBlock other) {
return (int)(this.getOffset() - other.getOffset());
}
};
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
@Override
public BlockCache[] getBlockCaches() {
return null;
}
}

View File

@ -2800,4 +2800,11 @@ public class HRegionServer extends HasThread implements
}
return result;
}
/**
* @return The cache config instance used by the regionserver.
*/
public CacheConfig getCacheConfig() {
return this.cacheConfig;
}
}

View File

@ -53,6 +53,10 @@ public class RSStatusServlet extends HttpServlet {
tmpl.setFormat(req.getParameter("format"));
if (req.getParameter("filter") != null)
tmpl.setFilter(req.getParameter("filter"));
if (req.getParameter("bcn") != null)
tmpl.setBcn(req.getParameter("bcn"));
if (req.getParameter("bcv") != null)
tmpl.setBcv(req.getParameter("bcv"));
tmpl.render(resp.getWriter(), hrs);
}
}

View File

@ -92,9 +92,9 @@
<ul class="nav navbar-nav">
<li><a href="/master-status">Home</a></li>
<li><a href="/tablesDetailed.jsp">Table Details</a></li>
<li><a href="/logs/">Local logs</a></li>
<li><a href="/logs/">Local Logs</a></li>
<li><a href="/logLevel">Log Level</a></li>
<li><a href="/dump">Debug dump</a></li>
<li><a href="/dump">Debug Dump</a></li>
<li><a href="/jmx">Metrics Dump</a></li>
<% if (HBaseConfiguration.isShowConfInServlet()) { %>
<li><a href="/conf">HBase Configuration</a></li>

View File

@ -94,9 +94,9 @@
<ul class="nav navbar-nav">
<li><a href="/master-status">Home</a></li>
<li><a href="/tablesDetailed.jsp">Table Details</a></li>
<li><a href="/logs/">Local logs</a></li>
<li><a href="/logs/">Local Logs</a></li>
<li><a href="/logLevel">Log Level</a></li>
<li><a href="/dump">Debug dump</a></li>
<li><a href="/dump">Debug Dump</a></li>
<li><a href="/jmx">Metrics Dump</a></li>
<% if (HBaseConfiguration.isShowConfInServlet()) { %>
<li><a href="/conf">HBase Configuration</a></li>
@ -170,9 +170,9 @@
<ul class="nav navbar-nav">
<li><a href="/master-status">Home</a></li>
<li><a href="/tablesDetailed.jsp">Table Details</a></li>
<li><a href="/logs/">Local logs</a></li>
<li><a href="/logs/">Local Logs</a></li>
<li><a href="/logLevel">Log Level</a></li>
<li><a href="/dump">Debug dump</a></li>
<li><a href="/dump">Debug Dump</a></li>
<li><a href="/jmx">Metrics Dump</a></li>
</ul>
</div><!--/.nav-collapse -->

View File

@ -62,9 +62,9 @@
<ul class="nav navbar-nav">
<li class="active"><a href="/master-status">Home</a></li>
<li><a href="/tablesDetailed.jsp">Table Details</a></li>
<li><a href="/logs/">Local logs</a></li>
<li><a href="/logs/">Local Logs</a></li>
<li><a href="/logLevel">Log Level</a></li>
<li><a href="/dump">Debug dump</a></li>
<li><a href="/dump">Debug Dump</a></li>
<li><a href="/jmx">Metrics Dump</a></li>
<% if (HBaseConfiguration.isShowConfInServlet()) { %>
<li><a href="/conf">HBase Configuration</a></li>

View File

@ -59,9 +59,9 @@
<ul class="nav navbar-nav">
<li><a href="/master-status">Home</a></li>
<li><a href="/tablesDetailed.jsp">Table Details</a></li>
<li><a href="/logs/">Local logs</a></li>
<li><a href="/logs/">Local Logs</a></li>
<li><a href="/logLevel">Log Level</a></li>
<li><a href="/dump">Debug dump</a></li>
<li><a href="/dump">Debug Dump</a></li>
<li><a href="/jmx">Metrics Dump</a></li>
<% if (HBaseConfiguration.isShowConfInServlet()) { %>
<li><a href="/conf">HBase Configuration</a></li>

View File

@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.io.hfile.LruCachedBlock;
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
import org.apache.hadoop.hbase.regionserver.DefaultMemStore;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -279,8 +279,8 @@ public class TestHeapSize {
// CachedBlock Fixed Overhead
// We really need "deep" sizing but ClassSize does not do this.
// Perhaps we should do all these more in this style....
cl = CachedBlock.class;
actual = CachedBlock.PER_BLOCK_OVERHEAD;
cl = LruCachedBlock.class;
actual = LruCachedBlock.PER_BLOCK_OVERHEAD;
expected = ClassSize.estimateBase(cl, false);
expected += ClassSize.estimateBase(String.class, false);
expected += ClassSize.estimateBase(ByteBuffer.class, false);

View File

@ -1,124 +0,0 @@
/**
*
* 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.hbase.SmallTests;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests the BlockCacheColumnFamilySummary class
*
*/
@Category(SmallTests.class)
public class TestBlockCacheColumnFamilySummary {
/**
*
*/
@Test
public void testEquals() {
BlockCacheColumnFamilySummary e1 = new BlockCacheColumnFamilySummary();
e1.setTable("table1");
e1.setColumnFamily("cf1");
BlockCacheColumnFamilySummary e2 = new BlockCacheColumnFamilySummary();
e2.setTable("table1");
e2.setColumnFamily("cf1");
assertEquals("bcse", e1, e2);
}
/**
*
*/
@Test
public void testNotEquals() {
BlockCacheColumnFamilySummary e1 = new BlockCacheColumnFamilySummary();
e1.setTable("table1");
e1.setColumnFamily("cf1");
BlockCacheColumnFamilySummary e2 = new BlockCacheColumnFamilySummary();
e2.setTable("tablexxxxxx");
e2.setColumnFamily("cf1");
assertTrue("bcse", ! e1.equals(e2));
}
/**
*
*/
@Test
public void testMapLookup() {
Map<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary> bcs =
new HashMap<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary>();
BlockCacheColumnFamilySummary e1 = new BlockCacheColumnFamilySummary("table1","cf1");
BlockCacheColumnFamilySummary lookup = bcs.get(e1);
if (lookup == null) {
lookup = BlockCacheColumnFamilySummary.create(e1);
bcs.put(e1,lookup);
lookup.incrementBlocks();
lookup.incrementHeapSize(100L);
}
BlockCacheColumnFamilySummary e2 = new BlockCacheColumnFamilySummary("table1","cf1");
BlockCacheColumnFamilySummary l2 = bcs.get(e2);
assertEquals("blocks",1,l2.getBlocks());
assertEquals("heap",100L,l2.getHeapSize());
}
/**
*
*/
@Test
public void testMapEntry() {
Map<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary> bcs =
new HashMap<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary>();
BlockCacheColumnFamilySummary e1 = new BlockCacheColumnFamilySummary("table1","cf1");
bcs.put(e1, e1);
BlockCacheColumnFamilySummary e2 = new BlockCacheColumnFamilySummary("table1","cf1");
bcs.put(e2, e2);
BlockCacheColumnFamilySummary e3 = new BlockCacheColumnFamilySummary("table1","cf1");
bcs.put(e3, e3);
assertEquals("mapSize",1,bcs.size());
}
}

View File

@ -0,0 +1,171 @@
/**
* 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 static org.junit.Assert.*;
import java.io.IOException;
import java.util.Map;
import java.util.NavigableSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.DataCacheEntry;
import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.IndexCacheEntry;
import org.codehaus.jackson.JsonGenerationException;
import org.codehaus.jackson.map.JsonMappingException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(SmallTests.class)
public class TestBlockCacheReporting {
private static final Log LOG = LogFactory.getLog(TestBlockCacheReporting.class);
private Configuration conf;
@Before
public void setUp() throws Exception {
CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
this.conf = HBaseConfiguration.create();
}
@After
public void tearDown() throws Exception {
// Let go of current block cache.
CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
}
private void addDataAndHits(final BlockCache bc, final int count) {
Cacheable dce = new DataCacheEntry();
Cacheable ice = new IndexCacheEntry();
for (int i = 0; i < count; i++) {
BlockCacheKey bckd = new BlockCacheKey("f", i);
BlockCacheKey bcki = new BlockCacheKey("f", i + count);
bc.getBlock(bckd, true, false, true);
bc.cacheBlock(bckd, dce);
bc.cacheBlock(bcki, ice);
bc.getBlock(bckd, true, false, true);
bc.getBlock(bcki, true, false, true);
}
assertEquals(2 * count /*Data and Index blocks*/, bc.getStats().getHitCount());
}
@Test
public void testSlabCacheConfig() throws JsonGenerationException, JsonMappingException, IOException {
this.conf.setFloat(CacheConfig.SLAB_CACHE_OFFHEAP_PERCENTAGE_KEY, 0.1f);
CacheConfig cc = new CacheConfig(this.conf);
assertTrue(cc.getBlockCache() instanceof DoubleBlockCache);
logPerBlock(cc.getBlockCache());
final int count = 3;
addDataAndHits(cc.getBlockCache(), count);
// The below has no asserts. It is just exercising toString and toJSON code.
LOG.info(cc.getBlockCache().getStats());
BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(cc.getBlockCache());
LOG.info(cbsbf);
logPerFile(cbsbf);
bucketCacheReport(cc.getBlockCache());
LOG.info(BlockCacheUtil.toJSON(cbsbf));
}
@Test
public void testBucketCache() throws JsonGenerationException, JsonMappingException, IOException {
this.conf.set(CacheConfig.BUCKET_CACHE_IOENGINE_KEY, "offheap");
this.conf.setInt(CacheConfig.BUCKET_CACHE_SIZE_KEY, 100);
this.conf.setFloat(CacheConfig.BUCKET_CACHE_COMBINED_PERCENTAGE_KEY, 0.8f);
CacheConfig cc = new CacheConfig(this.conf);
assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
logPerBlock(cc.getBlockCache());
final int count = 3;
addDataAndHits(cc.getBlockCache(), count);
// The below has no asserts. It is just exercising toString and toJSON code.
LOG.info(cc.getBlockCache().getStats());
BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(cc.getBlockCache());
LOG.info(cbsbf);
logPerFile(cbsbf);
bucketCacheReport(cc.getBlockCache());
LOG.info(BlockCacheUtil.toJSON(cbsbf));
}
@Test
public void testLruBlockCache() throws JsonGenerationException, JsonMappingException, IOException {
CacheConfig cc = new CacheConfig(this.conf);
assertTrue(cc.isBlockCacheEnabled());
assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
assertTrue(cc.getBlockCache() instanceof LruBlockCache);
logPerBlock(cc.getBlockCache());
addDataAndHits(cc.getBlockCache(), 3);
// The below has no asserts. It is just exercising toString and toJSON code.
BlockCache bc = cc.getBlockCache();
LOG.info("count=" + bc.getBlockCount() + ", currentSize=" + bc.getCurrentSize() +
", freeSize=" + bc.getFreeSize() );
LOG.info(cc.getBlockCache().getStats());
BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(cc.getBlockCache());
LOG.info(cbsbf);
logPerFile(cbsbf);
bucketCacheReport(cc.getBlockCache());
LOG.info(BlockCacheUtil.toJSON(cbsbf));
}
private void bucketCacheReport(final BlockCache bc) {
LOG.info(bc.getClass().getSimpleName() + ": " + bc.getStats());
BlockCache [] bcs = bc.getBlockCaches();
if (bcs != null) {
for (BlockCache sbc: bc.getBlockCaches()) {
bucketCacheReport(sbc);
}
}
}
private void logPerFile(final BlockCacheUtil.CachedBlocksByFile cbsbf)
throws JsonGenerationException, JsonMappingException, IOException {
for (Map.Entry<String, NavigableSet<CachedBlock>> e:
cbsbf.getCachedBlockStatsByFile().entrySet()) {
int count = 0;
long size = 0;
int countData = 0;
long sizeData = 0;
for (CachedBlock cb: e.getValue()) {
count++;
size += cb.getSize();
BlockType bt = cb.getBlockType();
if (bt != null && bt.isData()) {
countData++;
sizeData += cb.getSize();
}
}
LOG.info("filename=" + e.getKey() + ", count=" + count + ", countData=" + countData +
", size=" + size + ", sizeData=" + sizeData);
LOG.info(BlockCacheUtil.toJSON(e.getKey(), e.getValue()));
}
}
private BlockCacheUtil.CachedBlocksByFile logPerBlock(final BlockCache bc)
throws JsonGenerationException, JsonMappingException, IOException {
BlockCacheUtil.CachedBlocksByFile cbsbf = new BlockCacheUtil.CachedBlocksByFile();
for (CachedBlock cb: bc) {
LOG.info(cb.toString());
LOG.info(BlockCacheUtil.toJSON(bc));
cbsbf.update(cb);
}
return cbsbf;
}
}

View File

@ -24,6 +24,8 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.SmallTests;
@ -31,38 +33,65 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mortbay.log.Log;
/**
* Tests that {@link CacheConfig} does as expected.
*/
@Category(SmallTests.class)
public class TestCacheConfig {
private static final Log LOG = LogFactory.getLog(TestCacheConfig.class);
private Configuration conf;
static class Deserializer implements CacheableDeserializer<Cacheable> {
private final Cacheable cacheable;
Deserializer(final Cacheable c) {
this.cacheable = c;
}
@Override
public int getDeserialiserIdentifier() {
return 0;
}
@Override
public Cacheable deserialize(ByteBuffer b, boolean reuse) throws IOException {
LOG.info("Deserialized " + b + ", reuse=" + reuse);
return cacheable;
}
@Override
public Cacheable deserialize(ByteBuffer b) throws IOException {
LOG.info("Deserialized " + b);
return cacheable;
}
};
static class IndexCacheEntry extends DataCacheEntry {
private static IndexCacheEntry SINGLETON = new IndexCacheEntry();
public IndexCacheEntry() {
super(SINGLETON);
}
@Override
public BlockType getBlockType() {
return BlockType.ROOT_INDEX;
}
}
static class DataCacheEntry implements Cacheable {
private static final int SIZE = 1;
private static DataCacheEntry SINGLETON = new DataCacheEntry();
final CacheableDeserializer<Cacheable> deserializer;
private final CacheableDeserializer<Cacheable> deserializer =
new CacheableDeserializer<Cacheable>() {
@Override
public int getDeserialiserIdentifier() {
return 0;
}
DataCacheEntry() {
this(SINGLETON);
}
@Override
public Cacheable deserialize(ByteBuffer b, boolean reuse) throws IOException {
Log.info("Deserialized " + b + ", reuse=" + reuse);
return SINGLETON;
}
@Override
public Cacheable deserialize(ByteBuffer b) throws IOException {
Log.info("Deserialized " + b);
return SINGLETON;
}
};
DataCacheEntry(final Cacheable c) {
this.deserializer = new Deserializer(c);
}
public String toString() {
return "size=" + SIZE + ", type=" + getBlockType();
@ -80,7 +109,7 @@ public class TestCacheConfig {
@Override
public void serialize(ByteBuffer destination) {
Log.info("Serialized " + this + " to " + destination);
LOG.info("Serialized " + this + " to " + destination);
}
@Override
@ -119,7 +148,7 @@ public class TestCacheConfig {
* to onheap and offheap caches.
* @param sizing True if we should run sizing test (doesn't always apply).
*/
private void basicBlockCacheOps(final CacheConfig cc, final boolean doubling,
void basicBlockCacheOps(final CacheConfig cc, final boolean doubling,
final boolean sizing) {
assertTrue(cc.isBlockCacheEnabled());
assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());

View File

@ -40,7 +40,7 @@ public class TestCachedBlockQueue extends TestCase {
CachedBlock cb9 = new CachedBlock(1000, "cb9", 9);
CachedBlock cb10 = new CachedBlock(1500, "cb10", 10);
CachedBlockQueue queue = new CachedBlockQueue(10000,1000);
LruCachedBlockQueue queue = new LruCachedBlockQueue(10000,1000);
queue.add(cb1);
queue.add(cb2);
@ -78,7 +78,7 @@ public class TestCachedBlockQueue extends TestCase {
CachedBlock cb9 = new CachedBlock(1000, "cb9", 9);
CachedBlock cb10 = new CachedBlock(1500, "cb10", 10);
CachedBlockQueue queue = new CachedBlockQueue(10000,1000);
LruCachedBlockQueue queue = new LruCachedBlockQueue(10000,1000);
queue.add(cb1);
queue.add(cb2);
@ -110,7 +110,7 @@ public class TestCachedBlockQueue extends TestCase {
}
}
private static class CachedBlock extends org.apache.hadoop.hbase.io.hfile.CachedBlock
private static class CachedBlock extends org.apache.hadoop.hbase.io.hfile.LruCachedBlock
{
public CachedBlock(final long heapSize, String name, long accessTime) {
super(new BlockCacheKey(name, 0),

View File

@ -67,7 +67,7 @@ public class TestLruBlockCache {
// wait until at least one eviction has run
int n = 0;
while(cache.getEvictionCount() == 0) {
while(cache.getStats().getEvictionCount() == 0) {
Thread.sleep(200);
assertTrue("Eviction never happened.", n++ < 20);
}
@ -87,7 +87,7 @@ public class TestLruBlockCache {
assertTrue("Cache never stabilized.", n++ < 20);
}
long evictionCount = cache.getEvictionCount();
long evictionCount = cache.getStats().getEvictionCount();
assertTrue(evictionCount >= 1);
System.out.println("Background Evictions run: " + evictionCount);
}
@ -145,7 +145,7 @@ public class TestLruBlockCache {
}
// Expect no evictions
assertEquals(0, cache.getEvictionCount());
assertEquals(0, cache.getStats().getEvictionCount());
Thread t = new LruBlockCache.StatisticsThread(cache);
t.start();
t.join();
@ -170,7 +170,7 @@ public class TestLruBlockCache {
}
// A single eviction run should have occurred
assertEquals(1, cache.getEvictionCount());
assertEquals(1, cache.getStats().getEvictionCount());
// Our expected size overruns acceptable limit
assertTrue(expectedCacheSize >
@ -218,10 +218,10 @@ public class TestLruBlockCache {
}
// A single eviction run should have occurred
assertEquals(cache.getEvictionCount(), 1);
assertEquals(cache.getStats().getEvictionCount(), 1);
// We expect two entries evicted
assertEquals(cache.getEvictedCount(), 2);
assertEquals(cache.getStats().getEvictedCount(), 2);
// Our expected size overruns acceptable limit
assertTrue(expectedCacheSize >
@ -292,7 +292,7 @@ public class TestLruBlockCache {
}
// Do not expect any evictions yet
assertEquals(0, cache.getEvictionCount());
assertEquals(0, cache.getStats().getEvictionCount());
// Verify cache size
assertEquals(expectedCacheSize, cache.heapSize());
@ -301,8 +301,8 @@ public class TestLruBlockCache {
cache.cacheBlock(singleBlocks[3].cacheKey, singleBlocks[3]);
// Single eviction, one thing evicted
assertEquals(1, cache.getEvictionCount());
assertEquals(1, cache.getEvictedCount());
assertEquals(1, cache.getStats().getEvictionCount());
assertEquals(1, cache.getStats().getEvictedCount());
// Verify oldest single block is the one evicted
assertEquals(null, cache.getBlock(singleBlocks[0].cacheKey, true, false, true));
@ -314,8 +314,8 @@ public class TestLruBlockCache {
cache.cacheBlock(singleBlocks[4].cacheKey, singleBlocks[4]);
// Two evictions, two evicted.
assertEquals(2, cache.getEvictionCount());
assertEquals(2, cache.getEvictedCount());
assertEquals(2, cache.getStats().getEvictionCount());
assertEquals(2, cache.getStats().getEvictedCount());
// Oldest multi block should be evicted now
assertEquals(null, cache.getBlock(multiBlocks[0].cacheKey, true, false, true));
@ -324,8 +324,8 @@ public class TestLruBlockCache {
cache.cacheBlock(memoryBlocks[3].cacheKey, memoryBlocks[3], true);
// Three evictions, three evicted.
assertEquals(3, cache.getEvictionCount());
assertEquals(3, cache.getEvictedCount());
assertEquals(3, cache.getStats().getEvictionCount());
assertEquals(3, cache.getStats().getEvictedCount());
// Oldest memory block should be evicted now
assertEquals(null, cache.getBlock(memoryBlocks[0].cacheKey, true, false, true));
@ -335,8 +335,8 @@ public class TestLruBlockCache {
cache.cacheBlock(bigBlocks[0].cacheKey, bigBlocks[0]);
// Four evictions, six evicted (inserted block 3X size, expect +3 evicted)
assertEquals(4, cache.getEvictionCount());
assertEquals(6, cache.getEvictedCount());
assertEquals(4, cache.getStats().getEvictionCount());
assertEquals(6, cache.getStats().getEvictedCount());
// Expect three remaining singles to be evicted
assertEquals(null, cache.getBlock(singleBlocks[2].cacheKey, true, false, true));
@ -350,8 +350,8 @@ public class TestLruBlockCache {
cache.cacheBlock(bigBlocks[1].cacheKey, bigBlocks[1]);
// Five evictions, nine evicted (3 new)
assertEquals(5, cache.getEvictionCount());
assertEquals(9, cache.getEvictedCount());
assertEquals(5, cache.getStats().getEvictionCount());
assertEquals(9, cache.getStats().getEvictedCount());
// Expect three remaining multis to be evicted
assertEquals(null, cache.getBlock(singleBlocks[1].cacheKey, true, false, true));
@ -362,8 +362,8 @@ public class TestLruBlockCache {
cache.cacheBlock(bigBlocks[2].cacheKey, bigBlocks[2], true);
// Six evictions, twelve evicted (3 new)
assertEquals(6, cache.getEvictionCount());
assertEquals(12, cache.getEvictedCount());
assertEquals(6, cache.getStats().getEvictionCount());
assertEquals(12, cache.getStats().getEvictedCount());
// Expect three remaining in-memory to be evicted
assertEquals(null, cache.getBlock(memoryBlocks[1].cacheKey, true, false, true));
@ -407,23 +407,23 @@ public class TestLruBlockCache {
cache.cacheBlock(singleBlocks[4].cacheKey, singleBlocks[4]);
expectedCacheSize += singleBlocks[4].cacheBlockHeapSize();
// Do not expect any evictions yet
assertEquals(0, cache.getEvictionCount());
assertEquals(0, cache.getStats().getEvictionCount());
// Verify cache size
assertEquals(expectedCacheSize, cache.heapSize());
// 1. Insert a memory block, oldest single should be evicted, si:mu:me = 4:4:1
cache.cacheBlock(memoryBlocks[0].cacheKey, memoryBlocks[0], true);
// Single eviction, one block evicted
assertEquals(1, cache.getEvictionCount());
assertEquals(1, cache.getEvictedCount());
assertEquals(1, cache.getStats().getEvictionCount());
assertEquals(1, cache.getStats().getEvictedCount());
// Verify oldest single block (index = 0) is the one evicted
assertEquals(null, cache.getBlock(singleBlocks[0].cacheKey, true, false, true));
// 2. Insert another memory block, another single evicted, si:mu:me = 3:4:2
cache.cacheBlock(memoryBlocks[1].cacheKey, memoryBlocks[1], true);
// Two evictions, two evicted.
assertEquals(2, cache.getEvictionCount());
assertEquals(2, cache.getEvictedCount());
assertEquals(2, cache.getStats().getEvictionCount());
assertEquals(2, cache.getStats().getEvictedCount());
// Current oldest single block (index = 1) should be evicted now
assertEquals(null, cache.getBlock(singleBlocks[1].cacheKey, true, false, true));
@ -433,8 +433,8 @@ public class TestLruBlockCache {
cache.cacheBlock(memoryBlocks[4].cacheKey, memoryBlocks[4], true);
cache.cacheBlock(memoryBlocks[5].cacheKey, memoryBlocks[5], true);
// Three evictions, three evicted.
assertEquals(6, cache.getEvictionCount());
assertEquals(6, cache.getEvictedCount());
assertEquals(6, cache.getStats().getEvictionCount());
assertEquals(6, cache.getStats().getEvictedCount());
// two oldest single blocks and two oldest multi blocks evicted
assertEquals(null, cache.getBlock(singleBlocks[2].cacheKey, true, false, true));
assertEquals(null, cache.getBlock(singleBlocks[3].cacheKey, true, false, true));
@ -447,8 +447,8 @@ public class TestLruBlockCache {
cache.cacheBlock(memoryBlocks[7].cacheKey, memoryBlocks[7], true);
cache.cacheBlock(memoryBlocks[8].cacheKey, memoryBlocks[8], true);
// Three evictions, three evicted.
assertEquals(9, cache.getEvictionCount());
assertEquals(9, cache.getEvictedCount());
assertEquals(9, cache.getStats().getEvictionCount());
assertEquals(9, cache.getStats().getEvictedCount());
// one oldest single block and two oldest multi blocks evicted
assertEquals(null, cache.getBlock(singleBlocks[4].cacheKey, true, false, true));
assertEquals(null, cache.getBlock(multiBlocks[2].cacheKey, true, false, true));
@ -458,8 +458,8 @@ public class TestLruBlockCache {
// si:mu:me = 0:0:9
cache.cacheBlock(memoryBlocks[9].cacheKey, memoryBlocks[9], true);
// one eviction, one evicted.
assertEquals(10, cache.getEvictionCount());
assertEquals(10, cache.getEvictedCount());
assertEquals(10, cache.getStats().getEvictionCount());
assertEquals(10, cache.getStats().getEvictedCount());
// oldest memory block evicted
assertEquals(null, cache.getBlock(memoryBlocks[0].cacheKey, true, false, true));
@ -468,8 +468,8 @@ public class TestLruBlockCache {
// si:mu:me = 0:0:9 (no change)
cache.cacheBlock(singleBlocks[9].cacheKey, singleBlocks[9]);
// one eviction, one evicted.
assertEquals(11, cache.getEvictionCount());
assertEquals(11, cache.getEvictedCount());
assertEquals(11, cache.getStats().getEvictionCount());
assertEquals(11, cache.getStats().getEvictedCount());
// the single block just cached now evicted (can't evict memory)
assertEquals(null, cache.getBlock(singleBlocks[9].cacheKey, true, false, true));
}
@ -507,10 +507,10 @@ public class TestLruBlockCache {
}
// An eviction ran
assertEquals(1, cache.getEvictionCount());
assertEquals(1, cache.getStats().getEvictionCount());
// To drop down to 2/3 capacity, we'll need to evict 4 blocks
assertEquals(4, cache.getEvictedCount());
assertEquals(4, cache.getStats().getEvictedCount());
// Should have been taken off equally from single and multi
assertEquals(null, cache.getBlock(singleBlocks[0].cacheKey, true, false, true));
@ -530,11 +530,11 @@ public class TestLruBlockCache {
}
// 4 total evictions, 16 total evicted
assertEquals(4, cache.getEvictionCount());
assertEquals(16, cache.getEvictedCount());
assertEquals(4, cache.getStats().getEvictionCount());
assertEquals(16, cache.getStats().getEvictedCount());
// Should now have 7 total blocks
assertEquals(7, cache.size());
assertEquals(7, cache.getBlockCount());
}
@ -575,16 +575,16 @@ public class TestLruBlockCache {
}
// Do not expect any evictions yet
assertEquals(0, cache.getEvictionCount());
assertEquals(0, cache.getStats().getEvictionCount());
// Resize to half capacity plus an extra block (otherwise we evict an extra)
cache.setMaxSize((long)(maxSize * 0.5f));
// Should have run a single eviction
assertEquals(1, cache.getEvictionCount());
assertEquals(1, cache.getStats().getEvictionCount());
// And we expect 1/2 of the blocks to be evicted
assertEquals(15, cache.getEvictedCount());
assertEquals(15, cache.getStats().getEvictedCount());
// And the oldest 5 blocks from each category should be gone
for(int i=0;i<5;i++) {
@ -720,7 +720,7 @@ public class TestLruBlockCache {
(numEntries * ClassSize.CONCURRENT_HASHMAP_ENTRY) +
(LruBlockCache.DEFAULT_CONCURRENCY_LEVEL * ClassSize.CONCURRENT_HASHMAP_SEGMENT);
long negateBlockSize = (long)(totalOverhead/numEntries);
negateBlockSize += CachedBlock.PER_BLOCK_OVERHEAD;
negateBlockSize += LruCachedBlock.PER_BLOCK_OVERHEAD;
return ClassSize.align((long)Math.floor((roughBlockSize - negateBlockSize)*0.99f));
}
@ -732,7 +732,7 @@ public class TestLruBlockCache {
(numEntries * ClassSize.CONCURRENT_HASHMAP_ENTRY) +
(LruBlockCache.DEFAULT_CONCURRENCY_LEVEL * ClassSize.CONCURRENT_HASHMAP_SEGMENT);
long negateBlockSize = totalOverhead / numEntries;
negateBlockSize += CachedBlock.PER_BLOCK_OVERHEAD;
negateBlockSize += LruCachedBlock.PER_BLOCK_OVERHEAD;
return ClassSize.align((long)Math.floor((roughBlockSize - negateBlockSize)*
LruBlockCache.DEFAULT_ACCEPTABLE_FACTOR));
}
@ -754,7 +754,7 @@ public class TestLruBlockCache {
/** Size of the cache block holding this item. Used for verification. */
public long cacheBlockHeapSize() {
return CachedBlock.PER_BLOCK_OVERHEAD
return LruCachedBlock.PER_BLOCK_OVERHEAD
+ ClassSize.align(cacheKey.heapSize())
+ ClassSize.align(size);
}

View File

@ -23,9 +23,8 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.util.List;
import java.util.Iterator;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CoordinatedStateManager;
@ -34,10 +33,11 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.CacheStats;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache;
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerContext;
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerResult;
@ -321,25 +321,24 @@ public class TestHeapMemoryManager {
return 0;
}
@Override
public long getEvictedCount() {
return 0;
}
@Override
public long getBlockCount() {
return 0;
}
@Override
public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(Configuration conf)
throws IOException {
public void setMaxSize(long size) {
this.maxSize = size;
}
@Override
public Iterator<CachedBlock> iterator() {
return null;
}
@Override
public void setMaxSize(long size) {
this.maxSize = size;
public BlockCache[] getBlockCaches() {
return null;
}
}

View File

@ -1,151 +0,0 @@
/*
*
* 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableUtil;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Tests the block cache summary functionality in StoreFile,
* which contains the BlockCache
*
*/
@Category(MediumTests.class)
public class TestStoreFileBlockCacheSummary {
final Log LOG = LogFactory.getLog(getClass());
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static final String TEST_TABLE = "testTable";
private static final String TEST_TABLE2 = "testTable2";
private static final String TEST_CF = "testFamily";
private static byte [] FAMILY = Bytes.toBytes(TEST_CF);
private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
private static byte [] VALUE = Bytes.toBytes("testValue");
private final int TOTAL_ROWS = 4;
/**
* @throws java.lang.Exception exception
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster();
}
/**
* @throws java.lang.Exception exception
*/
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
private Put createPut(byte[] family, String row) {
Put put = new Put( Bytes.toBytes(row));
put.add(family, QUALIFIER, VALUE);
return put;
}
/**
* This test inserts data into multiple tables and then reads both tables to ensure
* they are in the block cache.
*
* @throws Exception exception
*/
@Test
public void testBlockCacheSummary() throws Exception {
HTable ht = TEST_UTIL.createTable(Bytes.toBytes(TEST_TABLE), FAMILY);
addRows(ht, FAMILY);
HTable ht2 = TEST_UTIL.createTable(Bytes.toBytes(TEST_TABLE2), FAMILY);
addRows(ht2, FAMILY);
TEST_UTIL.flush();
scan(ht, FAMILY);
scan(ht2, FAMILY);
BlockCache bc =
new CacheConfig(TEST_UTIL.getConfiguration()).getBlockCache();
List<BlockCacheColumnFamilySummary> bcs =
bc.getBlockCacheColumnFamilySummaries(TEST_UTIL.getConfiguration());
LOG.info("blockCacheSummary: " + bcs);
assertTrue("blockCache summary has " + bcs.size() + " entries", bcs.size() >= 2);
BlockCacheColumnFamilySummary e = bcs.get(bcs.size()-2);
assertEquals("table", TEST_TABLE, e.getTable());
assertEquals("cf", TEST_CF, e.getColumnFamily());
e = bcs.get(bcs.size()-1);
assertEquals("table", TEST_TABLE2, e.getTable());
assertEquals("cf", TEST_CF, e.getColumnFamily());
ht.close();
ht2.close();
}
private void addRows(HTable ht, byte[] family) throws IOException {
List<Row> rows = new ArrayList<Row>();
for (int i = 0; i < TOTAL_ROWS;i++) {
rows.add(createPut(family, "row" + i));
}
HTableUtil.bucketRsBatch( ht, rows);
}
private void scan(HTable ht, byte[] family) throws IOException {
Scan scan = new Scan();
scan.addColumn(family, QUALIFIER);
int count = 0;
for(@SuppressWarnings("unused") Result result : ht.getScanner(scan)) {
count++;
}
if (TOTAL_ROWS != count) {
throw new IOException("Incorrect number of rows!");
}
}
}

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.replication;
import java.io.IOException;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;

View File

@ -912,7 +912,7 @@
<disruptor.version>3.2.0</disruptor.version>
<collections.version>3.2.1</collections.version>
<httpclient.version>3.1</httpclient.version>
<metrics-core.version>2.1.2</metrics-core.version>
<metrics-core.version>2.2.0</metrics-core.version>
<guava.version>12.0.1</guava.version>
<jackson.version>1.8.8</jackson.version>
<jasper.version>5.5.23</jasper.version>