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