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