You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2014/07/16 08:15:33 UTC

[1/2] git commit: HBASE-11517 TestReplicaWithCluster turns zombie -- ADDS TIMEOUTS SO CAN DEBUG ZOMBIE

Repository: hbase
Updated Branches:
  refs/heads/branch-1 07643580b -> 14b331cca


HBASE-11517 TestReplicaWithCluster turns zombie -- ADDS TIMEOUTS SO CAN DEBUG ZOMBIE


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

Branch: refs/heads/branch-1
Commit: 7175f51f0817e93fe2c46aa39937fdff73e383d6
Parents: 0764358
Author: stack <st...@apache.org>
Authored: Tue Jul 15 10:01:32 2014 -0700
Committer: stack <st...@apache.org>
Committed: Tue Jul 15 23:14:35 2014 -0700

----------------------------------------------------------------------
 .../hbase/client/TestReplicaWithCluster.java    | 62 ++++++++++++--------
 1 file changed, 38 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7175f51f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index bf7a93b..2c1f52b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -129,7 +129,7 @@ public class TestReplicaWithCluster {
     HTU.shutdownMiniCluster();
   }
 
-  @Test
+  @Test (timeout=30000)
   public void testCreateDeleteTable() throws IOException {
     // Create table then get the single region for our new table.
     HTableDescriptor hdt = HTU.createTableDescriptor("testCreateDeleteTable");
@@ -162,7 +162,7 @@ public class TestReplicaWithCluster {
     HTU.deleteTable(hdt.getTableName());
   }
 
-  @Test
+  @Test (timeout=30000)
   public void testChangeTable() throws Exception {
     HTableDescriptor hdt = HTU.createTableDescriptor("testChangeTable");
     hdt.setRegionReplication(NB_SERVERS);
@@ -210,16 +210,20 @@ public class TestReplicaWithCluster {
 
     HTU.getHBaseCluster().stopMaster(0);
     HBaseAdmin admin = new HBaseAdmin(HTU.getConfiguration());
-    nHdt =admin.getTableDescriptor(hdt.getTableName());
-    Assert.assertEquals("fams=" + Arrays.toString(nHdt.getColumnFamilies()),
+    try {
+      nHdt = admin.getTableDescriptor(hdt.getTableName());
+      Assert.assertEquals("fams=" + Arrays.toString(nHdt.getColumnFamilies()),
         bHdt.getColumnFamilies().length + 1, nHdt.getColumnFamilies().length);
 
-    admin.disableTable(hdt.getTableName());
-    admin.deleteTable(hdt.getTableName());
-    HTU.getHBaseCluster().startMaster();
+      admin.disableTable(hdt.getTableName());
+      admin.deleteTable(hdt.getTableName());
+      HTU.getHBaseCluster().startMaster();
+    } finally {
+      if (admin != null) admin.close();
+    }
   }
 
-  @Test
+  @Test (timeout=30000)
   public void testReplicaAndReplication() throws Exception {
     HTableDescriptor hdt = HTU.createTableDescriptor("testReplicaAndReplication");
     hdt.setRegionReplication(NB_SERVERS);
@@ -242,19 +246,23 @@ public class TestReplicaWithCluster {
     HTU2.getHBaseAdmin().createTable(hdt, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
 
     ReplicationAdmin admin = new ReplicationAdmin(HTU.getConfiguration());
-    admin.addPeer("2", HTU2.getClusterKey());
+    try {
+      admin.addPeer("2", HTU2.getClusterKey());
+    } finally {
+      if (admin != null) admin.close();
+    }
 
     Put p = new Put(row);
     p.add(row, row, row);
     final HTable table = new HTable(HTU.getConfiguration(), hdt.getTableName());
-    table.put(p);
-
-    HTU.getHBaseAdmin().flush(table.getTableName());
-    LOG.info("Put & flush done on the first cluster. Now doing a get on the same cluster.");
+    try {
+      table.put(p);
+      HTU.getHBaseAdmin().flush(table.getTableName());
+      LOG.info("Put & flush done on the first cluster. Now doing a get on the same cluster.");
 
-    Waiter.waitFor(HTU.getConfiguration(), 1000, new Waiter.Predicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
+      Waiter.waitFor(HTU.getConfiguration(), 1000, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
         try {
           SlowMeCopro.cdl.set(new CountDownLatch(1));
           Get g = new Get(row);
@@ -265,15 +273,18 @@ public class TestReplicaWithCluster {
         } finally {
           SlowMeCopro.cdl.get().countDown();
           SlowMeCopro.sleepTime.set(0);
-        }      }
-    });
-
+        }
+      }});
+    } finally {
+      if (table != null) table.close();
+    }
     LOG.info("stale get on the first cluster done. Now for the second.");
 
     final HTable table2 = new HTable(HTU.getConfiguration(), hdt.getTableName());
-    Waiter.waitFor(HTU.getConfiguration(), 1000, new Waiter.Predicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
+    try {
+      Waiter.waitFor(HTU.getConfiguration(), 1000, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
         try {
           SlowMeCopro.cdl.set(new CountDownLatch(1));
           Get g = new Get(row);
@@ -284,8 +295,11 @@ public class TestReplicaWithCluster {
         } finally {
           SlowMeCopro.cdl.get().countDown();
           SlowMeCopro.sleepTime.set(0);
-        }      }
-    });
+        }
+      }});
+    } finally {
+      if (table2 != null) table2.close();
+    }
 
     HTU.getHBaseAdmin().disableTable(hdt.getTableName());
     HTU.deleteTable(hdt.getTableName());


