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