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