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