1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.util;
20
21 import java.io.IOException;
22 import java.nio.ByteBuffer;
23 import java.util.concurrent.Callable;
24 import java.util.concurrent.ExecutionException;
25 import java.util.concurrent.ExecutorService;
26 import java.util.concurrent.Future;
27 import java.util.concurrent.LinkedBlockingQueue;
28 import java.util.concurrent.ThreadPoolExecutor;
29 import java.util.concurrent.TimeUnit;
30
31 import org.apache.commons.logging.Log;
32 import org.apache.commons.logging.LogFactory;
33 import org.apache.hadoop.hbase.classification.InterfaceAudience;
34 import org.apache.hadoop.util.StringUtils;
35
36
37
38
39
40
41 @InterfaceAudience.Private
42 public class ByteBufferArray {
43 private static final Log LOG = LogFactory.getLog(ByteBufferArray.class);
44
45 static final int DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024;
46 ByteBuffer[] buffers;
47 private int bufferSize;
48 int bufferCount;
49
50
51
52
53
54
55
56
57 public ByteBufferArray(long capacity, boolean directByteBuffer, ByteBufferAllocator allocator)
58 throws IOException {
59 this.bufferSize = DEFAULT_BUFFER_SIZE;
60 if (this.bufferSize > (capacity / 16))
61 this.bufferSize = (int) roundUp(capacity / 16, 32768);
62 this.bufferCount = (int) (roundUp(capacity, bufferSize) / bufferSize);
63 LOG.info("Allocating buffers total=" + StringUtils.byteDesc(capacity)
64 + ", sizePerBuffer=" + StringUtils.byteDesc(bufferSize) + ", count="
65 + bufferCount + ", direct=" + directByteBuffer);
66 buffers = new ByteBuffer[bufferCount + 1];
67 createBuffers(directByteBuffer, allocator);
68 }
69
70 void createBuffers(boolean directByteBuffer, ByteBufferAllocator allocator)
71 throws IOException {
72 int threadCount = getThreadCount();
73 ExecutorService service = new ThreadPoolExecutor(threadCount, threadCount, 0L,
74 TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>());
75 int perThreadCount = (int)Math.floor((double) (bufferCount) / threadCount);
76 int lastThreadCount = bufferCount - (perThreadCount * (threadCount - 1));
77 Future<ByteBuffer[]>[] futures = new Future[threadCount];
78 try {
79 for (int i = 0; i < threadCount; i++) {
80
81 int buffersToCreate = (i == threadCount - 1) ? lastThreadCount : perThreadCount;
82 futures[i] = service.submit(
83 new BufferCreatorCallable(bufferSize, directByteBuffer, buffersToCreate, allocator));
84 }
85 int bufferIndex = 0;
86 for (Future<ByteBuffer[]> future : futures) {
87 try {
88 ByteBuffer[] buffers = future.get();
89 for (ByteBuffer buffer : buffers) {
90 this.buffers[bufferIndex++] = buffer;
91 }
92 } catch (InterruptedException | ExecutionException e) {
93 LOG.error("Buffer creation interrupted", e);
94 throw new IOException(e);
95 }
96 }
97 } finally {
98 service.shutdownNow();
99 }
100
101 this.buffers[bufferCount] = ByteBuffer.allocate(0);
102 }
103
104 int getThreadCount() {
105 return Runtime.getRuntime().availableProcessors();
106 }
107
108
109
110
111
112 private static class BufferCreatorCallable implements Callable<ByteBuffer[]> {
113 private final int bufferCapacity;
114 private final boolean directByteBuffer;
115 private final int bufferCount;
116 private final ByteBufferAllocator allocator;
117
118 BufferCreatorCallable(int bufferCapacity, boolean directByteBuffer, int bufferCount,
119 ByteBufferAllocator allocator) {
120 this.bufferCapacity = bufferCapacity;
121 this.directByteBuffer = directByteBuffer;
122 this.bufferCount = bufferCount;
123 this.allocator = allocator;
124 }
125
126 @Override
127 public ByteBuffer[] call() throws Exception {
128 ByteBuffer[] buffers = new ByteBuffer[this.bufferCount];
129 for (int i = 0; i < this.bufferCount; i++) {
130 buffers[i] = allocator.allocate(this.bufferCapacity, this.directByteBuffer);
131 }
132 return buffers;
133
134 }
135 }
136
137 private long roundUp(long n, long to) {
138 return ((n + to - 1) / to) * to;
139 }
140
141
142
143
144
145
146
147
148 public int getMultiple(long start, int len, byte[] dstArray) {
149 return getMultiple(start, len, dstArray, 0);
150 }
151
152
153
154
155
156
157
158
159
160
161 public int getMultiple(long start, int len, byte[] dstArray, int dstOffset) {
162 multiple(start, len, dstArray, dstOffset, GET_MULTIPLE_VISTOR);
163 return len;
164 }
165
166 private final static Visitor GET_MULTIPLE_VISTOR = new Visitor() {
167 @Override
168 public void visit(ByteBuffer bb, int pos, byte[] array, int arrayIdx, int len) {
169 ByteBufferUtils.copyFromBufferToArray(array, bb, pos, arrayIdx, len);
170 }
171 };
172
173
174
175
176
177
178
179 public void putMultiple(long start, int len, byte[] srcArray) {
180 putMultiple(start, len, srcArray, 0);
181 }
182
183
184
185
186
187
188
189
190
191 public void putMultiple(long start, int len, byte[] srcArray, int srcOffset) {
192 multiple(start, len, srcArray, srcOffset, PUT_MULTIPLE_VISITOR);
193 }
194
195 private final static Visitor PUT_MULTIPLE_VISITOR = new Visitor() {
196 @Override
197 public void visit(ByteBuffer bb, int pos, byte[] array, int arrayIdx, int len) {
198 ByteBufferUtils.copyFromArrayToBuffer(bb, pos, array, arrayIdx, len);
199 }
200 };
201
202 private interface Visitor {
203
204
205
206
207
208
209
210
211
212
213 void visit(ByteBuffer bb, int pos, byte[] array, int arrayOffset, int len);
214 }
215
216
217
218
219
220
221
222
223
224
225
226
227 void multiple(long start, int len, byte[] array, int arrayOffset, Visitor visitor) {
228 assert len >= 0;
229 long end = start + len;
230 int startBuffer = (int) (start / bufferSize), startOffset = (int) (start % bufferSize);
231 int endBuffer = (int) (end / bufferSize), endOffset = (int) (end % bufferSize);
232 assert array.length >= len + arrayOffset;
233 assert startBuffer >= 0 && startBuffer < bufferCount;
234 assert (endBuffer >= 0 && endBuffer < bufferCount)
235 || (endBuffer == bufferCount && endOffset == 0);
236 if (startBuffer >= buffers.length || startBuffer < 0) {
237 String msg = "Failed multiple, start=" + start + ",startBuffer="
238 + startBuffer + ",bufferSize=" + bufferSize;
239 LOG.error(msg);
240 throw new RuntimeException(msg);
241 }
242 int srcIndex = 0, cnt = -1;
243 for (int i = startBuffer; i <= endBuffer; ++i) {
244 ByteBuffer bb = buffers[i].duplicate();
245 int pos = 0;
246 if (i == startBuffer) {
247 cnt = bufferSize - startOffset;
248 if (cnt > len) {
249 cnt = len;
250 }
251 pos = startOffset;
252 } else if (i == endBuffer) {
253 cnt = endOffset;
254 } else {
255 cnt = bufferSize;
256 }
257 visitor.visit(bb, pos, array, srcIndex + arrayOffset, cnt);
258 srcIndex += cnt;
259 }
260 assert srcIndex == len;
261 }
262 }