You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by op...@apache.org on 2019/06/21 06:14:50 UTC

[hbase] branch HBASE-21879 updated: HBASE-22547 Align the config keys and add document for offheap read in HBase Book. (#301)

This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/HBASE-21879 by this push:
     new ff7e40d  HBASE-22547 Align the config keys and add document for offheap read in HBase Book. (#301)
ff7e40d is described below

commit ff7e40dd2c1fd90539e5daa296b1b6d9a7a379f3
Author: openinx <op...@gmail.com>
AuthorDate: Fri Jun 21 14:14:44 2019 +0800

    HBASE-22547 Align the config keys and add document for offheap read in HBase Book. (#301)
---
 .../apache/hadoop/hbase/io/ByteBuffAllocator.java  |  19 ++-
 .../hadoop/hbase/io/TestByteBuffAllocator.java     |  10 ++
 .../hadoop/hbase/master/MasterRpcServices.java     |   5 +-
 .../hadoop/hbase/regionserver/RSRpcServices.java   |   5 +-
 .../hbase/mob/TestMobWithByteBuffAllocator.java    |   4 +-
 src/main/asciidoc/_chapters/architecture.adoc      |  48 ------
 .../asciidoc/_chapters/offheap_read_write.adoc     | 186 +++++++++++++++++++++
 src/main/asciidoc/book.adoc                        |   1 +
 .../resources/images/bytebuff-allocator-stats.png  | Bin 0 -> 64120 bytes
 src/site/resources/images/offheap-overview.png     | Bin 0 -> 64687 bytes
 10 files changed, 219 insertions(+), 59 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBuffAllocator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBuffAllocator.java
index 5c2c8ff..9991e79 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBuffAllocator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBuffAllocator.java
@@ -68,10 +68,21 @@ public class ByteBuffAllocator {
   // default heap allocator, it will just allocate ByteBuffers from heap but wrapped by an ByteBuff.
   public static final ByteBuffAllocator HEAP = ByteBuffAllocator.createOnHeap();
 
+  public static final String ALLOCATOR_POOL_ENABLED_KEY = "hbase.server.allocator.pool.enabled";
+
   public static final String MAX_BUFFER_COUNT_KEY = "hbase.server.allocator.max.buffer.count";
 
   public static final String BUFFER_SIZE_KEY = "hbase.server.allocator.buffer.size";
 
+  public static final String MIN_ALLOCATE_SIZE_KEY = "hbase.server.allocator.minimal.allocate.size";
+
+  /**
+   * @deprecated use {@link ByteBuffAllocator#ALLOCATOR_POOL_ENABLED_KEY} instead.
+   */
+  @Deprecated
+  public static final String DEPRECATED_ALLOCATOR_POOL_ENABLED_KEY =
+      "hbase.ipc.server.reservoir.enabled";
+
   /**
    * @deprecated use {@link ByteBuffAllocator#MAX_BUFFER_COUNT_KEY} instead.
    */
@@ -88,9 +99,12 @@ public class ByteBuffAllocator {
    * The hbase.ipc.server.reservoir.initial.max and hbase.ipc.server.reservoir.initial.buffer.size
    * were introduced in HBase2.0.0, while in HBase3.0.0 the two config keys will be replaced by
    * {@link ByteBuffAllocator#MAX_BUFFER_COUNT_KEY} and {@link ByteBuffAllocator#BUFFER_SIZE_KEY}.
-   * Keep the two old config keys here for HBase2.x compatibility.
+   * Also the hbase.ipc.server.reservoir.enabled will be replaced by
+   * hbase.server.allocator.pool.enabled. Keep the three old config keys here for HBase2.x
+   * compatibility.
    */
   static {
+    Configuration.addDeprecation(DEPRECATED_ALLOCATOR_POOL_ENABLED_KEY, ALLOCATOR_POOL_ENABLED_KEY);
     Configuration.addDeprecation(DEPRECATED_MAX_BUFFER_COUNT_KEY, MAX_BUFFER_COUNT_KEY);
     Configuration.addDeprecation(DEPRECATED_BUFFER_SIZE_KEY, BUFFER_SIZE_KEY);
   }
@@ -113,9 +127,6 @@ public class ByteBuffAllocator {
    */
   public static final int DEFAULT_BUFFER_SIZE = 65 * 1024;
 
-  public static final String MIN_ALLOCATE_SIZE_KEY =
-      "hbase.ipc.server.reservoir.minimal.allocating.size";
-
   public static final Recycler NONE = () -> {
   };
 
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBuffAllocator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBuffAllocator.java
index 4c88b8f..a0605fc 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBuffAllocator.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestByteBuffAllocator.java
@@ -356,5 +356,15 @@ public class TestByteBuffAllocator {
     allocator = ByteBuffAllocator.create(conf, true);
     Assert.assertEquals(2048, allocator.getBufferSize());
     Assert.assertEquals(11, allocator.getTotalBufferCount());
+
+    conf = new Configuration();
+    conf.setBoolean(ByteBuffAllocator.DEPRECATED_ALLOCATOR_POOL_ENABLED_KEY, false);
+    Assert.assertFalse(conf.getBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, true));
+    conf.setBoolean(ByteBuffAllocator.DEPRECATED_ALLOCATOR_POOL_ENABLED_KEY, true);
+    Assert.assertTrue(conf.getBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, false));
+    conf.setBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, true);
+    Assert.assertTrue(conf.getBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, false));
+    conf.setBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, false);
+    Assert.assertFalse(conf.getBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, true));
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index b772fd7..0f1352c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
+import org.apache.hadoop.hbase.io.ByteBuffAllocator;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.PriorityFunction;
@@ -391,8 +392,8 @@ public class MasterRpcServices extends RSRpcServices
       RpcSchedulerFactory rpcSchedulerFactory, InetSocketAddress bindAddress, String name)
       throws IOException {
     // RpcServer at HM by default enable ByteBufferPool iff HM having user table region in it
-    boolean reservoirEnabled = conf.getBoolean(RESERVOIR_ENABLED_KEY,
-        LoadBalancer.isMasterCanHostUserRegions(conf));
+    boolean reservoirEnabled = conf.getBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY,
+      LoadBalancer.isMasterCanHostUserRegions(conf));
     try {
       return RpcServerFactory.createRpcServer(server, name, getServices(),
           bindAddress, // use final bindAddress for this server.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 2b8bba0..e27735a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -88,6 +88,7 @@ import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
 import org.apache.hadoop.hbase.exceptions.ScannerResetException;
 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
+import org.apache.hadoop.hbase.io.ByteBuffAllocator;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
@@ -291,8 +292,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    */
   static final int BATCH_ROWS_THRESHOLD_DEFAULT = 5000;
 
-  public static final String RESERVOIR_ENABLED_KEY = "hbase.ipc.server.reservoir.enabled";
-
   // Request counter. (Includes requests that are not serviced by regions.)
   // Count only once for requests with multiple actions like multi/caching-scan/replayBatch
   final LongAdder requestCount = new LongAdder();
@@ -1278,7 +1277,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   protected RpcServerInterface createRpcServer(Server server, Configuration conf,
       RpcSchedulerFactory rpcSchedulerFactory, InetSocketAddress bindAddress, String name)
       throws IOException {
-    boolean reservoirEnabled = conf.getBoolean(RESERVOIR_ENABLED_KEY, true);
+    boolean reservoirEnabled = conf.getBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, true);
     try {
       return RpcServerFactory.createRpcServer(server, name, getServices(),
           bindAddress, // use final bindAddress for this server.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobWithByteBuffAllocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobWithByteBuffAllocator.java
index a527740..e84af12 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobWithByteBuffAllocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobWithByteBuffAllocator.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.regionserver.RSRpcServices;
+import org.apache.hadoop.hbase.io.ByteBuffAllocator;
 import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -64,7 +64,7 @@ public class TestMobWithByteBuffAllocator {
   @BeforeClass
   public static void setUp() throws Exception {
     // Must use the ByteBuffAllocator here
-    CONF.setBoolean(RSRpcServices.RESERVOIR_ENABLED_KEY, true);
+    CONF.setBoolean(ByteBuffAllocator.ALLOCATOR_POOL_ENABLED_KEY, true);
     // Must use OFF-HEAP BucketCache here.
     CONF.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.1f);
     CONF.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 2c9aabd..3a94740 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -1005,54 +1005,6 @@ For a RegionServer hosting data that can comfortably fit into cache, or if your
 
 The compressed BlockCache is disabled by default. To enable it, set `hbase.block.data.cachecompressed` to `true` in _hbase-site.xml_ on all RegionServers.
 
-[[regionserver.offheap]]
-=== RegionServer Offheap Read/Write Path
-
-[[regionserver.offheap.readpath]]
-==== Offheap read-path
-In hbase-2.0.0, link:https://issues.apache.org/jira/browse/HBASE-11425[HBASE-11425] changed the HBase read path so it
-could hold the read-data off-heap avoiding copying of cached data on to the java heap.
-This reduces GC pauses given there is less garbage made and so less to clear. The off-heap read path has a performance
-that is similar/better to that of the on-heap LRU cache.  This feature is available since HBase 2.0.0.
-If the BucketCache is in `file` mode, fetching will always be slower compared to the native on-heap LruBlockCache.
-Refer to below blogs for more details and test results on off heaped read path
-link:https://blogs.apache.org/hbase/entry/offheaping_the_read_path_in[Offheaping the Read Path in Apache HBase: Part 1 of 2]
-and link:https://blogs.apache.org/hbase/entry/offheap-read-path-in-production[Offheap Read-Path in Production - The Alibaba story]
-
-For an end-to-end off-heaped read-path, first of all there should be an off-heap backed <<offheap.blockcache>>(BC). Configure 'hbase.bucketcache.ioengine' to off-heap in
-_hbase-site.xml_. Also specify the total capacity of the BC using `hbase.bucketcache.size` config. Please remember to adjust value of 'HBASE_OFFHEAPSIZE' in
-_hbase-env.sh_. This is how we specify the max possible off-heap memory allocation for the
-RegionServer java process. This should be bigger than the off-heap BC size. Please keep in mind that there is no default for `hbase.bucketcache.ioengine`
-which means the BC is turned OFF by default (See <<direct.memory>>). 
-
-Next thing to tune is the ByteBuffer pool on the RPC server side.
-The buffers from this pool will be used to accumulate the cell bytes and create a result cell block to send back to the client side.
-`hbase.ipc.server.reservoir.enabled` can be used to turn this pool ON or OFF. By default this pool is ON and available. HBase will create off heap ByteBuffers
-and pool them. Please make sure not to turn this OFF if you want end-to-end off-heaping in read path.
-If this pool is turned off, the server will create temp buffers on heap to accumulate the cell bytes and make a result cell block. This can impact the GC on a highly read loaded server.
-The user can tune this pool with respect to how many buffers are in the pool and what should be the size of each ByteBuffer. 
-Use the config `hbase.ipc.server.reservoir.initial.buffer.size` to tune each of the buffer sizes. Default is 64 KB. 
-
-When the read pattern is a random row read load and each of the rows are smaller in size compared to this 64 KB, try reducing this.
-When the result size is larger than one ByteBuffer size, the server will try to grab more than one buffer and make a result cell block out of these. When the pool is running out of buffers, the server will end up creating temporary on-heap buffers. 
-
-The maximum number of ByteBuffers in the pool can be tuned using the config 'hbase.ipc.server.reservoir.initial.max'. Its value defaults to 64 * region server handlers configured (See the config 'hbase.regionserver.handler.count'). The math is such that by default we consider 2 MB as the result cell block size per read result and each handler will be handling a read. For 2 MB size, we need 32 buffers each of size 64 KB (See default buffer size in pool). So per handler 32 ByteBuffers(BB). [...]
-
-If you still see GC issues even after making end-to-end read path off-heap, look for issues in the appropriate buffer pool. Check the below RegionServer log with INFO level:
-[source]
-----
-Pool already reached its max capacity : XXX and no free buffers now. Consider increasing the value for 'hbase.ipc.server.reservoir.initial.max' ?
-----
-
-The setting for _HBASE_OFFHEAPSIZE_ in _hbase-env.sh_ should consider this off heap buffer pool at the RPC side also. We need to config this max off heap size for the RegionServer as a bit higher than the sum of this max pool size and the off heap cache size. The TCP layer will also need to create direct bytebuffers for TCP communication. Also the DFS client will need some off-heap to do its workings especially if short-circuit reads are configured. Allocating an extra of 1 - 2 GB for th [...]
-
-If you are using co processors and refer the Cells in the read results, DO NOT store reference to these Cells out of the scope of the CP hook methods. Some times the CPs need store info about the cell (Like its row key) for considering in the next CP hook call etc. For such cases, pls clone the required fields of the entire Cell as per the use cases. [ See CellUtil#cloneXXX(Cell) APIs ]
-
-[[regionserver.offheap.writepath]]
-==== Offheap write-path
-
-TODO
-
 [[regionserver_splitting_implementation]]
 === RegionServer Splitting Implementation
 
diff --git a/src/main/asciidoc/_chapters/offheap_read_write.adoc b/src/main/asciidoc/_chapters/offheap_read_write.adoc
new file mode 100644
index 0000000..dad8ed0
--- /dev/null
+++ b/src/main/asciidoc/_chapters/offheap_read_write.adoc
@@ -0,0 +1,186 @@
+////
+/**
+ *
+ * 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.
+ */
+////
+
+[[offheap_read_write]]
+= RegionServer Offheap Read/Write Path
+:doctype: book
+:numbered:
+:toc: left
+:icons: font
+:experimental:
+
+[[regionserver.offheap.overview]]
+== Overview
+
+For reducing the Java GC impact to P99/P999 RPC latency, HBase 2.x has made the offheap read and write path. The cells are
+allocated from JVM offheap memory area, which won’t be garbage collected by JVM and need to be deallocated explicitly by
+upstream callers. In the write path, the request packet received from client will be allocated offheap and retained
+until those cells are successfully written to the WAL and Memstore. The memory data structure in Memstore does
+not directly store the cell memory, but reference to cells which are encoded in multiple chunks in MSLAB,  this is easier
+to manage the offheap memory. Similarly, in the read path, we’ll try to read the cache firstly, if the cache
+misses, go to the HFile and read the corresponding block. The workflow: from reading blocks to sending cells to
+client,  it's basically not involved in on-heap memory allocations.
+
+image::offheap-overview.png[]
+
+
+[[regionserver.offheap.readpath]]
+== Offheap read-path
+In HBase-2.0.0, link:https://issues.apache.org/jira/browse/HBASE-11425[HBASE-11425] changed the HBase read path so it
+could hold the read-data off-heap (from BucketCache) avoiding copying of cached data on to the java heap.
+This reduces GC pauses given there is less garbage made and so less to clear. The off-heap read path has a performance
+that is similar/better to that of the on-heap LRU cache.  This feature is available since HBase 2.0.0.
+If the BucketCache is in `file` mode, fetching will always be slower compared to the native on-heap LruBlockCache.
+Refer to below blogs for more details and test results on off heaped read path
+link:https://blogs.apache.org/hbase/entry/offheaping_the_read_path_in[Offheaping the Read Path in Apache HBase: Part 1 of 2]
+and link:https://blogs.apache.org/hbase/entry/offheap-read-path-in-production[Offheap Read-Path in Production - The Alibaba story]
+
+For an end-to-end off-heaped read-path, first of all there should be an off-heap backed <<offheap.blockcache, BucketCache>>. Configure 'hbase.bucketcache.ioengine' to off-heap in
+_hbase-site.xml_. Also specify the total capacity of the BucketCache using `hbase.bucketcache.size` config. Please remember to adjust value of 'HBASE_OFFHEAPSIZE' in
+_hbase-env.sh_. This is how we specify the max possible off-heap memory allocation for the RegionServer java process.
+This should be bigger than the off-heap BC size. Please keep in mind that there is no default for `hbase.bucketcache.ioengine`
+which means the BC is turned OFF by default (See <<direct.memory, Direct Memory Usage In HBase>>).
+
+Next thing to tune is the ByteBuffer pool on the RPC server side:
+
+NOTE: the config keys which start with prefix `hbase.ipc.server.reservoir` are deprecated in HBase3.x. If you are still
+in HBase2.x, then just use the old config keys. otherwise if in HBase3.x, please use the new config keys.
+(See <<regionserver.read.hdfs.block.offheap,deprecated and new configs in HBase3.x>>)
+
+The buffers from this pool will be used to accumulate the cell bytes and create a result cell block to send back to the client side.
+`hbase.ipc.server.reservoir.enabled` can be used to turn this pool ON or OFF. By default this pool is ON and available. HBase will create off heap ByteBuffers
+and pool them. Please make sure not to turn this OFF if you want end-to-end off-heaping in read path.
+If this pool is turned off, the server will create temp buffers on heap to accumulate the cell bytes and make a result cell block. This can impact the GC on a highly read loaded server.
+The user can tune this pool with respect to how many buffers are in the pool and what should be the size of each ByteBuffer.
+Use the config `hbase.ipc.server.reservoir.initial.buffer.size` to tune each of the buffer sizes. Default is 64 KB for HBase2.x, while it will be changed to 65KB by default for HBase3.x
+(see link:https://issues.apache.org/jira/browse/HBASE-22532[HBASE-22532])
+
+When the result size is larger than one ByteBuffer size, the server will try to grab more than one ByteBuffer and make a result cell block out of these.
+When the pool is running out of buffers, the server will end up creating temporary on-heap buffers.
+
+The maximum number of ByteBuffers in the pool can be tuned using the config `hbase.ipc.server.reservoir.initial.max`.
+Its value defaults to 64 * region server handlers configured (See the config `hbase.regionserver.handler.count`). The
+math is such that by default we consider 2 MB as the result cell block size per read result and each handler will be
+handling a read. For 2 MB size, we need 32 buffers each of size 64 KB (See default buffer size in pool). So per handler
+32 ByteBuffers(BB). We allocate twice this size as the max BBs count such that one handler can be creating the response
+and handing it to the RPC Responder thread and then handling a new request creating a new response cell block (using
+pooled buffers). Even if the responder could not send back the first TCP reply immediately, our count should allow that
+we should still have enough buffers in our pool without having to make temporary buffers on the heap. Again for smaller
+sized random row reads, tune this max count. There are lazily created buffers and the count is the max count to be pooled.
+
+If you still see GC issues even after making end-to-end read path off-heap, look for issues in the appropriate buffer
+pool. Check the below RegionServer log with INFO level in HBase2.x:
+
+[source]
+----
+Pool already reached its max capacity : XXX and no free buffers now. Consider increasing the value for 'hbase.ipc.server.reservoir.initial.max' ?
+----
+
+Or the following log message in HBase3.x:
+
+[source]
+----
+Pool already reached its max capacity : XXX and no free buffers now. Consider increasing the value for 'hbase.server.allocator.max.buffer.count' ?
+----
+
+The setting for _HBASE_OFFHEAPSIZE_ in _hbase-env.sh_ should consider this off heap buffer pool at the RPC side also.
+We need to config this max off heap size for the RegionServer as a bit higher than the sum of this max pool size and
+the off heap cache size. The TCP layer will also need to create direct bytebuffers for TCP communication. Also the DFS
+client will need some off-heap to do its workings especially if short-circuit reads are configured. Allocating an extra
+of 1 - 2 GB for the max direct memory size has worked in tests.
+
+If you are using co processors and refer the Cells in the read results, DO NOT store reference to these Cells out of
+the scope of the CP hook methods. Some times the CPs need store info about the cell (Like its row key) for considering
+in the next CP hook call etc. For such cases, pls clone the required fields of the entire Cell as per the use cases.
+[ See CellUtil#cloneXXX(Cell) APIs ]
+
+[[regionserver.read.hdfs.block.offheap]]
+== Read block from HDFS to offheap directly
+
+In HBase-2.x, the RegionServer will still read block from HDFS to a temporary heap ByteBuffer and then flush to BucketCache's
+IOEngine asynchronously, finally it will be an offheap one.  We can still observe much GC pressure when cache hit ratio
+is not very high (such as cacheHitRatio ~ 60% ), so in link:https://issues.apache.org/jira/browse/HBASE-21879[HBASE-21879]
+we redesigned the read path and made the HDFS block reading be offheap now. This feature will be available in HBASE-3.0.0.
+
+For more details about the design and performance improvement, please see the link:https://docs.google.com/document/d/1xSy9axGxafoH-Qc17zbD2Bd--rWjjI00xTWQZ8ZwI_E/edit?usp=sharing[document].
+Here we will share some best practice about the performance tuning:
+
+Firstly,  we introduced several configurations about the ByteBuffAllocator (which was abstracted to manage the memory application or release):
+
+1. `hbase.server.allocator.pool.enabled`: means whether the region server will use the pooled offheap ByteBuffer allocator. Its default
+value is true. In HBase2.x, we still use the deprecated `hbase.ipc.server.reservoir.enabled` config while we'll use the new
+one in HBase3.x.
+2. `hbase.server.allocator.minimal.allocate.size`: If the desired byte size is not less than this one, then it will
+be allocated as a pooled offheap ByteBuff, otherwise it will be allocated from heap directly because it
+is too wasting to allocate from pool with fixed-size ByteBuffers, default value is `hbase.server.allocator.buffer.size/6`.
+3. `hbase.server.allocator.max.buffer.count`: The ByteBuffAllocator will have many fixed-size ByteBuffers inside which
+are composited as a pool, this config indicate how many buffers are there in the pool. Its default value will be 2MB * 2 * hbase.regionserver.handler.count / 65KB,
+the default hbase.regionserver.handler.count is 30, then its value will be 1890.
+4. `hbase.server.allocator.buffer.size`: The byte size of each ByteBuffer, default value is 66560 (65KB), here we choose 65KB instead of 64KB
+because of link:https://issues.apache.org/jira/browse/HBASE-22532[HBASE-22532].
+
+The three config keys: `hbase.ipc.server.reservoir.enabled`, `hbase.ipc.server.reservoir.initial.buffer.size` and `hbase.ipc.server.reservoir.initial.max` are introduced in HBase2.x. while in HBase3.x
+they are deprecated now, instead please use the new config keys: `hbase.server.allocator.pool.enabled`, `hbase.server.allocator.buffer.size` and `hbase.server.allocator.max.buffer.count`.
+
+If you still use the deprecated three config keys in HBase3.0.0, you will get a WARN log message like:
+
+[source]
+----
+The config keys hbase.ipc.server.reservoir.initial.buffer.size and hbase.ipc.server.reservoir.initial.max are deprecated now, instead please use hbase.server.allocator.buffer.size and hbase.server.allocator.max.buffer.count. In future release we will remove the two deprecated configs.
+----
+
+Second, we have some suggestions about the performance:
+
+.Please make sure that there are enough pooled DirectByteBuffer in your ByteBuffAllocator.
+
+The ByteBuffAllocator will allocate ByteBuffer from DirectByteBuffer pool firstly, if there’s no available ByteBuffer
+from the pool,  then it will just allocate the ByteBuffers from heap, then the GC pressures will increase again.
+
+By default, we will pre-allocate 4MB for each RPC handlers ( The handler count is determined by the config:
+`hbase.regionserver.handler.count`, it has the default value 30) . That’s to say,  if your `hbase.server.allocator.buffer.size`
+is 65KB, then your pool will have 2MB * 2 / 65KB * 30 = 945 DirectByteBuffer.  If you have some large scan and have a big caching,
+say you may have a rpc response whose bytes size is greater than 2MB (another 2MB for receiving rpc request),  then it will
+be better to increase the `hbase.server.allocator.max.buffer.count`.
+
+The RegionServer web UI also has the statistic about ByteBuffAllocator:
+
+image::bytebuff-allocator-stats.png[]
+
+If the following condition meet, you may need to increase your max buffer.count:
+
+heapAllocationRatio >= hbase.server.allocator.minimal.allocate.size / hbase.server.allocator.buffer.size * 100%
+
+.Please make sure the buffer size is greater than your block size.
+
+We have the default block size=64KB, so almost all of the data block have a block size: 64KB + delta, whose delta is
+very small, depends on the size of last KeyValue. If we use the default `hbase.server.allocator.buffer.size`=64KB,
+then each block will be allocated as two ByteBuffers:  one 64KB DirectByteBuffer and one HeapByteBuffer with delta bytes,
+the HeapByteBuffer will increase the GC pressure. Ideally, we should let the data block to be allocated as one ByteBuffer,
+it has simpler data structure, faster access speed, less heap usage. On the other hand, If the blocks are composited by multiple ByteBuffers,
+so we have to validate the checksum by an temporary heap copying (see link:https://issues.apache.org/jira/browse/HBASE-21917[HBASE-21917]), while if it’s a single ByteBuffer,
+we can speed the checksum by calling the hadoop' checksum in native lib, it's more faster.
+
+Please also see: link:https://issues.apache.org/jira/browse/HBASE-22483[HBASE-22483]
+
+[[regionserver.offheap.writepath]]
+== Offheap write-path
+
+TODO
diff --git a/src/main/asciidoc/book.adoc b/src/main/asciidoc/book.adoc
index 51a2800..d6d66f3 100644
--- a/src/main/asciidoc/book.adoc
+++ b/src/main/asciidoc/book.adoc
@@ -65,6 +65,7 @@ include::_chapters/security.adoc[]
 include::_chapters/architecture.adoc[]
 include::_chapters/hbase_mob.adoc[]
 include::_chapters/inmemory_compaction.adoc[]
+include::_chapters/offheap_read_write.adoc[]
 include::_chapters/backup_restore.adoc[]
 include::_chapters/sync_replication.adoc[]
 include::_chapters/hbase_apis.adoc[]
diff --git a/src/site/resources/images/bytebuff-allocator-stats.png b/src/site/resources/images/bytebuff-allocator-stats.png
new file mode 100644
index 0000000..b13362d
Binary files /dev/null and b/src/site/resources/images/bytebuff-allocator-stats.png differ
diff --git a/src/site/resources/images/offheap-overview.png b/src/site/resources/images/offheap-overview.png
new file mode 100644
index 0000000..b46e237
Binary files /dev/null and b/src/site/resources/images/offheap-overview.png differ