You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2017/11/11 07:09:30 UTC

hbase git commit: HBASE-19187 Remove option to create on heap bucket cache.

Repository: hbase
Updated Branches:
  refs/heads/master 72270866c -> bff619ef7


HBASE-19187 Remove option to create on heap bucket cache.


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

Branch: refs/heads/master
Commit: bff619ef7b100e8b09f7f5eb0f6e289ca51de096
Parents: 7227086
Author: anoopsamjohn <an...@gmail.com>
Authored: Sat Nov 11 12:37:18 2017 +0530
Committer: anoopsamjohn <an...@gmail.com>
Committed: Sat Nov 11 12:37:18 2017 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/util/ByteBufferAllocator.java  |  3 +-
 .../hadoop/hbase/util/ByteBufferArray.java      | 18 +++----
 .../src/main/resources/hbase-default.xml        |  7 +--
 .../hadoop/hbase/util/TestByteBufferArray.java  | 38 +++++---------
 .../hbase/io/hfile/bucket/BucketCache.java      |  6 +--
 .../io/hfile/bucket/ByteBufferIOEngine.java     | 18 ++-----
 .../hbase/io/hfile/bucket/FileMmapEngine.java   | 14 ++----
 .../hadoop/hbase/io/util/MemorySizeUtil.java    | 44 +++-------------
 .../hbase/regionserver/HeapMemoryManager.java   |  8 ++-
 ...estAvoidCellReferencesIntoShippedBlocks.java |  2 +-
 .../client/TestBlockEvictionFromClient.java     |  2 +-
 .../hadoop/hbase/io/hfile/TestCacheConfig.java  |  6 ---
 .../io/hfile/TestScannerFromBucketCache.java    | 53 +++-----------------
 .../hbase/io/hfile/bucket/TestBucketCache.java  |  4 +-
 .../io/hfile/bucket/TestBucketWriterThread.java |  3 +-
 .../io/hfile/bucket/TestByteBufferIOEngine.java |  4 +-
 .../regionserver/TestHeapMemoryManager.java     | 52 -------------------
 17 files changed, 57 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferAllocator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferAllocator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferAllocator.java
