1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46 @InterfaceAudience.Private
47 @SuppressWarnings("NonAtomicVolatileUpdate")
48 public class BoundedByteBufferPool {
49 private static final Log LOG = LogFactory.getLog(BoundedByteBufferPool.class);
50
51 final Queue<ByteBuffer> buffers;
52
53
54 private final int maxByteBufferSizeToCache;
55
56
57 volatile int runningAverage;
58
59
60 private volatile int totalReservoirCapacity;
61
62
63 private AtomicLong allocations = new AtomicLong(0);
64
65 private ReentrantLock lock = new ReentrantLock();
66
67 private boolean createDirectByteBuffer;
68
69
70
71
72
73
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
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
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();
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 }