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