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