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