HBASE-17532 Replaced explicit type with diamond operator
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / KeyValueHeap.java
blob195e8f7491d024a19d873d88857fe4e51d5d59c3
1 /**
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 java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.Comparator;
25 import java.util.List;
26 import java.util.PriorityQueue;
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.hbase.Cell;
31 import org.apache.hadoop.hbase.CellComparator;
32 import org.apache.hadoop.hbase.classification.InterfaceAudience;
33 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
35 /**
36 * Implements a heap merge across any number of KeyValueScanners.
37 * <p>
38 * Implements KeyValueScanner itself.
39 * <p>
40 * This class is used at the Region level to merge across Stores
41 * and at the Store level to merge across the memstore and StoreFiles.
42 * <p>
43 * In the Region case, we also need InternalScanner.next(List), so this class
44 * also implements InternalScanner. WARNING: As is, if you try to use this
45 * as an InternalScanner at the Store level, you will get runtime exceptions.
47 @InterfaceAudience.Private
48 public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
49 implements KeyValueScanner, InternalScanner {
50 private static final Log LOG = LogFactory.getLog(KeyValueHeap.class);
51 protected PriorityQueue<KeyValueScanner> heap = null;
52 // Holds the scanners when a ever a eager close() happens. All such eagerly closed
53 // scans are collected and when the final scanner.close() happens will perform the
54 // actual close.
55 protected List<KeyValueScanner> scannersForDelayedClose = null;
57 /**
58 * The current sub-scanner, i.e. the one that contains the next key/value
59 * to return to the client. This scanner is NOT included in {@link #heap}
60 * (but we frequently add it back to the heap and pull the new winner out).
61 * We maintain an invariant that the current sub-scanner has already done
62 * a real seek, and that current.peek() is always a real key/value (or null)
63 * except for the fake last-key-on-row-column supplied by the multi-column
64 * Bloom filter optimization, which is OK to propagate to StoreScanner. In
65 * order to ensure that, always use {@link #pollRealKV()} to update current.
67 protected KeyValueScanner current = null;
69 protected KVScannerComparator comparator;
71 /**
72 * Constructor. This KeyValueHeap will handle closing of passed in
73 * KeyValueScanners.
74 * @param scanners
75 * @param comparator
77 public KeyValueHeap(List<? extends KeyValueScanner> scanners,
78 CellComparator comparator) throws IOException {
79 this(scanners, new KVScannerComparator(comparator));
82 /**
83 * Constructor.
84 * @param scanners
85 * @param comparator
86 * @throws IOException
88 KeyValueHeap(List<? extends KeyValueScanner> scanners,
89 KVScannerComparator comparator) throws IOException {
90 this.comparator = comparator;
91 this.scannersForDelayedClose = new ArrayList<>(scanners.size());
92 if (!scanners.isEmpty()) {
93 this.heap = new PriorityQueue<>(scanners.size(), this.comparator);
94 for (KeyValueScanner scanner : scanners) {
95 if (scanner.peek() != null) {
96 this.heap.add(scanner);
97 } else {
98 this.scannersForDelayedClose.add(scanner);
101 this.current = pollRealKV();
105 public Cell peek() {
106 if (this.current == null) {
107 return null;
109 return this.current.peek();
112 public Cell next() throws IOException {
113 if(this.current == null) {
114 return null;
116 Cell kvReturn = this.current.next();
117 Cell kvNext = this.current.peek();
118 if (kvNext == null) {
119 this.scannersForDelayedClose.add(this.current);
120 this.current = null;
121 this.current = pollRealKV();
122 } else {
123 KeyValueScanner topScanner = this.heap.peek();
124 // no need to add current back to the heap if it is the only scanner left
125 if (topScanner != null && this.comparator.compare(kvNext, topScanner.peek()) >= 0) {
126 this.heap.add(this.current);
127 this.current = null;
128 this.current = pollRealKV();
131 return kvReturn;
135 * Gets the next row of keys from the top-most scanner.
136 * <p>
137 * This method takes care of updating the heap.
138 * <p>
139 * This can ONLY be called when you are using Scanners that implement InternalScanner as well as
140 * KeyValueScanner (a {@link StoreScanner}).
141 * @param result
142 * @return true if more rows exist after this one, false if scanner is done
144 @Override
145 public boolean next(List<Cell> result) throws IOException {
146 return next(result, NoLimitScannerContext.getInstance());
149 @Override
150 public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
151 if (this.current == null) {
152 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
154 InternalScanner currentAsInternal = (InternalScanner)this.current;
155 boolean moreCells = currentAsInternal.next(result, scannerContext);
156 Cell pee = this.current.peek();
159 * By definition, any InternalScanner must return false only when it has no
160 * further rows to be fetched. So, we can close a scanner if it returns
161 * false. All existing implementations seem to be fine with this. It is much
162 * more efficient to close scanners which are not needed than keep them in
163 * the heap. This is also required for certain optimizations.
166 if (pee == null || !moreCells) {
167 // add the scanner that is to be closed
168 this.scannersForDelayedClose.add(this.current);
169 } else {
170 this.heap.add(this.current);
172 this.current = null;
173 this.current = pollRealKV();
174 if (this.current == null) {
175 moreCells = scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
177 return moreCells;
180 protected static class KVScannerComparator implements Comparator<KeyValueScanner> {
181 protected CellComparator kvComparator;
183 * Constructor
184 * @param kvComparator
186 public KVScannerComparator(CellComparator kvComparator) {
187 this.kvComparator = kvComparator;
189 public int compare(KeyValueScanner left, KeyValueScanner right) {
190 int comparison = compare(left.peek(), right.peek());
191 if (comparison != 0) {
192 return comparison;
193 } else {
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
196 // sort here.
197 return Long.compare(right.getScannerOrder(), left.getScannerOrder());
201 * Compares two KeyValue
202 * @param left
203 * @param right
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;
217 public void close() {
218 for (KeyValueScanner scanner : this.scannersForDelayedClose) {
219 scanner.close();
221 this.scannersForDelayedClose.clear();
222 if (this.current != null) {
223 this.current.close();
225 if (this.heap != null) {
226 KeyValueScanner scanner;
227 while ((scanner = this.heap.poll()) != null) {
228 scanner.close();
234 * Seeks all scanners at or below the specified seek key. If we earlied-out
235 * of a row, we may end up skipping values that were never reached yet.
236 * Rather than iterating down, we want to give the opportunity to re-seek.
237 * <p>
238 * As individual scanners may run past their ends, those scanners are
239 * automatically closed and removed from the heap.
240 * <p>
241 * This function (and {@link #reseek(Cell)}) does not do multi-column
242 * Bloom filter and lazy-seek optimizations. To enable those, call
243 * {@link #requestSeek(Cell, boolean, boolean)}.
244 * @param seekKey KeyValue to seek at or after
245 * @return true if KeyValues exist at or after specified key, false if not
246 * @throws IOException
248 @Override
249 public boolean seek(Cell seekKey) throws IOException {
250 return generalizedSeek(false, // This is not a lazy seek
251 seekKey,
252 false, // forward (false: this is not a reseek)
253 false); // Not using Bloom filters
257 * This function is identical to the {@link #seek(Cell)} function except
258 * that scanner.seek(seekKey) is changed to scanner.reseek(seekKey).
260 @Override
261 public boolean reseek(Cell seekKey) throws IOException {
262 return generalizedSeek(false, // This is not a lazy seek
263 seekKey,
264 true, // forward (true because this is reseek)
265 false); // Not using Bloom filters
269 * {@inheritDoc}
271 @Override
272 public boolean requestSeek(Cell key, boolean forward,
273 boolean useBloom) throws IOException {
274 return generalizedSeek(true, key, forward, useBloom);
278 * @param isLazy whether we are trying to seek to exactly the given row/col.
279 * Enables Bloom filter and most-recent-file-first optimizations for
280 * multi-column get/scan queries.
281 * @param seekKey key to seek to
282 * @param forward whether to seek forward (also known as reseek)
283 * @param useBloom whether to optimize seeks using Bloom filters
285 private boolean generalizedSeek(boolean isLazy, Cell seekKey,
286 boolean forward, boolean useBloom) throws IOException {
287 if (!isLazy && useBloom) {
288 throw new IllegalArgumentException("Multi-column Bloom filter " +
289 "optimization requires a lazy seek");
292 if (current == null) {
293 return false;
296 KeyValueScanner scanner = current;
297 try {
298 while (scanner != null) {
299 Cell topKey = scanner.peek();
300 if (comparator.getComparator().compare(seekKey, topKey) <= 0) {
301 // Top KeyValue is at-or-after Seek KeyValue. We only know that all
302 // scanners are at or after seekKey (because fake keys of
303 // scanners where a lazy-seek operation has been done are not greater
304 // than their real next keys) but we still need to enforce our
305 // invariant that the top scanner has done a real seek. This way
306 // StoreScanner and RegionScanner do not have to worry about fake
307 // keys.
308 heap.add(scanner);
309 scanner = null;
310 current = pollRealKV();
311 return current != null;
314 boolean seekResult;
315 if (isLazy && heap.size() > 0) {
316 // If there is only one scanner left, we don't do lazy seek.
317 seekResult = scanner.requestSeek(seekKey, forward, useBloom);
318 } else {
319 seekResult = NonLazyKeyValueScanner.doRealSeek(scanner, seekKey,
320 forward);
323 if (!seekResult) {
324 this.scannersForDelayedClose.add(scanner);
325 } else {
326 heap.add(scanner);
328 scanner = heap.poll();
329 if (scanner == null) {
330 current = null;
333 } catch (Exception e) {
334 if (scanner != null) {
335 try {
336 scanner.close();
337 } catch (Exception ce) {
338 LOG.warn("close KeyValueScanner error", ce);
341 throw e;
344 // Heap is returning empty, scanner is done
345 return false;
349 * Fetches the top sub-scanner from the priority queue, ensuring that a real
350 * seek has been done on it. Works by fetching the top sub-scanner, and if it
351 * has not done a real seek, making it do so (which will modify its top KV),
352 * putting it back, and repeating this until success. Relies on the fact that
353 * on a lazy seek we set the current key of a StoreFileScanner to a KV that
354 * is not greater than the real next KV to be read from that file, so the
355 * scanner that bubbles up to the top of the heap will have global next KV in
356 * this scanner heap if (1) it has done a real seek and (2) its KV is the top
357 * among all top KVs (some of which are fake) in the scanner heap.
359 protected KeyValueScanner pollRealKV() throws IOException {
360 KeyValueScanner kvScanner = heap.poll();
361 if (kvScanner == null) {
362 return null;
365 while (kvScanner != null && !kvScanner.realSeekDone()) {
366 if (kvScanner.peek() != null) {
367 try {
368 kvScanner.enforceSeek();
369 } catch (IOException ioe) {
370 // Add the item to delayed close set in case it is leak from close
371 this.scannersForDelayedClose.add(kvScanner);
372 throw ioe;
374 Cell curKV = kvScanner.peek();
375 if (curKV != null) {
376 KeyValueScanner nextEarliestScanner = heap.peek();
377 if (nextEarliestScanner == null) {
378 // The heap is empty. Return the only possible scanner.
379 return kvScanner;
382 // Compare the current scanner to the next scanner. We try to avoid
383 // putting the current one back into the heap if possible.
384 Cell nextKV = nextEarliestScanner.peek();
385 if (nextKV == null || comparator.compare(curKV, nextKV) < 0) {
386 // We already have the scanner with the earliest KV, so return it.
387 return kvScanner;
390 // Otherwise, put the scanner back into the heap and let it compete
391 // against all other scanners (both those that have done a "real
392 // seek" and a "lazy seek").
393 heap.add(kvScanner);
394 } else {
395 // Close the scanner because we did a real seek and found out there
396 // are no more KVs.
397 this.scannersForDelayedClose.add(kvScanner);
399 } else {
400 // Close the scanner because it has already run out of KVs even before
401 // we had to do a real seek on it.
402 this.scannersForDelayedClose.add(kvScanner);
404 kvScanner = heap.poll();
407 return kvScanner;
411 * @return the current Heap
413 public PriorityQueue<KeyValueScanner> getHeap() {
414 return this.heap;
418 * @see KeyValueScanner#getScannerOrder()
420 @Override
421 public long getScannerOrder() {
422 return 0;
425 KeyValueScanner getCurrentForTesting() {
426 return current;
429 @Override
430 public Cell getNextIndexedKey() {
431 // here we return the next index key from the top scanner
432 return current == null ? null : current.getNextIndexedKey();
435 @Override
436 public void shipped() throws IOException {
437 for (KeyValueScanner scanner : this.scannersForDelayedClose) {
438 scanner.close(); // There wont be further fetch of Cells from these scanners. Just close.
440 this.scannersForDelayedClose.clear();
441 if (this.current != null) {
442 this.current.shipped();
444 if (this.heap != null) {
445 for (KeyValueScanner scanner : this.heap) {
446 scanner.shipped();