View Javadoc

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