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.io.IOException;
23  import java.util.Comparator;
24  import java.util.List;
25  import java.util.PriorityQueue;
26  
27  import org.apache.hadoop.hbase.Cell;
28  import org.apache.hadoop.hbase.KeyValue.KVComparator;
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
31  
32  /**
33   * Implements a heap merge across any number of KeyValueScanners.
34   * <p>
35   * Implements KeyValueScanner itself.
36   * <p>
37   * This class is used at the Region level to merge across Stores
38   * and at the Store level to merge across the memstore and StoreFiles.
39   * <p>
40   * In the Region case, we also need InternalScanner.next(List), so this class
41   * also implements InternalScanner.  WARNING: As is, if you try to use this
42   * as an InternalScanner at the Store level, you will get runtime exceptions.
43   */
44  @InterfaceAudience.Private
45  public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
46      implements KeyValueScanner, InternalScanner {
47    protected PriorityQueue<KeyValueScanner> heap = null;
48  
49    /**
50     * The current sub-scanner, i.e. the one that contains the next key/value
51     * to return to the client. This scanner is NOT included in {@link #heap}
52     * (but we frequently add it back to the heap and pull the new winner out).
53     * We maintain an invariant that the current sub-scanner has already done
54     * a real seek, and that current.peek() is always a real key/value (or null)
55     * except for the fake last-key-on-row-column supplied by the multi-column
56     * Bloom filter optimization, which is OK to propagate to StoreScanner. In
57     * order to ensure that, always use {@link #pollRealKV()} to update current.
58     */
59    protected KeyValueScanner current = null;
60  
61    protected KVScannerComparator comparator;
62    
63    /**
64     * Constructor.  This KeyValueHeap will handle closing of passed in
65     * KeyValueScanners.
66     * @param scanners
67     * @param comparator
68     */
69    public KeyValueHeap(List<? extends KeyValueScanner> scanners,
70        KVComparator comparator) throws IOException {
71      this(scanners, new KVScannerComparator(comparator));
72    }
73  
74    /**
75     * Constructor.
76     * @param scanners
77     * @param comparator
78     * @throws IOException
79     */
80    KeyValueHeap(List<? extends KeyValueScanner> scanners,
81        KVScannerComparator comparator) throws IOException {
82      this.comparator = comparator;
83      if (!scanners.isEmpty()) {
84        this.heap = new PriorityQueue<KeyValueScanner>(scanners.size(),
85            this.comparator);
86        for (KeyValueScanner scanner : scanners) {
87          if (scanner.peek() != null) {
88            this.heap.add(scanner);
89          } else {
90            scanner.close();
91          }
92        }
93        this.current = pollRealKV();
94      }
95    }
96  
97    public Cell peek() {
98      if (this.current == null) {
99        return null;
100     }
101     return this.current.peek();
102   }
103 
104   public Cell next()  throws IOException {
105     if(this.current == null) {
106       return null;
107     }
108     Cell kvReturn = this.current.next();
109     Cell kvNext = this.current.peek();
110     if (kvNext == null) {
111       this.current.close();
112       this.current = null;
113       this.current = pollRealKV();
114     } else {
115       KeyValueScanner topScanner = this.heap.peek();
116       // no need to add current back to the heap if it is the only scanner left
117       if (topScanner != null && this.comparator.compare(kvNext, topScanner.peek()) >= 0) {
118         this.heap.add(this.current);
119         this.current = null;
120         this.current = pollRealKV();
121       }
122     }
123     return kvReturn;
124   }
125 
126   /**
127    * Gets the next row of keys from the top-most scanner.
128    * <p>
129    * This method takes care of updating the heap.
130    * <p>
131    * This can ONLY be called when you are using Scanners that implement InternalScanner as well as
132    * KeyValueScanner (a {@link StoreScanner}).
133    * @param result
134    * @return true if more rows exist after this one, false if scanner is done
135    */
136   @Override
137   public boolean next(List<Cell> result) throws IOException {
138     return next(result, NoLimitScannerContext.getInstance());
139   }
140 
141   @Override
142   public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
143     if (this.current == null) {
144       return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
145     }
146     InternalScanner currentAsInternal = (InternalScanner)this.current;
147     boolean moreCells = currentAsInternal.next(result, scannerContext);
148     Cell pee = this.current.peek();
149 
150     /*
151      * By definition, any InternalScanner must return false only when it has no
152      * further rows to be fetched. So, we can close a scanner if it returns
153      * false. All existing implementations seem to be fine with this. It is much
154      * more efficient to close scanners which are not needed than keep them in
155      * the heap. This is also required for certain optimizations.
156      */
157 
158     if (pee == null || !moreCells) {
159       this.current.close();
160     } else {
161       this.heap.add(this.current);
162     }
163     this.current = null;
164     this.current = pollRealKV();
165     if (this.current == null) {
166       moreCells = scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
167     }
168     return moreCells;
169   }
170 
171   protected static class KVScannerComparator implements Comparator<KeyValueScanner> {
172     protected KVComparator kvComparator;
173     /**
174      * Constructor
175      * @param kvComparator
176      */
177     public KVScannerComparator(KVComparator kvComparator) {
178       this.kvComparator = kvComparator;
179     }
180     public int compare(KeyValueScanner left, KeyValueScanner right) {
181       int comparison = compare(left.peek(), right.peek());
182       if (comparison != 0) {
183         return comparison;
184       } else {
185         // Since both the keys are exactly the same, we break the tie in favor
186         // of the key which came latest.
187         long leftSequenceID = left.getSequenceID();
188         long rightSequenceID = right.getSequenceID();
189         if (leftSequenceID > rightSequenceID) {
190           return -1;
191         } else if (leftSequenceID < rightSequenceID) {
192           return 1;
193         } else {
194           return 0;
195         }
196       }
197     }
198     /**
199      * Compares two KeyValue
200      * @param left
201      * @param right
202      * @return less than 0 if left is smaller, 0 if equal etc..
203      */
204     public int compare(Cell left, Cell right) {
205       return this.kvComparator.compare(left, right);
206     }
207     /**
208      * @return KVComparator
209      */
210     public KVComparator getComparator() {
211       return this.kvComparator;
212     }
213   }
214 
215   public void close() {
216     if (this.current != null) {
217       this.current.close();
218     }
219     if (this.heap != null) {
220       KeyValueScanner scanner;
221       while ((scanner = this.heap.poll()) != null) {
222         scanner.close();
223       }
224     }
225   }
226 
227   /**
228    * Seeks all scanners at or below the specified seek key.  If we earlied-out
229    * of a row, we may end up skipping values that were never reached yet.
230    * Rather than iterating down, we want to give the opportunity to re-seek.
231    * <p>
232    * As individual scanners may run past their ends, those scanners are
233    * automatically closed and removed from the heap.
234    * <p>
235    * This function (and {@link #reseek(Cell)}) does not do multi-column
236    * Bloom filter and lazy-seek optimizations. To enable those, call
237    * {@link #requestSeek(Cell, boolean, boolean)}.
238    * @param seekKey KeyValue to seek at or after
239    * @return true if KeyValues exist at or after specified key, false if not
240    * @throws IOException
241    */
242   @Override
243   public boolean seek(Cell seekKey) throws IOException {
244     return generalizedSeek(false,    // This is not a lazy seek
245                            seekKey,
246                            false,    // forward (false: this is not a reseek)
247                            false);   // Not using Bloom filters
248   }
249 
250   /**
251    * This function is identical to the {@link #seek(Cell)} function except
252    * that scanner.seek(seekKey) is changed to scanner.reseek(seekKey).
253    */
254   @Override
255   public boolean reseek(Cell seekKey) throws IOException {
256     return generalizedSeek(false,    // This is not a lazy seek
257                            seekKey,
258                            true,     // forward (true because this is reseek)
259                            false);   // Not using Bloom filters
260   }
261 
262   /**
263    * {@inheritDoc}
264    */
265   @Override
266   public boolean requestSeek(Cell key, boolean forward,
267       boolean useBloom) throws IOException {
268     return generalizedSeek(true, key, forward, useBloom);
269   }
270 
271   /**
272    * @param isLazy whether we are trying to seek to exactly the given row/col.
273    *          Enables Bloom filter and most-recent-file-first optimizations for
274    *          multi-column get/scan queries.
275    * @param seekKey key to seek to
276    * @param forward whether to seek forward (also known as reseek)
277    * @param useBloom whether to optimize seeks using Bloom filters
278    */
279   private boolean generalizedSeek(boolean isLazy, Cell seekKey,
280       boolean forward, boolean useBloom) throws IOException {
281     if (!isLazy && useBloom) {
282       throw new IllegalArgumentException("Multi-column Bloom filter " +
283           "optimization requires a lazy seek");
284     }
285 
286     if (current == null) {
287       return false;
288     }
289     heap.add(current);
290     current = null;
291 
292     KeyValueScanner scanner;
293     while ((scanner = heap.poll()) != null) {
294       Cell topKey = scanner.peek();
295       if (comparator.getComparator().compare(seekKey, topKey) <= 0) {
296         // Top KeyValue is at-or-after Seek KeyValue. We only know that all
297         // scanners are at or after seekKey (because fake keys of
298         // scanners where a lazy-seek operation has been done are not greater
299         // than their real next keys) but we still need to enforce our
300         // invariant that the top scanner has done a real seek. This way
301         // StoreScanner and RegionScanner do not have to worry about fake keys.
302         heap.add(scanner);
303         current = pollRealKV();
304         return current != null;
305       }
306 
307       boolean seekResult;
308       if (isLazy && heap.size() > 0) {
309         // If there is only one scanner left, we don't do lazy seek.
310         seekResult = scanner.requestSeek(seekKey, forward, useBloom);
311       } else {
312         seekResult = NonLazyKeyValueScanner.doRealSeek(
313             scanner, seekKey, forward);
314       }
315 
316       if (!seekResult) {
317         scanner.close();
318       } else {
319         heap.add(scanner);
320       }
321     }
322 
323     // Heap is returning empty, scanner is done
324     return false;
325   }
326 
327   /**
328    * Fetches the top sub-scanner from the priority queue, ensuring that a real
329    * seek has been done on it. Works by fetching the top sub-scanner, and if it
330    * has not done a real seek, making it do so (which will modify its top KV),
331    * putting it back, and repeating this until success. Relies on the fact that
332    * on a lazy seek we set the current key of a StoreFileScanner to a KV that
333    * is not greater than the real next KV to be read from that file, so the
334    * scanner that bubbles up to the top of the heap will have global next KV in
335    * this scanner heap if (1) it has done a real seek and (2) its KV is the top
336    * among all top KVs (some of which are fake) in the scanner heap.
337    */
338   protected KeyValueScanner pollRealKV() throws IOException {
339     KeyValueScanner kvScanner = heap.poll();
340     if (kvScanner == null) {
341       return null;
342     }
343 
344     while (kvScanner != null && !kvScanner.realSeekDone()) {
345       if (kvScanner.peek() != null) {
346         try {
347           kvScanner.enforceSeek();
348         } catch (IOException ioe) {
349           kvScanner.close();
350           throw ioe;
351         }
352         Cell curKV = kvScanner.peek();
353         if (curKV != null) {
354           KeyValueScanner nextEarliestScanner = heap.peek();
355           if (nextEarliestScanner == null) {
356             // The heap is empty. Return the only possible scanner.
357             return kvScanner;
358           }
359 
360           // Compare the current scanner to the next scanner. We try to avoid
361           // putting the current one back into the heap if possible.
362           Cell nextKV = nextEarliestScanner.peek();
363           if (nextKV == null || comparator.compare(curKV, nextKV) < 0) {
364             // We already have the scanner with the earliest KV, so return it.
365             return kvScanner;
366           }
367 
368           // Otherwise, put the scanner back into the heap and let it compete
369           // against all other scanners (both those that have done a "real
370           // seek" and a "lazy seek").
371           heap.add(kvScanner);
372         } else {
373           // Close the scanner because we did a real seek and found out there
374           // are no more KVs.
375           kvScanner.close();
376         }
377       } else {
378         // Close the scanner because it has already run out of KVs even before
379         // we had to do a real seek on it.
380         kvScanner.close();
381       }
382       kvScanner = heap.poll();
383     }
384 
385     return kvScanner;
386   }
387 
388   /**
389    * @return the current Heap
390    */
391   public PriorityQueue<KeyValueScanner> getHeap() {
392     return this.heap;
393   }
394 
395   @Override
396   public long getSequenceID() {
397     return 0;
398   }
399 
400   KeyValueScanner getCurrentForTesting() {
401     return current;
402   }
403 
404   @Override
405   public Cell getNextIndexedKey() {
406     // here we return the next index key from the top scanner
407     return current == null ? null : current.getNextIndexedKey();
408   }
409 }