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