1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.io.hfile.bucket;
22
23 import java.util.Arrays;
24 import java.util.Comparator;
25 import java.util.HashSet;
26 import java.util.Iterator;
27 import java.util.Map;
28 import java.util.Queue;
29 import java.util.Set;
30 import java.util.concurrent.atomic.AtomicLong;
31
32 import com.google.common.collect.MinMaxPriorityQueue;
33 import org.apache.commons.collections.map.LinkedMap;
34 import org.apache.commons.logging.Log;
35 import org.apache.commons.logging.LogFactory;
36 import org.apache.hadoop.hbase.classification.InterfaceAudience;
37 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
38 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
39 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.BucketEntry;
40
41 import com.google.common.base.Objects;
42 import com.google.common.base.Preconditions;
43 import com.google.common.primitives.Ints;
44
45
46
47
48
49
50
51
52
53 @InterfaceAudience.Private
54 public final class BucketAllocator {
55 private static final Log LOG = LogFactory.getLog(BucketAllocator.class);
56
57 public final static class Bucket {
58 private long baseOffset;
59 private int itemAllocationSize, sizeIndex;
60 private int itemCount;
61 private int freeList[];
62 private int freeCount, usedCount;
63
64 public Bucket(long offset) {
65 baseOffset = offset;
66 sizeIndex = -1;
67 }
68
69 void reconfigure(int sizeIndex, int[] bucketSizes, long bucketCapacity) {
70 Preconditions.checkElementIndex(sizeIndex, bucketSizes.length);
71 this.sizeIndex = sizeIndex;
72 itemAllocationSize = bucketSizes[sizeIndex];
73 itemCount = (int) (bucketCapacity / (long) itemAllocationSize);
74 freeCount = itemCount;
75 usedCount = 0;
76 freeList = new int[itemCount];
77 for (int i = 0; i < freeCount; ++i)
78 freeList[i] = i;
79 }
80
81 public boolean isUninstantiated() {
82 return sizeIndex == -1;
83 }
84
85 public int sizeIndex() {
86 return sizeIndex;
87 }
88
89 public int getItemAllocationSize() {
90 return itemAllocationSize;
91 }
92
93 public boolean hasFreeSpace() {
94 return freeCount > 0;
95 }
96
97 public boolean isCompletelyFree() {
98 return usedCount == 0;
99 }
100
101 public int freeCount() {
102 return freeCount;
103 }
104
105 public int usedCount() {
106 return usedCount;
107 }
108
109 public int getFreeBytes() {
110 return freeCount * itemAllocationSize;
111 }
112
113 public int getUsedBytes() {
114 return usedCount * itemAllocationSize;
115 }
116
117 public long getBaseOffset() {
118 return baseOffset;
119 }
120
121
122
123
124
125
126 public long allocate() {
127 assert freeCount > 0;
128 assert sizeIndex != -1;
129 ++usedCount;
130 long offset = baseOffset + (freeList[--freeCount] * itemAllocationSize);
131 assert offset >= 0;
132 return offset;
133 }
134
135 public void addAllocation(long offset) throws BucketAllocatorException {
136 offset -= baseOffset;
137 if (offset < 0 || offset % itemAllocationSize != 0)
138 throw new BucketAllocatorException(
139 "Attempt to add allocation for bad offset: " + offset + " base="
140 + baseOffset + ", bucket size=" + itemAllocationSize);
141 int idx = (int) (offset / itemAllocationSize);
142 boolean matchFound = false;
143 for (int i = 0; i < freeCount; ++i) {
144 if (matchFound) freeList[i - 1] = freeList[i];
145 else if (freeList[i] == idx) matchFound = true;
146 }
147 if (!matchFound)
148 throw new BucketAllocatorException("Couldn't find match for index "
149 + idx + " in free list");
150 ++usedCount;
151 --freeCount;
152 }
153
154 private void free(long offset) {
155 offset -= baseOffset;
156 assert offset >= 0;
157 assert offset < itemCount * itemAllocationSize;
158 assert offset % itemAllocationSize == 0;
159 assert usedCount > 0;
160 assert freeCount < itemCount;
161 int item = (int) (offset / (long) itemAllocationSize);
162 assert !freeListContains(item);
163 --usedCount;
164 freeList[freeCount++] = item;
165 }
166
167 private boolean freeListContains(int blockNo) {
168 for (int i = 0; i < freeCount; ++i) {
169 if (freeList[i] == blockNo) return true;
170 }
171 return false;
172 }
173 }
174
175 final class BucketSizeInfo {
176
177
178 private LinkedMap bucketList, freeBuckets, completelyFreeBuckets;
179 private int sizeIndex;
180
181 BucketSizeInfo(int sizeIndex) {
182 bucketList = new LinkedMap();
183 freeBuckets = new LinkedMap();
184 completelyFreeBuckets = new LinkedMap();
185 this.sizeIndex = sizeIndex;
186 }
187
188 public synchronized void instantiateBucket(Bucket b) {
189 assert b.isUninstantiated() || b.isCompletelyFree();
190 b.reconfigure(sizeIndex, bucketSizes, bucketCapacity);
191 bucketList.put(b, b);
192 freeBuckets.put(b, b);
193 completelyFreeBuckets.put(b, b);
194 }
195
196 public int sizeIndex() {
197 return sizeIndex;
198 }
199
200
201
202
203
204 public long allocateBlock() {
205 Bucket b = null;
206 if (freeBuckets.size() > 0) {
207
208 b = (Bucket) freeBuckets.lastKey();
209 }
210 if (b == null) {
211 b = grabGlobalCompletelyFreeBucket();
212 if (b != null) instantiateBucket(b);
213 }
214 if (b == null) return -1;
215 long result = b.allocate();
216 blockAllocated(b);
217 return result;
218 }
219
220 void blockAllocated(Bucket b) {
221 if (!b.isCompletelyFree()) completelyFreeBuckets.remove(b);
222 if (!b.hasFreeSpace()) freeBuckets.remove(b);
223 }
224
225 public Bucket findAndRemoveCompletelyFreeBucket() {
226 Bucket b = null;
227 assert bucketList.size() > 0;
228 if (bucketList.size() == 1) {
229
230 return null;
231 }
232
233 if (completelyFreeBuckets.size() > 0) {
234 b = (Bucket) completelyFreeBuckets.firstKey();
235 removeBucket(b);
236 }
237 return b;
238 }
239
240 private synchronized void removeBucket(Bucket b) {
241 assert b.isCompletelyFree();
242 bucketList.remove(b);
243 freeBuckets.remove(b);
244 completelyFreeBuckets.remove(b);
245 }
246
247 public void freeBlock(Bucket b, long offset) {
248 assert bucketList.containsKey(b);
249
250 assert (!completelyFreeBuckets.containsKey(b));
251 b.free(offset);
252 if (!freeBuckets.containsKey(b)) freeBuckets.put(b, b);
253 if (b.isCompletelyFree()) completelyFreeBuckets.put(b, b);
254 }
255
256 public synchronized IndexStatistics statistics() {
257 long free = 0, used = 0;
258 for (Object obj : bucketList.keySet()) {
259 Bucket b = (Bucket) obj;
260 free += b.freeCount();
261 used += b.usedCount();
262 }
263 return new IndexStatistics(free, used, bucketSizes[sizeIndex]);
264 }
265
266 @Override
267 public String toString() {
268 return Objects.toStringHelper(this.getClass())
269 .add("sizeIndex", sizeIndex)
270 .add("bucketSize", bucketSizes[sizeIndex])
271 .toString();
272 }
273 }
274
275
276
277
278
279
280 private static final int DEFAULT_BUCKET_SIZES[] = { 4 * 1024 + 1024, 8 * 1024 + 1024,
281 16 * 1024 + 1024, 32 * 1024 + 1024, 40 * 1024 + 1024, 48 * 1024 + 1024,
282 56 * 1024 + 1024, 64 * 1024 + 1024, 96 * 1024 + 1024, 128 * 1024 + 1024,
283 192 * 1024 + 1024, 256 * 1024 + 1024, 384 * 1024 + 1024,
284 512 * 1024 + 1024 };
285
286
287
288
289
290 public BucketSizeInfo roundUpToBucketSizeInfo(int blockSize) {
291 for (int i = 0; i < bucketSizes.length; ++i)
292 if (blockSize <= bucketSizes[i])
293 return bucketSizeInfos[i];
294 return null;
295 }
296
297
298
299
300 static public final int FEWEST_ITEMS_IN_BUCKET = 4;
301
302 private final int[] bucketSizes;
303 private final int bigItemSize;
304
305 private final long bucketCapacity;
306 private Bucket[] buckets;
307 private BucketSizeInfo[] bucketSizeInfos;
308 private final long totalSize;
309 private transient long usedSize = 0;
310
311 BucketAllocator(long availableSpace, int[] bucketSizes)
312 throws BucketAllocatorException {
313 this.bucketSizes = bucketSizes == null ? DEFAULT_BUCKET_SIZES : bucketSizes;
314 Arrays.sort(this.bucketSizes);
315 this.bigItemSize = Ints.max(this.bucketSizes);
316 this.bucketCapacity = (long) FEWEST_ITEMS_IN_BUCKET * bigItemSize;
317 buckets = new Bucket[(int) (availableSpace / bucketCapacity)];
318 if (buckets.length < this.bucketSizes.length)
319 throw new BucketAllocatorException("Bucket allocator size too small (" + buckets.length +
320 "); must have room for at least " + this.bucketSizes.length + " buckets");
321 bucketSizeInfos = new BucketSizeInfo[this.bucketSizes.length];
322 for (int i = 0; i < this.bucketSizes.length; ++i) {
323 bucketSizeInfos[i] = new BucketSizeInfo(i);
324 }
325 for (int i = 0; i < buckets.length; ++i) {
326 buckets[i] = new Bucket(bucketCapacity * i);
327 bucketSizeInfos[i < this.bucketSizes.length ? i : this.bucketSizes.length - 1]
328 .instantiateBucket(buckets[i]);
329 }
330 this.totalSize = ((long) buckets.length) * bucketCapacity;
331 if (LOG.isInfoEnabled()) {
332 LOG.info("Cache totalSize=" + this.totalSize + ", buckets=" + this.buckets.length +
333 ", bucket capacity=" + this.bucketCapacity +
334 "=(" + FEWEST_ITEMS_IN_BUCKET + "*" + this.bigItemSize + ")=" +
335 "(FEWEST_ITEMS_IN_BUCKET*(largest configured bucketcache size))");
336 }
337 }
338
339
340
341
342
343
344
345
346
347 BucketAllocator(long availableSpace, int[] bucketSizes, Map<BlockCacheKey, BucketEntry> map,
348 AtomicLong realCacheSize) throws BucketAllocatorException {
349 this(availableSpace, bucketSizes);
350
351
352
353
354
355 boolean[] reconfigured = new boolean[buckets.length];
356 int sizeNotMatchedCount = 0;
357 int insufficientCapacityCount = 0;
358 Iterator<Map.Entry<BlockCacheKey, BucketEntry>> iterator = map.entrySet().iterator();
359 while (iterator.hasNext()) {
360 Map.Entry<BlockCacheKey, BucketEntry> entry = iterator.next();
361 long foundOffset = entry.getValue().offset();
362 int foundLen = entry.getValue().getLength();
363 int bucketSizeIndex = -1;
364 for (int i = 0; i < this.bucketSizes.length; ++i) {
365 if (foundLen <= this.bucketSizes[i]) {
366 bucketSizeIndex = i;
367 break;
368 }
369 }
370 if (bucketSizeIndex == -1) {
371 sizeNotMatchedCount++;
372 iterator.remove();
373 continue;
374 }
375 int bucketNo = (int) (foundOffset / bucketCapacity);
376 if (bucketNo < 0 || bucketNo >= buckets.length) {
377 insufficientCapacityCount++;
378 iterator.remove();
379 continue;
380 }
381 Bucket b = buckets[bucketNo];
382 if (reconfigured[bucketNo]) {
383 if (b.sizeIndex() != bucketSizeIndex)
384 throw new BucketAllocatorException(
385 "Inconsistent allocation in bucket map;");
386 } else {
387 if (!b.isCompletelyFree())
388 throw new BucketAllocatorException("Reconfiguring bucket "
389 + bucketNo + " but it's already allocated; corrupt data");
390
391
392 BucketSizeInfo bsi = bucketSizeInfos[bucketSizeIndex];
393 BucketSizeInfo oldbsi = bucketSizeInfos[b.sizeIndex()];
394 oldbsi.removeBucket(b);
395 bsi.instantiateBucket(b);
396 reconfigured[bucketNo] = true;
397 }
398 realCacheSize.addAndGet(foundLen);
399 buckets[bucketNo].addAllocation(foundOffset);
400 usedSize += buckets[bucketNo].getItemAllocationSize();
401 bucketSizeInfos[bucketSizeIndex].blockAllocated(b);
402 }
403
404 if (sizeNotMatchedCount > 0) {
405 LOG.warn("There are " + sizeNotMatchedCount + " blocks which can't be rebuilt because "
406 + "there is no matching bucket size for these blocks");
407 }
408 if (insufficientCapacityCount > 0) {
409 LOG.warn("There are " + insufficientCapacityCount + " blocks which can't be rebuilt - "
410 + "did you shrink the cache?");
411 }
412 }
413
414 @Override
415 public String toString() {
416 StringBuilder sb = new StringBuilder(1024);
417 for (int i = 0; i < buckets.length; ++i) {
418 Bucket b = buckets[i];
419 if (i > 0) sb.append(", ");
420 sb.append("bucket.").append(i).append(": size=").append(b.getItemAllocationSize());
421 sb.append(", freeCount=").append(b.freeCount()).append(", used=").append(b.usedCount());
422 }
423 return sb.toString();
424 }
425
426 public long getUsedSize() {
427 return this.usedSize;
428 }
429
430 public long getFreeSize() {
431 return this.totalSize - getUsedSize();
432 }
433
434 public long getTotalSize() {
435 return this.totalSize;
436 }
437
438
439
440
441
442
443
444
445 public synchronized long allocateBlock(int blockSize) throws CacheFullException,
446 BucketAllocatorException {
447 assert blockSize > 0;
448 BucketSizeInfo bsi = roundUpToBucketSizeInfo(blockSize);
449 if (bsi == null) {
450 throw new BucketAllocatorException("Allocation too big size=" + blockSize +
451 "; adjust BucketCache sizes " + CacheConfig.BUCKET_CACHE_BUCKETS_KEY +
452 " to accomodate if size seems reasonable and you want it cached.");
453 }
454 long offset = bsi.allocateBlock();
455
456
457 if (offset < 0)
458 throw new CacheFullException(blockSize, bsi.sizeIndex());
459 usedSize += bucketSizes[bsi.sizeIndex()];
460 return offset;
461 }
462
463 private Bucket grabGlobalCompletelyFreeBucket() {
464 for (BucketSizeInfo bsi : bucketSizeInfos) {
465 Bucket b = bsi.findAndRemoveCompletelyFreeBucket();
466 if (b != null) return b;
467 }
468 return null;
469 }
470
471
472
473
474
475
476 public synchronized int freeBlock(long offset) {
477 int bucketNo = (int) (offset / bucketCapacity);
478 assert bucketNo >= 0 && bucketNo < buckets.length;
479 Bucket targetBucket = buckets[bucketNo];
480 bucketSizeInfos[targetBucket.sizeIndex()].freeBlock(targetBucket, offset);
481 usedSize -= targetBucket.getItemAllocationSize();
482 return targetBucket.getItemAllocationSize();
483 }
484
485 public int sizeIndexOfAllocation(long offset) {
486 int bucketNo = (int) (offset / bucketCapacity);
487 assert bucketNo >= 0 && bucketNo < buckets.length;
488 Bucket targetBucket = buckets[bucketNo];
489 return targetBucket.sizeIndex();
490 }
491
492 public int sizeOfAllocation(long offset) {
493 int bucketNo = (int) (offset / bucketCapacity);
494 assert bucketNo >= 0 && bucketNo < buckets.length;
495 Bucket targetBucket = buckets[bucketNo];
496 return targetBucket.getItemAllocationSize();
497 }
498
499 static class IndexStatistics {
500 private long freeCount, usedCount, itemSize, totalCount;
501
502 public long freeCount() {
503 return freeCount;
504 }
505
506 public long usedCount() {
507 return usedCount;
508 }
509
510 public long totalCount() {
511 return totalCount;
512 }
513
514 public long freeBytes() {
515 return freeCount * itemSize;
516 }
517
518 public long usedBytes() {
519 return usedCount * itemSize;
520 }
521
522 public long totalBytes() {
523 return totalCount * itemSize;
524 }
525
526 public long itemSize() {
527 return itemSize;
528 }
529
530 public IndexStatistics(long free, long used, long itemSize) {
531 setTo(free, used, itemSize);
532 }
533
534 public IndexStatistics() {
535 setTo(-1, -1, 0);
536 }
537
538 public void setTo(long free, long used, long itemSize) {
539 this.itemSize = itemSize;
540 this.freeCount = free;
541 this.usedCount = used;
542 this.totalCount = free + used;
543 }
544 }
545
546 public Bucket [] getBuckets() {
547 return this.buckets;
548 }
549
550 void logStatistics() {
551 IndexStatistics total = new IndexStatistics();
552 IndexStatistics[] stats = getIndexStatistics(total);
553 LOG.info("Bucket allocator statistics follow:\n");
554 LOG.info(" Free bytes=" + total.freeBytes() + "+; used bytes="
555 + total.usedBytes() + "; total bytes=" + total.totalBytes());
556 for (IndexStatistics s : stats) {
557 LOG.info(" Object size " + s.itemSize() + " used=" + s.usedCount()
558 + "; free=" + s.freeCount() + "; total=" + s.totalCount());
559 }
560 }
561
562 IndexStatistics[] getIndexStatistics(IndexStatistics grandTotal) {
563 IndexStatistics[] stats = getIndexStatistics();
564 long totalfree = 0, totalused = 0;
565 for (IndexStatistics stat : stats) {
566 totalfree += stat.freeBytes();
567 totalused += stat.usedBytes();
568 }
569 grandTotal.setTo(totalfree, totalused, 1);
570 return stats;
571 }
572
573 IndexStatistics[] getIndexStatistics() {
574 IndexStatistics[] stats = new IndexStatistics[bucketSizes.length];
575 for (int i = 0; i < stats.length; ++i)
576 stats[i] = bucketSizeInfos[i].statistics();
577 return stats;
578 }
579
580 public long freeBlock(long freeList[]) {
581 long sz = 0;
582 for (int i = 0; i < freeList.length; ++i)
583 sz += freeBlock(freeList[i]);
584 return sz;
585 }
586
587 public int getBucketIndex(long offset) {
588 return (int) (offset / bucketCapacity);
589 }
590
591
592
593
594
595
596
597
598
599
600
601
602 public Set<Integer> getLeastFilledBuckets(Set<Integer> excludedBuckets,
603 int bucketCount) {
604 Queue<Integer> queue = MinMaxPriorityQueue.<Integer>orderedBy(
605 new Comparator<Integer>() {
606 @Override
607 public int compare(Integer left, Integer right) {
608
609 return Float.compare(
610 ((float) buckets[left].usedCount) / buckets[left].itemCount,
611 ((float) buckets[right].usedCount) / buckets[right].itemCount);
612 }
613 }).maximumSize(bucketCount).create();
614
615 for (int i = 0; i < buckets.length; i ++ ) {
616 if (!excludedBuckets.contains(i) && !buckets[i].isUninstantiated() &&
617
618 bucketSizeInfos[buckets[i].sizeIndex()].bucketList.size() != 1) {
619 queue.add(i);
620 }
621 }
622
623 Set<Integer> result = new HashSet<>(bucketCount);
624 result.addAll(queue);
625
626 return result;
627 }
628 }