[2/2] git commit: HBASE-11520 Simplify offheap cache config by removing the confusing "hbase.bucketcache.percentage.in.combinedcache"

Posted by st...@apache.org.
HBASE-11520 Simplify offheap cache config by removing the confusing "hbase.bucketcache.percentage.in.combinedcache"


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

Branch: refs/heads/branch-1
Commit: 14b331ccab8ef18b00ff7b4cf9127e22c74c4bca
Parents: 7175f51
Author: stack <st...@apache.org>
Authored: Tue Jul 15 23:14:14 2014 -0700
Committer: stack <st...@apache.org>
Committed: Tue Jul 15 23:14:57 2014 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/io/hfile/CacheConfig.java      | 142 ++++++++++---------
 .../hadoop/hbase/io/hfile/LruBlockCache.java    |  13 +-
 .../hadoop/hbase/io/hfile/package-info.java     |  48 +------
 .../hadoop/hbase/io/hfile/TestCacheConfig.java  | 101 +++++++++++--
 src/main/docbkx/book.xml                        |  18 ++-
 5 files changed, 194 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/14b331cc/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
index d119c85..afaf7e9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
@@ -113,13 +113,6 @@ public class CacheConfig {
   public static final String BUCKET_CACHE_COMBINED_KEY = 
       "hbase.bucketcache.combinedcache.enabled";
 
-  /**
-   * A float which designates how much of the overall cache to give to bucket cache
-   * and how much to on-heap lru cache when {@link #BUCKET_CACHE_COMBINED_KEY} is set.
-   */
-  public static final String BUCKET_CACHE_COMBINED_PERCENTAGE_KEY =
-      "hbase.bucketcache.percentage.in.combinedcache";
-
   public static final String BUCKET_CACHE_WRITER_THREADS_KEY = "hbase.bucketcache.writer.threads";
   public static final String BUCKET_CACHE_WRITER_QUEUE_KEY = 
       "hbase.bucketcache.writer.queuelength";
@@ -451,7 +444,7 @@ public class CacheConfig {
   @VisibleForTesting
   static boolean blockCacheDisabled = false;
 
-  private static long getLruCacheSize(final Configuration conf, final MemoryUsage mu) {
+  static long getLruCacheSize(final Configuration conf, final MemoryUsage mu) {
     float cachePercentage = conf.getFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY,
       HConstants.HFILE_BLOCK_CACHE_SIZE_DEFAULT);
     if (cachePercentage <= 0.0001f) {
@@ -468,7 +461,68 @@ public class CacheConfig {
   }
 
   /**
+   * @param c Configuration to use.
+   * @param mu JMX Memory Bean
+   * @return An L1 instance.  Currently an instance of LruBlockCache.
+   */
+  private static LruBlockCache getL1(final Configuration c, final MemoryUsage mu) {
+    long lruCacheSize = getLruCacheSize(c, mu);
+    int blockSize = c.getInt("hbase.offheapcache.minblocksize", HConstants.DEFAULT_BLOCKSIZE);
+    LOG.info("Allocating LruBlockCache size=" +
+      StringUtils.byteDesc(lruCacheSize) + ", blockSize=" + StringUtils.byteDesc(blockSize));
+    return new LruBlockCache(lruCacheSize, blockSize, true, c);
+  }
+
+  /**
+   * @param c Configuration to use.
+   * @param mu JMX Memory Bean
+   * @return Returns L2 block cache instance (for now it is BucketCache BlockCache all the time)
+   * or null if not supposed to be a L2.
+   */
+  private static BucketCache getL2(final Configuration c, final MemoryUsage mu) {
+    // Check for L2.  ioengine name must be non-null.
+    String bucketCacheIOEngineName = c.get(BUCKET_CACHE_IOENGINE_KEY, null);
+    if (bucketCacheIOEngineName == null || bucketCacheIOEngineName.length() <= 0) return null;
+
+    int blockSize = c.getInt("hbase.offheapcache.minblocksize", HConstants.DEFAULT_BLOCKSIZE);
+    float bucketCachePercentage = c.getFloat(BUCKET_CACHE_SIZE_KEY, 0F);
+    long bucketCacheSize = (long) (bucketCachePercentage < 1? mu.getMax() * bucketCachePercentage:
+      bucketCachePercentage * 1024 * 1024);
+    if (bucketCacheSize <= 0) {
+      throw new IllegalStateException("bucketCacheSize <= 0; Check " +
+        BUCKET_CACHE_SIZE_KEY + " setting and/or server java heap size");
+    }
+    int writerThreads = c.getInt(BUCKET_CACHE_WRITER_THREADS_KEY,
+      DEFAULT_BUCKET_CACHE_WRITER_THREADS);
+    int writerQueueLen = c.getInt(BUCKET_CACHE_WRITER_QUEUE_KEY,
+      DEFAULT_BUCKET_CACHE_WRITER_QUEUE);
+    String persistentPath = c.get(BUCKET_CACHE_PERSISTENT_PATH_KEY);
+    String[] configuredBucketSizes = c.getStrings(BUCKET_CACHE_BUCKETS_KEY);
+    int [] bucketSizes = null;
+    if (configuredBucketSizes != null) {
+      bucketSizes = new int[configuredBucketSizes.length];
+      for (int i = 0; i < configuredBucketSizes.length; i++) {
+        bucketSizes[i] = Integer.parseInt(configuredBucketSizes[i]);
+      }
+    }
+    BucketCache bucketCache = null;
+    try {
+      int ioErrorsTolerationDuration = c.getInt(
+        "hbase.bucketcache.ioengine.errors.tolerated.duration",
+        BucketCache.DEFAULT_ERROR_TOLERATION_DURATION);
+      // Bucket cache logs its stats on creation internal to the constructor.
+      bucketCache = new BucketCache(bucketCacheIOEngineName,
+        bucketCacheSize, blockSize, bucketSizes, writerThreads, writerQueueLen, persistentPath,
+        ioErrorsTolerationDuration);
+    } catch (IOException ioex) {
+      LOG.error("Can't instantiate bucket cache", ioex); throw new RuntimeException(ioex);
+    }
+    return bucketCache;
+  }
+
+  /**
    * Returns the block cache or <code>null</code> in case none should be used.
+   * Sets GLOBAL_BLOCK_CACHE_INSTANCE
    *
    * @param conf  The current configuration.
    * @return The block cache or <code>null</code>.
@@ -477,67 +531,23 @@ public class CacheConfig {
     if (GLOBAL_BLOCK_CACHE_INSTANCE != null) return GLOBAL_BLOCK_CACHE_INSTANCE;
     if (blockCacheDisabled) return null;
     MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
-    long lruCacheSize = getLruCacheSize(conf, mu);
-    int blockSize = conf.getInt("hbase.offheapcache.minblocksize", HConstants.DEFAULT_BLOCKSIZE);
-
-    String bucketCacheIOEngineName = conf.get(BUCKET_CACHE_IOENGINE_KEY, null);
-    float bucketCachePercentage = conf.getFloat(BUCKET_CACHE_SIZE_KEY, 0F);
-    // A percentage of max heap size or a absolute value with unit megabytes
-    long bucketCacheSize = (long) (bucketCachePercentage < 1 ? mu.getMax()
-      * bucketCachePercentage : bucketCachePercentage * 1024 * 1024);
-
-    boolean combinedWithLru = conf.getBoolean(BUCKET_CACHE_COMBINED_KEY,
-      DEFAULT_BUCKET_CACHE_COMBINED);
-    BucketCache bucketCache = null;
-    if (bucketCacheIOEngineName != null && bucketCacheSize > 0) {
-      int writerThreads = conf.getInt(BUCKET_CACHE_WRITER_THREADS_KEY,
-        DEFAULT_BUCKET_CACHE_WRITER_THREADS);
-      int writerQueueLen = conf.getInt(BUCKET_CACHE_WRITER_QUEUE_KEY,
-        DEFAULT_BUCKET_CACHE_WRITER_QUEUE);
-      String persistentPath = conf.get(BUCKET_CACHE_PERSISTENT_PATH_KEY);
-      float combinedPercentage = conf.getFloat(BUCKET_CACHE_COMBINED_PERCENTAGE_KEY,
-        DEFAULT_BUCKET_CACHE_COMBINED_PERCENTAGE);
-      String[] configuredBucketSizes = conf.getStrings(BUCKET_CACHE_BUCKETS_KEY);
-      int [] bucketSizes = null;
-      if (configuredBucketSizes != null) {
-        bucketSizes = new int[configuredBucketSizes.length];
-        for (int i = 0; i < configuredBucketSizes.length; i++) {
-          bucketSizes[i] = Integer.parseInt(configuredBucketSizes[i]);
-        }
-      }
-      if (combinedWithLru) {
-        lruCacheSize = (long) ((1 - combinedPercentage) * bucketCacheSize);
-        bucketCacheSize = (long) (combinedPercentage * bucketCacheSize);
-      }
-      LOG.info("Allocating LruBlockCache size=" +
-        StringUtils.byteDesc(lruCacheSize) + ", blockSize=" + StringUtils.byteDesc(blockSize));
-      LruBlockCache lruCache = new LruBlockCache(lruCacheSize, blockSize, true, conf);
-      try {
-        int ioErrorsTolerationDuration = conf.getInt(
-          "hbase.bucketcache.ioengine.errors.tolerated.duration",
-          BucketCache.DEFAULT_ERROR_TOLERATION_DURATION);
-        bucketCache = new BucketCache(bucketCacheIOEngineName,
-          bucketCacheSize, blockSize, bucketSizes, writerThreads, writerQueueLen, persistentPath,
-          ioErrorsTolerationDuration);
-      } catch (IOException ioex) {
-        LOG.error("Can't instantiate bucket cache", ioex);
-        throw new RuntimeException(ioex);
-      }
+    LruBlockCache l1 = getL1(conf, mu);
+    BucketCache l2 = getL2(conf, mu);
+    if (l2 == null) {
+      GLOBAL_BLOCK_CACHE_INSTANCE = l1;
+    } else {
+      boolean combinedWithLru = conf.getBoolean(BUCKET_CACHE_COMBINED_KEY,
+        DEFAULT_BUCKET_CACHE_COMBINED);
       if (combinedWithLru) {
-        GLOBAL_BLOCK_CACHE_INSTANCE = new CombinedBlockCache(lruCache, bucketCache);
+        GLOBAL_BLOCK_CACHE_INSTANCE = new CombinedBlockCache(l1, l2);
       } else {
-        GLOBAL_BLOCK_CACHE_INSTANCE = lruCache;
+        // L1 and L2 are not 'combined'.  They are connected via the LruBlockCache victimhandler
+        // mechanism.  It is a little ugly but works according to the following: when the
+        // background eviction thread runs, blocks evicted from L1 will go to L2 AND when we get
+        // a block from the L1 cache, if not in L1, we will search L2.
+        l1.setVictimCache(l2);
+        GLOBAL_BLOCK_CACHE_INSTANCE = l1;
       }
-      lruCache.setVictimCache(bucketCache);
-    }
-    LOG.info("Allocating LruBlockCache size=" +
-      StringUtils.byteDesc(lruCacheSize) + ", blockSize=" + StringUtils.byteDesc(blockSize));
-    LruBlockCache lruCache = new LruBlockCache(lruCacheSize, blockSize);
-    lruCache.setVictimCache(bucketCache);
-    if (bucketCache != null && combinedWithLru) {
-      GLOBAL_BLOCK_CACHE_INSTANCE = new CombinedBlockCache(lruCache, bucketCache);
-    } else {
-      GLOBAL_BLOCK_CACHE_INSTANCE = lruCache;
     }
     return GLOBAL_BLOCK_CACHE_INSTANCE;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/14b331cc/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
index 5595439..ad533bf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
@@ -326,7 +326,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory,
       final boolean cacheDataInL1) {
     LruCachedBlock cb = map.get(cacheKey);
-    if(cb != null) {
+    if (cb != null) {
       // compare the contents, if they are not equal, we are in big trouble
       if (compare(buf, cb.getBuffer()) != 0) {
         throw new RuntimeException("Cached block contents differ, which should not have happened."
@@ -341,7 +341,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
     long newSize = updateSizeMetrics(cb, false);
     map.put(cacheKey, cb);
     elements.incrementAndGet();
-    if(newSize > acceptableSize() && !evictionInProgress) {
+    if (newSize > acceptableSize() && !evictionInProgress) {
       runEviction();
     }
   }
@@ -893,7 +893,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
 
   // Simple calculators of sizes given factors and maxSize
 
-  private long acceptableSize() {
+  long acceptableSize() {
     return (long)Math.floor(this.maxSize * this.acceptableFactor);
   }
   private long minSize() {
@@ -934,7 +934,8 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
 
   /** Clears the cache. Used in tests. */
   public void clearCache() {
-    map.clear();
+    this.map.clear();
+    this.elements.set(0);
   }
 
   /**
@@ -978,6 +979,10 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
     victimHandler = handler;
   }
 
+  BucketCache getVictimHandler() {
+    return this.victimHandler;
+  }
+
   @Override
   public BlockCache[] getBlockCaches() {
     return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/14b331cc/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/package-info.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/package-info.java
index 362181c..f298698 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/package-info.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/package-info.java
@@ -24,59 +24,21 @@
  * include the default, native on-heap {@link org.apache.hadoop.hbase.io.hfile.LruBlockCache} and a
  * {@link org.apache.hadoop.hbase.io.hfile.bucket.BucketCache} that has a bunch of deploy formats
  * including acting as a L2 for LruBlockCache -- when a block is evicted from LruBlockCache, it
- * goes to the BucketCache and when we search a block, we look in both places -- or
+ * goes to the BucketCache and when we search a block, we look in both places -- or, the
+ * most common deploy type,
  * using {@link org.apache.hadoop.hbase.io.hfile.CombinedBlockCache}, BucketCache is used as
  * a host for data blocks with meta blocks in an instance of LruBlockCache.  BucketCache
  * can also be onheap, offheap, and file-backed.
  * 
  * <h1>Which BlockCache should I use?</h1>
- * By default LruBlockCache is on.  If you would like to cache more, and offheap, try enabling
- * BucketCache. Fetching will always
+ * By default LruBlockCache is on.  If you would like to cache more, and offheap (offheap
+ * usually means less GC headache), try enabling * BucketCache. Fetching will always
  * be slower when fetching from BucketCache but latencies tend to be less erratic over time
  * (roughly because GC is less). See Nick Dimiduk's
  * <a href="http://www.n10k.com/blog/blockcache-101/">BlockCache 101</a> for some numbers.
  *
  * <h1>Enabling {@link org.apache.hadoop.hbase.io.hfile.bucket.BucketCache}</h2>
- * Read the options and defaults for BucketCache in the head of the
- * {@link org.apache.hadoop.hbase.io.hfile.CacheConfig}.
- * 
- * <p>Here is a simple example of how to enable a <code>4G</code> offheap bucket cache with 1G
- * onheap cache managed by {@link org.apache.hadoop.hbase.io.hfile.CombinedBlockCache}.
- * CombinedBlockCache will put DATA blocks in the BucketCache and META blocks -- INDEX and BLOOMS
- * -- in an instance of the LruBlockCache. For the
- * CombinedBlockCache (from the class comment), "[t]he smaller lruCache is used
- * to cache bloom blocks and index blocks, the larger bucketCache is used to
- * cache data blocks. getBlock reads first from the smaller lruCache before
- * looking for the block in the bucketCache. Metrics are the combined size and
- * hits and misses of both caches."  To disable CombinedBlockCache and have the BucketCache act
- * as a strict L2 cache to the L1 LruBlockCache (i.e. on eviction from L1, blocks go to L2), set
- * {@link org.apache.hadoop.hbase.io.hfile.CacheConfig#BUCKET_CACHE_COMBINED_KEY} to false. By
- * default, hbase.bucketcache.combinedcache.enabled (BUCKET_CACHE_COMBINED_KEY) is true.
- * 
- * <p>Back to the example of setting an onheap cache of 1G and offheap of 4G with the BlockCache
- * deploy managed by CombinedBlockCache. Setting hbase.bucketcache.ioengine and
- * hbase.bucketcache.size > 0 enables CombinedBlockCache.
- * In <code>hbase-env.sh</code> ensure the environment
- * variable <code>-XX:MaxDirectMemorySize</code> is enabled and is bigger than 4G, say 5G in size:
- * e.g. <code>-XX:MaxDirectMemorySize=5G</code>. This setting allows the JVM use offheap memory
- * up to this upper limit.  Allocate more than you need because there are other consumers of
- * offheap memory other than BlockCache (for example DFSClient in the RegionServer uses offheap).
- * In <code>hbase-site.xml</code> add the following configurations:
-<pre>&lt;property>
-  &lt;name>hbase.bucketcache.ioengine&lt;/name>
-  &lt;value>offheap&lt;/value>
-&lt;/property>
-&lt;property>
-  &lt;name>hbase.bucketcache.percentage.in.combinedcache&lt;/name>
-  &lt;value>0.8&lt;/value>
-&lt;/property>
-&lt;property>
-  &lt;name>hbase.bucketcache.size&lt;/name>
-  &lt;value>5120&lt;/value>
-&lt;/property></pre>.  Above we set a cache of 5G, 80% of which will be offheap (4G) and 1G onheap
- * (with DATA blocks in BucketCache and INDEX blocks in the onheap LruBlockCache).
- * Restart (or rolling restart) your cluster for the configs to take effect.  Check logs to ensure
- * your configurations came out as expected.
+ * See the HBase Reference Guide <a href="http://hbase.apache.org/book.html#enable.bucketcache">Enable BucketCache</a>.
  *
  */
 package org.apache.hadoop.hbase.io.hfile;

http://git-wip-us.apache.org/repos/asf/hbase/blob/14b331cc/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
index dfba277..156bfc1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
@@ -22,15 +22,22 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryUsage;
 import java.nio.ByteBuffer;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
+import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -198,11 +205,36 @@ public class TestCacheConfig {
     assertTrue(cc.getBlockCache() instanceof LruBlockCache);
   }
 
+  /**
+   * Assert that the caches are deployed with CombinedBlockCache and of the appropriate sizes.
+   */
   @Test
-  public void testBucketCacheConfig() {
+  public void testOffHeapBucketCacheConfig() {
     this.conf.set(CacheConfig.BUCKET_CACHE_IOENGINE_KEY, "offheap");
-    final float percent = 0.8f;
-    this.conf.setFloat(CacheConfig.BUCKET_CACHE_COMBINED_PERCENTAGE_KEY, percent);
+    doBucketCacheConfigTest();
+  }
+
+  @Test
+  public void testOnHeapBucketCacheConfig() {
+    this.conf.set(CacheConfig.BUCKET_CACHE_IOENGINE_KEY, "heap");
+    doBucketCacheConfigTest();
+  }
+
+  @Test
+  public void testFileBucketCacheConfig() throws IOException {
+    HBaseTestingUtility htu = new HBaseTestingUtility(this.conf);
+    try {
+      Path p = new Path(htu.getDataTestDir(), "bc.txt");
+      FileSystem fs = FileSystem.get(this.conf);
+      fs.create(p).close();
+      this.conf.set(CacheConfig.BUCKET_CACHE_IOENGINE_KEY, "file:" + p);
+      doBucketCacheConfigTest();
+    } finally {
+      htu.cleanupTestDir();
+    }
+  }
+
+  private void doBucketCacheConfigTest() {
     final int bcSize = 100;
     this.conf.setInt(CacheConfig.BUCKET_CACHE_SIZE_KEY, bcSize);
     CacheConfig cc = new CacheConfig(this.conf);
@@ -213,15 +245,66 @@ public class TestCacheConfig {
     BlockCache [] bcs = cbc.getBlockCaches();
     assertTrue(bcs[0] instanceof LruBlockCache);
     LruBlockCache lbc = (LruBlockCache)bcs[0];
-    long expectedBCSize = (long)(bcSize * (1.0f - percent));
-    long actualBCSize = lbc.getMaxSize() / (1024 * 1024);
-    assertTrue(expectedBCSize == actualBCSize);
+    assertEquals(CacheConfig.getLruCacheSize(this.conf,
+        ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()), lbc.getMaxSize());
     assertTrue(bcs[1] instanceof BucketCache);
     BucketCache bc = (BucketCache)bcs[1];
     // getMaxSize comes back in bytes but we specified size in MB
-    expectedBCSize = (long)(bcSize * percent);
-    actualBCSize = (long)(bc.getMaxSize() / (1024 * 1024));
-    assertTrue(expectedBCSize == actualBCSize);
+    assertEquals(bcSize, bc.getMaxSize() / (1024 * 1024));
+  }
+
+  /**
+   * Assert that when BUCKET_CACHE_COMBINED_KEY is false, the non-default, that we deploy
+   * LruBlockCache as L1 with a BucketCache for L2.
+   */
+  @Test (timeout=10000)
+  public void testBucketCacheConfigL1L2Setup() {
+    this.conf.set(CacheConfig.BUCKET_CACHE_IOENGINE_KEY, "offheap");
+    // Make lru size is smaller than bcSize for sure.  Need this to be true so when eviction
+    // from L1 happens, it does not fail because L2 can't take the eviction because block too big.
+    this.conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.001f);
+    MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
+    long lruExpectedSize = CacheConfig.getLruCacheSize(this.conf, mu);
+    final int bcSize = 100;
+    long bcExpectedSize = 100 * 1024 * 1024; // MB.
+    assertTrue(lruExpectedSize < bcExpectedSize);
+    this.conf.setInt(CacheConfig.BUCKET_CACHE_SIZE_KEY, bcSize);
+    this.conf.setBoolean(CacheConfig.BUCKET_CACHE_COMBINED_KEY, false);
+    CacheConfig cc = new CacheConfig(this.conf);
+    basicBlockCacheOps(cc, false, false);
+    assertTrue(cc.getBlockCache() instanceof LruBlockCache);
+    // TODO: Assert sizes allocated are right and proportions.
+    LruBlockCache lbc = (LruBlockCache)cc.getBlockCache();
+    assertEquals(lruExpectedSize, lbc.getMaxSize());
+    BucketCache bc = lbc.getVictimHandler();
+    // getMaxSize comes back in bytes but we specified size in MB
+    assertEquals(bcExpectedSize, bc.getMaxSize());
+    // Test the L1+L2 deploy works as we'd expect with blocks evicted from L1 going to L2.
+    long initialL1BlockCount = lbc.getBlockCount();
+    long initialL2BlockCount = bc.getBlockCount();
+    Cacheable c = new DataCacheEntry();
+    BlockCacheKey bck = new BlockCacheKey("bck", 0);
+    lbc.cacheBlock(bck, c, false, false);
+    assertEquals(initialL1BlockCount + 1, lbc.getBlockCount());
+    assertEquals(initialL2BlockCount, bc.getBlockCount());
+    // Force evictions by putting in a block too big.
+    final long justTooBigSize = lbc.acceptableSize() + 1;
+    lbc.cacheBlock(new BlockCacheKey("bck2", 0), new DataCacheEntry() {
+      @Override
+      public long heapSize() {
+        return justTooBigSize;
+      }
+
+      @Override
+      public int getSerializedLength() {
+        return (int)heapSize();
+      }
+    });
+    // The eviction thread in lrublockcache needs to run.
+    while (initialL1BlockCount != lbc.getBlockCount()) Threads.sleep(10);
+    assertEquals(initialL1BlockCount, lbc.getBlockCount());
+    long count = bc.getBlockCount();
+    assertTrue(initialL2BlockCount + 1 <= count);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/14b331cc/src/main/docbkx/book.xml
----------------------------------------------------------------------
diff --git a/src/main/docbkx/book.xml b/src/main/docbkx/book.xml
index 7e58c1c..82df25a 100644
--- a/src/main/docbkx/book.xml
+++ b/src/main/docbkx/book.xml
@@ -2159,7 +2159,7 @@ rs.close();
             <para>Restart (or rolling restart) your cluster for the configurations to take effect.
               Check logs for errors or unexpected behavior.</para>
           </section>
-          <section>
+          <section xml:id="enable.bucketcache">
             <title>Enable BucketCache</title>
                 <para>The usual deploy of BucketCache is via a
                     managing class that sets up two caching tiers: an L1 onheap cache
@@ -2172,8 +2172,7 @@ rs.close();
             setting <varname>cacheDataInL1</varname> via <programlisting>(HColumnDescriptor.setCacheDataInL1(true)</programlisting>
             or in the shell, creating or amending column families setting <varname>CACHE_DATA_IN_L1</varname>
             to true: e.g. <programlisting>hbase(main):003:0> create 't', {NAME => 't', CONFIGURATION => {CACHE_DATA_IN_L1 => 'true'}}</programlisting></para>
-        <para>The BucketCache deploy can be
-            onheap, offheap, or file based. You set which via the
+        <para>The BucketCache deploy can be onheap, offheap, or file based. You set which via the
             <varname>hbase.bucketcache.ioengine</varname> setting it to
             <varname>heap</varname> for BucketCache running as part of the java heap,
             <varname>offheap</varname> for BucketCache to make allocations offheap,
@@ -2191,6 +2190,8 @@ rs.close();
                   onheap cache. Configuration is performed on the RegionServer.  Setting
                   <varname>hbase.bucketcache.ioengine</varname> and 
                   <varname>hbase.bucketcache.size</varname> &gt; 0 enables CombinedBlockCache.
+                  Let us presume that the RegionServer has been set to run with a 5G heap:
+                  i.e. HBASE_HEAPSIZE=5g.
               </para>
               <step>
                 <para>First, edit the RegionServer's <filename>hbase-env.sh</filename> and set
@@ -2210,12 +2211,12 @@ rs.close();
   <value>offheap</value>
 </property>
 <property>
-  <name>hbase.bucketcache.percentage.in.combinedcache</name>
-  <value>0.8</value>
+  <name>hfile.block.cache.size</name>
+  <value>0.2</value>
 </property>
 <property>
   <name>hbase.bucketcache.size</name>
-  <value>5120</value>
+  <value>4196</value>
 </property>]]>
           </programlisting>
               </step>
@@ -2224,6 +2225,11 @@ rs.close();
                   issues.</para>
               </step>
             </procedure>
+            <para>In the above, we set bucketcache to be 4G.  The onheap lrublockcache we
+                configured to have 0.2 of the RegionServer's heap size (0.2 * 5G = 1G).
+                In other words, you configure the L1 LruBlockCache as you would normally,
+                as you would when there is no L2 BucketCache present.
+            </para>
           </section>
         </section>
       </section>