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.regionserver;
22
23 import java.lang.management.ManagementFactory;
24 import java.lang.management.RuntimeMXBean;
25 import java.rmi.UnexpectedException;
26 import java.util.Arrays;
27 import java.util.Collections;
28 import java.util.Iterator;
29 import java.util.List;
30 import java.util.NavigableSet;
31 import java.util.SortedSet;
32 import java.util.concurrent.atomic.AtomicLong;
33 import java.util.concurrent.locks.ReentrantReadWriteLock;
34
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.hbase.HBaseConfiguration;
39 import org.apache.hadoop.hbase.HConstants;
40 import org.apache.hadoop.hbase.KeyValue;
41 import org.apache.hadoop.hbase.client.Scan;
42 import org.apache.hadoop.hbase.io.HeapSize;
43 import org.apache.hadoop.hbase.regionserver.MemStoreLAB.Allocation;
44 import org.apache.hadoop.hbase.util.Bytes;
45 import org.apache.hadoop.hbase.util.ClassSize;
46
47
48
49
50
51
52
53
54
55
56
57
58 public class MemStore implements HeapSize {
59 private static final Log LOG = LogFactory.getLog(MemStore.class);
60
61 static final String USEMSLAB_KEY =
62 "hbase.hregion.memstore.mslab.enabled";
63 private static final boolean USEMSLAB_DEFAULT = false;
64
65 private Configuration conf;
66
67
68
69
70
71
72 volatile KeyValueSkipListSet kvset;
73
74
75 volatile KeyValueSkipListSet snapshot;
76
77 final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
78
79 final KeyValue.KVComparator comparator;
80
81
82 final KeyValue.KVComparator comparatorIgnoreType;
83
84
85 final KeyValue.KVComparator comparatorIgnoreTimestamp;
86
87
88 final AtomicLong size;
89
90 TimeRangeTracker timeRangeTracker;
91 TimeRangeTracker snapshotTimeRangeTracker;
92
93 MemStoreLAB allocator;
94
95
96
97
98
99
100 public MemStore() {
101 this(HBaseConfiguration.create(), KeyValue.COMPARATOR);
102 }
103
104
105
106
107
108 public MemStore(final Configuration conf,
109 final KeyValue.KVComparator c) {
110 this.conf = conf;
111 this.comparator = c;
112 this.comparatorIgnoreTimestamp =
113 this.comparator.getComparatorIgnoringTimestamps();
114 this.comparatorIgnoreType = this.comparator.getComparatorIgnoringType();
115 this.kvset = new KeyValueSkipListSet(c);
116 this.snapshot = new KeyValueSkipListSet(c);
117 timeRangeTracker = new TimeRangeTracker();
118 snapshotTimeRangeTracker = new TimeRangeTracker();
119 this.size = new AtomicLong(DEEP_OVERHEAD);
120 if (conf.getBoolean(USEMSLAB_KEY, USEMSLAB_DEFAULT)) {
121 this.allocator = new MemStoreLAB(conf);
122 } else {
123 this.allocator = null;
124 }
125 }
126
127 void dump() {
128 for (KeyValue kv: this.kvset) {
129 LOG.info(kv);
130 }
131 for (KeyValue kv: this.snapshot) {
132 LOG.info(kv);
133 }
134 }
135
136
137
138
139
140
141 void snapshot() {
142 this.lock.writeLock().lock();
143 try {
144
145
146 if (!this.snapshot.isEmpty()) {
147 LOG.warn("Snapshot called again without clearing previous. " +
148 "Doing nothing. Another ongoing flush or did we fail last attempt?");
149 } else {
150 if (!this.kvset.isEmpty()) {
151 this.snapshot = this.kvset;
152 this.kvset = new KeyValueSkipListSet(this.comparator);
153 this.snapshotTimeRangeTracker = this.timeRangeTracker;
154 this.timeRangeTracker = new TimeRangeTracker();
155
156 this.size.set(DEEP_OVERHEAD);
157
158 if (allocator != null) {
159 this.allocator = new MemStoreLAB(conf);
160 }
161 }
162 }
163 } finally {
164 this.lock.writeLock().unlock();
165 }
166 }
167
168
169
170
171
172
173
174
175
176 KeyValueSkipListSet getSnapshot() {
177 return this.snapshot;
178 }
179
180
181
182
183
184
185
186 void clearSnapshot(final SortedSet<KeyValue> ss)
187 throws UnexpectedException {
188 this.lock.writeLock().lock();
189 try {
190 if (this.snapshot != ss) {
191 throw new UnexpectedException("Current snapshot is " +
192 this.snapshot + ", was passed " + ss);
193 }
194
195
196 if (!ss.isEmpty()) {
197 this.snapshot = new KeyValueSkipListSet(this.comparator);
198 this.snapshotTimeRangeTracker = new TimeRangeTracker();
199 }
200 } finally {
201 this.lock.writeLock().unlock();
202 }
203 }
204
205
206
207
208
209
210 long add(final KeyValue kv) {
211 this.lock.readLock().lock();
212 try {
213 KeyValue toAdd = maybeCloneWithAllocator(kv);
214 return internalAdd(toAdd);
215 } finally {
216 this.lock.readLock().unlock();
217 }
218 }
219
220
221
222
223
224
225
226 private long internalAdd(final KeyValue toAdd) {
227 long s = heapSizeChange(toAdd, this.kvset.add(toAdd));
228 timeRangeTracker.includeTimestamp(toAdd);
229 this.size.addAndGet(s);
230 return s;
231 }
232
233 private KeyValue maybeCloneWithAllocator(KeyValue kv) {
234 if (allocator == null) {
235 return kv;
236 }
237
238 int len = kv.getLength();
239 Allocation alloc = allocator.allocateBytes(len);
240 if (alloc == null) {
241
242
243 return kv;
244 }
245 assert alloc != null && alloc.getData() != null;
246 System.arraycopy(kv.getBuffer(), kv.getOffset(), alloc.getData(), alloc.getOffset(), len);
247 KeyValue newKv = new KeyValue(alloc.getData(), alloc.getOffset(), len);
248 newKv.setMemstoreTS(kv.getMemstoreTS());
249 return newKv;
250 }
251
252
253
254
255
256
257 long delete(final KeyValue delete) {
258 long s = 0;
259 this.lock.readLock().lock();
260 try {
261 KeyValue toAdd = maybeCloneWithAllocator(delete);
262 s += heapSizeChange(toAdd, this.kvset.add(toAdd));
263 timeRangeTracker.includeTimestamp(toAdd);
264 } finally {
265 this.lock.readLock().unlock();
266 }
267 this.size.addAndGet(s);
268 return s;
269 }
270
271
272
273
274
275
276 KeyValue getNextRow(final KeyValue kv) {
277 this.lock.readLock().lock();
278 try {
279 return getLowest(getNextRow(kv, this.kvset), getNextRow(kv, this.snapshot));
280 } finally {
281 this.lock.readLock().unlock();
282 }
283 }
284
285
286
287
288
289
290 private KeyValue getLowest(final KeyValue a, final KeyValue b) {
291 if (a == null) {
292 return b;
293 }
294 if (b == null) {
295 return a;
296 }
297 return comparator.compareRows(a, b) <= 0? a: b;
298 }
299
300
301
302
303
304
305
306 private KeyValue getNextRow(final KeyValue key,
307 final NavigableSet<KeyValue> set) {
308 KeyValue result = null;
309 SortedSet<KeyValue> tail = key == null? set: set.tailSet(key);
310
311 for (KeyValue kv: tail) {
312 if (comparator.compareRows(kv, key) <= 0)
313 continue;
314
315
316 result = kv;
317 break;
318 }
319 return result;
320 }
321
322
323
324
325 void getRowKeyAtOrBefore(final GetClosestRowBeforeTracker state) {
326 this.lock.readLock().lock();
327 try {
328 getRowKeyAtOrBefore(kvset, state);
329 getRowKeyAtOrBefore(snapshot, state);
330 } finally {
331 this.lock.readLock().unlock();
332 }
333 }
334
335
336
337
338
339 private void getRowKeyAtOrBefore(final NavigableSet<KeyValue> set,
340 final GetClosestRowBeforeTracker state) {
341 if (set.isEmpty()) {
342 return;
343 }
344 if (!walkForwardInSingleRow(set, state.getTargetKey(), state)) {
345
346 getRowKeyBefore(set, state);
347 }
348 }
349
350
351
352
353
354
355
356
357
358
359
360 private boolean walkForwardInSingleRow(final SortedSet<KeyValue> set,
361 final KeyValue firstOnRow, final GetClosestRowBeforeTracker state) {
362 boolean foundCandidate = false;
363 SortedSet<KeyValue> tail = set.tailSet(firstOnRow);
364 if (tail.isEmpty()) return foundCandidate;
365 for (Iterator<KeyValue> i = tail.iterator(); i.hasNext();) {
366 KeyValue kv = i.next();
367
368 if (state.isTooFar(kv, firstOnRow)) break;
369 if (state.isExpired(kv)) {
370 i.remove();
371 continue;
372 }
373
374 if (state.handle(kv)) {
375 foundCandidate = true;
376 break;
377 }
378 }
379 return foundCandidate;
380 }
381
382
383
384
385
386
387
388 private void getRowKeyBefore(NavigableSet<KeyValue> set,
389 final GetClosestRowBeforeTracker state) {
390 KeyValue firstOnRow = state.getTargetKey();
391 for (Member p = memberOfPreviousRow(set, state, firstOnRow);
392 p != null; p = memberOfPreviousRow(p.set, state, firstOnRow)) {
393
394 if (!state.isTargetTable(p.kv)) break;
395
396 if (!state.isBetterCandidate(p.kv)) break;
397
398 firstOnRow = new KeyValue(p.kv.getRow(), HConstants.LATEST_TIMESTAMP);
399
400 if (walkForwardInSingleRow(p.set, firstOnRow, state)) break;
401 }
402 }
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419 public long updateColumnValue(byte[] row,
420 byte[] family,
421 byte[] qualifier,
422 long newValue,
423 long now) {
424 this.lock.readLock().lock();
425 try {
426 KeyValue firstKv = KeyValue.createFirstOnRow(
427 row, family, qualifier);
428
429 SortedSet<KeyValue> snSs = snapshot.tailSet(firstKv);
430 if (!snSs.isEmpty()) {
431 KeyValue snKv = snSs.first();
432
433 if (snKv.matchingRow(firstKv) && snKv.matchingQualifier(firstKv)) {
434 if (snKv.getTimestamp() == now) {
435
436 now += 1;
437 }
438 }
439 }
440
441
442
443
444
445
446
447 SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
448 Iterator<KeyValue> it = ss.iterator();
449 while ( it.hasNext() ) {
450 KeyValue kv = it.next();
451
452
453 if (!kv.matchingColumn(family,qualifier) || !kv.matchingRow(firstKv) ) {
454 break;
455 }
456
457
458 if (kv.getType() == KeyValue.Type.Put.getCode() &&
459 kv.getTimestamp() > now && firstKv.matchingQualifier(kv)) {
460 now = kv.getTimestamp();
461 }
462 }
463
464
465
466 return upsert(Arrays.asList(
467 new KeyValue(row, family, qualifier, now, Bytes.toBytes(newValue)))
468 );
469 } finally {
470 this.lock.readLock().unlock();
471 }
472 }
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491 public long upsert(List<KeyValue> kvs) {
492 this.lock.readLock().lock();
493 try {
494 long size = 0;
495 for (KeyValue kv : kvs) {
496 kv.setMemstoreTS(0);
497 size += upsert(kv);
498 }
499 return size;
500 } finally {
501 this.lock.readLock().unlock();
502 }
503 }
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519 private long upsert(KeyValue kv) {
520
521
522
523
524
525
526 long addedSize = internalAdd(kv);
527
528
529
530 KeyValue firstKv = KeyValue.createFirstOnRow(
531 kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
532 kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
533 kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
534 SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
535 Iterator<KeyValue> it = ss.iterator();
536 while ( it.hasNext() ) {
537 KeyValue cur = it.next();
538
539 if (kv == cur) {
540
541 continue;
542 }
543
544 if (!kv.matchingRow(cur)) {
545 break;
546 }
547
548
549 if (kv.matchingQualifier(cur)) {
550
551
552 if (kv.getType() == KeyValue.Type.Put.getCode() &&
553 kv.getMemstoreTS() == 0) {
554
555 addedSize -= heapSizeChange(kv, true);
556 it.remove();
557 }
558 } else {
559
560 break;
561 }
562 }
563 return addedSize;
564 }
565
566
567
568
569
570 private static class Member {
571 final KeyValue kv;
572 final NavigableSet<KeyValue> set;
573 Member(final NavigableSet<KeyValue> s, final KeyValue kv) {
574 this.kv = kv;
575 this.set = s;
576 }
577 }
578
579
580
581
582
583
584
585
586
587 private Member memberOfPreviousRow(NavigableSet<KeyValue> set,
588 final GetClosestRowBeforeTracker state, final KeyValue firstOnRow) {
589 NavigableSet<KeyValue> head = set.headSet(firstOnRow, false);
590 if (head.isEmpty()) return null;
591 for (Iterator<KeyValue> i = head.descendingIterator(); i.hasNext();) {
592 KeyValue found = i.next();
593 if (state.isExpired(found)) {
594 i.remove();
595 continue;
596 }
597 return new Member(head, found);
598 }
599 return null;
600 }
601
602
603
604
605 List<KeyValueScanner> getScanners() {
606 this.lock.readLock().lock();
607 try {
608 return Collections.<KeyValueScanner>singletonList(
609 new MemStoreScanner());
610 } finally {
611 this.lock.readLock().unlock();
612 }
613 }
614
615
616
617
618
619
620 public boolean shouldSeek(Scan scan) {
621 return timeRangeTracker.includesTimeRange(scan.getTimeRange()) ||
622 snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange());
623 }
624
625 public TimeRangeTracker getSnapshotTimeRangeTracker() {
626 return this.snapshotTimeRangeTracker;
627 }
628
629
630
631
632
633
634
635 protected class MemStoreScanner extends AbstractKeyValueScanner {
636
637 private KeyValue kvsetNextRow = null;
638 private KeyValue snapshotNextRow = null;
639
640
641 private Iterator<KeyValue> kvsetIt;
642 private Iterator<KeyValue> snapshotIt;
643
644
645 volatile KeyValueSkipListSet kvsetAtCreation;
646 volatile KeyValueSkipListSet snapshotAtCreation;
647
648
649 private SortedSet<KeyValue> kvTail;
650 private SortedSet<KeyValue> snapshotTail;
651
652
653 private KeyValue theNext;
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676 MemStoreScanner() {
677 super();
678
679 kvsetAtCreation = kvset;
680 snapshotAtCreation = snapshot;
681 }
682
683 protected KeyValue getNext(Iterator<KeyValue> it) {
684 long readPoint = MultiVersionConsistencyControl.getThreadReadPoint();
685
686 while (it.hasNext()) {
687 KeyValue v = it.next();
688 if (v.getMemstoreTS() <= readPoint) {
689 return v;
690 }
691 }
692
693 return null;
694 }
695
696
697
698
699
700
701
702
703 @Override
704 public synchronized boolean seek(KeyValue key) {
705 if (key == null) {
706 close();
707 return false;
708 }
709
710
711
712 kvTail = kvsetAtCreation.tailSet(key);
713 snapshotTail = snapshotAtCreation.tailSet(key);
714
715 return seekInSubLists(key);
716 }
717
718
719
720
721
722 private synchronized boolean seekInSubLists(KeyValue key){
723 kvsetIt = kvTail.iterator();
724 snapshotIt = snapshotTail.iterator();
725
726 kvsetNextRow = getNext(kvsetIt);
727 snapshotNextRow = getNext(snapshotIt);
728
729
730 theNext = getLowest(kvsetNextRow, snapshotNextRow);
731
732
733 return (theNext != null);
734 }
735
736
737
738
739
740
741
742 @Override
743 public synchronized boolean reseek(KeyValue key) {
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762 kvTail = kvTail.tailSet(key);
763 snapshotTail = snapshotTail.tailSet(key);
764
765 return seekInSubLists(key);
766 }
767
768
769 @Override
770 public synchronized KeyValue peek() {
771
772 return theNext;
773 }
774
775 @Override
776 public synchronized KeyValue next() {
777 if (theNext == null) {
778 return null;
779 }
780
781 final KeyValue ret = theNext;
782
783
784 if (theNext == kvsetNextRow) {
785 kvsetNextRow = getNext(kvsetIt);
786 } else {
787 snapshotNextRow = getNext(snapshotIt);
788 }
789
790
791 theNext = getLowest(kvsetNextRow, snapshotNextRow);
792
793
794
795
796 return ret;
797 }
798
799
800
801
802
803
804 protected KeyValue getLowest(KeyValue first, KeyValue second) {
805 if (first == null && second == null) {
806 return null;
807 }
808 if (first != null && second != null) {
809 int compare = comparator.compare(first, second);
810 return (compare <= 0 ? first : second);
811 }
812 return (first != null ? first : second);
813 }
814
815 public synchronized void close() {
816 this.kvsetNextRow = null;
817 this.snapshotNextRow = null;
818
819 this.kvsetIt = null;
820 this.snapshotIt = null;
821 }
822
823
824
825
826
827 @Override
828 public long getSequenceID() {
829 return Long.MAX_VALUE;
830 }
831 }
832
833 public final static long FIXED_OVERHEAD = ClassSize.align(
834 ClassSize.OBJECT + (11 * ClassSize.REFERENCE));
835
836 public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
837 ClassSize.REENTRANT_LOCK + ClassSize.ATOMIC_LONG +
838 ClassSize.COPYONWRITE_ARRAYSET + ClassSize.COPYONWRITE_ARRAYLIST +
839 (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
840
841
842
843
844
845
846
847
848 long heapSizeChange(final KeyValue kv, final boolean notpresent) {
849 return notpresent ?
850 ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize()):
851 0;
852 }
853
854
855
856
857
858 @Override
859 public long heapSize() {
860 return size.get();
861 }
862
863
864
865
866 public long keySize() {
867 return heapSize() - DEEP_OVERHEAD;
868 }
869
870
871
872
873
874
875
876
877 public static void main(String [] args) {
878 RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
879 LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
880 runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
881 LOG.info("vmInputArguments=" + runtime.getInputArguments());
882 MemStore memstore1 = new MemStore();
883
884 long size = 0;
885 final int count = 10000;
886 byte [] fam = Bytes.toBytes("col");
887 byte [] qf = Bytes.toBytes("umn");
888 byte [] empty = new byte[0];
889 for (int i = 0; i < count; i++) {
890
891 size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
892 }
893 LOG.info("memstore1 estimated size=" + size);
894 for (int i = 0; i < count; i++) {
895 size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
896 }
897 LOG.info("memstore1 estimated size (2nd loading of same data)=" + size);
898
899 MemStore memstore2 = new MemStore();
900 for (int i = 0; i < count; i++) {
901 size += memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i,
902 new byte[i]));
903 }
904 LOG.info("memstore2 estimated size=" + size);
905 final int seconds = 30;
906 LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
907 for (int i = 0; i < seconds; i++) {
908
909 }
910 LOG.info("Exiting.");
911 }
912 }