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.Arrays;
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.conf.Configuration;
38  import org.apache.hadoop.hbase.HBaseConfiguration;
39  import org.apache.hadoop.hbase.HConstants;
40  import org.apache.hadoop.hbase.KeyValue;
41  import org.apache.hadoop.hbase.client.Scan;
42  import org.apache.hadoop.hbase.io.HeapSize;
43  import org.apache.hadoop.hbase.regionserver.MemStoreLAB.Allocation;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.hbase.util.ClassSize;
46  
47  /**
48   * The MemStore holds in-memory modifications to the Store.  Modifications
49   * are {@link KeyValue}s.  When asked to flush, current memstore is moved
50   * to snapshot and is cleared.  We continue to serve edits out of new memstore
51   * and backing snapshot until flusher reports in that the flush succeeded. At
52   * this point we let the snapshot go.
53   * TODO: Adjust size of the memstore when we remove items because they have
54   * been deleted.
55   * TODO: With new KVSLS, need to make sure we update HeapSize with difference
56   * in KV size.
57   */
58  public class MemStore implements HeapSize {
59    private static final Log LOG = LogFactory.getLog(MemStore.class);
60  
61    static final String USEMSLAB_KEY =
62      "hbase.hregion.memstore.mslab.enabled";
63    private static final boolean USEMSLAB_DEFAULT = false;
64  
65    private Configuration conf;
66  
67    // MemStore.  Use a KeyValueSkipListSet rather than SkipListSet because of the
68    // better semantics.  The Map will overwrite if passed a key it already had
69    // whereas the Set will not add new KV if key is same though value might be
70    // different.  Value is not important -- just make sure always same
71    // reference passed.
72    volatile KeyValueSkipListSet kvset;
73  
74    // Snapshot of memstore.  Made for flusher.
75    volatile KeyValueSkipListSet snapshot;
76  
77    final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
78  
79    final KeyValue.KVComparator comparator;
80  
81    // Used comparing versions -- same r/c and ts but different type.
82    final KeyValue.KVComparator comparatorIgnoreType;
83  
84    // Used comparing versions -- same r/c and type but different timestamp.
85    final KeyValue.KVComparator comparatorIgnoreTimestamp;
86  
87    // Used to track own heapSize
88    final AtomicLong size;
89  
90    TimeRangeTracker timeRangeTracker;
91    TimeRangeTracker snapshotTimeRangeTracker;
92    
93    MemStoreLAB allocator;
94  
95  
96  
97    /**
98     * Default constructor. Used for tests.
99     */
100   public MemStore() {
101     this(HBaseConfiguration.create(), KeyValue.COMPARATOR);
102   }
103 
104   /**
105    * Constructor.
106    * @param c Comparator
107    */
108   public MemStore(final Configuration conf,
109                   final KeyValue.KVComparator c) {
110     this.conf = conf;
111     this.comparator = c;
112     this.comparatorIgnoreTimestamp =
113       this.comparator.getComparatorIgnoringTimestamps();
114     this.comparatorIgnoreType = this.comparator.getComparatorIgnoringType();
115     this.kvset = new KeyValueSkipListSet(c);
116     this.snapshot = new KeyValueSkipListSet(c);
117     timeRangeTracker = new TimeRangeTracker();
118     snapshotTimeRangeTracker = new TimeRangeTracker();
119     this.size = new AtomicLong(DEEP_OVERHEAD);
120     if (conf.getBoolean(USEMSLAB_KEY, USEMSLAB_DEFAULT)) {
121       this.allocator = new MemStoreLAB(conf);
122     } else {
123       this.allocator = 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    * Creates a snapshot of the current memstore.
138    * Snapshot must be cleared by call to {@link #clearSnapshot(SortedSet<KeyValue>)}
139    * To get the snapshot made by this method, use {@link #getSnapshot()}
140    */
141   void snapshot() {
142     this.lock.writeLock().lock();
143     try {
144       // If snapshot currently has entries, then flusher failed or didn't call
145       // cleanup.  Log a warning.
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           // Reset heap to not include any keys
156           this.size.set(DEEP_OVERHEAD);
157           // Reset allocator so we get a fresh buffer for the new memstore
158           if (allocator != null) {
159             this.allocator = new MemStoreLAB(conf);
160           }
161         }
162       }
163     } finally {
164       this.lock.writeLock().unlock();
165     }
166   }
167 
168   /**
169    * Return the current snapshot.
170    * Called by flusher to get current snapshot made by a previous
171    * call to {@link #snapshot()}
172    * @return Return snapshot.
173    * @see {@link #snapshot()}
174    * @see {@link #clearSnapshot(SortedSet<KeyValue>)}
175    */
176   KeyValueSkipListSet getSnapshot() {
177     return this.snapshot;
178   }
179 
180   /**
181    * The passed snapshot was successfully persisted; it can be let go.
182    * @param ss The snapshot to clean out.
183    * @throws UnexpectedException
184    * @see {@link #snapshot()}
185    */
186   void clearSnapshot(final SortedSet<KeyValue> ss)
187   throws UnexpectedException {
188     this.lock.writeLock().lock();
189     try {
190       if (this.snapshot != ss) {
191         throw new UnexpectedException("Current snapshot is " +
192           this.snapshot + ", was passed " + ss);
193       }
194       // OK. Passed in snapshot is same as current snapshot.  If not-empty,
195       // create a new snapshot and let the old one go.
196       if (!ss.isEmpty()) {
197         this.snapshot = new KeyValueSkipListSet(this.comparator);
198         this.snapshotTimeRangeTracker = new TimeRangeTracker();
199       }
200     } finally {
201       this.lock.writeLock().unlock();
202     }
203   }
204 
205   /**
206    * Write an update
207    * @param kv
208    * @return approximate size of the passed key and value.
209    */
210   long add(final KeyValue kv) {
211     this.lock.readLock().lock();
212     try {
213       KeyValue toAdd = maybeCloneWithAllocator(kv);
214       return internalAdd(toAdd);
215     } finally {
216       this.lock.readLock().unlock();
217     }
218   }
219   
220   /**
221    * Internal version of add() that doesn't clone KVs with the
222    * allocator, and doesn't take the lock.
223    * 
224    * Callers should ensure they already have the read lock taken
225    */
226   private long internalAdd(final KeyValue toAdd) {
227     long s = heapSizeChange(toAdd, this.kvset.add(toAdd));
228     timeRangeTracker.includeTimestamp(toAdd);
229     this.size.addAndGet(s);
230     return s;
231   }
232 
233   private KeyValue maybeCloneWithAllocator(KeyValue kv) {
234     if (allocator == null) {
235       return kv;
236     }
237 
238     int len = kv.getLength();
239     Allocation alloc = allocator.allocateBytes(len);
240     if (alloc == null) {
241       // The allocation was too large, allocator decided
242       // not to do anything with it.
243       return kv;
244     }
245     assert alloc != null && alloc.getData() != null;
246     System.arraycopy(kv.getBuffer(), kv.getOffset(), alloc.getData(), alloc.getOffset(), len);
247     KeyValue newKv = new KeyValue(alloc.getData(), alloc.getOffset(), len);
248     newKv.setMemstoreTS(kv.getMemstoreTS());
249     return newKv;
250   }
251 
252   /**
253    * Write a delete
254    * @param delete
255    * @return approximate size of the passed key and value.
256    */
257   long delete(final KeyValue delete) {
258     long s = 0;
259     this.lock.readLock().lock();
260     try {
261       KeyValue toAdd = maybeCloneWithAllocator(delete);
262       s += heapSizeChange(toAdd, this.kvset.add(toAdd));
263       timeRangeTracker.includeTimestamp(toAdd);
264     } finally {
265       this.lock.readLock().unlock();
266     }
267     this.size.addAndGet(s);
268     return s;
269   }
270 
271   /**
272    * @param kv Find the row that comes after this one.  If null, we return the
273    * first.
274    * @return Next row or null if none found.
275    */
276   KeyValue getNextRow(final KeyValue kv) {
277     this.lock.readLock().lock();
278     try {
279       return getLowest(getNextRow(kv, this.kvset), getNextRow(kv, this.snapshot));
280     } finally {
281       this.lock.readLock().unlock();
282     }
283   }
284 
285   /*
286    * @param a
287    * @param b
288    * @return Return lowest of a or b or null if both a and b are null
289    */
290   private KeyValue getLowest(final KeyValue a, final KeyValue b) {
291     if (a == null) {
292       return b;
293     }
294     if (b == null) {
295       return a;
296     }
297     return comparator.compareRows(a, b) <= 0? a: b;
298   }
299 
300   /*
301    * @param key Find row that follows this one.  If null, return first.
302    * @param map Set to look in for a row beyond <code>row</code>.
303    * @return Next row or null if none found.  If one found, will be a new
304    * KeyValue -- can be destroyed by subsequent calls to this method.
305    */
306   private KeyValue getNextRow(final KeyValue key,
307       final NavigableSet<KeyValue> set) {
308     KeyValue result = null;
309     SortedSet<KeyValue> tail = key == null? set: set.tailSet(key);
310     // Iterate until we fall into the next row; i.e. move off current row
311     for (KeyValue kv: tail) {
312       if (comparator.compareRows(kv, key) <= 0)
313         continue;
314       // Note: Not suppressing deletes or expired cells.  Needs to be handled
315       // by higher up functions.
316       result = kv;
317       break;
318     }
319     return result;
320   }
321 
322   /**
323    * @param state column/delete tracking state
324    */
325   void getRowKeyAtOrBefore(final GetClosestRowBeforeTracker state) {
326     this.lock.readLock().lock();
327     try {
328       getRowKeyAtOrBefore(kvset, state);
329       getRowKeyAtOrBefore(snapshot, state);
330     } finally {
331       this.lock.readLock().unlock();
332     }
333   }
334 
335   /*
336    * @param set
337    * @param state Accumulates deletes and candidates.
338    */
339   private void getRowKeyAtOrBefore(final NavigableSet<KeyValue> set,
340       final GetClosestRowBeforeTracker state) {
341     if (set.isEmpty()) {
342       return;
343     }
344     if (!walkForwardInSingleRow(set, state.getTargetKey(), state)) {
345       // Found nothing in row.  Try backing up.
346       getRowKeyBefore(set, state);
347     }
348   }
349 
350   /*
351    * Walk forward in a row from <code>firstOnRow</code>.  Presumption is that
352    * we have been passed the first possible key on a row.  As we walk forward
353    * we accumulate deletes until we hit a candidate on the row at which point
354    * we return.
355    * @param set
356    * @param firstOnRow First possible key on this row.
357    * @param state
358    * @return True if we found a candidate walking this row.
359    */
360   private boolean walkForwardInSingleRow(final SortedSet<KeyValue> set,
361       final KeyValue firstOnRow, final GetClosestRowBeforeTracker state) {
362     boolean foundCandidate = false;
363     SortedSet<KeyValue> tail = set.tailSet(firstOnRow);
364     if (tail.isEmpty()) return foundCandidate;
365     for (Iterator<KeyValue> i = tail.iterator(); i.hasNext();) {
366       KeyValue kv = i.next();
367       // Did we go beyond the target row? If so break.
368       if (state.isTooFar(kv, firstOnRow)) break;
369       if (state.isExpired(kv)) {
370         i.remove();
371         continue;
372       }
373       // If we added something, this row is a contender. break.
374       if (state.handle(kv)) {
375         foundCandidate = true;
376         break;
377       }
378     }
379     return foundCandidate;
380   }
381 
382   /*
383    * Walk backwards through the passed set a row at a time until we run out of
384    * set or until we get a candidate.
385    * @param set
386    * @param state
387    */
388   private void getRowKeyBefore(NavigableSet<KeyValue> set,
389       final GetClosestRowBeforeTracker state) {
390     KeyValue firstOnRow = state.getTargetKey();
391     for (Member p = memberOfPreviousRow(set, state, firstOnRow);
392         p != null; p = memberOfPreviousRow(p.set, state, firstOnRow)) {
393       // Make sure we don't fall out of our table.
394       if (!state.isTargetTable(p.kv)) break;
395       // Stop looking if we've exited the better candidate range.
396       if (!state.isBetterCandidate(p.kv)) break;
397       // Make into firstOnRow
398       firstOnRow = new KeyValue(p.kv.getRow(), HConstants.LATEST_TIMESTAMP);
399       // If we find something, break;
400       if (walkForwardInSingleRow(p.set, firstOnRow, state)) break;
401     }
402   }
403 
404   /**
405    * Given the specs of a column, update it, first by inserting a new record,
406    * then removing the old one.  Since there is only 1 KeyValue involved, the memstoreTS
407    * will be set to 0, thus ensuring that they instantly appear to anyone. The underlying
408    * store will ensure that the insert/delete each are atomic. A scanner/reader will either
409    * get the new value, or the old value and all readers will eventually only see the new
410    * value after the old was removed.
411    *
412    * @param row
413    * @param family
414    * @param qualifier
415    * @param newValue
416    * @param now
417    * @return  Timestamp
418    */
419   public long updateColumnValue(byte[] row,
420                                 byte[] family,
421                                 byte[] qualifier,
422                                 long newValue,
423                                 long now) {
424    this.lock.readLock().lock();
425     try {
426       KeyValue firstKv = KeyValue.createFirstOnRow(
427           row, family, qualifier);
428       // Is there a KeyValue in 'snapshot' with the same TS? If so, upgrade the timestamp a bit.
429       SortedSet<KeyValue> snSs = snapshot.tailSet(firstKv);
430       if (!snSs.isEmpty()) {
431         KeyValue snKv = snSs.first();
432         // is there a matching KV in the snapshot?
433         if (snKv.matchingRow(firstKv) && snKv.matchingQualifier(firstKv)) {
434           if (snKv.getTimestamp() == now) {
435             // poop,
436             now += 1;
437           }
438         }
439       }
440 
441       // logic here: the new ts MUST be at least 'now'. But it could be larger if necessary.
442       // But the timestamp should also be max(now, mostRecentTsInMemstore)
443 
444       // so we cant add the new KV w/o knowing what's there already, but we also
445       // want to take this chance to delete some kvs. So two loops (sad)
446 
447       SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
448       Iterator<KeyValue> it = ss.iterator();
449       while ( it.hasNext() ) {
450         KeyValue kv = it.next();
451 
452         // if this isnt the row we are interested in, then bail:
453         if (!kv.matchingColumn(family,qualifier) || !kv.matchingRow(firstKv) ) {
454           break; // rows dont match, bail.
455         }
456 
457         // if the qualifier matches and it's a put, just RM it out of the kvset.
458         if (kv.getType() == KeyValue.Type.Put.getCode() &&
459             kv.getTimestamp() > now && firstKv.matchingQualifier(kv)) {
460           now = kv.getTimestamp();
461         }
462       }
463 
464       // create or update (upsert) a new KeyValue with
465       // 'now' and a 0 memstoreTS == immediately visible
466       return upsert(Arrays.asList(
467           new KeyValue(row, family, qualifier, now, Bytes.toBytes(newValue)))
468       );
469     } finally {
470       this.lock.readLock().unlock();
471     }
472   }
473 
474   /**
475    * Update or insert the specified KeyValues.
476    * <p>
477    * For each KeyValue, insert into MemStore.  This will atomically upsert the
478    * value for that row/family/qualifier.  If a KeyValue did already exist,
479    * it will then be removed.
480    * <p>
481    * Currently the memstoreTS is kept at 0 so as each insert happens, it will
482    * be immediately visible.  May want to change this so it is atomic across
483    * all KeyValues.
484    * <p>
485    * This is called under row lock, so Get operations will still see updates
486    * atomically.  Scans will only see each KeyValue update as atomic.
487    *
488    * @param kvs
489    * @return change in memstore size
490    */
491   public long upsert(List<KeyValue> kvs) {
492    this.lock.readLock().lock();
493     try {
494       long size = 0;
495       for (KeyValue kv : kvs) {
496         kv.setMemstoreTS(0);
497         size += upsert(kv);
498       }
499       return size;
500     } finally {
501       this.lock.readLock().unlock();
502     }
503   }
504 
505   /**
506    * Inserts the specified KeyValue into MemStore and deletes any existing
507    * versions of the same row/family/qualifier as the specified KeyValue.
508    * <p>
509    * First, the specified KeyValue is inserted into the Memstore.
510    * <p>
511    * If there are any existing KeyValues in this MemStore with the same row,
512    * family, and qualifier, they are removed.
513    * <p>
514    * Callers must hold the read lock.
515    * 
516    * @param kv
517    * @return change in size of MemStore
518    */
519   private long upsert(KeyValue kv) {
520     // Add the KeyValue to the MemStore
521     // Use the internalAdd method here since we (a) already have a lock
522     // and (b) cannot safely use the MSLAB here without potentially
523     // hitting OOME - see TestMemStore.testUpsertMSLAB for a
524     // test that triggers the pathological case if we don't avoid MSLAB
525     // here.
526     long addedSize = internalAdd(kv);
527 
528     // Get the KeyValues for the row/family/qualifier regardless of timestamp.
529     // For this case we want to clean up any other puts
530     KeyValue firstKv = KeyValue.createFirstOnRow(
531         kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
532         kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
533         kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
534     SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
535     Iterator<KeyValue> it = ss.iterator();
536     while ( it.hasNext() ) {
537       KeyValue cur = it.next();
538 
539       if (kv == cur) {
540         // ignore the one just put in
541         continue;
542       }
543       // if this isn't the row we are interested in, then bail
544       if (!kv.matchingRow(cur)) {
545         break;
546       }
547 
548       // if the qualifier matches and it's a put, remove it
549       if (kv.matchingQualifier(cur)) {
550 
551         // to be extra safe we only remove Puts that have a memstoreTS==0
552         if (kv.getType() == KeyValue.Type.Put.getCode() &&
553             kv.getMemstoreTS() == 0) {
554           // false means there was a change, so give us the size.
555           addedSize -= heapSizeChange(kv, true);
556           it.remove();
557         }
558       } else {
559         // past the column, done
560         break;
561       }
562     }
563     return addedSize;
564   }
565 
566   /*
567    * Immutable data structure to hold member found in set and the set it was
568    * found in.  Include set because it is carrying context.
569    */
570   private static class Member {
571     final KeyValue kv;
572     final NavigableSet<KeyValue> set;
573     Member(final NavigableSet<KeyValue> s, final KeyValue kv) {
574       this.kv = kv;
575       this.set = s;
576     }
577   }
578 
579   /*
580    * @param set Set to walk back in.  Pass a first in row or we'll return
581    * same row (loop).
582    * @param state Utility and context.
583    * @param firstOnRow First item on the row after the one we want to find a
584    * member in.
585    * @return Null or member of row previous to <code>firstOnRow</code>
586    */
587   private Member memberOfPreviousRow(NavigableSet<KeyValue> set,
588       final GetClosestRowBeforeTracker state, final KeyValue firstOnRow) {
589     NavigableSet<KeyValue> head = set.headSet(firstOnRow, false);
590     if (head.isEmpty()) return null;
591     for (Iterator<KeyValue> i = head.descendingIterator(); i.hasNext();) {
592       KeyValue found = i.next();
593       if (state.isExpired(found)) {
594         i.remove();
595         continue;
596       }
597       return new Member(head, found);
598     }
599     return null;
600   }
601 
602   /**
603    * @return scanner on memstore and snapshot in this order.
604    */
605   List<KeyValueScanner> getScanners() {
606     this.lock.readLock().lock();
607     try {
608       return Collections.<KeyValueScanner>singletonList(
609           new MemStoreScanner());
610     } finally {
611       this.lock.readLock().unlock();
612     }
613   }
614 
615   /**
616    * Check if this memstore may contain the required keys
617    * @param scan
618    * @return False if the key definitely does not exist in this Memstore
619    */
620   public boolean shouldSeek(Scan scan) {
621     return timeRangeTracker.includesTimeRange(scan.getTimeRange()) ||
622         snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange());
623   }
624 
625   public TimeRangeTracker getSnapshotTimeRangeTracker() {
626     return this.snapshotTimeRangeTracker;
627   }
628 
629   /*
630    * MemStoreScanner implements the KeyValueScanner.
631    * It lets the caller scan the contents of a memstore -- both current
632    * map and snapshot.
633    * This behaves as if it were a real scanner but does not maintain position.
634    */
635   protected class MemStoreScanner extends AbstractKeyValueScanner {
636     // Next row information for either kvset or snapshot
637     private KeyValue kvsetNextRow = null;
638     private KeyValue snapshotNextRow = null;
639 
640     // iterator based scanning.
641     private Iterator<KeyValue> kvsetIt;
642     private Iterator<KeyValue> snapshotIt;
643 
644     // The kvset and snapshot at the time of creating this scanner
645     volatile KeyValueSkipListSet kvsetAtCreation;
646     volatile KeyValueSkipListSet snapshotAtCreation;
647 
648     // Sub lists on which we're iterating
649     private SortedSet<KeyValue> kvTail;
650     private SortedSet<KeyValue> snapshotTail;
651 
652     // the pre-calculated KeyValue to be returned by peek() or next()
653     private KeyValue theNext;
654 
655     /*
656     Some notes...
657 
658      So memstorescanner is fixed at creation time. this includes pointers/iterators into
659     existing kvset/snapshot.  during a snapshot creation, the kvset is null, and the
660     snapshot is moved.  since kvset is null there is no point on reseeking on both,
661       we can save us the trouble. During the snapshot->hfile transition, the memstore
662       scanner is re-created by StoreScanner#updateReaders().  StoreScanner should
663       potentially do something smarter by adjusting the existing memstore scanner.
664 
665       But there is a greater problem here, that being once a scanner has progressed
666       during a snapshot scenario, we currently iterate past the kvset then 'finish' up.
667       if a scan lasts a little while, there is a chance for new entries in kvset to
668       become available but we will never see them.  This needs to be handled at the
669       StoreScanner level with coordination with MemStoreScanner.
670 
671       Currently, this problem is only partly managed: during the small amount of time
672       when the StoreScanner has not yet created a new MemStoreScanner, we will miss
673       the adds to kvset in the MemStoreScanner.
674     */
675 
676     MemStoreScanner() {
677       super();
678 
679       kvsetAtCreation = kvset;
680       snapshotAtCreation = snapshot;
681     }
682 
683     protected KeyValue getNext(Iterator<KeyValue> it) {
684       long readPoint = MultiVersionConsistencyControl.getThreadReadPoint();
685 
686       while (it.hasNext()) {
687         KeyValue v = it.next();
688         if (v.getMemstoreTS() <= readPoint) {
689           return v;
690         }
691       }
692 
693       return null;
694     }
695 
696     /**
697      *  Set the scanner at the seek key.
698      *  Must be called only once: there is no thread safety between the scanner
699      *   and the memStore.
700      * @param key seek value
701      * @return false if the key is null or if there is no data
702      */
703     @Override
704     public synchronized boolean seek(KeyValue key) {
705       if (key == null) {
706         close();
707         return false;
708       }
709 
710       // kvset and snapshot will never be null.
711       // if tailSet can't find anything, SortedSet is empty (not null).
712       kvTail = kvsetAtCreation.tailSet(key);
713       snapshotTail = snapshotAtCreation.tailSet(key);
714 
715       return seekInSubLists(key);
716     }
717 
718 
719     /**
720      * (Re)initialize the iterators after a seek or a reseek.
721      */
722     private synchronized boolean seekInSubLists(KeyValue key){
723       kvsetIt = kvTail.iterator();
724       snapshotIt = snapshotTail.iterator();
725 
726       kvsetNextRow = getNext(kvsetIt);
727       snapshotNextRow = getNext(snapshotIt);
728 
729       // Calculate the next value
730       theNext = getLowest(kvsetNextRow, snapshotNextRow);
731 
732       // has data
733       return (theNext != null);
734     }
735 
736 
737     /**
738      * Move forward on the sub-lists set previously by seek.
739      * @param key seek value (should be non-null)
740      * @return true if there is at least one KV to read, false otherwise
741      */
742     @Override
743     public synchronized boolean reseek(KeyValue key) {
744       /*
745       See HBASE-4195 & HBASE-3855 for the background on this implementation.
746       This code is executed concurrently with flush and puts, without locks.
747       Two points must be known when working on this code:
748       1) It's not possible to use the 'kvTail' and 'snapshot'
749        variables, as they are modified during a flush.
750       2) The ideal implementation for performances would use the sub skip list
751        implicitly pointed by the iterators 'kvsetIt' and
752        'snapshotIt'. Unfortunately the Java API does not offer a method to
753        get it. So we're using the skip list that we kept when we created
754        the iterators. As these iterators could have been moved forward after
755        their creation, we're doing a kind of rewind here. It has a small
756        performance impact (we're using a wider list than necessary), and we
757        could see values that were not here when we read the list the first
758        time. We expect that the new values will be skipped by the test on
759        readpoint performed in the next() function.
760        */
761 
762       kvTail = kvTail.tailSet(key);
763       snapshotTail = snapshotTail.tailSet(key);
764 
765       return seekInSubLists(key);
766     }
767 
768 
769     @Override
770     public synchronized KeyValue peek() {
771       //DebugPrint.println(" MS@" + hashCode() + " peek = " + getLowest());
772       return theNext;
773     }
774 
775     @Override
776     public synchronized KeyValue next() {
777       if (theNext == null) {
778           return null;
779       }
780 
781       final KeyValue ret = theNext;
782 
783       // Advance one of the iterators
784       if (theNext == kvsetNextRow) {
785         kvsetNextRow = getNext(kvsetIt);
786       } else {
787         snapshotNextRow = getNext(snapshotIt);
788       }
789 
790       // Calculate the next value
791       theNext = getLowest(kvsetNextRow, snapshotNextRow);
792 
793       //long readpoint = ReadWriteConsistencyControl.getThreadReadPoint();
794       //DebugPrint.println(" MS@" + hashCode() + " next: " + theNext + " next_next: " +
795       //    getLowest() + " threadpoint=" + readpoint);
796       return ret;
797     }
798 
799     /*
800      * Returns the lower of the two key values, or null if they are both null.
801      * This uses comparator.compare() to compare the KeyValue using the memstore
802      * comparator.
803      */
804     protected KeyValue getLowest(KeyValue first, KeyValue second) {
805       if (first == null && second == null) {
806         return null;
807       }
808       if (first != null && second != null) {
809         int compare = comparator.compare(first, second);
810         return (compare <= 0 ? first : second);
811       }
812       return (first != null ? first : second);
813     }
814 
815     public synchronized void close() {
816       this.kvsetNextRow = null;
817       this.snapshotNextRow = null;
818 
819       this.kvsetIt = null;
820       this.snapshotIt = null;
821     }
822 
823     /**
824      * MemStoreScanner returns max value as sequence id because it will
825      * always have the latest data among all files.
826      */
827     @Override
828     public long getSequenceID() {
829       return Long.MAX_VALUE;
830     }
831   }
832 
833   public final static long FIXED_OVERHEAD = ClassSize.align(
834       ClassSize.OBJECT + (11 * ClassSize.REFERENCE));
835 
836   public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
837       ClassSize.REENTRANT_LOCK + ClassSize.ATOMIC_LONG +
838       ClassSize.COPYONWRITE_ARRAYSET + ClassSize.COPYONWRITE_ARRAYLIST +
839       (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
840 
841   /*
842    * Calculate how the MemStore size has changed.  Includes overhead of the
843    * backing Map.
844    * @param kv
845    * @param notpresent True if the kv was NOT present in the set.
846    * @return Size
847    */
848   long heapSizeChange(final KeyValue kv, final boolean notpresent) {
849     return notpresent ?
850         ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize()):
851         0;
852   }
853 
854   /**
855    * Get the entire heap usage for this MemStore not including keys in the
856    * snapshot.
857    */
858   @Override
859   public long heapSize() {
860     return size.get();
861   }
862 
863   /**
864    * Get the heap usage of KVs in this MemStore.
865    */
866   public long keySize() {
867     return heapSize() - DEEP_OVERHEAD;
868   }
869 
870   /**
871    * Code to help figure if our approximation of object heap sizes is close
872    * enough.  See hbase-900.  Fills memstores then waits so user can heap
873    * dump and bring up resultant hprof in something like jprofiler which
874    * allows you get 'deep size' on objects.
875    * @param args main args
876    */
877   public static void main(String [] args) {
878     RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
879     LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
880       runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
881     LOG.info("vmInputArguments=" + runtime.getInputArguments());
882     MemStore memstore1 = new MemStore();
883     // TODO: x32 vs x64
884     long size = 0;
885     final int count = 10000;
886     byte [] fam = Bytes.toBytes("col");
887     byte [] qf = Bytes.toBytes("umn");
888     byte [] empty = new byte[0];
889     for (int i = 0; i < count; i++) {
890       // Give each its own ts
891       size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
892     }
893     LOG.info("memstore1 estimated size=" + size);
894     for (int i = 0; i < count; i++) {
895       size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
896     }
897     LOG.info("memstore1 estimated size (2nd loading of same data)=" + size);
898     // Make a variably sized memstore.
899     MemStore memstore2 = new MemStore();
900     for (int i = 0; i < count; i++) {
901       size += memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i,
902         new byte[i]));
903     }
904     LOG.info("memstore2 estimated size=" + size);
905     final int seconds = 30;
906     LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
907     for (int i = 0; i < seconds; i++) {
908       // Thread.sleep(1000);
909     }
910     LOG.info("Exiting.");
911   }
912 }