View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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.hbase.classification.InterfaceAudience;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.Cell;
37  import org.apache.hadoop.hbase.CellUtil;
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.KeyValueUtil;
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.Pair;
49  import org.apache.hadoop.hbase.util.ReflectionUtils;
50  import org.apache.htrace.Trace;
51  
52  /**
53   * The MemStore holds in-memory modifications to the Store.  Modifications
54   * are {@link Cell}s.  When asked to flush, current memstore is moved
55   * to snapshot and is cleared.  We continue to serve edits out of new memstore
56   * and backing snapshot until flusher reports in that the flush succeeded. At
57   * this point we let the snapshot go.
58   *  <p>
59   * The MemStore functions should not be called in parallel. Callers should hold
60   *  write and read locks. This is done in {@link HStore}.
61   *  </p>
62   *
63   * TODO: Adjust size of the memstore when we remove items because they have
64   * been deleted.
65   * TODO: With new KVSLS, need to make sure we update HeapSize with difference
66   * in KV size.
67   */
68  @InterfaceAudience.Private
69  public class DefaultMemStore implements MemStore {
70    private static final Log LOG = LogFactory.getLog(DefaultMemStore.class);
71    static final String USEMSLAB_KEY = "hbase.hregion.memstore.mslab.enabled";
72    private static final boolean USEMSLAB_DEFAULT = true;
73    static final String MSLAB_CLASS_NAME = "hbase.regionserver.mslab.class";
74  
75    private Configuration conf;
76  
77    // MemStore.  Use a CellSkipListSet rather than SkipListSet because of the
78    // better semantics.  The Map will overwrite if passed a key it already had
79    // whereas the Set will not add new Cell if key is same though value might be
80    // different.  Value is not important -- just make sure always same
81    // reference passed.
82    volatile CellSkipListSet cellSet;
83  
84    // Snapshot of memstore.  Made for flusher.
85    volatile CellSkipListSet snapshot;
86  
87    final KeyValue.KVComparator comparator;
88  
89    // Used to track own heapSize
90    final AtomicLong size;
91    private volatile long snapshotSize;
92  
93    // Used to track when to flush
94    volatile long timeOfOldestEdit = Long.MAX_VALUE;
95  
96    TimeRangeTracker timeRangeTracker;
97    TimeRangeTracker snapshotTimeRangeTracker;
98  
99    volatile MemStoreLAB allocator;
100   volatile MemStoreLAB snapshotAllocator;
101   volatile long snapshotId;
102 
103   /**
104    * Default constructor. Used for tests.
105    */
106   public DefaultMemStore() {
107     this(HBaseConfiguration.create(), KeyValue.COMPARATOR);
108   }
109 
110   /**
111    * Constructor.
112    * @param c Comparator
113    */
114   public DefaultMemStore(final Configuration conf,
115                   final KeyValue.KVComparator 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    * Creates a snapshot of the current memstore.
144    * Snapshot must be cleared by call to {@link #clearSnapshot(long)}
145    */
146   @Override
147   public MemStoreSnapshot snapshot() {
148     // If snapshot currently has entries, then flusher failed or didn't call
149     // cleanup.  Log a warning.
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         // Reset heap to not include any keys
162         this.size.set(DEEP_OVERHEAD);
163         this.snapshotAllocator = this.allocator;
164         // Reset allocator so we get a fresh buffer for the new memstore
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     return new MemStoreSnapshot(this.snapshotId, snapshot.size(), this.snapshotSize,
174         this.snapshotTimeRangeTracker, new CollectionBackedScanner(snapshot, this.comparator));
175   }
176 
177   /**
178    * The passed snapshot was successfully persisted; it can be let go.
179    * @param id Id of the snapshot to clean out.
180    * @throws UnexpectedStateException
181    * @see #snapshot()
182    */
183   @Override
184   public void clearSnapshot(long id) throws UnexpectedStateException {
185     MemStoreLAB tmpAllocator = null;
186     if (this.snapshotId != id) {
187       throw new UnexpectedStateException("Current snapshot id is " + this.snapshotId + ",passed "
188           + id);
189     }
190     // OK. Passed in snapshot is same as current snapshot. If not-empty,
191     // create a new snapshot and let the old one go.
192     if (!this.snapshot.isEmpty()) {
193       this.snapshot = new CellSkipListSet(this.comparator);
194       this.snapshotTimeRangeTracker = new TimeRangeTracker();
195     }
196     this.snapshotSize = 0;
197     this.snapshotId = -1;
198     if (this.snapshotAllocator != null) {
199       tmpAllocator = this.snapshotAllocator;
200       this.snapshotAllocator = null;
201     }
202     if (tmpAllocator != null) {
203       tmpAllocator.close();
204     }
205   }
206 
207   @Override
208   public long getFlushableSize() {
209     return this.snapshotSize > 0 ? this.snapshotSize : keySize();
210   }
211 
212   @Override
213   public long getSnapshotSize() {
214     return this.snapshotSize;
215   }
216 
217   /**
218    * Write an update
219    * @param cell
220    * @return approximate size of the passed KV & newly added KV which maybe different than the
221    *         passed-in KV
222    */
223   @Override
224   public Pair<Long, Cell> add(Cell cell) {
225     Cell toAdd = maybeCloneWithAllocator(cell);
226     return new Pair<Long, Cell>(internalAdd(toAdd), toAdd);
227   }
228 
229   @Override
230   public long timeOfOldestEdit() {
231     return timeOfOldestEdit;
232   }
233 
234   private boolean addToCellSet(Cell e) {
235     boolean b = this.cellSet.add(e);
236     setOldestEditTimeToNow();
237     return b;
238   }
239 
240   private boolean removeFromCellSet(Cell e) {
241     boolean b = this.cellSet.remove(e);
242     setOldestEditTimeToNow();
243     return b;
244   }
245 
246   void setOldestEditTimeToNow() {
247     if (timeOfOldestEdit == Long.MAX_VALUE) {
248       timeOfOldestEdit = EnvironmentEdgeManager.currentTime();
249     }
250   }
251 
252   /**
253    * Internal version of add() that doesn't clone Cells with the
254    * allocator, and doesn't take the lock.
255    *
256    * Callers should ensure they already have the read lock taken
257    */
258   private long internalAdd(final Cell toAdd) {
259     long s = heapSizeChange(toAdd, addToCellSet(toAdd));
260     timeRangeTracker.includeTimestamp(toAdd);
261     this.size.addAndGet(s);
262     return s;
263   }
264 
265   private Cell maybeCloneWithAllocator(Cell cell) {
266     if (allocator == null) {
267       return cell;
268     }
269 
270     int len = KeyValueUtil.length(cell);
271     ByteRange alloc = allocator.allocateBytes(len);
272     if (alloc == null) {
273       // The allocation was too large, allocator decided
274       // not to do anything with it.
275       return cell;
276     }
277     assert alloc.getBytes() != null;
278     KeyValueUtil.appendToByteArray(cell, alloc.getBytes(), alloc.getOffset());
279     KeyValue newKv = new KeyValue(alloc.getBytes(), alloc.getOffset(), len);
280     newKv.setSequenceId(cell.getSequenceId());
281     return newKv;
282   }
283 
284   /**
285    * Remove n key from the memstore. Only cells that have the same key and the
286    * same memstoreTS are removed.  It is ok to not update timeRangeTracker
287    * in this call. It is possible that we can optimize this method by using
288    * tailMap/iterator, but since this method is called rarely (only for
289    * error recovery), we can leave those optimization for the future.
290    * @param cell
291    */
292   @Override
293   public void rollback(Cell cell) {
294     // If the key is in the snapshot, delete it. We should not update
295     // this.size, because that tracks the size of only the memstore and
296     // not the snapshot. The flush of this snapshot to disk has not
297     // yet started because Store.flush() waits for all rwcc transactions to
298     // commit before starting the flush to disk.
299     Cell found = this.snapshot.get(cell);
300     if (found != null && found.getSequenceId() == cell.getSequenceId()) {
301       this.snapshot.remove(cell);
302       long sz = heapSizeChange(cell, true);
303       this.snapshotSize -= sz;
304     }
305     // If the key is in the memstore, delete it. Update this.size.
306     found = this.cellSet.get(cell);
307     if (found != null && found.getSequenceId() == cell.getSequenceId()) {
308       removeFromCellSet(cell);
309       long s = heapSizeChange(cell, true);
310       this.size.addAndGet(-s);
311     }
312   }
313 
314   /**
315    * Write a delete
316    * @param deleteCell
317    * @return approximate size of the passed key and value.
318    */
319   @Override
320   public long delete(Cell deleteCell) {
321     long s = 0;
322     Cell toAdd = maybeCloneWithAllocator(deleteCell);
323     s += heapSizeChange(toAdd, addToCellSet(toAdd));
324     timeRangeTracker.includeTimestamp(toAdd);
325     this.size.addAndGet(s);
326     return s;
327   }
328 
329   /**
330    * @param cell Find the row that comes after this one.  If null, we return the
331    * first.
332    * @return Next row or null if none found.
333    */
334   Cell getNextRow(final Cell cell) {
335     return getLowest(getNextRow(cell, this.cellSet), getNextRow(cell, this.snapshot));
336   }
337 
338   /*
339    * @param a
340    * @param b
341    * @return Return lowest of a or b or null if both a and b are null
342    */
343   private Cell getLowest(final Cell a, final Cell b) {
344     if (a == null) {
345       return b;
346     }
347     if (b == null) {
348       return a;
349     }
350     return comparator.compareRows(a, b) <= 0? a: b;
351   }
352 
353   /*
354    * @param key Find row that follows this one.  If null, return first.
355    * @param map Set to look in for a row beyond <code>row</code>.
356    * @return Next row or null if none found.  If one found, will be a new
357    * KeyValue -- can be destroyed by subsequent calls to this method.
358    */
359   private Cell getNextRow(final Cell key,
360       final NavigableSet<Cell> set) {
361     Cell result = null;
362     SortedSet<Cell> tail = key == null? set: set.tailSet(key);
363     // Iterate until we fall into the next row; i.e. move off current row
364     for (Cell cell: tail) {
365       if (comparator.compareRows(cell, key) <= 0)
366         continue;
367       // Note: Not suppressing deletes or expired cells.  Needs to be handled
368       // by higher up functions.
369       result = cell;
370       break;
371     }
372     return result;
373   }
374 
375   /**
376    * @param state column/delete tracking state
377    */
378   @Override
379   public void getRowKeyAtOrBefore(final GetClosestRowBeforeTracker state) {
380     getRowKeyAtOrBefore(cellSet, state);
381     getRowKeyAtOrBefore(snapshot, state);
382   }
383 
384   /*
385    * @param set
386    * @param state Accumulates deletes and candidates.
387    */
388   private void getRowKeyAtOrBefore(final NavigableSet<Cell> set,
389       final GetClosestRowBeforeTracker state) {
390     if (set.isEmpty()) {
391       return;
392     }
393     if (!walkForwardInSingleRow(set, state.getTargetKey(), state)) {
394       // Found nothing in row.  Try backing up.
395       getRowKeyBefore(set, state);
396     }
397   }
398 
399   /*
400    * Walk forward in a row from <code>firstOnRow</code>.  Presumption is that
401    * we have been passed the first possible key on a row.  As we walk forward
402    * we accumulate deletes until we hit a candidate on the row at which point
403    * we return.
404    * @param set
405    * @param firstOnRow First possible key on this row.
406    * @param state
407    * @return True if we found a candidate walking this row.
408    */
409   private boolean walkForwardInSingleRow(final SortedSet<Cell> set,
410       final Cell firstOnRow, final GetClosestRowBeforeTracker state) {
411     boolean foundCandidate = false;
412     SortedSet<Cell> tail = set.tailSet(firstOnRow);
413     if (tail.isEmpty()) return foundCandidate;
414     for (Iterator<Cell> i = tail.iterator(); i.hasNext();) {
415       Cell kv = i.next();
416       // Did we go beyond the target row? If so break.
417       if (state.isTooFar(kv, firstOnRow)) break;
418       if (state.isExpired(kv)) {
419         i.remove();
420         continue;
421       }
422       // If we added something, this row is a contender. break.
423       if (state.handle(kv)) {
424         foundCandidate = true;
425         break;
426       }
427     }
428     return foundCandidate;
429   }
430 
431   /*
432    * Walk backwards through the passed set a row at a time until we run out of
433    * set or until we get a candidate.
434    * @param set
435    * @param state
436    */
437   private void getRowKeyBefore(NavigableSet<Cell> set,
438       final GetClosestRowBeforeTracker state) {
439     Cell firstOnRow = state.getTargetKey();
440     for (Member p = memberOfPreviousRow(set, state, firstOnRow);
441         p != null; p = memberOfPreviousRow(p.set, state, firstOnRow)) {
442       // Make sure we don't fall out of our table.
443       if (!state.isTargetTable(p.cell)) break;
444       // Stop looking if we've exited the better candidate range.
445       if (!state.isBetterCandidate(p.cell)) break;
446       // Make into firstOnRow
447       firstOnRow = new KeyValue(p.cell.getRowArray(), p.cell.getRowOffset(), p.cell.getRowLength(),
448           HConstants.LATEST_TIMESTAMP);
449       // If we find something, break;
450       if (walkForwardInSingleRow(p.set, firstOnRow, state)) break;
451     }
452   }
453 
454   /**
455    * Only used by tests. TODO: Remove
456    *
457    * Given the specs of a column, update it, first by inserting a new record,
458    * then removing the old one.  Since there is only 1 KeyValue involved, the memstoreTS
459    * will be set to 0, thus ensuring that they instantly appear to anyone. The underlying
460    * store will ensure that the insert/delete each are atomic. A scanner/reader will either
461    * get the new value, or the old value and all readers will eventually only see the new
462    * value after the old was removed.
463    *
464    * @param row
465    * @param family
466    * @param qualifier
467    * @param newValue
468    * @param now
469    * @return  Timestamp
470    */
471   @Override
472   public long updateColumnValue(byte[] row,
473                                 byte[] family,
474                                 byte[] qualifier,
475                                 long newValue,
476                                 long now) {
477     Cell firstCell = KeyValueUtil.createFirstOnRow(row, family, qualifier);
478     // Is there a Cell in 'snapshot' with the same TS? If so, upgrade the timestamp a bit.
479     SortedSet<Cell> snSs = snapshot.tailSet(firstCell);
480     if (!snSs.isEmpty()) {
481       Cell snc = snSs.first();
482       // is there a matching Cell in the snapshot?
483       if (CellUtil.matchingRow(snc, firstCell) && CellUtil.matchingQualifier(snc, firstCell)) {
484         if (snc.getTimestamp() == now) {
485           // poop,
486           now += 1;
487         }
488       }
489     }
490 
491     // logic here: the new ts MUST be at least 'now'. But it could be larger if necessary.
492     // But the timestamp should also be max(now, mostRecentTsInMemstore)
493 
494     // so we cant add the new Cell w/o knowing what's there already, but we also
495     // want to take this chance to delete some cells. So two loops (sad)
496 
497     SortedSet<Cell> ss = cellSet.tailSet(firstCell);
498     for (Cell cell : ss) {
499       // if this isnt the row we are interested in, then bail:
500       if (!CellUtil.matchingColumn(cell, family, qualifier)
501           || !CellUtil.matchingRow(cell, firstCell)) {
502         break; // rows dont match, bail.
503       }
504 
505       // if the qualifier matches and it's a put, just RM it out of the cellSet.
506       if (cell.getTypeByte() == KeyValue.Type.Put.getCode() &&
507           cell.getTimestamp() > now && CellUtil.matchingQualifier(firstCell, cell)) {
508         now = cell.getTimestamp();
509       }
510     }
511 
512     // create or update (upsert) a new Cell with
513     // 'now' and a 0 memstoreTS == immediately visible
514     List<Cell> cells = new ArrayList<Cell>(1);
515     cells.add(new KeyValue(row, family, qualifier, now, Bytes.toBytes(newValue)));
516     return upsert(cells, 1L);
517   }
518 
519   /**
520    * Update or insert the specified KeyValues.
521    * <p>
522    * For each KeyValue, insert into MemStore.  This will atomically upsert the
523    * value for that row/family/qualifier.  If a KeyValue did already exist,
524    * it will then be removed.
525    * <p>
526    * Currently the memstoreTS is kept at 0 so as each insert happens, it will
527    * be immediately visible.  May want to change this so it is atomic across
528    * all KeyValues.
529    * <p>
530    * This is called under row lock, so Get operations will still see updates
531    * atomically.  Scans will only see each KeyValue update as atomic.
532    *
533    * @param cells
534    * @param readpoint readpoint below which we can safely remove duplicate KVs
535    * @return change in memstore size
536    */
537   @Override
538   public long upsert(Iterable<Cell> cells, long readpoint) {
539     long size = 0;
540     for (Cell cell : cells) {
541       size += upsert(cell, readpoint);
542     }
543     return size;
544   }
545 
546   /**
547    * Inserts the specified KeyValue into MemStore and deletes any existing
548    * versions of the same row/family/qualifier as the specified KeyValue.
549    * <p>
550    * First, the specified KeyValue is inserted into the Memstore.
551    * <p>
552    * If there are any existing KeyValues in this MemStore with the same row,
553    * family, and qualifier, they are removed.
554    * <p>
555    * Callers must hold the read lock.
556    *
557    * @param cell
558    * @return change in size of MemStore
559    */
560   private long upsert(Cell cell, long readpoint) {
561     // Add the Cell to the MemStore
562     // Use the internalAdd method here since we (a) already have a lock
563     // and (b) cannot safely use the MSLAB here without potentially
564     // hitting OOME - see TestMemStore.testUpsertMSLAB for a
565     // test that triggers the pathological case if we don't avoid MSLAB
566     // here.
567     long addedSize = internalAdd(cell);
568 
569     // Get the Cells for the row/family/qualifier regardless of timestamp.
570     // For this case we want to clean up any other puts
571     Cell firstCell = KeyValueUtil.createFirstOnRow(
572         cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
573         cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
574         cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
575     SortedSet<Cell> ss = cellSet.tailSet(firstCell);
576     Iterator<Cell> it = ss.iterator();
577     // versions visible to oldest scanner
578     int versionsVisible = 0;
579     while ( it.hasNext() ) {
580       Cell cur = it.next();
581 
582       if (cell == cur) {
583         // ignore the one just put in
584         continue;
585       }
586       // check that this is the row and column we are interested in, otherwise bail
587       if (CellUtil.matchingRow(cell, cur) && CellUtil.matchingQualifier(cell, cur)) {
588         // only remove Puts that concurrent scanners cannot possibly see
589         if (cur.getTypeByte() == KeyValue.Type.Put.getCode() &&
590             cur.getSequenceId() <= readpoint) {
591           if (versionsVisible >= 1) {
592             // if we get here we have seen at least one version visible to the oldest scanner,
593             // which means we can prove that no scanner will see this version
594 
595             // false means there was a change, so give us the size.
596             long delta = heapSizeChange(cur, true);
597             addedSize -= delta;
598             this.size.addAndGet(-delta);
599             it.remove();
600             setOldestEditTimeToNow();
601           } else {
602             versionsVisible++;
603           }
604         }
605       } else {
606         // past the row or column, done
607         break;
608       }
609     }
610     return addedSize;
611   }
612 
613   /*
614    * Immutable data structure to hold member found in set and the set it was
615    * found in. Include set because it is carrying context.
616    */
617   private static class Member {
618     final Cell cell;
619     final NavigableSet<Cell> set;
620     Member(final NavigableSet<Cell> s, final Cell kv) {
621       this.cell = kv;
622       this.set = s;
623     }
624   }
625 
626   /*
627    * @param set Set to walk back in.  Pass a first in row or we'll return
628    * same row (loop).
629    * @param state Utility and context.
630    * @param firstOnRow First item on the row after the one we want to find a
631    * member in.
632    * @return Null or member of row previous to <code>firstOnRow</code>
633    */
634   private Member memberOfPreviousRow(NavigableSet<Cell> set,
635       final GetClosestRowBeforeTracker state, final Cell firstOnRow) {
636     NavigableSet<Cell> head = set.headSet(firstOnRow, false);
637     if (head.isEmpty()) return null;
638     for (Iterator<Cell> i = head.descendingIterator(); i.hasNext();) {
639       Cell found = i.next();
640       if (state.isExpired(found)) {
641         i.remove();
642         continue;
643       }
644       return new Member(head, found);
645     }
646     return null;
647   }
648 
649   /**
650    * @return scanner on memstore and snapshot in this order.
651    */
652   @Override
653   public List<KeyValueScanner> getScanners(long readPt) {
654     return Collections.<KeyValueScanner> singletonList(new MemStoreScanner(readPt));
655   }
656 
657   /**
658    * Check if this memstore may contain the required keys
659    * @param scan
660    * @return False if the key definitely does not exist in this Memstore
661    */
662   public boolean shouldSeek(Scan scan, long oldestUnexpiredTS) {
663     return (timeRangeTracker.includesTimeRange(scan.getTimeRange()) ||
664         snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange()))
665         && (Math.max(timeRangeTracker.getMaximumTimestamp(),
666                      snapshotTimeRangeTracker.getMaximumTimestamp()) >=
667             oldestUnexpiredTS);
668   }
669 
670   /*
671    * MemStoreScanner implements the KeyValueScanner.
672    * It lets the caller scan the contents of a memstore -- both current
673    * map and snapshot.
674    * This behaves as if it were a real scanner but does not maintain position.
675    */
676   protected class MemStoreScanner extends NonLazyKeyValueScanner {
677     // Next row information for either cellSet or snapshot
678     private Cell cellSetNextRow = null;
679     private Cell snapshotNextRow = null;
680 
681     // last iterated Cells for cellSet and snapshot (to restore iterator state after reseek)
682     private Cell cellSetItRow = null;
683     private Cell snapshotItRow = null;
684     
685     // iterator based scanning.
686     private Iterator<Cell> cellSetIt;
687     private Iterator<Cell> snapshotIt;
688 
689     // The cellSet and snapshot at the time of creating this scanner
690     private CellSkipListSet cellSetAtCreation;
691     private CellSkipListSet snapshotAtCreation;
692 
693     // the pre-calculated Cell to be returned by peek() or next()
694     private Cell theNext;
695 
696     // The allocator and snapshot allocator at the time of creating this scanner
697     volatile MemStoreLAB allocatorAtCreation;
698     volatile MemStoreLAB snapshotAllocatorAtCreation;
699     
700     // A flag represents whether could stop skipping Cells for MVCC
701     // if have encountered the next row. Only used for reversed scan
702     private boolean stopSkippingCellsIfNextRow = false;
703 
704     private long readPoint;
705 
706     /*
707     Some notes...
708 
709      So memstorescanner is fixed at creation time. this includes pointers/iterators into
710     existing kvset/snapshot.  during a snapshot creation, the kvset is null, and the
711     snapshot is moved.  since kvset is null there is no point on reseeking on both,
712       we can save us the trouble. During the snapshot->hfile transition, the memstore
713       scanner is re-created by StoreScanner#updateReaders().  StoreScanner should
714       potentially do something smarter by adjusting the existing memstore scanner.
715 
716       But there is a greater problem here, that being once a scanner has progressed
717       during a snapshot scenario, we currently iterate past the kvset then 'finish' up.
718       if a scan lasts a little while, there is a chance for new entries in kvset to
719       become available but we will never see them.  This needs to be handled at the
720       StoreScanner level with coordination with MemStoreScanner.
721 
722       Currently, this problem is only partly managed: during the small amount of time
723       when the StoreScanner has not yet created a new MemStoreScanner, we will miss
724       the adds to kvset in the MemStoreScanner.
725     */
726 
727     MemStoreScanner(long readPoint) {
728       super();
729 
730       this.readPoint = readPoint;
731       cellSetAtCreation = cellSet;
732       snapshotAtCreation = snapshot;
733       if (allocator != null) {
734         this.allocatorAtCreation = allocator;
735         this.allocatorAtCreation.incScannerCount();
736       }
737       if (snapshotAllocator != null) {
738         this.snapshotAllocatorAtCreation = snapshotAllocator;
739         this.snapshotAllocatorAtCreation.incScannerCount();
740       }
741       if (Trace.isTracing() && Trace.currentSpan() != null) {
742         Trace.currentSpan().addTimelineAnnotation("Creating MemStoreScanner");
743       }
744     }
745 
746     /**
747      * Lock on 'this' must be held by caller.
748      * @param it
749      * @return Next Cell
750      */
751     private Cell getNext(Iterator<Cell> it) {
752       Cell startCell = theNext;
753       Cell v = null;
754       try {
755         while (it.hasNext()) {
756           v = it.next();
757           if (v.getSequenceId() <= this.readPoint) {
758             return v;
759           }
760           if (stopSkippingCellsIfNextRow && startCell != null
761               && comparator.compareRows(v, startCell) > 0) {
762             return null;
763           }
764         }
765 
766         return null;
767       } finally {
768         if (v != null) {
769           // in all cases, remember the last Cell iterated to
770           if (it == snapshotIt) {
771             snapshotItRow = v;
772           } else {
773             cellSetItRow = v;
774           }
775         }
776       }
777     }
778 
779     /**
780      *  Set the scanner at the seek key.
781      *  Must be called only once: there is no thread safety between the scanner
782      *   and the memStore.
783      * @param key seek value
784      * @return false if the key is null or if there is no data
785      */
786     @Override
787     public synchronized boolean seek(Cell key) {
788       if (key == null) {
789         close();
790         return false;
791       }
792       // kvset and snapshot will never be null.
793       // if tailSet can't find anything, SortedSet is empty (not null).
794       cellSetIt = cellSetAtCreation.tailSet(key).iterator();
795       snapshotIt = snapshotAtCreation.tailSet(key).iterator();
796       cellSetItRow = null;
797       snapshotItRow = null;
798 
799       return seekInSubLists(key);
800     }
801 
802 
803     /**
804      * (Re)initialize the iterators after a seek or a reseek.
805      */
806     private synchronized boolean seekInSubLists(Cell key){
807       cellSetNextRow = getNext(cellSetIt);
808       snapshotNextRow = getNext(snapshotIt);
809 
810       // Calculate the next value
811       theNext = getLowest(cellSetNextRow, snapshotNextRow);
812 
813       // has data
814       return (theNext != null);
815     }
816 
817 
818     /**
819      * Move forward on the sub-lists set previously by seek.
820      * @param key seek value (should be non-null)
821      * @return true if there is at least one KV to read, false otherwise
822      */
823     @Override
824     public synchronized boolean reseek(Cell key) {
825       /*
826       See HBASE-4195 & HBASE-3855 & HBASE-6591 for the background on this implementation.
827       This code is executed concurrently with flush and puts, without locks.
828       Two points must be known when working on this code:
829       1) It's not possible to use the 'kvTail' and 'snapshot'
830        variables, as they are modified during a flush.
831       2) The ideal implementation for performance would use the sub skip list
832        implicitly pointed by the iterators 'kvsetIt' and
833        'snapshotIt'. Unfortunately the Java API does not offer a method to
834        get it. So we remember the last keys we iterated to and restore
835        the reseeked set to at least that point.
836        */
837       cellSetIt = cellSetAtCreation.tailSet(getHighest(key, cellSetItRow)).iterator();
838       snapshotIt = snapshotAtCreation.tailSet(getHighest(key, snapshotItRow)).iterator();
839 
840       return seekInSubLists(key);
841     }
842 
843 
844     @Override
845     public synchronized Cell peek() {
846       //DebugPrint.println(" MS@" + hashCode() + " peek = " + getLowest());
847       return theNext;
848     }
849 
850     @Override
851     public synchronized Cell next() {
852       if (theNext == null) {
853           return null;
854       }
855 
856       final Cell ret = theNext;
857 
858       // Advance one of the iterators
859       if (theNext == cellSetNextRow) {
860         cellSetNextRow = getNext(cellSetIt);
861       } else {
862         snapshotNextRow = getNext(snapshotIt);
863       }
864 
865       // Calculate the next value
866       theNext = getLowest(cellSetNextRow, snapshotNextRow);
867 
868       //long readpoint = ReadWriteConsistencyControl.getThreadReadPoint();
869       //DebugPrint.println(" MS@" + hashCode() + " next: " + theNext + " next_next: " +
870       //    getLowest() + " threadpoint=" + readpoint);
871       return ret;
872     }
873 
874     /*
875      * Returns the lower of the two key values, or null if they are both null.
876      * This uses comparator.compare() to compare the KeyValue using the memstore
877      * comparator.
878      */
879     private Cell getLowest(Cell first, Cell second) {
880       if (first == null && second == null) {
881         return null;
882       }
883       if (first != null && second != null) {
884         int compare = comparator.compare(first, second);
885         return (compare <= 0 ? first : second);
886       }
887       return (first != null ? first : second);
888     }
889 
890     /*
891      * Returns the higher of the two cells, or null if they are both null.
892      * This uses comparator.compare() to compare the Cell using the memstore
893      * comparator.
894      */
895     private Cell getHighest(Cell first, Cell second) {
896       if (first == null && second == null) {
897         return null;
898       }
899       if (first != null && second != null) {
900         int compare = comparator.compare(first, second);
901         return (compare > 0 ? first : second);
902       }
903       return (first != null ? first : second);
904     }
905 
906     public synchronized void close() {
907       this.cellSetNextRow = null;
908       this.snapshotNextRow = null;
909 
910       this.cellSetIt = null;
911       this.snapshotIt = null;
912       
913       if (allocatorAtCreation != null) {
914         this.allocatorAtCreation.decScannerCount();
915         this.allocatorAtCreation = null;
916       }
917       if (snapshotAllocatorAtCreation != null) {
918         this.snapshotAllocatorAtCreation.decScannerCount();
919         this.snapshotAllocatorAtCreation = null;
920       }
921 
922       this.cellSetItRow = null;
923       this.snapshotItRow = null;
924     }
925 
926     /**
927      * MemStoreScanner returns max value as sequence id because it will
928      * always have the latest data among all files.
929      */
930     @Override
931     public long getSequenceID() {
932       return Long.MAX_VALUE;
933     }
934 
935     @Override
936     public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
937         long oldestUnexpiredTS) {
938       return shouldSeek(scan, oldestUnexpiredTS);
939     }
940 
941     /**
942      * Seek scanner to the given key first. If it returns false(means
943      * peek()==null) or scanner's peek row is bigger than row of given key, seek
944      * the scanner to the previous row of given key
945      */
946     @Override
947     public synchronized boolean backwardSeek(Cell key) {
948       seek(key);
949       if (peek() == null || comparator.compareRows(peek(), key) > 0) {
950         return seekToPreviousRow(key);
951       }
952       return true;
953     }
954 
955     /**
956      * Separately get the KeyValue before the specified key from kvset and
957      * snapshotset, and use the row of higher one as the previous row of
958      * specified key, then seek to the first KeyValue of previous row
959      */
960     @Override
961     public synchronized boolean seekToPreviousRow(Cell originalKey) {
962       boolean keepSeeking = false;
963       Cell key = originalKey;
964       do {
965         Cell firstKeyOnRow = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(),
966             key.getRowLength());
967         SortedSet<Cell> cellHead = cellSetAtCreation.headSet(firstKeyOnRow);
968         Cell cellSetBeforeRow = cellHead.isEmpty() ? null : cellHead.last();
969         SortedSet<Cell> snapshotHead = snapshotAtCreation
970             .headSet(firstKeyOnRow);
971         Cell snapshotBeforeRow = snapshotHead.isEmpty() ? null : snapshotHead
972             .last();
973         Cell lastCellBeforeRow = getHighest(cellSetBeforeRow, snapshotBeforeRow);
974         if (lastCellBeforeRow == null) {
975           theNext = null;
976           return false;
977         }
978         Cell firstKeyOnPreviousRow = KeyValueUtil.createFirstOnRow(lastCellBeforeRow.getRowArray(),
979             lastCellBeforeRow.getRowOffset(), lastCellBeforeRow.getRowLength());
980         this.stopSkippingCellsIfNextRow = true;
981         seek(firstKeyOnPreviousRow);
982         this.stopSkippingCellsIfNextRow = false;
983         if (peek() == null
984             || comparator.compareRows(peek(), firstKeyOnPreviousRow) > 0) {
985           keepSeeking = true;
986           key = firstKeyOnPreviousRow;
987           continue;
988         } else {
989           keepSeeking = false;
990         }
991       } while (keepSeeking);
992       return true;
993     }
994 
995     @Override
996     public synchronized boolean seekToLastRow() {
997       Cell first = cellSetAtCreation.isEmpty() ? null : cellSetAtCreation
998           .last();
999       Cell second = snapshotAtCreation.isEmpty() ? null
1000           : snapshotAtCreation.last();
1001       Cell higherCell = getHighest(first, second);
1002       if (higherCell == null) {
1003         return false;
1004       }
1005       Cell firstCellOnLastRow = KeyValueUtil.createFirstOnRow(higherCell.getRowArray(),
1006           higherCell.getRowOffset(), higherCell.getRowLength());
1007       if (seek(firstCellOnLastRow)) {
1008         return true;
1009       } else {
1010         return seekToPreviousRow(higherCell);
1011       }
1012 
1013     }
1014   }
1015 
1016   public final static long FIXED_OVERHEAD = ClassSize.align(
1017       ClassSize.OBJECT + (9 * ClassSize.REFERENCE) + (3 * Bytes.SIZEOF_LONG));
1018 
1019   public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
1020       ClassSize.ATOMIC_LONG + (2 * ClassSize.TIMERANGE_TRACKER) +
1021       (2 * ClassSize.CELL_SKIPLIST_SET) + (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
1022 
1023   /*
1024    * Calculate how the MemStore size has changed.  Includes overhead of the
1025    * backing Map.
1026    * @param cell
1027    * @param notpresent True if the cell was NOT present in the set.
1028    * @return Size
1029    */
1030   static long heapSizeChange(final Cell cell, final boolean notpresent) {
1031     return notpresent ? ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY
1032         + CellUtil.estimatedHeapSizeOf(cell)) : 0;
1033   }
1034 
1035   private long keySize() {
1036     return heapSize() - DEEP_OVERHEAD;
1037   }
1038 
1039   /**
1040    * Get the entire heap usage for this MemStore not including keys in the
1041    * snapshot.
1042    */
1043   @Override
1044   public long heapSize() {
1045     return size.get();
1046   }
1047 
1048   @Override
1049   public long size() {
1050     return heapSize();
1051   }
1052 
1053   /**
1054    * Code to help figure if our approximation of object heap sizes is close
1055    * enough.  See hbase-900.  Fills memstores then waits so user can heap
1056    * dump and bring up resultant hprof in something like jprofiler which
1057    * allows you get 'deep size' on objects.
1058    * @param args main args
1059    */
1060   public static void main(String [] args) {
1061     RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
1062     LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
1063       runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
1064     LOG.info("vmInputArguments=" + runtime.getInputArguments());
1065     DefaultMemStore memstore1 = new DefaultMemStore();
1066     // TODO: x32 vs x64
1067     long size = 0;
1068     final int count = 10000;
1069     byte [] fam = Bytes.toBytes("col");
1070     byte [] qf = Bytes.toBytes("umn");
1071     byte [] empty = new byte[0];
1072     for (int i = 0; i < count; i++) {
1073       // Give each its own ts
1074       Pair<Long, Cell> ret = memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
1075       size += ret.getFirst();
1076     }
1077     LOG.info("memstore1 estimated size=" + size);
1078     for (int i = 0; i < count; i++) {
1079       Pair<Long, Cell> ret = memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
1080       size += ret.getFirst();
1081     }
1082     LOG.info("memstore1 estimated size (2nd loading of same data)=" + size);
1083     // Make a variably sized memstore.
1084     DefaultMemStore memstore2 = new DefaultMemStore();
1085     for (int i = 0; i < count; i++) {
1086       Pair<Long, Cell> ret = memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i,
1087         new byte[i]));
1088       size += ret.getFirst();
1089     }
1090     LOG.info("memstore2 estimated size=" + size);
1091     final int seconds = 30;
1092     LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
1093     for (int i = 0; i < seconds; i++) {
1094       // Thread.sleep(1000);
1095     }
1096     LOG.info("Exiting.");
1097   }
1098 
1099 }