View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.io;
19  
20  import java.nio.ByteBuffer;
21  import java.util.Queue;
22  import java.util.concurrent.atomic.AtomicLong;
23  import java.util.concurrent.locks.ReentrantLock;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.util.BoundedArrayQueue;
29  
30  /**
31   * Like Hadoops' ByteBufferPool only you do not specify desired size when getting a ByteBuffer.
32   * This pool keeps an upper bound on the count of ByteBuffers in the pool and on the maximum size
33   * of ByteBuffer that it will retain (Hence the pool is 'bounded' as opposed to, say,
34   * Hadoop's ElasticByteBuffferPool).
35   * If a ByteBuffer is bigger than the configured threshold, we will just let the ByteBuffer go
36   * rather than add it to the pool. If more ByteBuffers than the configured maximum instances,
37   * we will not add the passed ByteBuffer to the pool; we will just drop it
38   * (we will log a WARN in this case that we are at capacity).
39   *
40   * <p>The intended use case is a reservoir of bytebuffers that an RPC can reuse; buffers tend to
41   * achieve a particular 'run' size over time give or take a few extremes. Set TRACE level on this
42   * class for a couple of seconds to get reporting on how it is running when deployed.
43   *
44   * <p>This class is thread safe.
45   */
46  @InterfaceAudience.Private
47  @SuppressWarnings("NonAtomicVolatileUpdate") // Suppress error-prone warning, see HBASE-21162
48  public class BoundedByteBufferPool {
49    private static final Log LOG = LogFactory.getLog(BoundedByteBufferPool.class);
50  
51    final Queue<ByteBuffer> buffers;
52  
53    // Maximum size of a ByteBuffer to retain in pool
54    private final int maxByteBufferSizeToCache;
55  
56    // A running average only it only rises, it never recedes
57    volatile int runningAverage;
58  
59    // Scratch that keeps rough total size of pooled bytebuffers
60    private volatile int totalReservoirCapacity;
61  
62    // For reporting
63    private AtomicLong allocations = new AtomicLong(0);
64  
65    private ReentrantLock lock =  new ReentrantLock();
66  
67    private boolean createDirectByteBuffer;
68  
69    /**
70     * @param maxByteBufferSizeToCache
71     * @param initialByteBufferSize
72     * @param maxToCache
73     * @param createDirectByteBuffer whether the buffers created by this pool to be off heap
74     */
75    public BoundedByteBufferPool(final int maxByteBufferSizeToCache, final int initialByteBufferSize,
76        final int maxToCache, final boolean createDirectByteBuffer) {
77      this.maxByteBufferSizeToCache = maxByteBufferSizeToCache;
78      this.runningAverage = initialByteBufferSize;
79      this.buffers = new BoundedArrayQueue<ByteBuffer>(maxToCache);
80      this.createDirectByteBuffer = createDirectByteBuffer;
81    }
82  
83    public ByteBuffer getBuffer() {
84      ByteBuffer bb = null;
85      lock.lock();
86      try {
87        bb = this.buffers.poll();
88        if (bb != null) {
89          this.totalReservoirCapacity -= bb.capacity();
90        }
91      } finally {
92        lock.unlock();
93      }
94      if (bb != null) {
95        // Clear sets limit == capacity. Postion == 0.
96        bb.clear();
97      } else {
98        bb = this.createDirectByteBuffer ? ByteBuffer.allocateDirect(this.runningAverage)
99            : ByteBuffer.allocate(this.runningAverage);
100       this.allocations.incrementAndGet();
101     }
102     if (LOG.isTraceEnabled()) {
103       LOG.trace("runningAverage=" + this.runningAverage +
104         ", totalCapacity=" + this.totalReservoirCapacity + ", count=" + this.buffers.size() +
105         ", allocations=" + this.allocations.get());
106     }
107     return bb;
108   }
109 
110   public void putBuffer(ByteBuffer bb) {
111     // If buffer is larger than we want to keep around, just let it go.
112     if (bb.capacity() > this.maxByteBufferSizeToCache) return;
113     boolean success = false;
114     int average = 0;
115     lock.lock();
116     try {
117       success = this.buffers.offer(bb);
118       if (success) {
119         this.totalReservoirCapacity += bb.capacity();
120         average = this.totalReservoirCapacity / this.buffers.size(); // size will never be 0.
121       }
122     } finally {
123       lock.unlock();
124     }
125     if (!success) {
126       if (LOG.isDebugEnabled()) {
127         LOG.debug("At capacity: " + this.buffers.size());
128       }
129     } else {
130       if (average > this.runningAverage && average < this.maxByteBufferSizeToCache) {
131         this.runningAverage = average;
132       }
133     }
134   }
135 }