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