001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io;
019
020import java.nio.ByteBuffer;
021import java.util.ArrayList;
022import java.util.List;
023import java.util.Queue;
024import java.util.concurrent.ConcurrentLinkedQueue;
025import java.util.concurrent.atomic.AtomicInteger;
026import java.util.concurrent.atomic.LongAdder;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.hbase.HConstants;
029import org.apache.hadoop.hbase.nio.ByteBuff;
030import org.apache.hadoop.hbase.nio.SingleByteBuff;
031import org.apache.hadoop.hbase.util.ReflectionUtils;
032import org.apache.hadoop.hbase.util.UnsafeAccess;
033import org.apache.yetus.audience.InterfaceAudience;
034import org.slf4j.Logger;
035import org.slf4j.LoggerFactory;
036
037import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
038
039/**
040 * ByteBuffAllocator is a nio ByteBuffer pool. It returns {@link ByteBuff}s which are wrappers of
041 * offheap {@link ByteBuffer} usually. If we are sure that the returned ByteBuffs have reached the
042 * end of their life cycle, we must call {@link ByteBuff#release()} to return buffers to the pool
043 * otherwise the pool will leak. If the desired memory size is larger than what the ByteBufferPool
044 * has available, we'll downgrade to allocate ByteBuffers from the heap. Increase the ByteBufferPool
045 * size if detect this case.<br/>
046 * <br/>
047 * For better memory/pool utilization, there is a lower bound named
048 * <code>minSizeForReservoirUse</code> in this allocator, and if the desired size is less than
049 * <code>minSizeForReservoirUse</code>, the allocator will just allocate the ByteBuffer from heap
050 * and let the JVM manage memory, because it better to not waste pool slots allocating a single
051 * fixed-size ByteBuffer for a small object.<br/>
052 * <br/>
053 * This pool can be used anywhere it makes sense managing memory. Currently used at least by RPC.
054 */
055@InterfaceAudience.Private
056public class ByteBuffAllocator {
057
058  private static final Logger LOG = LoggerFactory.getLogger(ByteBuffAllocator.class);
059
060  // The on-heap allocator is mostly used for testing but also has some non-test usage such as
061  // for scanning snapshot. This implementation will just allocate ByteBuffers from heap but
062  // wrapped by ByteBuff.
063  public static final ByteBuffAllocator HEAP = ByteBuffAllocator.createOnHeap();
064
065  public static final String ALLOCATOR_POOL_ENABLED_KEY = "hbase.server.allocator.pool.enabled";
066
067  public static final String MAX_BUFFER_COUNT_KEY = "hbase.server.allocator.max.buffer.count";
068
069  public static final String BUFFER_SIZE_KEY = "hbase.server.allocator.buffer.size";
070
071  public static final String MIN_ALLOCATE_SIZE_KEY = "hbase.server.allocator.minimal.allocate.size";
072
073  /**
074   * Set an alternate bytebuffallocator by setting this config, e.g. we can config
075   * {@link DeallocateRewriteByteBuffAllocator} to find out prematurely release issues
076   */
077  public static final String BYTEBUFF_ALLOCATOR_CLASS = "hbase.bytebuff.allocator.class";
078
079  /**
080   * @deprecated since 2.3.0 and will be removed in 4.0.0. Use
081   *             {@link ByteBuffAllocator#ALLOCATOR_POOL_ENABLED_KEY} instead.
082   */
083  @Deprecated
084  public static final String DEPRECATED_ALLOCATOR_POOL_ENABLED_KEY =
085    "hbase.ipc.server.reservoir.enabled";
086
087  /**
088   * @deprecated since 2.3.0 and will be removed in 4.0.0. Use
089   *             {@link ByteBuffAllocator#MAX_BUFFER_COUNT_KEY} instead.
090   */
091  @Deprecated
092  static final String DEPRECATED_MAX_BUFFER_COUNT_KEY = "hbase.ipc.server.reservoir.initial.max";
093
094  /**
095   * @deprecated since 2.3.0 and will be removed in 4.0.0. Use
096   *             {@link ByteBuffAllocator#BUFFER_SIZE_KEY} instead.
097   */
098  @Deprecated
099  static final String DEPRECATED_BUFFER_SIZE_KEY = "hbase.ipc.server.reservoir.initial.buffer.size";
100
101  /**
102   * There're some reasons why better to choose 65KB(rather than 64KB) as the default buffer size:
103   * <p>
104   * 1. Almost all of the data blocks have the block size: 64KB + delta, whose delta is very small,
105   * depends on the size of lastKeyValue. If we set buffer.size=64KB, then each block will be
106   * allocated as a MultiByteBuff: one 64KB DirectByteBuffer and delta bytes HeapByteBuffer, the
107   * HeapByteBuffer will increase the GC pressure. Ideally, we should let the data block to be
108   * allocated as a SingleByteBuff, it has simpler data structure, faster access speed, less heap
109   * usage.
110   * <p>
111   * 2. Since the blocks are MultiByteBuff when using buffer.size=64KB, so we have to calculate the
112   * checksum by an temp heap copying (see HBASE-21917), while if it's a SingleByteBuff, we can
113   * speed the checksum by calling the hadoop' checksum in native lib, which is more faster.
114   * <p>
115   * For performance comparison, please see HBASE-22483.
116   */
117  public static final int DEFAULT_BUFFER_SIZE = 65 * 1024;
118
119  public static final Recycler NONE = () -> {
120  };
121
122  public interface Recycler {
123    void free();
124  }
125
126  protected final boolean reservoirEnabled;
127  protected final int bufSize;
128  private final int maxBufCount;
129  private final AtomicInteger usedBufCount = new AtomicInteger(0);
130
131  private boolean maxPoolSizeInfoLevelLogged = false;
132
133  // If the desired size is at least this size, it'll allocated from ByteBufferPool, otherwise it'll
134  // allocated from heap for better utilization. We make this to be 1/6th of the pool buffer size.
135  private final int minSizeForReservoirUse;
136
137  private final Queue<ByteBuffer> buffers = new ConcurrentLinkedQueue<>();
138
139  // Metrics to track the pool allocation bytes and heap allocation bytes. If heap allocation
140  // bytes is increasing so much, then we may need to increase the max.buffer.count .
141  private final LongAdder poolAllocationBytes = new LongAdder();
142  private final LongAdder heapAllocationBytes = new LongAdder();
143  private long lastPoolAllocationBytes = 0;
144  private long lastHeapAllocationBytes = 0;
145
146  /**
147   * Initialize an {@link ByteBuffAllocator} which will try to allocate ByteBuffers from off-heap if
148   * reservoir is enabled and the reservoir has enough buffers, otherwise the allocator will just
149   * allocate the insufficient buffers from on-heap to meet the requirement.
150   * @param conf             which get the arguments to initialize the allocator.
151   * @param reservoirEnabled indicate whether the reservoir is enabled or disabled. NOTICE: if
152   *                         reservoir is enabled, then we will use the pool allocator to allocate
153   *                         off-heap ByteBuffers and use the HEAP allocator to allocate heap
154   *                         ByteBuffers. Otherwise if reservoir is disabled then all allocations
155   *                         will happen in HEAP instance.
156   * @return ByteBuffAllocator to manage the byte buffers.
157   */
158  public static ByteBuffAllocator create(Configuration conf, boolean reservoirEnabled) {
159    int poolBufSize = conf.getInt(BUFFER_SIZE_KEY, DEFAULT_BUFFER_SIZE);
160    if (reservoirEnabled) {
161      // The max number of buffers to be pooled in the ByteBufferPool. The default value been
162      // selected based on the #handlers configured. When it is read request, 2 MB is the max size
163      // at which we will send back one RPC request. Means max we need 2 MB for creating the
164      // response cell block. (Well it might be much lesser than this because in 2 MB size calc, we
165      // include the heap size overhead of each cells also.) Considering 2 MB, we will need
166      // (2 * 1024 * 1024) / poolBufSize buffers to make the response cell block. Pool buffer size
167      // is by default 64 KB.
168      // In case of read request, at the end of the handler process, we will make the response
169      // cellblock and add the Call to connection's response Q and a single Responder thread takes
170      // connections and responses from that one by one and do the socket write. So there is chances
171      // that by the time a handler originated response is actually done writing to socket and so
172      // released the BBs it used, the handler might have processed one more read req. On an avg 2x
173      // we consider and consider that also for the max buffers to pool
174      if (poolBufSize <= 0) {
175        throw new IllegalArgumentException(BUFFER_SIZE_KEY + " must be positive. Please disable "
176          + "the reservoir rather than setting the size of the buffer to zero or negative.");
177      }
178      int bufsForTwoMB = (2 * 1024 * 1024) / poolBufSize;
179      int maxBuffCount =
180        conf.getInt(MAX_BUFFER_COUNT_KEY, conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,
181          HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT) * bufsForTwoMB * 2);
182      int minSizeForReservoirUse = conf.getInt(MIN_ALLOCATE_SIZE_KEY, poolBufSize / 6);
183      if (minSizeForReservoirUse <= 0) {
184        LOG.warn("The minimal size for reservoir use is less or equal to zero, all allocations "
185          + "will be from the pool. Set a higher " + MIN_ALLOCATE_SIZE_KEY + " to avoid this.");
186      }
187      Class<?> clazz = conf.getClass(BYTEBUFF_ALLOCATOR_CLASS, ByteBuffAllocator.class);
188      return (ByteBuffAllocator) ReflectionUtils.newInstance(clazz, true, maxBuffCount, poolBufSize,
189        minSizeForReservoirUse);
190    } else {
191      return HEAP;
192    }
193  }
194
195  /**
196   * Initialize an {@link ByteBuffAllocator} which only allocate ByteBuffer from on-heap, it's
197   * designed for testing purpose or disabled reservoir case.
198   * @return allocator to allocate on-heap ByteBuffer.
199   */
200  private static ByteBuffAllocator createOnHeap() {
201    return new ByteBuffAllocator(false, 0, DEFAULT_BUFFER_SIZE, Integer.MAX_VALUE);
202  }
203
204  protected ByteBuffAllocator(boolean reservoirEnabled, int maxBufCount, int bufSize,
205    int minSizeForReservoirUse) {
206    this.reservoirEnabled = reservoirEnabled;
207    this.maxBufCount = maxBufCount;
208    this.bufSize = bufSize;
209    this.minSizeForReservoirUse = minSizeForReservoirUse;
210  }
211
212  public boolean isReservoirEnabled() {
213    return reservoirEnabled;
214  }
215
216  public long getHeapAllocationBytes() {
217    return heapAllocationBytes.sum();
218  }
219
220  public long getPoolAllocationBytes() {
221    return poolAllocationBytes.sum();
222  }
223
224  public int getBufferSize() {
225    return this.bufSize;
226  }
227
228  public int getUsedBufferCount() {
229    return this.usedBufCount.intValue();
230  }
231
232  /**
233   * The {@link ConcurrentLinkedQueue#size()} is O(N) complexity and time-consuming, so DO NOT use
234   * the method except in UT.
235   */
236  public int getFreeBufferCount() {
237    return this.buffers.size();
238  }
239
240  public int getTotalBufferCount() {
241    return maxBufCount;
242  }
243
244  public static long getHeapAllocationBytes(ByteBuffAllocator... allocators) {
245    long heapAllocBytes = 0;
246    for (ByteBuffAllocator alloc : Sets.newHashSet(allocators)) {
247      heapAllocBytes += alloc.getHeapAllocationBytes();
248    }
249    return heapAllocBytes;
250  }
251
252  public static double getHeapAllocationRatio(ByteBuffAllocator... allocators) {
253    double heapDelta = 0.0, poolDelta = 0.0;
254    long heapAllocBytes, poolAllocBytes;
255    // If disabled the pool allocator, then we use the global HEAP allocator. otherwise we use
256    // the pool allocator to allocate offheap ByteBuffers and use the HEAP to allocate heap
257    // ByteBuffers. So here we use a HashSet to remove the duplicated allocator object in disable
258    // case.
259    for (ByteBuffAllocator alloc : Sets.newHashSet(allocators)) {
260      heapAllocBytes = alloc.heapAllocationBytes.sum();
261      poolAllocBytes = alloc.poolAllocationBytes.sum();
262      heapDelta += (heapAllocBytes - alloc.lastHeapAllocationBytes);
263      poolDelta += (poolAllocBytes - alloc.lastPoolAllocationBytes);
264      alloc.lastHeapAllocationBytes = heapAllocBytes;
265      alloc.lastPoolAllocationBytes = poolAllocBytes;
266    }
267    // Calculate the heap allocation ratio.
268    if (Math.abs(heapDelta + poolDelta) < 1e-3) {
269      return 0.0;
270    }
271    return heapDelta / (heapDelta + poolDelta);
272  }
273
274  /**
275   * Allocate an buffer with buffer size from ByteBuffAllocator, Note to call the
276   * {@link ByteBuff#release()} if no need any more, otherwise the memory leak happen in NIO
277   * ByteBuffer pool.
278   * @return an ByteBuff with the buffer size.
279   */
280  public SingleByteBuff allocateOneBuffer() {
281    if (isReservoirEnabled()) {
282      ByteBuffer bb = getBuffer();
283      if (bb != null) {
284        return new SingleByteBuff(() -> putbackBuffer(bb), bb);
285      }
286    }
287    // Allocated from heap, let the JVM free its memory.
288    return (SingleByteBuff) ByteBuff.wrap(allocateOnHeap(bufSize));
289  }
290
291  private ByteBuffer allocateOnHeap(int size) {
292    heapAllocationBytes.add(size);
293    return ByteBuffer.allocate(size);
294  }
295
296  /**
297   * Allocate size bytes from the ByteBufAllocator, Note to call the {@link ByteBuff#release()} if
298   * no need any more, otherwise the memory leak happen in NIO ByteBuffer pool.
299   * @param size to allocate
300   * @return an ByteBuff with the desired size.
301   */
302  public ByteBuff allocate(int size) {
303    if (size < 0) {
304      throw new IllegalArgumentException("size to allocate should >=0");
305    }
306    // If disabled the reservoir, just allocate it from on-heap.
307    if (!isReservoirEnabled() || size == 0) {
308      return ByteBuff.wrap(allocateOnHeap(size));
309    }
310    int reminder = size % bufSize;
311    int len = size / bufSize + (reminder > 0 ? 1 : 0);
312    List<ByteBuffer> bbs = new ArrayList<>(len);
313    // Allocate from ByteBufferPool until the remaining is less than minSizeForReservoirUse or
314    // reservoir is exhausted.
315    int remain = size;
316    while (remain >= minSizeForReservoirUse) {
317      ByteBuffer bb = this.getBuffer();
318      if (bb == null) {
319        break;
320      }
321      bbs.add(bb);
322      remain -= bufSize;
323    }
324    int lenFromReservoir = bbs.size();
325    if (remain > 0) {
326      // If the last ByteBuffer is too small or the reservoir can not provide more ByteBuffers, we
327      // just allocate the ByteBuffer from on-heap.
328      bbs.add(allocateOnHeap(remain));
329    }
330
331    ByteBuff bb;
332    // we only need a recycler if we successfully pulled from the pool
333    // this matters for determining whether to add leak detection in RefCnt
334    if (lenFromReservoir == 0) {
335      bb = ByteBuff.wrap(bbs);
336    } else {
337      bb = ByteBuff.wrap(bbs, () -> {
338        for (int i = 0; i < lenFromReservoir; i++) {
339          this.putbackBuffer(bbs.get(i));
340        }
341      });
342    }
343
344    bb.limit(size);
345    return bb;
346  }
347
348  /**
349   * Free all direct buffers if allocated, mainly used for testing.
350   */
351  public void clean() {
352    while (!buffers.isEmpty()) {
353      ByteBuffer b = buffers.poll();
354      if (b.isDirect()) {
355        UnsafeAccess.freeDirectBuffer(b);
356      }
357    }
358    this.usedBufCount.set(0);
359    this.maxPoolSizeInfoLevelLogged = false;
360    this.poolAllocationBytes.reset();
361    this.heapAllocationBytes.reset();
362    this.lastPoolAllocationBytes = 0;
363    this.lastHeapAllocationBytes = 0;
364  }
365
366  /**
367   * @return One free DirectByteBuffer from the pool. If no free ByteBuffer and we have not reached
368   *         the maximum pool size, it will create a new one and return. In case of max pool size
369   *         also reached, will return null. When pool returned a ByteBuffer, make sure to return it
370   *         back to pool after use.
371   */
372  private ByteBuffer getBuffer() {
373    ByteBuffer bb = buffers.poll();
374    if (bb != null) {
375      // To reset the limit to capacity and position to 0, must clear here.
376      bb.clear();
377      poolAllocationBytes.add(bufSize);
378      return bb;
379    }
380    while (true) {
381      int c = this.usedBufCount.intValue();
382      if (c >= this.maxBufCount) {
383        if (!maxPoolSizeInfoLevelLogged) {
384          LOG.info("Pool already reached its max capacity : {} and no free buffers now. Consider "
385            + "increasing the value for '{}' ?", maxBufCount, MAX_BUFFER_COUNT_KEY);
386          maxPoolSizeInfoLevelLogged = true;
387        }
388        return null;
389      }
390      if (!this.usedBufCount.compareAndSet(c, c + 1)) {
391        continue;
392      }
393      poolAllocationBytes.add(bufSize);
394      return ByteBuffer.allocateDirect(bufSize);
395    }
396  }
397
398  /**
399   * Return back a ByteBuffer after its use. Don't read/write the ByteBuffer after the returning.
400   * @param buf ByteBuffer to return.
401   */
402  protected void putbackBuffer(ByteBuffer buf) {
403    if (buf.capacity() != bufSize || (reservoirEnabled ^ buf.isDirect())) {
404      LOG.warn("Trying to put a buffer, not created by this pool! Will be just ignored");
405      return;
406    }
407    buffers.offer(buf);
408  }
409}