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