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 long sz = heapSizeChange(kv, true);
305 this.snapshotSize -= sz;
306 }
307
308 found = this.kvset.get(kv);
309 if (found != null && found.getMvccVersion() == kv.getMvccVersion()) {
310 removeFromKVSet(kv);
311 long s = heapSizeChange(kv, true);
312 this.size.addAndGet(-s);
313 }
314 }
315
316
317
318
319
320
321 long delete(final KeyValue delete) {
322 long s = 0;
323 KeyValue toAdd = maybeCloneWithAllocator(delete);
324 s += heapSizeChange(toAdd, addToKVSet(toAdd));
325 timeRangeTracker.includeTimestamp(toAdd);
326 this.size.addAndGet(s);
327 return s;
328 }
329
330
331
332
333
334
335 KeyValue getNextRow(final KeyValue kv) {
336 return getLowest(getNextRow(kv, this.kvset), getNextRow(kv, this.snapshot));
337 }
338
339
340
341
342
343
344 private KeyValue getLowest(final KeyValue a, final KeyValue b) {
345 if (a == null) {
346 return b;
347 }
348 if (b == null) {
349 return a;
350 }
351 return comparator.compareRows(a, b) <= 0? a: b;
352 }
353
354
355
356
357
358
359
360 private KeyValue getNextRow(final KeyValue key,
361 final NavigableSet<KeyValue> set) {
362 KeyValue result = null;
363 SortedSet<KeyValue> tail = key == null? set: set.tailSet(key);
364
365 for (KeyValue kv: tail) {
366 if (comparator.compareRows(kv, key) <= 0)
367 continue;
368
369
370 result = kv;
371 break;
372 }
373 return result;
374 }
375
376
377
378
379 void getRowKeyAtOrBefore(final GetClosestRowBeforeTracker state) {
380 getRowKeyAtOrBefore(kvset, state);
381 getRowKeyAtOrBefore(snapshot, state);
382 }
383
384
385
386
387
388 private void getRowKeyAtOrBefore(final NavigableSet<KeyValue> set,
389 final GetClosestRowBeforeTracker state) {
390 if (set.isEmpty()) {
391 return;
392 }
393 if (!walkForwardInSingleRow(set, state.getTargetKey(), state)) {
394
395 getRowKeyBefore(set, state);
396 }
397 }
398
399
400
401
402
403
404
405
406
407
408
409 private boolean walkForwardInSingleRow(final SortedSet<KeyValue> set,
410 final KeyValue firstOnRow, final GetClosestRowBeforeTracker state) {
411 boolean foundCandidate = false;
412 SortedSet<KeyValue> tail = set.tailSet(firstOnRow);
413 if (tail.isEmpty()) return foundCandidate;
414 for (Iterator<KeyValue> i = tail.iterator(); i.hasNext();) {
415 KeyValue kv = i.next();
416
417 if (state.isTooFar(kv, firstOnRow)) break;
418 if (state.isExpired(kv)) {
419 i.remove();
420 continue;
421 }
422
423 if (state.handle(kv)) {
424 foundCandidate = true;
425 break;
426 }
427 }
428 return foundCandidate;
429 }
430
431
432
433
434
435
436
437 private void getRowKeyBefore(NavigableSet<KeyValue> set,
438 final GetClosestRowBeforeTracker state) {
439 KeyValue firstOnRow = state.getTargetKey();
440 for (Member p = memberOfPreviousRow(set, state, firstOnRow);
441 p != null; p = memberOfPreviousRow(p.set, state, firstOnRow)) {
442
443 if (!state.isTargetTable(p.kv)) break;
444
445 if (!state.isBetterCandidate(p.kv)) break;
446
447 firstOnRow = new KeyValue(p.kv.getRowArray(), p.kv.getRowOffset(), p.kv.getRowLength(),
448 HConstants.LATEST_TIMESTAMP);
449
450 if (walkForwardInSingleRow(p.set, firstOnRow, state)) break;
451 }
452 }
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471 long updateColumnValue(byte[] row,
472 byte[] family,
473 byte[] qualifier,
474 long newValue,
475 long now) {
476 KeyValue firstKv = KeyValue.createFirstOnRow(
477 row, family, qualifier);
478
479 SortedSet<KeyValue> snSs = snapshot.tailSet(firstKv);
480 if (!snSs.isEmpty()) {
481 KeyValue snKv = snSs.first();
482
483 if (snKv.matchingRow(firstKv) && snKv.matchingQualifier(firstKv)) {
484 if (snKv.getTimestamp() == now) {
485
486 now += 1;
487 }
488 }
489 }
490
491
492
493
494
495
496
497 SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
498 for (KeyValue kv : ss) {
499
500 if (!kv.matchingColumn(family, qualifier) || !kv.matchingRow(firstKv)) {
501 break;
502 }
503
504
505 if (kv.getTypeByte() == KeyValue.Type.Put.getCode() &&
506 kv.getTimestamp() > now && firstKv.matchingQualifier(kv)) {
507 now = kv.getTimestamp();
508 }
509 }
510
511
512
513 List<Cell> cells = new ArrayList<Cell>(1);
514 cells.add(new KeyValue(row, family, qualifier, now, Bytes.toBytes(newValue)));
515 return upsert(cells, 1L);
516 }
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536 public long upsert(Iterable<Cell> cells, long readpoint) {
537 long size = 0;
538 for (Cell cell : cells) {
539 size += upsert(cell, readpoint);
540 }
541 return size;
542 }
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558 private long upsert(Cell cell, long readpoint) {
559
560
561
562
563
564
565 KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
566 long addedSize = internalAdd(kv);
567
568
569
570 KeyValue firstKv = KeyValue.createFirstOnRow(
571 kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
572 kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
573 kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
574 SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
575 Iterator<KeyValue> it = ss.iterator();
576
577 int versionsVisible = 0;
578 while ( it.hasNext() ) {
579 KeyValue cur = it.next();
580
581 if (kv == cur) {
582
583 continue;
584 }
585
586 if (kv.matchingRow(cur) && kv.matchingQualifier(cur)) {
587
588 if (cur.getTypeByte() == KeyValue.Type.Put.getCode() &&
589 cur.getMvccVersion() <= readpoint) {
590 if (versionsVisible > 1) {
591
592
593
594
595 long delta = heapSizeChange(cur, true);
596 addedSize -= delta;
597 this.size.addAndGet(-delta);
598 it.remove();
599 setOldestEditTimeToNow();
600 } else {
601 versionsVisible++;
602 }
603 }
604 } else {
605
606 break;
607 }
608 }
609 return addedSize;
610 }
611
612
613
614
615
616 private static class Member {
617 final KeyValue kv;
618 final NavigableSet<KeyValue> set;
619 Member(final NavigableSet<KeyValue> s, final KeyValue kv) {
620 this.kv = kv;
621 this.set = s;
622 }
623 }
624
625
626
627
628
629
630
631
632
633 private Member memberOfPreviousRow(NavigableSet<KeyValue> set,
634 final GetClosestRowBeforeTracker state, final KeyValue firstOnRow) {
635 NavigableSet<KeyValue> head = set.headSet(firstOnRow, false);
636 if (head.isEmpty()) return null;
637 for (Iterator<KeyValue> i = head.descendingIterator(); i.hasNext();) {
638 KeyValue found = i.next();
639 if (state.isExpired(found)) {
640 i.remove();
641 continue;
642 }
643 return new Member(head, found);
644 }
645 return null;
646 }
647
648
649
650
651 List<KeyValueScanner> getScanners(long readPt) {
652 return Collections.<KeyValueScanner>singletonList(
653 new MemStoreScanner(readPt));
654 }
655
656
657
658
659
660
661 public boolean shouldSeek(Scan scan, long oldestUnexpiredTS) {
662 return (timeRangeTracker.includesTimeRange(scan.getTimeRange()) ||
663 snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange()))
664 && (Math.max(timeRangeTracker.getMaximumTimestamp(),
665 snapshotTimeRangeTracker.getMaximumTimestamp()) >=
666 oldestUnexpiredTS);
667 }
668
669 public TimeRangeTracker getSnapshotTimeRangeTracker() {
670 return this.snapshotTimeRangeTracker;
671 }
672
673
674
675
676
677
678
679 protected class MemStoreScanner extends NonLazyKeyValueScanner {
680
681 private KeyValue kvsetNextRow = null;
682 private KeyValue snapshotNextRow = null;
683
684
685 private KeyValue kvsetItRow = null;
686 private KeyValue snapshotItRow = null;
687
688
689 private Iterator<KeyValue> kvsetIt;
690 private Iterator<KeyValue> snapshotIt;
691
692
693 private KeyValueSkipListSet kvsetAtCreation;
694 private KeyValueSkipListSet snapshotAtCreation;
695
696
697 private KeyValue theNext;
698
699
700 volatile MemStoreLAB allocatorAtCreation;
701 volatile MemStoreLAB snapshotAllocatorAtCreation;
702
703
704
705 private boolean stopSkippingKVsIfNextRow = false;
706
707 private long readPoint;
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730 MemStoreScanner(long readPoint) {
731 super();
732
733 this.readPoint = readPoint;
734 kvsetAtCreation = kvset;
735 snapshotAtCreation = snapshot;
736 if (allocator != null) {
737 this.allocatorAtCreation = allocator;
738 this.allocatorAtCreation.incScannerCount();
739 }
740 if (snapshotAllocator != null) {
741 this.snapshotAllocatorAtCreation = snapshotAllocator;
742 this.snapshotAllocatorAtCreation.incScannerCount();
743 }
744 }
745
746 private KeyValue getNext(Iterator<KeyValue> it) {
747 KeyValue startKV = theNext;
748 KeyValue v = null;
749 try {
750 while (it.hasNext()) {
751 v = it.next();
752 if (v.getMvccVersion() <= this.readPoint) {
753 return v;
754 }
755 if (stopSkippingKVsIfNextRow && startKV != null
756 && comparator.compareRows(v, startKV) > 0) {
757 return null;
758 }
759 }
760
761 return null;
762 } finally {
763 if (v != null) {
764
765 if (it == snapshotIt) {
766 snapshotItRow = v;
767 } else {
768 kvsetItRow = v;
769 }
770 }
771 }
772 }
773
774
775
776
777
778
779
780
781 @Override
782 public synchronized boolean seek(KeyValue key) {
783 if (key == null) {
784 close();
785 return false;
786 }
787
788
789
790 kvsetIt = kvsetAtCreation.tailSet(key).iterator();
791 snapshotIt = snapshotAtCreation.tailSet(key).iterator();
792 kvsetItRow = null;
793 snapshotItRow = null;
794
795 return seekInSubLists(key);
796 }
797
798
799
800
801
802 private synchronized boolean seekInSubLists(KeyValue key){
803 kvsetNextRow = getNext(kvsetIt);
804 snapshotNextRow = getNext(snapshotIt);
805
806
807 theNext = getLowest(kvsetNextRow, snapshotNextRow);
808
809
810 return (theNext != null);
811 }
812
813
814
815
816
817
818
819 @Override
820 public synchronized boolean reseek(KeyValue key) {
821
822
823
824
825
826
827
828
829
830
831
832
833
834 kvsetIt = kvsetAtCreation.tailSet(getHighest(key, kvsetItRow)).iterator();
835 snapshotIt = snapshotAtCreation.tailSet(getHighest(key, snapshotItRow)).iterator();
836
837 return seekInSubLists(key);
838 }
839
840
841 @Override
842 public synchronized KeyValue peek() {
843
844 return theNext;
845 }
846
847 @Override
848 public synchronized KeyValue next() {
849 if (theNext == null) {
850 return null;
851 }
852
853 final KeyValue ret = theNext;
854
855
856 if (theNext == kvsetNextRow) {
857 kvsetNextRow = getNext(kvsetIt);
858 } else {
859 snapshotNextRow = getNext(snapshotIt);
860 }
861
862
863 theNext = getLowest(kvsetNextRow, snapshotNextRow);
864
865
866
867
868 return ret;
869 }
870
871
872
873
874
875
876 private KeyValue getLowest(KeyValue first, KeyValue second) {
877 if (first == null && second == null) {
878 return null;
879 }
880 if (first != null && second != null) {
881 int compare = comparator.compare(first, second);
882 return (compare <= 0 ? first : second);
883 }
884 return (first != null ? first : second);
885 }
886
887
888
889
890
891
892 private KeyValue getHighest(KeyValue first, KeyValue second) {
893 if (first == null && second == null) {
894 return null;
895 }
896 if (first != null && second != null) {
897 int compare = comparator.compare(first, second);
898 return (compare > 0 ? first : second);
899 }
900 return (first != null ? first : second);
901 }
902
903 public synchronized void close() {
904 this.kvsetNextRow = null;
905 this.snapshotNextRow = null;
906
907 this.kvsetIt = null;
908 this.snapshotIt = null;
909
910 if (allocatorAtCreation != null) {
911 this.allocatorAtCreation.decScannerCount();
912 this.allocatorAtCreation = null;
913 }
914 if (snapshotAllocatorAtCreation != null) {
915 this.snapshotAllocatorAtCreation.decScannerCount();
916 this.snapshotAllocatorAtCreation = null;
917 }
918
919 this.kvsetItRow = null;
920 this.snapshotItRow = null;
921 }
922
923
924
925
926
927 @Override
928 public long getSequenceID() {
929 return Long.MAX_VALUE;
930 }
931
932 @Override
933 public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
934 long oldestUnexpiredTS) {
935 return shouldSeek(scan, oldestUnexpiredTS);
936 }
937
938
939
940
941
942
943 @Override
944 public synchronized boolean backwardSeek(KeyValue key) {
945 seek(key);
946 if (peek() == null || comparator.compareRows(peek(), key) > 0) {
947 return seekToPreviousRow(key);
948 }
949 return true;
950 }
951
952
953
954
955
956
957 @Override
958 public synchronized boolean seekToPreviousRow(KeyValue key) {
959 KeyValue firstKeyOnRow = KeyValue.createFirstOnRow(key.getRow());
960 SortedSet<KeyValue> kvHead = kvsetAtCreation.headSet(firstKeyOnRow);
961 KeyValue kvsetBeforeRow = kvHead.isEmpty() ? null : kvHead.last();
962 SortedSet<KeyValue> snapshotHead = snapshotAtCreation
963 .headSet(firstKeyOnRow);
964 KeyValue snapshotBeforeRow = snapshotHead.isEmpty() ? null : snapshotHead
965 .last();
966 KeyValue lastKVBeforeRow = getHighest(kvsetBeforeRow, snapshotBeforeRow);
967 if (lastKVBeforeRow == null) {
968 theNext = null;
969 return false;
970 }
971 KeyValue firstKeyOnPreviousRow = KeyValue
972 .createFirstOnRow(lastKVBeforeRow.getRow());
973 this.stopSkippingKVsIfNextRow = true;
974 seek(firstKeyOnPreviousRow);
975 this.stopSkippingKVsIfNextRow = false;
976 if (peek() == null
977 || comparator.compareRows(peek(), firstKeyOnPreviousRow) > 0) {
978 return seekToPreviousRow(lastKVBeforeRow);
979 }
980 return true;
981 }
982
983 @Override
984 public synchronized boolean seekToLastRow() {
985 KeyValue first = kvsetAtCreation.isEmpty() ? null : kvsetAtCreation
986 .last();
987 KeyValue second = snapshotAtCreation.isEmpty() ? null
988 : snapshotAtCreation.last();
989 KeyValue higherKv = getHighest(first, second);
990 if (higherKv == null) {
991 return false;
992 }
993 KeyValue firstKvOnLastRow = KeyValue.createFirstOnRow(higherKv.getRow());
994 if (seek(firstKvOnLastRow)) {
995 return true;
996 } else {
997 return seekToPreviousRow(higherKv);
998 }
999
1000 }
1001 }
1002
1003 public final static long FIXED_OVERHEAD = ClassSize.align(
1004 ClassSize.OBJECT + (10 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG));
1005
1006 public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
1007 ClassSize.ATOMIC_LONG + (2 * ClassSize.TIMERANGE_TRACKER) +
1008 (2 * ClassSize.KEYVALUE_SKIPLIST_SET) + (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
1009
1010
1011
1012
1013
1014
1015
1016
1017 static long heapSizeChange(final KeyValue kv, final boolean notpresent) {
1018 return notpresent ?
1019 ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize()):
1020 0;
1021 }
1022
1023
1024
1025
1026
1027 @Override
1028 public long heapSize() {
1029 return size.get();
1030 }
1031
1032
1033
1034
1035 public long keySize() {
1036 return heapSize() - DEEP_OVERHEAD;
1037 }
1038
1039
1040
1041
1042
1043
1044
1045
1046 public static void main(String [] args) {
1047 RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
1048 LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
1049 runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
1050 LOG.info("vmInputArguments=" + runtime.getInputArguments());
1051 MemStore memstore1 = new MemStore();
1052
1053 long size = 0;
1054 final int count = 10000;
1055 byte [] fam = Bytes.toBytes("col");
1056 byte [] qf = Bytes.toBytes("umn");
1057 byte [] empty = new byte[0];
1058 for (int i = 0; i < count; i++) {
1059
1060 size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
1061 }
1062 LOG.info("memstore1 estimated size=" + size);
1063 for (int i = 0; i < count; i++) {
1064 size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
1065 }
1066 LOG.info("memstore1 estimated size (2nd loading of same data)=" + size);
1067
1068 MemStore memstore2 = new MemStore();
1069 for (int i = 0; i < count; i++) {
1070 size += memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i,
1071 new byte[i]));
1072 }
1073 LOG.info("memstore2 estimated size=" + size);
1074 final int seconds = 30;
1075 LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
1076 for (int i = 0; i < seconds; i++) {
1077
1078 }
1079 LOG.info("Exiting.");
1080 }
1081 }