View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.regionserver;
22  
23  import java.io.IOException;
24  import java.lang.management.ManagementFactory;
25  import java.lang.management.RuntimeMXBean;
26  import java.rmi.UnexpectedException;
27  import java.util.Collections;
28  import java.util.Iterator;
29  import java.util.List;
30  import java.util.NavigableSet;
31  import java.util.SortedSet;
32  import java.util.concurrent.atomic.AtomicLong;
33  import java.util.concurrent.locks.ReentrantReadWriteLock;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.hbase.HConstants;
38  import org.apache.hadoop.hbase.KeyValue;
39  import org.apache.hadoop.hbase.client.Scan;
40  import org.apache.hadoop.hbase.io.HeapSize;
41  import org.apache.hadoop.hbase.util.Bytes;
42  import org.apache.hadoop.hbase.util.ClassSize;
43  
44  /**
45   * The MemStore holds in-memory modifications to the Store.  Modifications
46   * are {@link KeyValue}s.  When asked to flush, current memstore is moved
47   * to snapshot and is cleared.  We continue to serve edits out of new memstore
48   * and backing snapshot until flusher reports in that the flush succeeded. At
49   * this point we let the snapshot go.
50   * TODO: Adjust size of the memstore when we remove items because they have
51   * been deleted.
52   * TODO: With new KVSLS, need to make sure we update HeapSize with difference
53   * in KV size.
54   */
55  public class MemStore implements HeapSize {
56    private static final Log LOG = LogFactory.getLog(MemStore.class);
57  
58    // MemStore.  Use a KeyValueSkipListSet rather than SkipListSet because of the
59    // better semantics.  The Map will overwrite if passed a key it already had
60    // whereas the Set will not add new KV if key is same though value might be
61    // different.  Value is not important -- just make sure always same
62    // reference passed.
63    volatile KeyValueSkipListSet kvset;
64  
65    // Snapshot of memstore.  Made for flusher.
66    volatile KeyValueSkipListSet snapshot;
67  
68    final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
69  
70    final KeyValue.KVComparator comparator;
71  
72    // Used comparing versions -- same r/c and ts but different type.
73    final KeyValue.KVComparator comparatorIgnoreType;
74  
75    // Used comparing versions -- same r/c and type but different timestamp.
76    final KeyValue.KVComparator comparatorIgnoreTimestamp;
77  
78    // Used to track own heapSize
79    final AtomicLong size;
80  
81    TimeRangeTracker timeRangeTracker;
82    TimeRangeTracker snapshotTimeRangeTracker;
83  
84    /**
85     * Default constructor. Used for tests.
86     */
87    public MemStore() {
88      this(KeyValue.COMPARATOR);
89    }
90  
91    /**
92     * Constructor.
93     * @param c Comparator
94     */
95    public MemStore(final KeyValue.KVComparator c) {
96      this.comparator = c;
97      this.comparatorIgnoreTimestamp =
98        this.comparator.getComparatorIgnoringTimestamps();
99      this.comparatorIgnoreType = this.comparator.getComparatorIgnoringType();
100     this.kvset = new KeyValueSkipListSet(c);
101     this.snapshot = new KeyValueSkipListSet(c);
102     timeRangeTracker = new TimeRangeTracker();
103     snapshotTimeRangeTracker = new TimeRangeTracker();
104     this.size = new AtomicLong(DEEP_OVERHEAD);
105   }
106 
107   void dump() {
108     for (KeyValue kv: this.kvset) {
109       LOG.info(kv);
110     }
111     for (KeyValue kv: this.snapshot) {
112       LOG.info(kv);
113     }
114   }
115 
116   /**
117    * Creates a snapshot of the current memstore.
118    * Snapshot must be cleared by call to {@link #clearSnapshot(SortedSet<KeyValue>)}
119    * To get the snapshot made by this method, use {@link #getSnapshot()}
120    */
121   void snapshot() {
122     this.lock.writeLock().lock();
123     try {
124       // If snapshot currently has entries, then flusher failed or didn't call
125       // cleanup.  Log a warning.
126       if (!this.snapshot.isEmpty()) {
127         LOG.warn("Snapshot called again without clearing previous. " +
128           "Doing nothing. Another ongoing flush or did we fail last attempt?");
129       } else {
130         if (!this.kvset.isEmpty()) {
131           this.snapshot = this.kvset;
132           this.kvset = new KeyValueSkipListSet(this.comparator);
133           this.snapshotTimeRangeTracker = this.timeRangeTracker;
134           this.timeRangeTracker = new TimeRangeTracker();
135           // Reset heap to not include any keys
136           this.size.set(DEEP_OVERHEAD);
137         }
138       }
139     } finally {
140       this.lock.writeLock().unlock();
141     }
142   }
143 
144   /**
145    * Return the current snapshot.
146    * Called by flusher to get current snapshot made by a previous
147    * call to {@link #snapshot()}
148    * @return Return snapshot.
149    * @see {@link #snapshot()}
150    * @see {@link #clearSnapshot(SortedSet<KeyValue>)}
151    */
152   KeyValueSkipListSet getSnapshot() {
153     return this.snapshot;
154   }
155 
156   /**
157    * The passed snapshot was successfully persisted; it can be let go.
158    * @param ss The snapshot to clean out.
159    * @throws UnexpectedException
160    * @see {@link #snapshot()}
161    */
162   void clearSnapshot(final SortedSet<KeyValue> ss)
163   throws UnexpectedException {
164     this.lock.writeLock().lock();
165     try {
166       if (this.snapshot != ss) {
167         throw new UnexpectedException("Current snapshot is " +
168           this.snapshot + ", was passed " + ss);
169       }
170       // OK. Passed in snapshot is same as current snapshot.  If not-empty,
171       // create a new snapshot and let the old one go.
172       if (!ss.isEmpty()) {
173         this.snapshot = new KeyValueSkipListSet(this.comparator);
174         this.snapshotTimeRangeTracker = new TimeRangeTracker();
175       }
176     } finally {
177       this.lock.writeLock().unlock();
178     }
179   }
180 
181   /**
182    * Write an update
183    * @param kv
184    * @return approximate size of the passed key and value.
185    */
186   long add(final KeyValue kv) {
187     long s = -1;
188     this.lock.readLock().lock();
189     try {
190       s = heapSizeChange(kv, this.kvset.add(kv));
191       timeRangeTracker.includeTimestamp(kv);
192       this.size.addAndGet(s);
193     } finally {
194       this.lock.readLock().unlock();
195     }
196     return s;
197   }
198 
199   /**
200    * Write a delete
201    * @param delete
202    * @return approximate size of the passed key and value.
203    */
204   long delete(final KeyValue delete) {
205     long s = 0;
206     this.lock.readLock().lock();
207     try {
208       s += heapSizeChange(delete, this.kvset.add(delete));
209       timeRangeTracker.includeTimestamp(delete);
210     } finally {
211       this.lock.readLock().unlock();
212     }
213     this.size.addAndGet(s);
214     return s;
215   }
216 
217   /**
218    * @param kv Find the row that comes after this one.  If null, we return the
219    * first.
220    * @return Next row or null if none found.
221    */
222   KeyValue getNextRow(final KeyValue kv) {
223     this.lock.readLock().lock();
224     try {
225       return getLowest(getNextRow(kv, this.kvset), getNextRow(kv, this.snapshot));
226     } finally {
227       this.lock.readLock().unlock();
228     }
229   }
230 
231   /*
232    * @param a
233    * @param b
234    * @return Return lowest of a or b or null if both a and b are null
235    */
236   private KeyValue getLowest(final KeyValue a, final KeyValue b) {
237     if (a == null) {
238       return b;
239     }
240     if (b == null) {
241       return a;
242     }
243     return comparator.compareRows(a, b) <= 0? a: b;
244   }
245 
246   /*
247    * @param key Find row that follows this one.  If null, return first.
248    * @param map Set to look in for a row beyond <code>row</code>.
249    * @return Next row or null if none found.  If one found, will be a new
250    * KeyValue -- can be destroyed by subsequent calls to this method.
251    */
252   private KeyValue getNextRow(final KeyValue key,
253       final NavigableSet<KeyValue> set) {
254     KeyValue result = null;
255     SortedSet<KeyValue> tail = key == null? set: set.tailSet(key);
256     // Iterate until we fall into the next row; i.e. move off current row
257     for (KeyValue kv: tail) {
258       if (comparator.compareRows(kv, key) <= 0)
259         continue;
260       // Note: Not suppressing deletes or expired cells.  Needs to be handled
261       // by higher up functions.
262       result = kv;
263       break;
264     }
265     return result;
266   }
267 
268   /**
269    * @param state column/delete tracking state
270    */
271   void getRowKeyAtOrBefore(final GetClosestRowBeforeTracker state) {
272     this.lock.readLock().lock();
273     try {
274       getRowKeyAtOrBefore(kvset, state);
275       getRowKeyAtOrBefore(snapshot, state);
276     } finally {
277       this.lock.readLock().unlock();
278     }
279   }
280 
281   /*
282    * @param set
283    * @param state Accumulates deletes and candidates.
284    */
285   private void getRowKeyAtOrBefore(final NavigableSet<KeyValue> set,
286       final GetClosestRowBeforeTracker state) {
287     if (set.isEmpty()) {
288       return;
289     }
290     if (!walkForwardInSingleRow(set, state.getTargetKey(), state)) {
291       // Found nothing in row.  Try backing up.
292       getRowKeyBefore(set, state);
293     }
294   }
295 
296   /*
297    * Walk forward in a row from <code>firstOnRow</code>.  Presumption is that
298    * we have been passed the first possible key on a row.  As we walk forward
299    * we accumulate deletes until we hit a candidate on the row at which point
300    * we return.
301    * @param set
302    * @param firstOnRow First possible key on this row.
303    * @param state
304    * @return True if we found a candidate walking this row.
305    */
306   private boolean walkForwardInSingleRow(final SortedSet<KeyValue> set,
307       final KeyValue firstOnRow, final GetClosestRowBeforeTracker state) {
308     boolean foundCandidate = false;
309     SortedSet<KeyValue> tail = set.tailSet(firstOnRow);
310     if (tail.isEmpty()) return foundCandidate;
311     for (Iterator<KeyValue> i = tail.iterator(); i.hasNext();) {
312       KeyValue kv = i.next();
313       // Did we go beyond the target row? If so break.
314       if (state.isTooFar(kv, firstOnRow)) break;
315       if (state.isExpired(kv)) {
316         i.remove();
317         continue;
318       }
319       // If we added something, this row is a contender. break.
320       if (state.handle(kv)) {
321         foundCandidate = true;
322         break;
323       }
324     }
325     return foundCandidate;
326   }
327 
328   /*
329    * Walk backwards through the passed set a row at a time until we run out of
330    * set or until we get a candidate.
331    * @param set
332    * @param state
333    */
334   private void getRowKeyBefore(NavigableSet<KeyValue> set,
335       final GetClosestRowBeforeTracker state) {
336     KeyValue firstOnRow = state.getTargetKey();
337     for (Member p = memberOfPreviousRow(set, state, firstOnRow);
338         p != null; p = memberOfPreviousRow(p.set, state, firstOnRow)) {
339       // Make sure we don't fall out of our table.
340       if (!state.isTargetTable(p.kv)) break;
341       // Stop looking if we've exited the better candidate range.
342       if (!state.isBetterCandidate(p.kv)) break;
343       // Make into firstOnRow
344       firstOnRow = new KeyValue(p.kv.getRow(), HConstants.LATEST_TIMESTAMP);
345       // If we find something, break;
346       if (walkForwardInSingleRow(p.set, firstOnRow, state)) break;
347     }
348   }
349 
350   /**
351    * Given the specs of a column, update it, first by inserting a new record,
352    * then removing the old one.  Since there is only 1 KeyValue involved, the memstoreTS
353    * will be set to 0, thus ensuring that they instantly appear to anyone. The underlying
354    * store will ensure that the insert/delete each are atomic. A scanner/reader will either
355    * get the new value, or the old value and all readers will eventually only see the new
356    * value after the old was removed.
357    *
358    * @param row
359    * @param family
360    * @param qualifier
361    * @param newValue
362    * @param now
363    * @return
364    */
365   public long updateColumnValue(byte[] row,
366                                 byte[] family,
367                                 byte[] qualifier,
368                                 long newValue,
369                                 long now) {
370    this.lock.readLock().lock();
371     try {
372       KeyValue firstKv = KeyValue.createFirstOnRow(
373           row, family, qualifier);
374       // create a new KeyValue with 'now' and a 0 memstoreTS == immediately visible
375       KeyValue newKv;
376       // Is there a KeyValue in 'snapshot' with the same TS? If so, upgrade the timestamp a bit.
377       SortedSet<KeyValue> snSs = snapshot.tailSet(firstKv);
378       if (!snSs.isEmpty()) {
379         KeyValue snKv = snSs.first();
380         // is there a matching KV in the snapshot?
381         if (snKv.matchingRow(firstKv) && snKv.matchingQualifier(firstKv)) {
382           if (snKv.getTimestamp() == now) {
383             // poop,
384             now += 1;
385           }
386         }
387       }
388 
389       // logic here: the new ts MUST be at least 'now'. But it could be larger if necessary.
390       // But the timestamp should also be max(now, mostRecentTsInMemstore)
391 
392       // so we cant add the new KV w/o knowing what's there already, but we also
393       // want to take this chance to delete some kvs. So two loops (sad)
394 
395       SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
396       Iterator<KeyValue> it = ss.iterator();
397       while ( it.hasNext() ) {
398         KeyValue kv = it.next();
399 
400         // if this isnt the row we are interested in, then bail:
401         if (!firstKv.matchingRow(kv)) {
402           break; // rows dont match, bail.
403         }
404 
405         // if the qualifier matches and it's a put, just RM it out of the kvset.
406         if (firstKv.matchingQualifier(kv)) {
407           // to be extra safe we only remove Puts that have a memstoreTS==0
408           if (kv.getType() == KeyValue.Type.Put.getCode()) {
409             now = Math.max(now, kv.getTimestamp());
410           }
411         }
412       }
413 
414 
415       // add the new value now. this might have the same TS as an existing KV, thus confusing
416       // readers slightly for a MOMENT until we erase the old one (and thus old value).
417       newKv = new KeyValue(row, family, qualifier,
418           now,
419           Bytes.toBytes(newValue));
420       long addedSize = add(newKv);
421 
422       // remove extra versions.
423       ss = kvset.tailSet(firstKv);
424       it = ss.iterator();
425       while ( it.hasNext() ) {
426         KeyValue kv = it.next();
427 
428         if (kv == newKv) {
429           // ignore the one i just put in (heh)
430           continue;
431         }
432 
433         // if this isnt the row we are interested in, then bail:
434         if (!firstKv.matchingRow(kv)) {
435           break; // rows dont match, bail.
436         }
437 
438         // if the qualifier matches and it's a put, just RM it out of the kvset.
439         if (firstKv.matchingQualifier(kv)) {
440           // to be extra safe we only remove Puts that have a memstoreTS==0
441           if (kv.getType() == KeyValue.Type.Put.getCode()) {
442             // false means there was a change, so give us the size.
443             addedSize -= heapSizeChange(kv, false);
444 
445             it.remove();
446           }
447         }
448       }
449 
450       return addedSize;
451     } finally {
452       this.lock.readLock().unlock();
453     }
454   }
455 
456   /*
457    * Immutable data structure to hold member found in set and the set it was
458    * found in.  Include set because it is carrying context.
459    */
460   private static class Member {
461     final KeyValue kv;
462     final NavigableSet<KeyValue> set;
463     Member(final NavigableSet<KeyValue> s, final KeyValue kv) {
464       this.kv = kv;
465       this.set = s;
466     }
467   }
468 
469   /*
470    * @param set Set to walk back in.  Pass a first in row or we'll return
471    * same row (loop).
472    * @param state Utility and context.
473    * @param firstOnRow First item on the row after the one we want to find a
474    * member in.
475    * @return Null or member of row previous to <code>firstOnRow</code>
476    */
477   private Member memberOfPreviousRow(NavigableSet<KeyValue> set,
478       final GetClosestRowBeforeTracker state, final KeyValue firstOnRow) {
479     NavigableSet<KeyValue> head = set.headSet(firstOnRow, false);
480     if (head.isEmpty()) return null;
481     for (Iterator<KeyValue> i = head.descendingIterator(); i.hasNext();) {
482       KeyValue found = i.next();
483       if (state.isExpired(found)) {
484         i.remove();
485         continue;
486       }
487       return new Member(head, found);
488     }
489     return null;
490   }
491 
492   /**
493    * @return scanner on memstore and snapshot in this order.
494    */
495   List<KeyValueScanner> getScanners() {
496     this.lock.readLock().lock();
497     try {
498       return Collections.<KeyValueScanner>singletonList(
499           new MemStoreScanner());
500     } finally {
501       this.lock.readLock().unlock();
502     }
503   }
504 
505   /**
506    * Check if this memstore may contain the required keys
507    * @param scan
508    * @return False if the key definitely does not exist in this Memstore
509    */
510   public boolean shouldSeek(Scan scan) {
511     return timeRangeTracker.includesTimeRange(scan.getTimeRange()) ||
512         snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange());
513   }
514 
515   public TimeRangeTracker getSnapshotTimeRangeTracker() {
516     return this.snapshotTimeRangeTracker;
517   }
518 
519   /*
520    * MemStoreScanner implements the KeyValueScanner.
521    * It lets the caller scan the contents of a memstore -- both current
522    * map and snapshot.
523    * This behaves as if it were a real scanner but does not maintain position.
524    */
525   protected class MemStoreScanner implements KeyValueScanner {
526     // Next row information for either kvset or snapshot
527     private KeyValue kvsetNextRow = null;
528     private KeyValue snapshotNextRow = null;
529 
530     // iterator based scanning.
531     Iterator<KeyValue> kvsetIt;
532     Iterator<KeyValue> snapshotIt;
533 
534     /*
535     Some notes...
536 
537      So memstorescanner is fixed at creation time. this includes pointers/iterators into
538     existing kvset/snapshot.  during a snapshot creation, the kvset is null, and the
539     snapshot is moved.  since kvset is null there is no point on reseeking on both,
540       we can save us the trouble. During the snapshot->hfile transition, the memstore
541       scanner is re-created by StoreScanner#updateReaders().  StoreScanner should
542       potentially do something smarter by adjusting the existing memstore scanner.
543 
544       But there is a greater problem here, that being once a scanner has progressed
545       during a snapshot scenario, we currently iterate past the kvset then 'finish' up.
546       if a scan lasts a little while, there is a chance for new entries in kvset to
547       become available but we will never see them.  This needs to be handled at the
548       StoreScanner level with coordination with MemStoreScanner.
549 
550     */
551 
552     MemStoreScanner() {
553       super();
554 
555       //DebugPrint.println(" MS new@" + hashCode());
556     }
557 
558     protected KeyValue getNext(Iterator<KeyValue> it) {
559       KeyValue ret = null;
560       long readPoint = ReadWriteConsistencyControl.getThreadReadPoint();
561       //DebugPrint.println( " MS@" + hashCode() + ": threadpoint = " + readPoint);
562 
563       while (ret == null && it.hasNext()) {
564         KeyValue v = it.next();
565         if (v.getMemstoreTS() <= readPoint) {
566           // keep it.
567           ret = v;
568         }
569       }
570       return ret;
571     }
572 
573     public synchronized boolean seek(KeyValue key) {
574       if (key == null) {
575         close();
576         return false;
577       }
578 
579       // kvset and snapshot will never be empty.
580       // if tailSet cant find anything, SS is empty (not null).
581       SortedSet<KeyValue> kvTail = kvset.tailSet(key);
582       SortedSet<KeyValue> snapshotTail = snapshot.tailSet(key);
583 
584       kvsetIt = kvTail.iterator();
585       snapshotIt = snapshotTail.iterator();
586 
587       kvsetNextRow = getNext(kvsetIt);
588       snapshotNextRow = getNext(snapshotIt);
589 
590 
591       //long readPoint = ReadWriteConsistencyControl.getThreadReadPoint();
592       //DebugPrint.println( " MS@" + hashCode() + " kvset seek: " + kvsetNextRow + " with size = " +
593       //    kvset.size() + " threadread = " + readPoint);
594       //DebugPrint.println( " MS@" + hashCode() + " snapshot seek: " + snapshotNextRow + " with size = " +
595       //    snapshot.size() + " threadread = " + readPoint);
596 
597 
598       KeyValue lowest = getLowest();
599 
600       // has data := (lowest != null)
601       return lowest != null;
602     }
603 
604     public synchronized KeyValue peek() {
605       //DebugPrint.println(" MS@" + hashCode() + " peek = " + getLowest());
606       return getLowest();
607     }
608 
609 
610     public synchronized KeyValue next() {
611       KeyValue theNext = getLowest();
612 
613       if (theNext == null) {
614           return null;
615       }
616 
617       // Advance one of the iterators
618       if (theNext == kvsetNextRow) {
619         kvsetNextRow = getNext(kvsetIt);
620       } else {
621         snapshotNextRow = getNext(snapshotIt);
622       }
623 
624       //long readpoint = ReadWriteConsistencyControl.getThreadReadPoint();
625       //DebugPrint.println(" MS@" + hashCode() + " next: " + theNext + " next_next: " +
626       //    getLowest() + " threadpoint=" + readpoint);
627       return theNext;
628     }
629 
630     protected KeyValue getLowest() {
631       return getLower(kvsetNextRow,
632           snapshotNextRow);
633     }
634 
635     /*
636      * Returns the lower of the two key values, or null if they are both null.
637      * This uses comparator.compare() to compare the KeyValue using the memstore
638      * comparator.
639      */
640     protected KeyValue getLower(KeyValue first, KeyValue second) {
641       if (first == null && second == null) {
642         return null;
643       }
644       if (first != null && second != null) {
645         int compare = comparator.compare(first, second);
646         return (compare <= 0 ? first : second);
647       }
648       return (first != null ? first : second);
649     }
650 
651     public synchronized void close() {
652       this.kvsetNextRow = null;
653       this.snapshotNextRow = null;
654 
655       this.kvsetIt = null;
656       this.snapshotIt = null;
657     }
658   }
659 
660   public final static long FIXED_OVERHEAD = ClassSize.align(
661       ClassSize.OBJECT + (9 * ClassSize.REFERENCE));
662 
663   public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
664       ClassSize.REENTRANT_LOCK + ClassSize.ATOMIC_LONG +
665       ClassSize.COPYONWRITE_ARRAYSET + ClassSize.COPYONWRITE_ARRAYLIST +
666       (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
667 
668   /*
669    * Calculate how the MemStore size has changed.  Includes overhead of the
670    * backing Map.
671    * @param kv
672    * @param notpresent True if the kv was NOT present in the set.
673    * @return Size
674    */
675   long heapSizeChange(final KeyValue kv, final boolean notpresent) {
676     return notpresent ?
677         ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize()):
678         0;
679   }
680 
681   /**
682    * Get the entire heap usage for this MemStore not including keys in the
683    * snapshot.
684    */
685   @Override
686   public long heapSize() {
687     return size.get();
688   }
689 
690   /**
691    * Get the heap usage of KVs in this MemStore.
692    */
693   public long keySize() {
694     return heapSize() - DEEP_OVERHEAD;
695   }
696 
697   /**
698    * Code to help figure if our approximation of object heap sizes is close
699    * enough.  See hbase-900.  Fills memstores then waits so user can heap
700    * dump and bring up resultant hprof in something like jprofiler which
701    * allows you get 'deep size' on objects.
702    * @param args main args
703    */
704   public static void main(String [] args) {
705     RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
706     LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
707       runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
708     LOG.info("vmInputArguments=" + runtime.getInputArguments());
709     MemStore memstore1 = new MemStore();
710     // TODO: x32 vs x64
711     long size = 0;
712     final int count = 10000;
713     byte [] fam = Bytes.toBytes("col");
714     byte [] qf = Bytes.toBytes("umn");
715     byte [] empty = new byte[0];
716     for (int i = 0; i < count; i++) {
717       // Give each its own ts
718       size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
719     }
720     LOG.info("memstore1 estimated size=" + size);
721     for (int i = 0; i < count; i++) {
722       size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
723     }
724     LOG.info("memstore1 estimated size (2nd loading of same data)=" + size);
725     // Make a variably sized memstore.
726     MemStore memstore2 = new MemStore();
727     for (int i = 0; i < count; i++) {
728       size += memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i,
729         new byte[i]));
730     }
731     LOG.info("memstore2 estimated size=" + size);
732     final int seconds = 30;
733     LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
734     for (int i = 0; i < seconds; i++) {
735       // Thread.sleep(1000);
736     }
737     LOG.info("Exiting.");
738   }
739 }