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