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