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