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