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