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