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
11 * http://www.apache.org/licenses/LICENSE-2.0
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.
20 package org
.apache
.hadoop
.hbase
.regionserver
;
22 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.annotations
.VisibleForTesting
;
24 import java
.io
.IOException
;
25 import java
.util
.ArrayList
;
26 import java
.util
.Comparator
;
27 import java
.util
.List
;
28 import java
.util
.PriorityQueue
;
30 import org
.apache
.hadoop
.hbase
.Cell
;
31 import org
.apache
.hadoop
.hbase
.CellComparator
;
32 import org
.apache
.yetus
.audience
.InterfaceAudience
;
33 import org
.slf4j
.Logger
;
34 import org
.slf4j
.LoggerFactory
;
35 import org
.apache
.hadoop
.hbase
.regionserver
.ScannerContext
.NextState
;
38 * Implements a heap merge across any number of KeyValueScanners.
40 * Implements KeyValueScanner itself.
42 * This class is used at the Region level to merge across Stores
43 * and at the Store level to merge across the memstore and StoreFiles.
45 * In the Region case, we also need InternalScanner.next(List), so this class
46 * also implements InternalScanner. WARNING: As is, if you try to use this
47 * as an InternalScanner at the Store level, you will get runtime exceptions.
49 @InterfaceAudience.Private
50 public class KeyValueHeap
extends NonReversedNonLazyKeyValueScanner
51 implements KeyValueScanner
, InternalScanner
{
52 private static final Logger LOG
= LoggerFactory
.getLogger(KeyValueHeap
.class);
53 protected PriorityQueue
<KeyValueScanner
> heap
= null;
54 // Holds the scanners when a ever a eager close() happens. All such eagerly closed
55 // scans are collected and when the final scanner.close() happens will perform the
57 protected List
<KeyValueScanner
> scannersForDelayedClose
= null;
60 * The current sub-scanner, i.e. the one that contains the next key/value
61 * to return to the client. This scanner is NOT included in {@link #heap}
62 * (but we frequently add it back to the heap and pull the new winner out).
63 * We maintain an invariant that the current sub-scanner has already done
64 * a real seek, and that current.peek() is always a real key/value (or null)
65 * except for the fake last-key-on-row-column supplied by the multi-column
66 * Bloom filter optimization, which is OK to propagate to StoreScanner. In
67 * order to ensure that, always use {@link #pollRealKV()} to update current.
69 protected KeyValueScanner current
= null;
71 protected KVScannerComparator comparator
;
74 * Constructor. This KeyValueHeap will handle closing of passed in
79 public KeyValueHeap(List
<?
extends KeyValueScanner
> scanners
,
80 CellComparator comparator
) throws IOException
{
81 this(scanners
, new KVScannerComparator(comparator
));
90 KeyValueHeap(List
<?
extends KeyValueScanner
> scanners
,
91 KVScannerComparator comparator
) throws IOException
{
92 this.comparator
= comparator
;
93 this.scannersForDelayedClose
= new ArrayList
<>(scanners
.size());
94 if (!scanners
.isEmpty()) {
95 this.heap
= new PriorityQueue
<>(scanners
.size(), this.comparator
);
96 for (KeyValueScanner scanner
: scanners
) {
97 if (scanner
.peek() != null) {
98 this.heap
.add(scanner
);
100 this.scannersForDelayedClose
.add(scanner
);
103 this.current
= pollRealKV();
109 if (this.current
== null) {
112 return this.current
.peek();
116 public Cell
next() throws IOException
{
117 if(this.current
== null) {
120 Cell kvReturn
= this.current
.next();
121 Cell kvNext
= this.current
.peek();
122 if (kvNext
== null) {
123 this.scannersForDelayedClose
.add(this.current
);
125 this.current
= pollRealKV();
127 KeyValueScanner topScanner
= this.heap
.peek();
128 // no need to add current back to the heap if it is the only scanner left
129 if (topScanner
!= null && this.comparator
.compare(kvNext
, topScanner
.peek()) >= 0) {
130 this.heap
.add(this.current
);
132 this.current
= pollRealKV();
139 * Gets the next row of keys from the top-most scanner.
141 * This method takes care of updating the heap.
143 * This can ONLY be called when you are using Scanners that implement InternalScanner as well as
144 * KeyValueScanner (a {@link StoreScanner}).
145 * @return true if more rows exist after this one, false if scanner is done
148 public boolean next(List
<Cell
> result
, ScannerContext scannerContext
) throws IOException
{
149 if (this.current
== null) {
150 return scannerContext
.setScannerState(NextState
.NO_MORE_VALUES
).hasMoreValues();
152 InternalScanner currentAsInternal
= (InternalScanner
)this.current
;
153 boolean moreCells
= currentAsInternal
.next(result
, scannerContext
);
154 Cell pee
= this.current
.peek();
157 * By definition, any InternalScanner must return false only when it has no
158 * further rows to be fetched. So, we can close a scanner if it returns
159 * false. All existing implementations seem to be fine with this. It is much
160 * more efficient to close scanners which are not needed than keep them in
161 * the heap. This is also required for certain optimizations.
164 if (pee
== null || !moreCells
) {
165 // add the scanner that is to be closed
166 this.scannersForDelayedClose
.add(this.current
);
168 this.heap
.add(this.current
);
171 this.current
= pollRealKV();
172 if (this.current
== null) {
173 moreCells
= scannerContext
.setScannerState(NextState
.NO_MORE_VALUES
).hasMoreValues();
178 protected static class KVScannerComparator
implements Comparator
<KeyValueScanner
> {
179 protected CellComparator kvComparator
;
182 * @param kvComparator
184 public KVScannerComparator(CellComparator kvComparator
) {
185 this.kvComparator
= kvComparator
;
189 public int compare(KeyValueScanner left
, KeyValueScanner right
) {
190 int comparison
= compare(left
.peek(), right
.peek());
191 if (comparison
!= 0) {
194 // Since both the keys are exactly the same, we break the tie in favor of higher ordered
195 // scanner since it'll have newer data. Since higher value should come first, we reverse
197 return Long
.compare(right
.getScannerOrder(), left
.getScannerOrder());
201 * Compares two KeyValue
204 * @return less than 0 if left is smaller, 0 if equal etc..
206 public int compare(Cell left
, Cell right
) {
207 return this.kvComparator
.compare(left
, right
);
210 * @return KVComparator
212 public CellComparator
getComparator() {
213 return this.kvComparator
;
218 public void close() {
219 for (KeyValueScanner scanner
: this.scannersForDelayedClose
) {
222 this.scannersForDelayedClose
.clear();
223 if (this.current
!= null) {
224 this.current
.close();
226 if (this.heap
!= null) {
227 // Order of closing the scanners shouldn't matter here, so simply iterate and close them.
228 for (KeyValueScanner scanner
: heap
) {
235 * Seeks all scanners at or below the specified seek key. If we earlied-out
236 * of a row, we may end up skipping values that were never reached yet.
237 * Rather than iterating down, we want to give the opportunity to re-seek.
239 * As individual scanners may run past their ends, those scanners are
240 * automatically closed and removed from the heap.
242 * This function (and {@link #reseek(Cell)}) does not do multi-column
243 * Bloom filter and lazy-seek optimizations. To enable those, call
244 * {@link #requestSeek(Cell, boolean, boolean)}.
245 * @param seekKey KeyValue to seek at or after
246 * @return true if KeyValues exist at or after specified key, false if not
247 * @throws IOException
250 public boolean seek(Cell seekKey
) throws IOException
{
251 return generalizedSeek(false, // This is not a lazy seek
253 false, // forward (false: this is not a reseek)
254 false); // Not using Bloom filters
258 * This function is identical to the {@link #seek(Cell)} function except
259 * that scanner.seek(seekKey) is changed to scanner.reseek(seekKey).
262 public boolean reseek(Cell seekKey
) throws IOException
{
263 return generalizedSeek(false, // This is not a lazy seek
265 true, // forward (true because this is reseek)
266 false); // Not using Bloom filters
273 public boolean requestSeek(Cell key
, boolean forward
,
274 boolean useBloom
) throws IOException
{
275 return generalizedSeek(true, key
, forward
, useBloom
);
279 * @param isLazy whether we are trying to seek to exactly the given row/col.
280 * Enables Bloom filter and most-recent-file-first optimizations for
281 * multi-column get/scan queries.
282 * @param seekKey key to seek to
283 * @param forward whether to seek forward (also known as reseek)
284 * @param useBloom whether to optimize seeks using Bloom filters
286 private boolean generalizedSeek(boolean isLazy
, Cell seekKey
,
287 boolean forward
, boolean useBloom
) throws IOException
{
288 if (!isLazy
&& useBloom
) {
289 throw new IllegalArgumentException("Multi-column Bloom filter " +
290 "optimization requires a lazy seek");
293 if (current
== null) {
297 KeyValueScanner scanner
= current
;
299 while (scanner
!= null) {
300 Cell topKey
= scanner
.peek();
301 if (comparator
.getComparator().compare(seekKey
, topKey
) <= 0) {
302 // Top KeyValue is at-or-after Seek KeyValue. We only know that all
303 // scanners are at or after seekKey (because fake keys of
304 // scanners where a lazy-seek operation has been done are not greater
305 // than their real next keys) but we still need to enforce our
306 // invariant that the top scanner has done a real seek. This way
307 // StoreScanner and RegionScanner do not have to worry about fake
311 current
= pollRealKV();
312 return current
!= null;
316 if (isLazy
&& heap
.size() > 0) {
317 // If there is only one scanner left, we don't do lazy seek.
318 seekResult
= scanner
.requestSeek(seekKey
, forward
, useBloom
);
320 seekResult
= NonLazyKeyValueScanner
.doRealSeek(scanner
, seekKey
,
325 this.scannersForDelayedClose
.add(scanner
);
329 scanner
= heap
.poll();
330 if (scanner
== null) {
334 } catch (Exception e
) {
335 if (scanner
!= null) {
338 } catch (Exception ce
) {
339 LOG
.warn("close KeyValueScanner error", ce
);
345 // Heap is returning empty, scanner is done
350 * Fetches the top sub-scanner from the priority queue, ensuring that a real
351 * seek has been done on it. Works by fetching the top sub-scanner, and if it
352 * has not done a real seek, making it do so (which will modify its top KV),
353 * putting it back, and repeating this until success. Relies on the fact that
354 * on a lazy seek we set the current key of a StoreFileScanner to a KV that
355 * is not greater than the real next KV to be read from that file, so the
356 * scanner that bubbles up to the top of the heap will have global next KV in
357 * this scanner heap if (1) it has done a real seek and (2) its KV is the top
358 * among all top KVs (some of which are fake) in the scanner heap.
360 protected KeyValueScanner
pollRealKV() throws IOException
{
361 KeyValueScanner kvScanner
= heap
.poll();
362 if (kvScanner
== null) {
366 while (kvScanner
!= null && !kvScanner
.realSeekDone()) {
367 if (kvScanner
.peek() != null) {
369 kvScanner
.enforceSeek();
370 } catch (IOException ioe
) {
371 // Add the item to delayed close set in case it is leak from close
372 this.scannersForDelayedClose
.add(kvScanner
);
375 Cell curKV
= kvScanner
.peek();
377 KeyValueScanner nextEarliestScanner
= heap
.peek();
378 if (nextEarliestScanner
== null) {
379 // The heap is empty. Return the only possible scanner.
383 // Compare the current scanner to the next scanner. We try to avoid
384 // putting the current one back into the heap if possible.
385 Cell nextKV
= nextEarliestScanner
.peek();
386 if (nextKV
== null || comparator
.compare(curKV
, nextKV
) < 0) {
387 // We already have the scanner with the earliest KV, so return it.
391 // Otherwise, put the scanner back into the heap and let it compete
392 // against all other scanners (both those that have done a "real
393 // seek" and a "lazy seek").
396 // Close the scanner because we did a real seek and found out there
398 this.scannersForDelayedClose
.add(kvScanner
);
401 // Close the scanner because it has already run out of KVs even before
402 // we had to do a real seek on it.
403 this.scannersForDelayedClose
.add(kvScanner
);
405 kvScanner
= heap
.poll();
412 * @return the current Heap
414 public PriorityQueue
<KeyValueScanner
> getHeap() {
420 KeyValueScanner
getCurrentForTesting() {
425 public Cell
getNextIndexedKey() {
426 // here we return the next index key from the top scanner
427 return current
== null ?
null : current
.getNextIndexedKey();
431 public void shipped() throws IOException
{
432 for (KeyValueScanner scanner
: this.scannersForDelayedClose
) {
433 scanner
.close(); // There wont be further fetch of Cells from these scanners. Just close.
435 this.scannersForDelayedClose
.clear();
436 if (this.current
!= null) {
437 this.current
.shipped();
439 if (this.heap
!= null) {
440 for (KeyValueScanner scanner
: this.heap
) {