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.rmi.UnexpectedException;
25  import java.util.ArrayList;
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  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.hbase.classification.InterfaceAudience;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.hbase.Cell;
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.io.HeapSize;
44  import org.apache.hadoop.hbase.regionserver.MemStoreLAB.Allocation;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.apache.hadoop.hbase.util.ClassSize;
47  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
48  import org.cloudera.htrace.Trace;
49  
50  /**
51   * The MemStore holds in-memory modifications to the Store.  Modifications
52   * are {@link KeyValue}s.  When asked to flush, current memstore is moved
53   * to snapshot and is cleared.  We continue to serve edits out of new memstore
54   * and backing snapshot until flusher reports in that the flush succeeded. At
55   * this point we let the snapshot go.
56   *  <p>
57   * The MemStore functions should not be called in parallel. Callers should hold
58   *  write and read locks. This is done in {@link HStore}.
59   *  </p>
60   *
61   * TODO: Adjust size of the memstore when we remove items because they have
62   * been deleted.
63   * TODO: With new KVSLS, need to make sure we update HeapSize with difference
64   * in KV size.
65   */
66  @InterfaceAudience.Private
67  public class MemStore implements HeapSize {
68    private static final Log LOG = LogFactory.getLog(MemStore.class);
69  
70    static final String USEMSLAB_KEY =
71      "hbase.hregion.memstore.mslab.enabled";
72    private static final boolean USEMSLAB_DEFAULT = true;
73  
74    private Configuration conf;
75  
76    // MemStore.  Use a KeyValueSkipListSet rather than SkipListSet because of the
77    // better semantics.  The Map will overwrite if passed a key it already had
78    // whereas the Set will not add new KV if key is same though value might be
79    // different.  Value is not important -- just make sure always same
80    // reference passed.
81    volatile KeyValueSkipListSet kvset;
82  
83    // Snapshot of memstore.  Made for flusher.
84    volatile KeyValueSkipListSet snapshot;
85  
86    final KeyValue.KVComparator comparator;
87  
88    // Used to track own heapSize
89    final AtomicLong size;
90    private volatile long snapshotSize;
91  
92    // Used to track when to flush
93    volatile long timeOfOldestEdit = Long.MAX_VALUE;
94  
95    TimeRangeTracker timeRangeTracker;
96    TimeRangeTracker snapshotTimeRangeTracker;
97  
98    MemStoreChunkPool chunkPool;
99    volatile MemStoreLAB allocator;
100   volatile MemStoreLAB snapshotAllocator;
101 
102   /**
103    * Default constructor. Used for tests.
104    */
105   public MemStore() {
106     this(HBaseConfiguration.create(), KeyValue.COMPARATOR);
107   }
108 
109   /**
110    * Constructor.
111    * @param c Comparator
112    */
113   public MemStore(final Configuration conf,
114                   final KeyValue.KVComparator c) {
115     this.conf = conf;
116     this.comparator = c;
117     this.kvset = new KeyValueSkipListSet(c);
118     this.snapshot = new KeyValueSkipListSet(c);
119     timeRangeTracker = new TimeRangeTracker();
120     snapshotTimeRangeTracker = new TimeRangeTracker();
121     this.size = new AtomicLong(DEEP_OVERHEAD);
122     this.snapshotSize = 0;
123     if (conf.getBoolean(USEMSLAB_KEY, USEMSLAB_DEFAULT)) {
124       this.chunkPool = MemStoreChunkPool.getPool(conf);
125       this.allocator = new MemStoreLAB(conf, chunkPool);
126     } else {
127       this.allocator = null;
128       this.chunkPool = null;
129     }
130   }
131 
132   void dump() {
133     for (KeyValue kv: this.kvset) {
134       LOG.info(kv);
135     }
136     for (KeyValue kv: this.snapshot) {
137       LOG.info(kv);
138     }
139   }
140 
141   /**
142    * Creates a snapshot of the current memstore.
143    * Snapshot must be cleared by call to {@link #clearSnapshot(SortedSet)}
144    * To get the snapshot made by this method, use {@link #getSnapshot()}
145    */
146   void snapshot() {
147     // If snapshot currently has entries, then flusher failed or didn't call
148     // cleanup.  Log a warning.
149     if (!this.snapshot.isEmpty()) {
150       LOG.warn("Snapshot called again without clearing previous. " +
151           "Doing nothing. Another ongoing flush or did we fail last attempt?");
152     } else {
153       if (!this.kvset.isEmpty()) {
154         this.snapshotSize = keySize();
155         this.snapshot = this.kvset;
156         this.kvset = new KeyValueSkipListSet(this.comparator);
157         this.snapshotTimeRangeTracker = this.timeRangeTracker;
158         this.timeRangeTracker = new TimeRangeTracker();
159         // Reset heap to not include any keys
160         this.size.set(DEEP_OVERHEAD);
161         this.snapshotAllocator = this.allocator;
162         // Reset allocator so we get a fresh buffer for the new memstore
163         if (allocator != null) {
164           this.allocator = new MemStoreLAB(conf, chunkPool);
165         }
166         timeOfOldestEdit = Long.MAX_VALUE;
167       }
168     }
169   }
170 
171   /**
172    * Return the current snapshot.
173    * Called by flusher to get current snapshot made by a previous
174    * call to {@link #snapshot()}
175    * @return Return snapshot.
176    * @see #snapshot()
177    * @see #clearSnapshot(SortedSet)
178    */
179   KeyValueSkipListSet getSnapshot() {
180     return this.snapshot;
181   }
182 
183   /**
184    * On flush, how much memory we will clear.
185    * Flush will first clear out the data in snapshot if any (It will take a second flush
186    * invocation to clear the current Cell set). If snapshot is empty, current
187    * Cell set will be flushed.
188    *
189    * @return size of data that is going to be flushed
190    */
191   long getFlushableSize() {
192     return this.snapshotSize > 0 ? this.snapshotSize : keySize();
193   }
194 
195   /**
196    * The passed snapshot was successfully persisted; it can be let go.
197    * @param ss The snapshot to clean out.
198    * @throws UnexpectedException
199    * @see #snapshot()
200    */
201   void clearSnapshot(final SortedSet<KeyValue> ss)
202   throws UnexpectedException {
203     MemStoreLAB tmpAllocator = null;
204     if (this.snapshot != ss) {
205       throw new UnexpectedException("Current snapshot is " +
206           this.snapshot + ", was passed " + ss);
207     }
208     // OK. Passed in snapshot is same as current snapshot.  If not-empty,
209     // create a new snapshot and let the old one go.
210     if (!ss.isEmpty()) {
211       this.snapshot = new KeyValueSkipListSet(this.comparator);
212       this.snapshotTimeRangeTracker = new TimeRangeTracker();
213     }
214     this.snapshotSize = 0;
215     if (this.snapshotAllocator != null) {
216       tmpAllocator = this.snapshotAllocator;
217       this.snapshotAllocator = null;
218     }
219     if (tmpAllocator != null) {
220       tmpAllocator.close();
221     }
222   }
223 
224   /**
225    * Write an update
226    * @param kv
227    * @return approximate size of the passed key and value.
228    */
229   long add(final KeyValue kv) {
230     KeyValue toAdd = maybeCloneWithAllocator(kv);
231     return internalAdd(toAdd);
232   }
233 
234   long timeOfOldestEdit() {
235     return timeOfOldestEdit;
236   }
237 
238   private boolean addToKVSet(KeyValue e) {
239     boolean b = this.kvset.add(e);
240     setOldestEditTimeToNow();
241     return b;
242   }
243 
244   private boolean removeFromKVSet(KeyValue e) {
245     boolean b = this.kvset.remove(e);
246     setOldestEditTimeToNow();
247     return b;
248   }
249 
250   void setOldestEditTimeToNow() {
251     if (timeOfOldestEdit == Long.MAX_VALUE) {
252       timeOfOldestEdit = EnvironmentEdgeManager.currentTimeMillis();
253     }
254   }
255 
256   /**
257    * Internal version of add() that doesn't clone KVs with the
258    * allocator, and doesn't take the lock.
259    *
260    * Callers should ensure they already have the read lock taken
261    */
262   private long internalAdd(final KeyValue toAdd) {
263     long s = heapSizeChange(toAdd, addToKVSet(toAdd));
264     timeRangeTracker.includeTimestamp(toAdd);
265     this.size.addAndGet(s);
266     return s;
267   }
268 
269   private KeyValue maybeCloneWithAllocator(KeyValue kv) {
270     if (allocator == null) {
271       return kv;
272     }
273 
274     int len = kv.getLength();
275     Allocation alloc = allocator.allocateBytes(len);
276     if (alloc == null) {
277       // The allocation was too large, allocator decided
278       // not to do anything with it.
279       return kv;
280     }
281     assert alloc.getData() != null;
282     System.arraycopy(kv.getBuffer(), kv.getOffset(), alloc.getData(), alloc.getOffset(), len);
283     KeyValue newKv = new KeyValue(alloc.getData(), alloc.getOffset(), len);
284     newKv.setMvccVersion(kv.getMvccVersion());
285     return newKv;
286   }
287 
288   /**
289    * Remove n key from the memstore. Only kvs that have the same key and the
290    * same memstoreTS are removed.  It is ok to not update timeRangeTracker
291    * in this call. It is possible that we can optimize this method by using
292    * tailMap/iterator, but since this method is called rarely (only for
293    * error recovery), we can leave those optimization for the future.
294    * @param kv
295    */
296   void rollback(final KeyValue kv) {
297     // If the key is in the snapshot, delete it. We should not update
298     // this.size, because that tracks the size of only the memstore and
299     // not the snapshot. The flush of this snapshot to disk has not
300     // yet started because Store.flush() waits for all rwcc transactions to
301     // commit before starting the flush to disk.
302     KeyValue found = this.snapshot.get(kv);
303     if (found != null && found.getMvccVersion() == kv.getMvccVersion()) {
304       this.snapshot.remove(kv);
305       long sz = heapSizeChange(kv, true);
306       this.snapshotSize -= sz;
307     }
308     // If the key is in the memstore, delete it. Update this.size.
309     found = this.kvset.get(kv);
310     if (found != null && found.getMvccVersion() == kv.getMvccVersion()) {
311       removeFromKVSet(kv);
312       long s = heapSizeChange(kv, true);
313       this.size.addAndGet(-s);
314     }
315   }
316 
317   /**
318    * Write a delete
319    * @param delete
320    * @return approximate size of the passed key and value.
321    */
322   long delete(final KeyValue delete) {
323     long s = 0;
324     KeyValue toAdd = maybeCloneWithAllocator(delete);
325     s += heapSizeChange(toAdd, addToKVSet(toAdd));
326     timeRangeTracker.includeTimestamp(toAdd);
327     this.size.addAndGet(s);
328     return s;
329   }
330 
331   /**
332    * @param kv Find the row that comes after this one.  If null, we return the
333    * first.
334    * @return Next row or null if none found.
335    */
336   KeyValue getNextRow(final KeyValue kv) {
337     return getLowest(getNextRow(kv, this.kvset), getNextRow(kv, this.snapshot));
338   }
339 
340   /*
341    * @param a
342    * @param b
343    * @return Return lowest of a or b or null if both a and b are null
344    */
345   private KeyValue getLowest(final KeyValue a, final KeyValue b) {
346     if (a == null) {
347       return b;
348     }
349     if (b == null) {
350       return a;
351     }
352     return comparator.compareRows(a, b) <= 0? a: b;
353   }
354 
355   /*
356    * @param key Find row that follows this one.  If null, return first.
357    * @param map Set to look in for a row beyond <code>row</code>.
358    * @return Next row or null if none found.  If one found, will be a new
359    * KeyValue -- can be destroyed by subsequent calls to this method.
360    */
361   private KeyValue getNextRow(final KeyValue key,
362       final NavigableSet<KeyValue> set) {
363     KeyValue result = null;
364     SortedSet<KeyValue> tail = key == null? set: set.tailSet(key);
365     // Iterate until we fall into the next row; i.e. move off current row
366     for (KeyValue kv: tail) {
367       if (comparator.compareRows(kv, key) <= 0)
368         continue;
369       // Note: Not suppressing deletes or expired cells.  Needs to be handled
370       // by higher up functions.
371       result = kv;
372       break;
373     }
374     return result;
375   }
376 
377   /**
378    * @param state column/delete tracking state
379    */
380   void getRowKeyAtOrBefore(final GetClosestRowBeforeTracker state) {
381     getRowKeyAtOrBefore(kvset, state);
382     getRowKeyAtOrBefore(snapshot, state);
383   }
384 
385   /*
386    * @param set
387    * @param state Accumulates deletes and candidates.
388    */
389   private void getRowKeyAtOrBefore(final NavigableSet<KeyValue> set,
390       final GetClosestRowBeforeTracker state) {
391     if (set.isEmpty()) {
392       return;
393     }
394     if (!walkForwardInSingleRow(set, state.getTargetKey(), state)) {
395       // Found nothing in row.  Try backing up.
396       getRowKeyBefore(set, state);
397     }
398   }
399 
400   /*
401    * Walk forward in a row from <code>firstOnRow</code>.  Presumption is that
402    * we have been passed the first possible key on a row.  As we walk forward
403    * we accumulate deletes until we hit a candidate on the row at which point
404    * we return.
405    * @param set
406    * @param firstOnRow First possible key on this row.
407    * @param state
408    * @return True if we found a candidate walking this row.
409    */
410   private boolean walkForwardInSingleRow(final SortedSet<KeyValue> set,
411       final KeyValue firstOnRow, final GetClosestRowBeforeTracker state) {
412     boolean foundCandidate = false;
413     SortedSet<KeyValue> tail = set.tailSet(firstOnRow);
414     if (tail.isEmpty()) return foundCandidate;
415     for (Iterator<KeyValue> i = tail.iterator(); i.hasNext();) {
416       KeyValue kv = i.next();
417       // Did we go beyond the target row? If so break.
418       if (state.isTooFar(kv, firstOnRow)) break;
419       if (state.isExpired(kv)) {
420         i.remove();
421         continue;
422       }
423       // If we added something, this row is a contender. break.
424       if (state.handle(kv)) {
425         foundCandidate = true;
426         break;
427       }
428     }
429     return foundCandidate;
430   }
431 
432   /*
433    * Walk backwards through the passed set a row at a time until we run out of
434    * set or until we get a candidate.
435    * @param set
436    * @param state
437    */
438   private void getRowKeyBefore(NavigableSet<KeyValue> set,
439       final GetClosestRowBeforeTracker state) {
440     KeyValue firstOnRow = state.getTargetKey();
441     for (Member p = memberOfPreviousRow(set, state, firstOnRow);
442         p != null; p = memberOfPreviousRow(p.set, state, firstOnRow)) {
443       // Make sure we don't fall out of our table.
444       if (!state.isTargetTable(p.kv)) break;
445       // Stop looking if we've exited the better candidate range.
446       if (!state.isBetterCandidate(p.kv)) break;
447       // Make into firstOnRow
448       firstOnRow = new KeyValue(p.kv.getRowArray(), p.kv.getRowOffset(), p.kv.getRowLength(),
449           HConstants.LATEST_TIMESTAMP);
450       // If we find something, break;
451       if (walkForwardInSingleRow(p.set, firstOnRow, state)) break;
452     }
453   }
454 
455   /**
456    * Only used by tests. TODO: Remove
457    *
458    * Given the specs of a column, update it, first by inserting a new record,
459    * then removing the old one.  Since there is only 1 KeyValue involved, the memstoreTS
460    * will be set to 0, thus ensuring that they instantly appear to anyone. The underlying
461    * store will ensure that the insert/delete each are atomic. A scanner/reader will either
462    * get the new value, or the old value and all readers will eventually only see the new
463    * value after the old was removed.
464    *
465    * @param row
466    * @param family
467    * @param qualifier
468    * @param newValue
469    * @param now
470    * @return  Timestamp
471    */
472   long updateColumnValue(byte[] row,
473                                 byte[] family,
474                                 byte[] qualifier,
475                                 long newValue,
476                                 long now) {
477     KeyValue firstKv = KeyValue.createFirstOnRow(
478         row, family, qualifier);
479     // Is there a KeyValue in 'snapshot' with the same TS? If so, upgrade the timestamp a bit.
480     SortedSet<KeyValue> snSs = snapshot.tailSet(firstKv);
481     if (!snSs.isEmpty()) {
482       KeyValue snKv = snSs.first();
483       // is there a matching KV in the snapshot?
484       if (snKv.matchingRow(firstKv) && snKv.matchingQualifier(firstKv)) {
485         if (snKv.getTimestamp() == now) {
486           // poop,
487           now += 1;
488         }
489       }
490     }
491 
492     // logic here: the new ts MUST be at least 'now'. But it could be larger if necessary.
493     // But the timestamp should also be max(now, mostRecentTsInMemstore)
494 
495     // so we cant add the new KV w/o knowing what's there already, but we also
496     // want to take this chance to delete some kvs. So two loops (sad)
497 
498     SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
499     for (KeyValue kv : ss) {
500       // if this isnt the row we are interested in, then bail:
501       if (!kv.matchingColumn(family, qualifier) || !kv.matchingRow(firstKv)) {
502         break; // rows dont match, bail.
503       }
504 
505       // if the qualifier matches and it's a put, just RM it out of the kvset.
506       if (kv.getTypeByte() == KeyValue.Type.Put.getCode() &&
507           kv.getTimestamp() > now && firstKv.matchingQualifier(kv)) {
508         now = kv.getTimestamp();
509       }
510     }
511 
512     // create or update (upsert) a new KeyValue 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   public long upsert(Iterable<Cell> cells, long readpoint) {
538     long size = 0;
539     for (Cell cell : cells) {
540       size += upsert(cell, readpoint);
541     }
542     return size;
543   }
544 
545   /**
546    * Inserts the specified KeyValue into MemStore and deletes any existing
547    * versions of the same row/family/qualifier as the specified KeyValue.
548    * <p>
549    * First, the specified KeyValue is inserted into the Memstore.
550    * <p>
551    * If there are any existing KeyValues in this MemStore with the same row,
552    * family, and qualifier, they are removed.
553    * <p>
554    * Callers must hold the read lock.
555    *
556    * @param cell
557    * @return change in size of MemStore
558    */
559   private long upsert(Cell cell, long readpoint) {
560     // Add the KeyValue to the MemStore
561     // Use the internalAdd method here since we (a) already have a lock
562     // and (b) cannot safely use the MSLAB here without potentially
563     // hitting OOME - see TestMemStore.testUpsertMSLAB for a
564     // test that triggers the pathological case if we don't avoid MSLAB
565     // here.
566     KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
567     long addedSize = internalAdd(kv);
568 
569     // Get the KeyValues for the row/family/qualifier regardless of timestamp.
570     // For this case we want to clean up any other puts
571     KeyValue firstKv = KeyValue.createFirstOnRow(
572         kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(),
573         kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
574         kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
575     SortedSet<KeyValue> ss = kvset.tailSet(firstKv);
576     Iterator<KeyValue> it = ss.iterator();
577     // versions visible to oldest scanner
578     int versionsVisible = 0;
579     while ( it.hasNext() ) {
580       KeyValue cur = it.next();
581 
582       if (kv == 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 (kv.matchingRow(cur) && kv.matchingQualifier(cur)) {
588         // only remove Puts that concurrent scanners cannot possibly see
589         if (cur.getTypeByte() == KeyValue.Type.Put.getCode() &&
590             cur.getMvccVersion() <= 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 KeyValue kv;
619     final NavigableSet<KeyValue> set;
620     Member(final NavigableSet<KeyValue> s, final KeyValue kv) {
621       this.kv = 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<KeyValue> set,
635       final GetClosestRowBeforeTracker state, final KeyValue firstOnRow) {
636     NavigableSet<KeyValue> head = set.headSet(firstOnRow, false);
637     if (head.isEmpty()) return null;
638     for (Iterator<KeyValue> i = head.descendingIterator(); i.hasNext();) {
639       KeyValue 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   List<KeyValueScanner> getScanners(long readPt) {
653     return Collections.<KeyValueScanner>singletonList(
654         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   public TimeRangeTracker getSnapshotTimeRangeTracker() {
671     return this.snapshotTimeRangeTracker;
672   }
673 
674   /*
675    * MemStoreScanner implements the KeyValueScanner.
676    * It lets the caller scan the contents of a memstore -- both current
677    * map and snapshot.
678    * This behaves as if it were a real scanner but does not maintain position.
679    */
680   protected class MemStoreScanner extends NonLazyKeyValueScanner {
681     // Next row information for either kvset or snapshot
682     private KeyValue kvsetNextRow = null;
683     private KeyValue snapshotNextRow = null;
684 
685     // last iterated KVs for kvset and snapshot (to restore iterator state after reseek)
686     private KeyValue kvsetItRow = null;
687     private KeyValue snapshotItRow = null;
688     
689     // iterator based scanning.
690     private Iterator<KeyValue> kvsetIt;
691     private Iterator<KeyValue> snapshotIt;
692 
693     // The kvset and snapshot at the time of creating this scanner
694     private KeyValueSkipListSet kvsetAtCreation;
695     private KeyValueSkipListSet snapshotAtCreation;
696 
697     // the pre-calculated KeyValue to be returned by peek() or next()
698     private KeyValue theNext;
699 
700     // The allocator and snapshot allocator at the time of creating this scanner
701     volatile MemStoreLAB allocatorAtCreation;
702     volatile MemStoreLAB snapshotAllocatorAtCreation;
703     
704     // A flag represents whether could stop skipping KeyValues for MVCC
705     // if have encountered the next row. Only used for reversed scan
706     private boolean stopSkippingKVsIfNextRow = false;
707 
708     private long readPoint;
709 
710     /*
711     Some notes...
712 
713      So memstorescanner is fixed at creation time. this includes pointers/iterators into
714     existing kvset/snapshot.  during a snapshot creation, the kvset is null, and the
715     snapshot is moved.  since kvset is null there is no point on reseeking on both,
716       we can save us the trouble. During the snapshot->hfile transition, the memstore
717       scanner is re-created by StoreScanner#updateReaders().  StoreScanner should
718       potentially do something smarter by adjusting the existing memstore scanner.
719 
720       But there is a greater problem here, that being once a scanner has progressed
721       during a snapshot scenario, we currently iterate past the kvset then 'finish' up.
722       if a scan lasts a little while, there is a chance for new entries in kvset to
723       become available but we will never see them.  This needs to be handled at the
724       StoreScanner level with coordination with MemStoreScanner.
725 
726       Currently, this problem is only partly managed: during the small amount of time
727       when the StoreScanner has not yet created a new MemStoreScanner, we will miss
728       the adds to kvset in the MemStoreScanner.
729     */
730 
731     MemStoreScanner(long readPoint) {
732       super();
733 
734       this.readPoint = readPoint;
735       kvsetAtCreation = kvset;
736       snapshotAtCreation = snapshot;
737       if (allocator != null) {
738         this.allocatorAtCreation = allocator;
739         this.allocatorAtCreation.incScannerCount();
740       }
741       if (snapshotAllocator != null) {
742         this.snapshotAllocatorAtCreation = snapshotAllocator;
743         this.snapshotAllocatorAtCreation.incScannerCount();
744       }
745       if (Trace.isTracing() && Trace.currentSpan() != null) {
746         Trace.currentSpan().addTimelineAnnotation("Creating MemStoreScanner");
747       }
748     }
749 
750     private KeyValue getNext(Iterator<KeyValue> it) {
751       KeyValue startKV = theNext;
752       KeyValue v = null;
753       try {
754         while (it.hasNext()) {
755           v = it.next();
756           if (v.getMvccVersion() <= this.readPoint) {
757             return v;
758           }
759           if (stopSkippingKVsIfNextRow && startKV != null
760               && comparator.compareRows(v, startKV) > 0) {
761             return null;
762           }
763         }
764 
765         return null;
766       } finally {
767         if (v != null) {
768           // in all cases, remember the last KV iterated to
769           if (it == snapshotIt) {
770             snapshotItRow = v;
771           } else {
772             kvsetItRow = v;
773           }
774         }
775       }
776     }
777 
778     /**
779      *  Set the scanner at the seek key.
780      *  Must be called only once: there is no thread safety between the scanner
781      *   and the memStore.
782      * @param key seek value
783      * @return false if the key is null or if there is no data
784      */
785     @Override
786     public synchronized boolean seek(KeyValue key) {
787       if (key == null) {
788         close();
789         return false;
790       }
791 
792       // kvset and snapshot will never be null.
793       // if tailSet can't find anything, SortedSet is empty (not null).
794       kvsetIt = kvsetAtCreation.tailSet(key).iterator();
795       snapshotIt = snapshotAtCreation.tailSet(key).iterator();
796       kvsetItRow = 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(KeyValue key){
807       kvsetNextRow = getNext(kvsetIt);
808       snapshotNextRow = getNext(snapshotIt);
809 
810       // Calculate the next value
811       theNext = getLowest(kvsetNextRow, 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(KeyValue 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 
838       kvsetIt = kvsetAtCreation.tailSet(getHighest(key, kvsetItRow)).iterator();
839       snapshotIt = snapshotAtCreation.tailSet(getHighest(key, snapshotItRow)).iterator();
840 
841       return seekInSubLists(key);
842     }
843 
844 
845     @Override
846     public synchronized KeyValue peek() {
847       //DebugPrint.println(" MS@" + hashCode() + " peek = " + getLowest());
848       return theNext;
849     }
850 
851     @Override
852     public synchronized KeyValue next() {
853       if (theNext == null) {
854           return null;
855       }
856 
857       final KeyValue ret = theNext;
858 
859       // Advance one of the iterators
860       if (theNext == kvsetNextRow) {
861         kvsetNextRow = getNext(kvsetIt);
862       } else {
863         snapshotNextRow = getNext(snapshotIt);
864       }
865 
866       // Calculate the next value
867       theNext = getLowest(kvsetNextRow, snapshotNextRow);
868 
869       //long readpoint = ReadWriteConsistencyControl.getThreadReadPoint();
870       //DebugPrint.println(" MS@" + hashCode() + " next: " + theNext + " next_next: " +
871       //    getLowest() + " threadpoint=" + readpoint);
872       return ret;
873     }
874 
875     /*
876      * Returns the lower of the two key values, or null if they are both null.
877      * This uses comparator.compare() to compare the KeyValue using the memstore
878      * comparator.
879      */
880     private KeyValue getLowest(KeyValue first, KeyValue second) {
881       if (first == null && second == null) {
882         return null;
883       }
884       if (first != null && second != null) {
885         int compare = comparator.compare(first, second);
886         return (compare <= 0 ? first : second);
887       }
888       return (first != null ? first : second);
889     }
890 
891     /*
892      * Returns the higher of the two key values, or null if they are both null.
893      * This uses comparator.compare() to compare the KeyValue using the memstore
894      * comparator.
895      */
896     private KeyValue getHighest(KeyValue first, KeyValue second) {
897       if (first == null && second == null) {
898         return null;
899       }
900       if (first != null && second != null) {
901         int compare = comparator.compare(first, second);
902         return (compare > 0 ? first : second);
903       }
904       return (first != null ? first : second);
905     }
906 
907     public synchronized void close() {
908       this.kvsetNextRow = null;
909       this.snapshotNextRow = null;
910 
911       this.kvsetIt = null;
912       this.snapshotIt = null;
913       
914       if (allocatorAtCreation != null) {
915         this.allocatorAtCreation.decScannerCount();
916         this.allocatorAtCreation = null;
917       }
918       if (snapshotAllocatorAtCreation != null) {
919         this.snapshotAllocatorAtCreation.decScannerCount();
920         this.snapshotAllocatorAtCreation = null;
921       }
922 
923       this.kvsetItRow = null;
924       this.snapshotItRow = null;
925     }
926 
927     /**
928      * MemStoreScanner returns max value as sequence id because it will
929      * always have the latest data among all files.
930      */
931     @Override
932     public long getSequenceID() {
933       return Long.MAX_VALUE;
934     }
935 
936     @Override
937     public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
938         long oldestUnexpiredTS) {
939       return shouldSeek(scan, oldestUnexpiredTS);
940     }
941 
942     /**
943      * Seek scanner to the given key first. If it returns false(means
944      * peek()==null) or scanner's peek row is bigger than row of given key, seek
945      * the scanner to the previous row of given key
946      */
947     @Override
948     public synchronized boolean backwardSeek(KeyValue key) {
949       seek(key);
950       if (peek() == null || comparator.compareRows(peek(), key) > 0) {
951         return seekToPreviousRow(key);
952       }
953       return true;
954     }
955 
956     /**
957      * Separately get the KeyValue before the specified key from kvset and
958      * snapshotset, and use the row of higher one as the previous row of
959      * specified key, then seek to the first KeyValue of previous row
960      */
961     @Override
962     public synchronized boolean seekToPreviousRow(KeyValue key) {
963       KeyValue firstKeyOnRow = KeyValue.createFirstOnRow(key.getRow());
964       SortedSet<KeyValue> kvHead = kvsetAtCreation.headSet(firstKeyOnRow);
965       KeyValue kvsetBeforeRow = kvHead.isEmpty() ? null : kvHead.last();
966       SortedSet<KeyValue> snapshotHead = snapshotAtCreation
967           .headSet(firstKeyOnRow);
968       KeyValue snapshotBeforeRow = snapshotHead.isEmpty() ? null : snapshotHead
969           .last();
970       KeyValue lastKVBeforeRow = getHighest(kvsetBeforeRow, snapshotBeforeRow);
971       if (lastKVBeforeRow == null) {
972         theNext = null;
973         return false;
974       }
975       KeyValue firstKeyOnPreviousRow = KeyValue
976           .createFirstOnRow(lastKVBeforeRow.getRow());
977       this.stopSkippingKVsIfNextRow = true;
978       seek(firstKeyOnPreviousRow);
979       this.stopSkippingKVsIfNextRow = false;
980       if (peek() == null
981           || comparator.compareRows(peek(), firstKeyOnPreviousRow) > 0) {
982         return seekToPreviousRow(lastKVBeforeRow);
983       }
984       return true;
985     }
986 
987     @Override
988     public synchronized boolean seekToLastRow() {
989       KeyValue first = kvsetAtCreation.isEmpty() ? null : kvsetAtCreation
990           .last();
991       KeyValue second = snapshotAtCreation.isEmpty() ? null
992           : snapshotAtCreation.last();
993       KeyValue higherKv = getHighest(first, second);
994       if (higherKv == null) {
995         return false;
996       }
997       KeyValue firstKvOnLastRow = KeyValue.createFirstOnRow(higherKv.getRow());
998       if (seek(firstKvOnLastRow)) {
999         return true;
1000       } else {
1001         return seekToPreviousRow(higherKv);
1002       }
1003 
1004     }
1005   }
1006 
1007   public final static long FIXED_OVERHEAD = ClassSize.align(
1008       ClassSize.OBJECT + (10 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG));
1009 
1010   public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
1011       ClassSize.ATOMIC_LONG + (2 * ClassSize.TIMERANGE_TRACKER) +
1012       (2 * ClassSize.KEYVALUE_SKIPLIST_SET) + (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
1013 
1014   /*
1015    * Calculate how the MemStore size has changed.  Includes overhead of the
1016    * backing Map.
1017    * @param kv
1018    * @param notpresent True if the kv was NOT present in the set.
1019    * @return Size
1020    */
1021   static long heapSizeChange(final KeyValue kv, final boolean notpresent) {
1022     return notpresent ?
1023         ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize()):
1024         0;
1025   }
1026 
1027   /**
1028    * Get the entire heap usage for this MemStore not including keys in the
1029    * snapshot.
1030    */
1031   @Override
1032   public long heapSize() {
1033     return size.get();
1034   }
1035 
1036   /**
1037    * Get the heap usage of KVs in this MemStore.
1038    */
1039   public long keySize() {
1040     return heapSize() - DEEP_OVERHEAD;
1041   }
1042 
1043   /**
1044    * Code to help figure if our approximation of object heap sizes is close
1045    * enough.  See hbase-900.  Fills memstores then waits so user can heap
1046    * dump and bring up resultant hprof in something like jprofiler which
1047    * allows you get 'deep size' on objects.
1048    * @param args main args
1049    */
1050   public static void main(String [] args) {
1051     RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
1052     LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
1053       runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
1054     LOG.info("vmInputArguments=" + runtime.getInputArguments());
1055     MemStore memstore1 = new MemStore();
1056     // TODO: x32 vs x64
1057     long size = 0;
1058     final int count = 10000;
1059     byte [] fam = Bytes.toBytes("col");
1060     byte [] qf = Bytes.toBytes("umn");
1061     byte [] empty = new byte[0];
1062     for (int i = 0; i < count; i++) {
1063       // Give each its own ts
1064       size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
1065     }
1066     LOG.info("memstore1 estimated size=" + size);
1067     for (int i = 0; i < count; i++) {
1068       size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty));
1069     }
1070     LOG.info("memstore1 estimated size (2nd loading of same data)=" + size);
1071     // Make a variably sized memstore.
1072     MemStore memstore2 = new MemStore();
1073     for (int i = 0; i < count; i++) {
1074       size += memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i,
1075         new byte[i]));
1076     }
1077     LOG.info("memstore2 estimated size=" + size);
1078     final int seconds = 30;
1079     LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
1080     for (int i = 0; i < seconds; i++) {
1081       // Thread.sleep(1000);
1082     }
1083     LOG.info("Exiting.");
1084   }
1085 }