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 = true;
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
258
259
260 void rollback(final KeyValue kv) {
261 this.lock.readLock().lock();
262 try {
263
264
265
266
267
268 KeyValue found = this.snapshot.get(kv);
269 if (found != null && found.getMemstoreTS() == kv.getMemstoreTS()) {
270 this.snapshot.remove(kv);
271 }
272
273 found = this.kvset.get(kv);
274 if (found != null && found.getMemstoreTS() == kv.getMemstoreTS()) {
275 this.kvset.remove(kv);
276 long s = heapSizeChange(kv, true);
277 this.size.addAndGet(-s);
278 }
279 } finally {
280 this.lock.readLock().unlock();
281 }
282 }
283
284
285
286
287
288
289 long delete(final KeyValue delete) {
290 long s = 0;
291 this.lock.readLock().lock();
292 try {
293 KeyValue toAdd = maybeCloneWithAllocator(delete);
294 s += heapSizeChange(toAdd, this.kvset.add(toAdd));
295 timeRangeTracker.includeTimestamp(toAdd);
296 } finally {
297 this.lock.readLock().unlock();
298 }
299 this.size.addAndGet(s);
300 return s;
301 }
302
303
304
305
306
307
308 KeyValue getNextRow(final KeyValue kv) {
309 this.lock.readLock().lock();
310 try {
311 return getLowest(getNextRow(kv, this.kvset), getNextRow(kv, this.snapshot));
312 } finally {
313 this.lock.readLock().unlock();
314 }
315 }
316
317
318
319
320
321
322 private KeyValue getLowest(final KeyValue a, final KeyValue b) {
323 if (a == null) {
324 return b;
325 }
326 if (b == null) {
327 return a;
328 }
329 return comparator.compareRows(a, b) <= 0? a: b;
330 }
331
332
333
334
335
336
337
338 private KeyValue getNextRow(final KeyValue key,
339 final NavigableSet<KeyValue> set) {
340 KeyValue result = null;
341 SortedSet<KeyValue> tail = key == null? set: set.tailSet(key);
342
343 for (KeyValue kv: tail) {
344 if (comparator.compareRows(kv, key) <= 0)
345 continue;
346
347
348 result = kv;
349 break;
350 }
351 return result;
352 }
353
354
355
356
357 void getRowKeyAtOrBefore(final GetClosestRowBeforeTracker state) {
358 this.lock.readLock().lock();
359 try {
360 getRowKeyAtOrBefore(kvset, state);
361 getRowKeyAtOrBefore(snapshot, state);
362 } finally {
363 this.lock.readLock().unlock();
364 }
365 }
366
367
368
369
370
371 private void getRowKeyAtOrBefore(final NavigableSet<KeyValue> set,
372 final GetClosestRowBeforeTracker state) {
373 if (set.isEmpty()) {
374 return;
375 }
376 if (!walkForwardInSingleRow(set, state.getTargetKey(), state)) {
377
378 getRowKeyBefore(set, state);
379 }
380 }
381
382
383
384
385
386
387
388
389
390
391
392 private boolean walkForwardInSingleRow(final SortedSet<KeyValue> set,
393 final KeyValue firstOnRow, final GetClosestRowBeforeTracker state) {
394 boolean foundCandidate = false;
395 SortedSet<KeyValue> tail = set.tailSet(firstOnRow);
396 if (tail.isEmpty()) return foundCandidate;
397 for (Iterator<KeyValue> i = tail.iterator(); i.hasNext();) {
398 KeyValue kv = i.next();
399
400 if (state.isTooFar(kv, firstOnRow)) break;
401 if (state.isExpired(kv)) {
402 i.remove();
403 continue;
404 }
405
406 if (state.handle(kv)) {
407 foundCandidate = true;
408 break;
409 }
410 }
411 return foundCandidate;
412 }
413
414
415
416
417
418
419
420 private void getRowKeyBefore(NavigableSet<KeyValue> set,
421 final GetClosestRowBeforeTracker state) {
422 KeyValue firstOnRow = state.getTargetKey();
423 for (Member p = memberOfPreviousRow(set, state, firstOnRow);
424 p != null; p = memberOfPreviousRow(p.set, state, firstOnRow)) {
425
426 if (!state.isTargetTable(p.kv)) break;
427
428 if (!state.isBetterCandidate(p.kv)) break;
429
430 firstOnRow = new KeyValue(p.kv.getRow(), HConstants.LATEST_TIMESTAMP);
431
432 if (walkForwardInSingleRow(p.set, firstOnRow, state)) break;
433 }
434 }
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451 public long updateColumnValue(byte[] row,
452 byte[] family,
453 byte[] qualifier,
454 long newValue,
455 long now) {
456 this.lock.readLock().lock();
457 try {
458 KeyValue firstKv = KeyValue.createFirstOnRow(
459 row, family, qualifier);
460
461 SortedSet<KeyValue> snSs = snapshot.tailSet(firstKv);
462 if (!snSs.isEmpty()) {
463 KeyValue snKv = snSs.first();
464
465 if (snKv.matchingRow(firstKv) && snKv.matchingQualifier(firstKv)) {
466 if (snKv.getTimestamp() == now) {
467
468 now += 1;
469 }
470 }
471 }
472
473
474
475
476
477
478
479 SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
480 Iterator<KeyValue> it = ss.iterator();
481 while ( it.hasNext() ) {
482 KeyValue kv = it.next();
483
484
485 if (!kv.matchingColumn(family,qualifier) || !kv.matchingRow(firstKv) ) {
486 break;
487 }
488
489
490 if (kv.getType() == KeyValue.Type.Put.getCode() &&
491 kv.getTimestamp() > now && firstKv.matchingQualifier(kv)) {
492 now = kv.getTimestamp();
493 }
494 }
495
496
497
498 return upsert(Arrays.asList(
499 new KeyValue(row, family, qualifier, now, Bytes.toBytes(newValue)))
500 );
501 } finally {
502 this.lock.readLock().unlock();
503 }
504 }
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523 public long upsert(List<KeyValue> kvs) {
524 this.lock.readLock().lock();
525 try {
526 long size = 0;
527 for (KeyValue kv : kvs) {
528 kv.setMemstoreTS(0);
529 size += upsert(kv);
530 }
531 return size;
532 } finally {
533 this.lock.readLock().unlock();
534 }
535 }
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551 private long upsert(KeyValue kv) {
552
553
554
555
556
557
558 long addedSize = internalAdd(kv);
559
560
561
562 KeyValue firstKv = KeyValue.createFirstOnRow(
563 kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
564 kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
565 kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
566 SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
567 Iterator<KeyValue> it = ss.iterator();
568 while ( it.hasNext() ) {
569 KeyValue cur = it.next();
570
571 if (kv == cur) {
572
573 continue;
574 }
575
576 if (!kv.matchingRow(cur)) {
577 break;
578 }
579
580
581 if (kv.matchingQualifier(cur)) {
582
583
584 if (kv.getType() == KeyValue.Type.Put.getCode() &&
585 kv.getMemstoreTS() == 0) {
586
587 long delta = heapSizeChange(cur, true);
588 addedSize -= delta;
589 this.size.addAndGet(-delta);
590 it.remove();
591 }
592 } else {
593
594 break;
595 }
596 }
597 return addedSize;
598 }
599
600
601
602
603
604 private static class Member {
605 final KeyValue kv;
606 final NavigableSet<KeyValue> set;
607 Member(final NavigableSet<KeyValue> s, final KeyValue kv) {
608 this.kv = kv;
609 this.set = s;
610 }
611 }
612
613
614
615
616
617
618
619
620
621 private Member memberOfPreviousRow(NavigableSet<KeyValue> set,
622 final GetClosestRowBeforeTracker state, final KeyValue firstOnRow) {
623 NavigableSet<KeyValue> head = set.headSet(firstOnRow, false);
624 if (head.isEmpty()) return null;
625 for (Iterator<KeyValue> i = head.descendingIterator(); i.hasNext();) {
626 KeyValue found = i.next();
627 if (state.isExpired(found)) {
628 i.remove();
629 continue;
630 }
631 return new Member(head, found);
632 }
633 return null;
634 }
635
636
637
638
639 List<KeyValueScanner> getScanners() {
640 this.lock.readLock().lock();
641 try {
642 return Collections.<KeyValueScanner>singletonList(
643 new MemStoreScanner());
644 } finally {
645 this.lock.readLock().unlock();
646 }
647 }
648
649
650
651
652
653
654 public boolean shouldSeek(Scan scan, long oldestUnexpiredTS) {
655 return (timeRangeTracker.includesTimeRange(scan.getTimeRange()) ||
656 snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange()))
657 && (Math.max(timeRangeTracker.getMaximumTimestamp(),
658 snapshotTimeRangeTracker.getMaximumTimestamp()) >=
659 oldestUnexpiredTS);
660 }
661
662 public TimeRangeTracker getSnapshotTimeRangeTracker() {
663 return this.snapshotTimeRangeTracker;
664 }
665
666
667
668
669
670
671
672 protected class MemStoreScanner extends NonLazyKeyValueScanner {
673
674 private KeyValue kvsetNextRow = null;
675 private KeyValue snapshotNextRow = null;
676
677
678 private KeyValue kvsetItRow = null;
679 private KeyValue snapshotItRow = null;
680
681
682 private Iterator<KeyValue> kvsetIt;
683 private Iterator<KeyValue> snapshotIt;
684
685
686 volatile KeyValueSkipListSet kvsetAtCreation;
687 volatile KeyValueSkipListSet snapshotAtCreation;
688
689
690 private KeyValue theNext;
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713 MemStoreScanner() {
714 super();
715
716 kvsetAtCreation = kvset;
717 snapshotAtCreation = snapshot;
718 }
719
720 private KeyValue getNext(Iterator<KeyValue> it) {
721 long readPoint = MultiVersionConsistencyControl.getThreadReadPoint();
722
723 KeyValue v = null;
724 try {
725 while (it.hasNext()) {
726 v = it.next();
727 if (v.getMemstoreTS() <= readPoint) {
728 return v;
729 }
730 }
731
732 return null;
733 } finally {
734 if (v != null) {
735
736 if (it == snapshotIt) {
737 snapshotItRow = v;
738 } else {
739 kvsetItRow = v;
740 }
741 }
742 }
743 }
744
745
746
747
748
749
750
751
752 @Override
753 public synchronized boolean seek(KeyValue key) {
754 if (key == null) {
755 close();
756 return false;
757 }
758
759
760
761 kvsetIt = kvsetAtCreation.tailSet(key).iterator();
762 snapshotIt = snapshotAtCreation.tailSet(key).iterator();
763 kvsetItRow = null;
764 snapshotItRow = null;
765
766 return seekInSubLists(key);
767 }
768
769
770
771
772
773 private synchronized boolean seekInSubLists(KeyValue key){
774 kvsetNextRow = getNext(kvsetIt);
775 snapshotNextRow = getNext(snapshotIt);
776
777
778 theNext = getLowest(kvsetNextRow, snapshotNextRow);
779
780
781 return (theNext != null);
782 }
783
784
785
786
787
788
789
790 @Override
791 public synchronized boolean reseek(KeyValue key) {
792
793
794
795
796
797
798
799
800
801
802
803
804
805 kvsetIt = kvsetAtCreation.tailSet(getHighest(key, kvsetItRow)).iterator();
806 snapshotIt = snapshotAtCreation.tailSet(getHighest(key, snapshotItRow)).iterator();
807
808 return seekInSubLists(key);
809 }
810
811
812 @Override
813 public synchronized KeyValue peek() {
814
815 return theNext;
816 }
817
818 @Override
819 public synchronized KeyValue next() {
820 if (theNext == null) {
821 return null;
822 }
823
824 final KeyValue ret = theNext;
825
826
827 if (theNext == kvsetNextRow) {
828 kvsetNextRow = getNext(kvsetIt);
829 } else {
830 snapshotNextRow = getNext(snapshotIt);
831 }
832
833
834 theNext = getLowest(kvsetNextRow, snapshotNextRow);
835
836
837
838
839 return ret;
840 }
841
842
843
844
845
846
847 private KeyValue getLowest(KeyValue first, KeyValue second) {
848 if (first == null && second == null) {
849 return null;
850 }
851 if (first != null && second != null) {
852 int compare = comparator.compare(first, second);
853 return (compare <= 0 ? first : second);
854 }
855 return (first != null ? first : second);
856 }
857
858
859
860
861
862
863 private KeyValue getHighest(KeyValue first, KeyValue second) {
864 if (first == null && second == null) {
865 return null;
866 }
867 if (first != null && second != null) {
868 int compare = comparator.compare(first, second);
869 return (compare > 0 ? first : second);
870 }
871 return (first != null ? first : second);
872 }
873
874 public synchronized void close() {
875 this.kvsetNextRow = null;
876 this.snapshotNextRow = null;
877
878 this.kvsetIt = null;
879 this.snapshotIt = null;
880
881 this.kvsetItRow = null;
882 this.snapshotItRow = null;
883 }
884
885
886
887
888
889 @Override
890 public long getSequenceID() {
891 return Long.MAX_VALUE;
892 }
893
894 @Override
895 public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
896 long oldestUnexpiredTS) {
897 return shouldSeek(scan, oldestUnexpiredTS);
898 }
899 }
900
901 public final static long FIXED_OVERHEAD = ClassSize.align(
902 ClassSize.OBJECT + (11 * ClassSize.REFERENCE));
903
904 public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
905 ClassSize.REENTRANT_LOCK + ClassSize.ATOMIC_LONG +
906 ClassSize.COPYONWRITE_ARRAYSET + ClassSize.COPYONWRITE_ARRAYLIST +
907 (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
908
909
910 public static final boolean NO_PERSISTENT_TS = false;
911
912
913
914
915
916
917
918
919 long heapSizeChange(final KeyValue kv, final boolean notpresent) {
920 return notpresent ?
921 ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize()):
922 0;
923 }
924
925
926
927
928
929 @Override
930 public long heapSize() {
931 return size.get();
932 }
933
934
935
936
937 public long keySize() {
938 return heapSize() - DEEP_OVERHEAD;
939 }
940
941
942
943
944
945
946
947
948 public static void main(String [] args) {
949 RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
950 LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
951 runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
952 LOG.info("vmInputArguments=" + runtime.getInputArguments());
953 MemStore memstore1 = new MemStore();
954
955 long size = 0;
956 final int count = 10000;
957 byte [] fam = Bytes.toBytes("col");
958 byte [] qf = Bytes.toBytes("umn");
959 byte [] empty = new byte[0];
960 for (int i = 0; i < count; i++) {
961
962 size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
963 }
964 LOG.info("memstore1 estimated size=" + size);
965 for (int i = 0; i < count; i++) {
966 size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
967 }
968 LOG.info("memstore1 estimated size (2nd loading of same data)=" + size);
969
970 MemStore memstore2 = new MemStore();
971 for (int i = 0; i < count; i++) {
972 size += memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i,
973 new byte[i]));
974 }
975 LOG.info("memstore2 estimated size=" + size);
976 final int seconds = 30;
977 LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
978 for (int i = 0; i < seconds; i++) {
979
980 }
981 LOG.info("Exiting.");
982 }
983 }