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