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