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