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