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