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