1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.io.hfile.slab;
21
22 import java.nio.ByteBuffer;
23 import java.util.concurrent.ConcurrentLinkedQueue;
24 import java.util.concurrent.LinkedBlockingQueue;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.hbase.util.ClassSize;
29 import org.apache.hadoop.hbase.util.DirectMemoryUtils;
30 import com.google.common.base.Preconditions;
31
32
33
34
35
36
37
38 class Slab implements org.apache.hadoop.hbase.io.HeapSize {
39 static final Log LOG = LogFactory.getLog(Slab.class);
40
41
42 private LinkedBlockingQueue<ByteBuffer> buffers;
43
44
45 private ConcurrentLinkedQueue<ByteBuffer> slabs;
46
47 private final int blockSize;
48 private final int numBlocks;
49 private long heapSize;
50
51 Slab(int blockSize, int numBlocks) {
52 buffers = new LinkedBlockingQueue<ByteBuffer>();
53 slabs = new ConcurrentLinkedQueue<ByteBuffer>();
54
55 this.blockSize = blockSize;
56 this.numBlocks = numBlocks;
57
58 this.heapSize = ClassSize.estimateBase(this.getClass(), false);
59
60 int maxBlocksPerSlab = Integer.MAX_VALUE / blockSize;
61 int maxSlabSize = maxBlocksPerSlab * blockSize;
62
63 int numFullSlabs = numBlocks / maxBlocksPerSlab;
64 int partialSlabSize = (numBlocks % maxBlocksPerSlab) * blockSize;
65 for (int i = 0; i < numFullSlabs; i++) {
66 allocateAndSlice(maxSlabSize, blockSize);
67 }
68
69 if (partialSlabSize > 0) {
70 allocateAndSlice(partialSlabSize, blockSize);
71 }
72 }
73
74 private void allocateAndSlice(int size, int sliceSize) {
75 ByteBuffer newSlab = ByteBuffer.allocateDirect(size);
76 slabs.add(newSlab);
77 for (int j = 0; j < newSlab.capacity(); j += sliceSize) {
78 newSlab.limit(j + sliceSize).position(j);
79 ByteBuffer aSlice = newSlab.slice();
80 buffers.add(aSlice);
81 heapSize += ClassSize.estimateBase(aSlice.getClass(), false);
82 }
83 }
84
85
86
87
88
89
90 void shutdown() {
91 for (ByteBuffer aSlab : slabs) {
92 try {
93 DirectMemoryUtils.destroyDirectByteBuffer(aSlab);
94 } catch (Exception e) {
95 LOG.warn("Unable to deallocate direct memory during shutdown", e);
96 }
97 }
98 }
99
100 int getBlockSize() {
101 return this.blockSize;
102 }
103
104 int getBlockCapacity() {
105 return this.numBlocks;
106 }
107
108 int getBlocksRemaining() {
109 return this.buffers.size();
110 }
111
112
113
114
115
116 ByteBuffer alloc(int bufferSize) throws InterruptedException {
117 int newCapacity = Preconditions.checkPositionIndex(bufferSize, blockSize);
118
119 ByteBuffer returnedBuffer = buffers.take();
120
121 returnedBuffer.clear().limit(newCapacity);
122 return returnedBuffer;
123 }
124
125 void free(ByteBuffer toBeFreed) {
126 Preconditions.checkArgument(toBeFreed.capacity() == blockSize);
127 buffers.add(toBeFreed);
128 }
129
130 @Override
131 public long heapSize() {
132 return heapSize;
133 }
134 }