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