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