index 8e2ac16..2c5eac8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferAllocator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferAllocator.java
@@ -31,9 +31,8 @@ public interface ByteBufferAllocator {
   /**
    * Allocates a bytebuffer
    * @param size the size of the bytebuffer
-   * @param directByteBuffer indicator to create a direct bytebuffer
    * @return the bytebuffer that is created
    * @throws IOException exception thrown if there is an error while creating the ByteBuffer
    */
-  ByteBuffer allocate(long size, boolean directByteBuffer) throws IOException;
+  ByteBuffer allocate(long size) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
index d1d5e7d..2715740 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
@@ -59,11 +59,10 @@ public class ByteBufferArray {
    * of the array bounds for the last byte(see {@link ByteBufferArray#multiple}),
    * we will allocate one additional buffer with capacity 0;
    * @param capacity total size of the byte buffer array
-   * @param directByteBuffer true if we allocate direct buffer
    * @param allocator the ByteBufferAllocator that will create the buffers
    * @throws IOException throws IOException if there is an exception thrown by the allocator
    */
-  public ByteBufferArray(long capacity, boolean directByteBuffer, ByteBufferAllocator allocator)
+  public ByteBufferArray(long capacity, ByteBufferAllocator allocator)
       throws IOException {
     this.bufferSize = DEFAULT_BUFFER_SIZE;
     if (this.bufferSize > (capacity / 16))
@@ -71,13 +70,13 @@ public class ByteBufferArray {
     this.bufferCount = (int) (roundUp(capacity, bufferSize) / bufferSize);
     LOG.info("Allocating buffers total=" + StringUtils.byteDesc(capacity)
         + ", sizePerBuffer=" + StringUtils.byteDesc(bufferSize) + ", count="
-        + bufferCount + ", direct=" + directByteBuffer);
+        + bufferCount);
     buffers = new ByteBuffer[bufferCount + 1];
-    createBuffers(directByteBuffer, allocator);
+    createBuffers(allocator);
   }
 
   @VisibleForTesting
-  void createBuffers(boolean directByteBuffer, ByteBufferAllocator allocator)
+  void createBuffers(ByteBufferAllocator allocator)
       throws IOException {
     int threadCount = getThreadCount();
     ExecutorService service = new ThreadPoolExecutor(threadCount, threadCount, 0L,
@@ -90,7 +89,7 @@ public class ByteBufferArray {
         // Last thread will have to deal with a different number of buffers
         int buffersToCreate = (i == threadCount - 1) ? lastThreadCount : perThreadCount;
         futures[i] = service.submit(
-          new BufferCreatorCallable(bufferSize, directByteBuffer, buffersToCreate, allocator));
+          new BufferCreatorCallable(bufferSize, buffersToCreate, allocator));
       }
       int bufferIndex = 0;
       for (Future<ByteBuffer[]> future : futures) {
@@ -122,14 +121,11 @@ public class ByteBufferArray {
    */
   private static class BufferCreatorCallable implements Callable<ByteBuffer[]> {
     private final int bufferCapacity;
-    private final boolean directByteBuffer;
     private final int bufferCount;
     private final ByteBufferAllocator allocator;
 
-    BufferCreatorCallable(int bufferCapacity, boolean directByteBuffer, int bufferCount,
-        ByteBufferAllocator allocator) {
+    BufferCreatorCallable(int bufferCapacity, int bufferCount, ByteBufferAllocator allocator) {
       this.bufferCapacity = bufferCapacity;
-      this.directByteBuffer = directByteBuffer;
       this.bufferCount = bufferCount;
       this.allocator = allocator;
     }
@@ -138,7 +134,7 @@ public class ByteBufferArray {
     public ByteBuffer[] call() throws Exception {
       ByteBuffer[] buffers = new ByteBuffer[this.bufferCount];
       for (int i = 0; i < this.bufferCount; i++) {
-        buffers[i] = allocator.allocate(this.bufferCapacity, this.directByteBuffer);
+        buffers[i] = allocator.allocate(this.bufferCapacity);
       }
       return buffers;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 28e9b4d..be23dc8 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -894,9 +894,10 @@ possible configurations would overwhelm and obscure the important.
     <property>
     <name>hbase.bucketcache.ioengine</name>
     <value></value>
-    <description>Where to store the contents of the bucketcache. One of: heap,
-    offheap, or file. If a file, set it to file:PATH_TO_FILE. See
-    http://hbase.apache.org/book.html#offheap.blockcache for more information.
+    <description>Where to store the contents of the bucketcache. One of: offheap,
+    file, files or mmap. If a file or files, set it to file(s):PATH_TO_FILE.
+    mmap means the content will be in an mmaped file. Use mmap:PATH_TO_FILE.
+    See http://hbase.apache.org/book.html#offheap.blockcache for more information.
     </description>
   </property>
   <property>

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
index 7077643..4b87945 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestByteBufferArray.java
@@ -38,16 +38,11 @@ public class TestByteBufferArray {
     int capacity = 4 * 1024 * 1024;
     ByteBufferAllocator allocator = new ByteBufferAllocator() {
       @Override
-      public ByteBuffer allocate(long size, boolean directByteBuffer)
-          throws IOException {
-        if (directByteBuffer) {
-          return ByteBuffer.allocateDirect((int) size);
-        } else {
-          return ByteBuffer.allocate((int) size);
-        }
+      public ByteBuffer allocate(long size) throws IOException {
+        return ByteBuffer.allocateDirect((int) size);
       }
     };
-    ByteBufferArray array = new ByteBufferArray(capacity, false, allocator);
+    ByteBufferArray array = new ByteBufferArray(capacity, allocator);
     ByteBuff subBuf = array.asSubByteBuff(0, capacity);
     subBuf.position(capacity - 1);// Position to the last byte
     assertTrue(subBuf.hasRemaining());
@@ -61,15 +56,11 @@ public class TestByteBufferArray {
     int capacity = 470 * 1021 * 1023;
     ByteBufferAllocator allocator = new ByteBufferAllocator() {
       @Override
-      public ByteBuffer allocate(long size, boolean directByteBuffer) throws IOException {
-        if (directByteBuffer) {
-          return ByteBuffer.allocateDirect((int) size);
-        } else {
-          return ByteBuffer.allocate((int) size);
-        }
+      public ByteBuffer allocate(long size) throws IOException {
+        return ByteBuffer.allocateDirect((int) size);
       }
     };
-    ByteBufferArray array = new ByteBufferArray(capacity, false, allocator);
+    ByteBufferArray array = new ByteBufferArray(capacity, allocator);
     assertEquals(119, array.buffers.length);
     for (int i = 0; i < array.buffers.length; i++) {
       if (i == array.buffers.length - 1) {
@@ -84,19 +75,15 @@ public class TestByteBufferArray {
   public void testByteBufferCreation1() throws Exception {
     ByteBufferAllocator allocator = new ByteBufferAllocator() {
       @Override
-      public ByteBuffer allocate(long size, boolean directByteBuffer) throws IOException {
-        if (directByteBuffer) {
-          return ByteBuffer.allocateDirect((int) size);
-        } else {
-          return ByteBuffer.allocate((int) size);
-        }
+      public ByteBuffer allocate(long size) throws IOException {
+        return ByteBuffer.allocateDirect((int) size);
       }
     };
-    ByteBufferArray array = new DummyByteBufferArray(7 * 1024 * 1024, false, allocator);
+    ByteBufferArray array = new DummyByteBufferArray(7 * 1024 * 1024, allocator);
     // overwrite
     array.bufferCount = 25;
     array.buffers = new ByteBuffer[array.bufferCount + 1];
-    array.createBuffers(true, allocator);
+    array.createBuffers(allocator);
     for (int i = 0; i < array.buffers.length; i++) {
       if (i == array.buffers.length - 1) {
         assertEquals(array.buffers[i].capacity(), 0);
@@ -108,9 +95,8 @@ public class TestByteBufferArray {
 
   private static class DummyByteBufferArray extends ByteBufferArray {
 
-    public DummyByteBufferArray(long capacity, boolean directByteBuffer,
-        ByteBufferAllocator allocator) throws IOException {
-      super(capacity, directByteBuffer, allocator);
+    public DummyByteBufferArray(long capacity, ByteBufferAllocator allocator) throws IOException {
+      super(capacity, allocator);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index 6f95b6f..b0011d7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -381,14 +381,12 @@ public class BucketCache implements BlockCache, HeapSize {
           .split(FileIOEngine.FILE_DELIMITER);
       return new FileIOEngine(capacity, persistencePath != null, filePaths);
     } else if (ioEngineName.startsWith("offheap")) {
-      return new ByteBufferIOEngine(capacity, true);
-    } else if (ioEngineName.startsWith("heap")) {
-      return new ByteBufferIOEngine(capacity, false);
+      return new ByteBufferIOEngine(capacity);
     } else if (ioEngineName.startsWith("mmap:")) {
       return new FileMmapEngine(ioEngineName.substring(5), capacity);
     } else {
       throw new IllegalArgumentException(
-          "Don't understand io engine name for cache - prefix with file:, heap or offheap");
+          "Don't understand io engine name for cache- prefix with file:, files:, mmap: or offheap");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
index 4fefa97..9f4ffba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/ByteBufferIOEngine.java
@@ -68,36 +68,28 @@ So said all these, when we read a block it may be possible that the bytes of tha
 public class ByteBufferIOEngine implements IOEngine {
   private ByteBufferArray bufferArray;
   private final long capacity;
-  private final boolean direct;
 
   /**
    * Construct the ByteBufferIOEngine with the given capacity
    * @param capacity
-   * @param direct true if allocate direct buffer
    * @throws IOException ideally here no exception to be thrown from the allocator
    */
-  public ByteBufferIOEngine(long capacity, boolean direct)
+  public ByteBufferIOEngine(long capacity)
       throws IOException {
     this.capacity = capacity;
-    this.direct = direct;
     ByteBufferAllocator allocator = new ByteBufferAllocator() {
       @Override
-      public ByteBuffer allocate(long size, boolean directByteBuffer)
-          throws IOException {
-        if (directByteBuffer) {
-          return ByteBuffer.allocateDirect((int) size);
-        } else {
-          return ByteBuffer.allocate((int) size);
-        }
+      public ByteBuffer allocate(long size) throws IOException {
+        return ByteBuffer.allocateDirect((int) size);
       }
     };
-    bufferArray = new ByteBufferArray(capacity, direct, allocator);
+    bufferArray = new ByteBufferArray(capacity, allocator);
   }
 
   @Override
   public String toString() {
     return "ioengine=" + this.getClass().getSimpleName() + ", capacity=" +
-      String.format("%,d", this.capacity) + ", direct=" + this.direct;
+      String.format("%,d", this.capacity);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java
index 45036e5..4fe39d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileMmapEngine.java
@@ -71,20 +71,14 @@ public class FileMmapEngine implements IOEngine {
     ByteBufferAllocator allocator = new ByteBufferAllocator() {
       int pos = 0;
       @Override
-      public ByteBuffer allocate(long size, boolean directByteBuffer) throws IOException {
-        ByteBuffer buffer = null;
-        if (directByteBuffer) {
-          buffer = fileChannel.map(java.nio.channels.FileChannel.MapMode.READ_WRITE, pos * size,
-              size);
-        } else {
-          throw new IllegalArgumentException(
-              "Only Direct Bytebuffers allowed with FileMMap engine");
-        }
+      public ByteBuffer allocate(long size) throws IOException {
+        ByteBuffer buffer = fileChannel.map(java.nio.channels.FileChannel.MapMode.READ_WRITE,
+            pos * size, size);
         pos++;
         return buffer;
       }
     };
-    bufferArray = new ByteBufferArray(fileSize, true, allocator);
+    bufferArray = new ByteBufferArray(fileSize, allocator);
   }
 
   private long roundUp(long n, long to) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java
index dbfb295..3689643 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java
@@ -205,29 +205,7 @@ public class MemorySizeUtil {
     // L1 block cache is always on heap
     float l1CachePercent = conf.getFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY,
         HConstants.HFILE_BLOCK_CACHE_SIZE_DEFAULT);
-    float l2CachePercent = getL2BlockCacheHeapPercent(conf);
-    return l1CachePercent + l2CachePercent;
-  }
-
-  /**
-   * @param conf
-   * @return The on heap size for L2 block cache.
-   */
-  public static float getL2BlockCacheHeapPercent(Configuration conf) {
-    float l2CachePercent = 0.0F;
-    String bucketCacheIOEngineName = conf.get(HConstants.BUCKET_CACHE_IOENGINE_KEY, null);
-    // L2 block cache can be on heap when IOEngine is "heap"
-    if (bucketCacheIOEngineName != null && bucketCacheIOEngineName.startsWith("heap")) {
-      float bucketCachePercentage = conf.getFloat(HConstants.BUCKET_CACHE_SIZE_KEY, 0F);
-      long max = -1L;
-      final MemoryUsage usage = safeGetHeapMemoryUsage();
-      if (usage != null) {
-        max = usage.getMax();
-      }
-      l2CachePercent = bucketCachePercentage < 1 ? bucketCachePercentage
-          : (bucketCachePercentage * 1024 * 1024) / max;
-    }
-    return l2CachePercent;
+    return l1CachePercent;
   }
 
   /**
@@ -260,21 +238,13 @@ public class MemorySizeUtil {
    * @return the number of bytes to use for bucket cache, negative if disabled.
    */
   public static long getBucketCacheSize(final Configuration conf) {
-    final float bucketCachePercentage = conf.getFloat(HConstants.BUCKET_CACHE_SIZE_KEY, 0F);
-    long bucketCacheSize;
-    // Values < 1 are treated as % of heap
-    if (bucketCachePercentage < 1) {
-      long max = -1L;
-      final MemoryUsage usage = safeGetHeapMemoryUsage();
-      if (usage != null) {
-        max = usage.getMax();
-      }
-      bucketCacheSize = (long)(max * bucketCachePercentage);
-    // values >= 1 are treated as # of MiB
-    } else {
-      bucketCacheSize = (long)(bucketCachePercentage * 1024 * 1024);
+    // Size configured in MBs
+    float bucketCacheSize = conf.getFloat(HConstants.BUCKET_CACHE_SIZE_KEY, 0F);
+    if (bucketCacheSize < 1) {
+      throw new IllegalArgumentException("Bucket Cache should be minimum 1 MB in size."
+          + "Configure 'hbase.bucketcache.size' with > 1 value");
     }
-    return bucketCacheSize;
+    return (long) (bucketCacheSize * 1024 * 1024);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
index 14021ff..b64937e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
@@ -73,7 +73,6 @@ public class HeapMemoryManager {
   private float blockCachePercent;
   private float blockCachePercentMinRange;
   private float blockCachePercentMaxRange;
-  private float l2BlockCachePercent;
 
   private float heapOccupancyPercent;
 
@@ -183,8 +182,7 @@ public class HeapMemoryManager {
     }
 
     int gml = (int) (globalMemStorePercentMaxRange * CONVERT_TO_PERCENTAGE);
-    this.l2BlockCachePercent = MemorySizeUtil.getL2BlockCacheHeapPercent(conf);
-    int bcul = (int) ((blockCachePercentMinRange + l2BlockCachePercent) * CONVERT_TO_PERCENTAGE);
+    int bcul = (int) ((blockCachePercentMinRange) * CONVERT_TO_PERCENTAGE);
     if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) {
       throw new RuntimeException("Current heap configuration for MemStore and BlockCache exceeds "
           + "the threshold required for successful cluster operation. "
@@ -195,7 +193,7 @@ public class HeapMemoryManager {
           + blockCachePercentMinRange);
     }
     gml = (int) (globalMemStorePercentMinRange * CONVERT_TO_PERCENTAGE);
-    bcul = (int) ((blockCachePercentMaxRange + l2BlockCachePercent) * CONVERT_TO_PERCENTAGE);
+    bcul = (int) ((blockCachePercentMaxRange) * CONVERT_TO_PERCENTAGE);
     if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) {
       throw new RuntimeException("Current heap configuration for MemStore and BlockCache exceeds "
           + "the threshold required for successful cluster operation. "
@@ -361,7 +359,7 @@ public class HeapMemoryManager {
           blockCacheSize = blockCachePercentMaxRange;
         }
         int gml = (int) (memstoreSize * CONVERT_TO_PERCENTAGE);
-        int bcul = (int) ((blockCacheSize + l2BlockCachePercent) * CONVERT_TO_PERCENTAGE);
+        int bcul = (int) ((blockCacheSize) * CONVERT_TO_PERCENTAGE);
         if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) {
           LOG.info("Current heap configuration from HeapMemoryTuner exceeds "
               + "the threshold required for successful cluster operation. "

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
index baf0145..7ad4e78 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
@@ -101,7 +101,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
                                                         // tests
     conf.setInt("hbase.regionserver.handler.count", 20);
     conf.setInt("hbase.bucketcache.size", 400);
-    conf.setStrings("hbase.bucketcache.ioengine", "offheap");
+    conf.setStrings(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
     conf.setInt("hbase.hstore.compactionThreshold", 7);
     conf.setFloat("hfile.block.cache.size", 0.2f);
     conf.setFloat("hbase.regionserver.global.memstore.size", 0.1f);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index a1cbb04..d6f9d71 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -110,7 +110,7 @@ public class TestBlockEvictionFromClient {
                                                         // tests
     conf.setInt("hbase.regionserver.handler.count", 20);
     conf.setInt("hbase.bucketcache.size", 400);
-    conf.setStrings("hbase.bucketcache.ioengine", "offheap");
+    conf.setStrings(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
     conf.setFloat("hfile.block.cache.size", 0.2f);
     conf.setFloat("hbase.regionserver.global.memstore.size", 0.1f);
     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);// do not retry

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/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 e16e51e..710d408 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
@@ -300,12 +300,6 @@ public class TestCacheConfig {
   }
 
   @Test
-  public void testOnHeapBucketCacheConfig() {
-    this.conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "heap");
-    doBucketCacheConfigTest();
-  }
-
-  @Test
   public void testFileBucketCacheConfig() throws IOException {
     HBaseTestingUtility htu = new HBaseTestingUtility(this.conf);
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
index c6e26cb..7e2a0a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.KeyValue;
@@ -71,16 +72,12 @@ public class TestScannerFromBucketCache {
   // Test names
   private TableName tableName;
 
-  private void setUp(boolean offheap, boolean useBucketCache) throws IOException {
+  private void setUp(boolean useBucketCache) throws IOException {
     test_util = HBaseTestingUtility.createLocalHTU();
     conf = test_util.getConfiguration();
     if (useBucketCache) {
       conf.setInt("hbase.bucketcache.size", 400);
-      if (offheap) {
-        conf.setStrings("hbase.bucketcache.ioengine", "offheap");
-      } else {
-        conf.setStrings("hbase.bucketcache.ioengine", "heap");
-      }
+      conf.setStrings(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
       conf.setInt("hbase.bucketcache.writer.threads", 10);
       conf.setFloat("hfile.block.cache.size", 0.2f);
       conf.setFloat("hbase.regionserver.global.memstore.size", 0.1f);
@@ -102,7 +99,7 @@ public class TestScannerFromBucketCache {
 
   @Test
   public void testBasicScanWithLRUCache() throws IOException {
-    setUp(false, false);
+    setUp(false);
     byte[] row1 = Bytes.toBytes("row1");
     byte[] qf1 = Bytes.toBytes("qualifier1");
     byte[] qf2 = Bytes.toBytes("qualifier2");
@@ -140,7 +137,7 @@ public class TestScannerFromBucketCache {
 
   @Test
   public void testBasicScanWithOffheapBucketCache() throws IOException {
-    setUp(true, true);
+    setUp(true);
     byte[] row1 = Bytes.toBytes("row1offheap");
     byte[] qf1 = Bytes.toBytes("qualifier1");
     byte[] qf2 = Bytes.toBytes("qualifier2");
@@ -181,7 +178,7 @@ public class TestScannerFromBucketCache {
 
   @Test
   public void testBasicScanWithOffheapBucketCacheWithMBB() throws IOException {
-    setUp(true, true);
+    setUp(true);
     byte[] row1 = Bytes.toBytes("row1offheap");
     byte[] qf1 = Bytes.toBytes("qualifier1");
     byte[] qf2 = Bytes.toBytes("qualifier2");
@@ -231,44 +228,6 @@ public class TestScannerFromBucketCache {
     }
   }
 
-  @Test
-  public void testBasicScanWithOnheapBucketCache() throws IOException {
-    setUp(false, true);
-    byte[] row1 = Bytes.toBytes("row1onheap");
-    byte[] qf1 = Bytes.toBytes("qualifier1");
-    byte[] qf2 = Bytes.toBytes("qualifier2");
-    byte[] fam1 = Bytes.toBytes("famonheap");
-
-    long ts1 = 1; // System.currentTimeMillis();
-    long ts2 = ts1 + 1;
-    long ts3 = ts1 + 2;
-
-    // Setting up region
-    String method = this.getName();
-    this.region = initHRegion(tableName, method, conf, test_util, fam1);
-    try {
-      List<Cell> expected = insertData(row1, qf1, qf2, fam1, ts1, ts2, ts3, false);
-
-      List<Cell> actual = performScan(row1, fam1);
-      // Verify result
-      for (int i = 0; i < expected.size(); i++) {
-        assertFalse(actual.get(i) instanceof ByteBufferKeyValue);
-        assertTrue(PrivateCellUtil.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
-      }
-      // do the scan again and verify. This time it should be from the bucket cache in onheap mode
-      actual = performScan(row1, fam1);
-      // Verify result
-      for (int i = 0; i < expected.size(); i++) {
-        assertFalse(actual.get(i) instanceof ByteBufferKeyValue);
-        assertTrue(PrivateCellUtil.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
-      }
-
-    } finally {
-      HBaseTestingUtility.closeRegionAndWAL(this.region);
-      this.region = null;
-    }
-  }
-
   private List<Cell> insertData(byte[] row1, byte[] qf1, byte[] qf2, byte[] fam1, long ts1,
       long ts2, long ts3, boolean withVal) throws IOException {
     // Putting data in Region

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
index 5a2a51c..1c7f951 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
@@ -20,14 +20,12 @@ package org.apache.hadoop.hbase.io.hfile.bucket;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -92,7 +90,7 @@ public class TestBucketCache {
   final long capacitySize = 32 * 1024 * 1024;
   final int writeThreads = BucketCache.DEFAULT_WRITER_THREADS;
   final int writerQLen = BucketCache.DEFAULT_WRITER_QUEUE_ITEMS;
-  String ioEngineName = "heap";
+  String ioEngineName = "offheap";
   String persistencePath = null;
 
   private class MockedBucketCache extends BucketCache {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java
index e789b4f..b8949ed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketWriterThread.java
@@ -36,7 +36,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
 
 import static org.hamcrest.CoreMatchers.is;
@@ -80,7 +79,7 @@ public class TestBucketWriterThread {
     // Run with one writer thread only. Means there will be one writer queue only too.  We depend
     // on this in below.
     final int writerThreadsCount = 1;
-    this.bc = new MockBucketCache("heap", capacity, 1, new int [] {1}, writerThreadsCount,
+    this.bc = new MockBucketCache("offheap", capacity, 1, new int [] {1}, writerThreadsCount,
       capacity, null, 100/*Tolerate ioerrors for 100ms*/);
     assertEquals(writerThreadsCount, bc.writerThreads.length);
     assertEquals(writerThreadsCount, bc.writerQueues.size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java
index bf15a59..ab2276a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestByteBufferIOEngine.java
@@ -43,7 +43,7 @@ public class TestByteBufferIOEngine {
     int capacity = 32 * 1024 * 1024; // 32 MB
     int testNum = 100;
     int maxBlockSize = 64 * 1024;
-    ByteBufferIOEngine ioEngine = new ByteBufferIOEngine(capacity, false);
+    ByteBufferIOEngine ioEngine = new ByteBufferIOEngine(capacity);
     int testOffsetAtStartNum = testNum / 10;
     int testOffsetAtEndNum = testNum / 10;
     for (int i = 0; i < testNum; i++) {
@@ -113,7 +113,7 @@ public class TestByteBufferIOEngine {
     int capacity = 32 * 1024 * 1024; // 32 MB
     int testNum = 100;
     int maxBlockSize = 64 * 1024;
-    ByteBufferIOEngine ioEngine = new ByteBufferIOEngine(capacity, false);
+    ByteBufferIOEngine ioEngine = new ByteBufferIOEngine(capacity);
     int testOffsetAtStartNum = testNum / 10;
     int testOffsetAtEndNum = testNum / 10;
     for (int i = 0; i < testNum; i++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bff619ef/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index aae04df..b16feca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -602,58 +602,6 @@ public class TestHeapMemoryManager {
     assertEquals(oldBlockCacheSize, blockCache.maxSize);
   }
 
-  @Test
-  public void testWhenL2BlockCacheIsOnHeap() throws Exception {
-    HeapMemoryManager heapMemoryManager = null;
-    BlockCacheStub blockCache = new BlockCacheStub((long) (maxHeapSize * 0.4));
-    MemstoreFlusherStub memStoreFlusher = new MemstoreFlusherStub((long) (maxHeapSize * 0.3));
-    Configuration conf = HBaseConfiguration.create();
-    conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.7f);
-    conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.1f);
-    conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f);
-    conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MIN_RANGE_KEY, 0.1f);
-    conf.setInt(DefaultHeapMemoryTuner.NUM_PERIODS_TO_IGNORE, 0);
-    conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.4F);
-    conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.3F);
-    conf.setFloat(HConstants.BUCKET_CACHE_SIZE_KEY, 0.1F);
-    conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "heap");
-
-    conf.setLong(HeapMemoryManager.HBASE_RS_HEAP_MEMORY_TUNER_PERIOD, 1000);
-    conf.setClass(HeapMemoryManager.HBASE_RS_HEAP_MEMORY_TUNER_CLASS, CustomHeapMemoryTuner.class,
-        HeapMemoryTuner.class);
-
-    try {
-      heapMemoryManager = new HeapMemoryManager(blockCache, memStoreFlusher,
-          new RegionServerStub(conf), new RegionServerAccountingStub(conf));
-      fail("Should have failed as the collective heap memory need is above 80%");
-    } catch (Exception e) {
-    }
-
-    // Change the max/min ranges for memstore and bock cache so as to pass the criteria check
-    conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.6f);
-    conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.6f);
-    heapMemoryManager = new HeapMemoryManager(blockCache, memStoreFlusher,
-        new RegionServerStub(conf), new RegionServerAccountingStub(conf));
-    long oldMemstoreSize = memStoreFlusher.memstoreSize;
-    long oldBlockCacheSize = blockCache.maxSize;
-    final ChoreService choreService = new ChoreService("TEST_SERVER_NAME");
-    heapMemoryManager.start(choreService);
-    CustomHeapMemoryTuner.memstoreSize = 0.4f;
-    CustomHeapMemoryTuner.blockCacheSize = 0.4f;
-    // Allow the tuner to run once and do necessary memory up
-    Thread.sleep(1500);
-    // The size should not get changes as the collection of memstore size and L1 and L2 block cache
-    // size will cross the ax allowed 80% mark
-    assertEquals(oldMemstoreSize, memStoreFlusher.memstoreSize);
-    assertEquals(oldBlockCacheSize, blockCache.maxSize);
-    CustomHeapMemoryTuner.memstoreSize = 0.1f;
-    CustomHeapMemoryTuner.blockCacheSize = 0.5f;
-    // Allow the tuner to run once and do necessary memory up
-    waitForTune(memStoreFlusher, memStoreFlusher.memstoreSize);
-    assertHeapSpace(0.1f, memStoreFlusher.memstoreSize);
-    assertHeapSpace(0.5f, blockCache.maxSize);
-  }
-
   private void assertHeapSpace(float expectedHeapPercentage, long currentHeapSpace) {
     long expected = (long) (this.maxHeapSize * expectedHeapPercentage);
     assertEquals(expected, currentHeapSpace);