001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.regionserver; 019 020import java.io.FileNotFoundException; 021import java.io.IOException; 022import java.util.ArrayList; 023import java.util.Collection; 024import java.util.Collections; 025import java.util.List; 026import java.util.Optional; 027import java.util.PriorityQueue; 028import java.util.concurrent.atomic.LongAdder; 029import java.util.function.IntConsumer; 030import org.apache.hadoop.fs.Path; 031import org.apache.hadoop.hbase.Cell; 032import org.apache.hadoop.hbase.CellComparator; 033import org.apache.hadoop.hbase.ExtendedCell; 034import org.apache.hadoop.hbase.HBaseInterfaceAudience; 035import org.apache.hadoop.hbase.HConstants; 036import org.apache.hadoop.hbase.PrivateCellUtil; 037import org.apache.hadoop.hbase.client.Scan; 038import org.apache.hadoop.hbase.io.TimeRange; 039import org.apache.hadoop.hbase.io.hfile.HFileScanner; 040import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher; 041import org.apache.yetus.audience.InterfaceAudience; 042import org.apache.yetus.audience.InterfaceStability; 043 044/** 045 * KeyValueScanner adaptor over the Reader. It also provides hooks into bloom filter things. 046 */ 047@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.PHOENIX) 048@InterfaceStability.Evolving 049public class StoreFileScanner implements KeyValueScanner { 050 // the reader it comes from: 051 private final StoreFileReader reader; 052 private final HFileScanner hfs; 053 private ExtendedCell cur = null; 054 private boolean closed = false; 055 056 private boolean realSeekDone; 057 private boolean delayedReseek; 058 private ExtendedCell delayedSeekKV; 059 060 private final boolean enforceMVCC; 061 private final boolean hasMVCCInfo; 062 // A flag represents whether could stop skipping KeyValues for MVCC 063 // if have encountered the next row. Only used for reversed scan 064 private boolean stopSkippingKVsIfNextRow = false; 065 // A Cell that represents the row before the most previously seeked to row in seekToPreviousRow 066 // Note: Oftentimes this will contain an instance of a KeyOnly implementation of the Cell as it's 067 // not returned to callers and only used as a hint for seeking (so we can save on 068 // memory/allocations) 069 private ExtendedCell previousRow = null; 070 // Whether the underlying HFile is using a data block encoding that has lower cost for seeking to 071 // a row from the beginning of a block (i.e. RIV1). If the data block encoding has a high cost for 072 // seeks, then we can use a modified reverse scanning algorithm to reduce seeks from the beginning 073 // of the block 074 private final boolean isFastSeekingEncoding; 075 076 private static LongAdder seekCount; 077 078 private final boolean canOptimizeForNonNullColumn; 079 080 private final long readPt; 081 082 // Order of this scanner relative to other scanners when duplicate key-value is found. 083 // Higher values means scanner has newer data. 084 private final long scannerOrder; 085 086 /** 087 * Implements a {@link KeyValueScanner} on top of the specified {@link HFileScanner} 088 * @param useMVCC If true, scanner will filter out updates with MVCC larger 089 * than {@code readPt}. 090 * @param readPt MVCC value to use to filter out the updates newer than this 091 * scanner. 092 * @param hasMVCC Set to true if underlying store file reader has MVCC info. 093 * @param scannerOrder Order of the scanner relative to other scanners. See 094 * {@link KeyValueScanner#getScannerOrder()}. 095 * @param canOptimizeForNonNullColumn {@code true} if we can make sure there is no null column, 096 * otherwise {@code false}. This is a hint for optimization. 097 * @param isFastSeekingEncoding {@code true} if the data block encoding can seek quickly 098 * from the beginning of a block (i.e. RIV1), otherwise 099 * {@code false}. This is a hint for optimization. 100 */ 101 public StoreFileScanner(StoreFileReader reader, HFileScanner hfs, boolean useMVCC, 102 boolean hasMVCC, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn, 103 boolean isFastSeekingEncoding) { 104 this.readPt = readPt; 105 this.reader = reader; 106 this.hfs = hfs; 107 this.enforceMVCC = useMVCC; 108 this.hasMVCCInfo = hasMVCC; 109 this.scannerOrder = scannerOrder; 110 this.canOptimizeForNonNullColumn = canOptimizeForNonNullColumn; 111 this.isFastSeekingEncoding = isFastSeekingEncoding; 112 this.reader.incrementRefCount(); 113 } 114 115 /** 116 * Return an array of scanners corresponding to the given set of store files. 117 */ 118 public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files, 119 boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean useDropBehind, long readPt) 120 throws IOException { 121 return getScannersForStoreFiles(files, cacheBlocks, usePread, isCompaction, useDropBehind, null, 122 readPt); 123 } 124 125 /** 126 * Return an array of scanners corresponding to the given set of store files, And set the 127 * ScanQueryMatcher for each store file scanner for further optimization 128 */ 129 public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files, 130 boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop, 131 ScanQueryMatcher matcher, long readPt) throws IOException { 132 if (files.isEmpty()) { 133 return Collections.emptyList(); 134 } 135 List<StoreFileScanner> scanners = new ArrayList<>(files.size()); 136 boolean canOptimizeForNonNullColumn = matcher != null ? !matcher.hasNullColumnInQuery() : false; 137 PriorityQueue<HStoreFile> sortedFiles = 138 new PriorityQueue<>(files.size(), StoreFileComparators.SEQ_ID); 139 for (HStoreFile file : files) { 140 // The sort function needs metadata so we need to open reader first before sorting the list. 141 file.initReader(); 142 sortedFiles.add(file); 143 } 144 boolean succ = false; 145 try { 146 for (int i = 0, n = files.size(); i < n; i++) { 147 HStoreFile sf = sortedFiles.remove(); 148 StoreFileScanner scanner; 149 if (usePread) { 150 scanner = sf.getPreadScanner(cacheBlocks, readPt, i, canOptimizeForNonNullColumn); 151 } else { 152 scanner = sf.getStreamScanner(canUseDrop, cacheBlocks, isCompaction, readPt, i, 153 canOptimizeForNonNullColumn); 154 } 155 scanners.add(scanner); 156 } 157 succ = true; 158 } finally { 159 if (!succ) { 160 for (StoreFileScanner scanner : scanners) { 161 scanner.close(); 162 } 163 } 164 } 165 return scanners; 166 } 167 168 /** 169 * Get scanners for compaction. We will create a separated reader for each store file to avoid 170 * contention with normal read request. 171 */ 172 public static List<StoreFileScanner> getScannersForCompaction(Collection<HStoreFile> files, 173 boolean canUseDropBehind, long readPt) throws IOException { 174 List<StoreFileScanner> scanners = new ArrayList<>(files.size()); 175 List<HStoreFile> sortedFiles = new ArrayList<>(files); 176 Collections.sort(sortedFiles, StoreFileComparators.SEQ_ID); 177 boolean succ = false; 178 try { 179 for (int i = 0, n = sortedFiles.size(); i < n; i++) { 180 scanners.add( 181 sortedFiles.get(i).getStreamScanner(canUseDropBehind, false, true, readPt, i, false)); 182 } 183 succ = true; 184 } finally { 185 if (!succ) { 186 for (StoreFileScanner scanner : scanners) { 187 scanner.close(); 188 } 189 } 190 } 191 return scanners; 192 } 193 194 @Override 195 public String toString() { 196 return "StoreFileScanner[" + hfs.toString() + ", cur=" + cur + "]"; 197 } 198 199 @Override 200 public ExtendedCell peek() { 201 return cur; 202 } 203 204 @Override 205 public ExtendedCell next() throws IOException { 206 ExtendedCell retKey = cur; 207 208 try { 209 // only seek if we aren't at the end. cur == null implies 'end'. 210 if (cur != null) { 211 hfs.next(); 212 setCurrentCell(hfs.getCell()); 213 if (hasMVCCInfo || this.reader.isBulkLoaded()) { 214 skipKVsNewerThanReadpoint(); 215 } 216 } 217 } catch (FileNotFoundException e) { 218 throw e; 219 } catch (IOException e) { 220 throw new IOException("Could not iterate " + this, e); 221 } 222 return retKey; 223 } 224 225 @Override 226 public boolean seek(ExtendedCell key) throws IOException { 227 if (seekCount != null) seekCount.increment(); 228 229 try { 230 try { 231 if (!seekAtOrAfter(hfs, key)) { 232 this.cur = null; 233 return false; 234 } 235 236 setCurrentCell(hfs.getCell()); 237 238 if (!hasMVCCInfo && this.reader.isBulkLoaded()) { 239 return skipKVsNewerThanReadpoint(); 240 } else { 241 return !hasMVCCInfo ? true : skipKVsNewerThanReadpoint(); 242 } 243 } finally { 244 realSeekDone = true; 245 previousRow = null; 246 } 247 } catch (FileNotFoundException e) { 248 throw e; 249 } catch (IOException ioe) { 250 throw new IOException("Could not seek " + this + " to key " + key, ioe); 251 } 252 } 253 254 @Override 255 public boolean reseek(ExtendedCell key) throws IOException { 256 if (seekCount != null) seekCount.increment(); 257 258 try { 259 try { 260 if (!reseekAtOrAfter(hfs, key)) { 261 this.cur = null; 262 return false; 263 } 264 setCurrentCell(hfs.getCell()); 265 266 if (!hasMVCCInfo && this.reader.isBulkLoaded()) { 267 return skipKVsNewerThanReadpoint(); 268 } else { 269 return !hasMVCCInfo ? true : skipKVsNewerThanReadpoint(); 270 } 271 } finally { 272 realSeekDone = true; 273 previousRow = null; 274 } 275 } catch (FileNotFoundException e) { 276 throw e; 277 } catch (IOException ioe) { 278 throw new IOException("Could not reseek " + this + " to key " + key, ioe); 279 } 280 } 281 282 protected void setCurrentCell(ExtendedCell newVal) throws IOException { 283 this.cur = newVal; 284 if (this.cur != null && this.reader.isBulkLoaded() && !this.reader.isSkipResetSeqId()) { 285 PrivateCellUtil.setSequenceId(cur, this.reader.getSequenceID()); 286 } 287 } 288 289 protected boolean skipKVsNewerThanReadpoint() throws IOException { 290 // We want to ignore all key-values that are newer than our current 291 // readPoint 292 Cell startKV = cur; 293 while (enforceMVCC && cur != null && (cur.getSequenceId() > readPt)) { 294 boolean hasNext = hfs.next(); 295 setCurrentCell(hfs.getCell()); 296 if ( 297 hasNext && this.stopSkippingKVsIfNextRow && getComparator().compareRows(cur, startKV) > 0 298 ) { 299 return false; 300 } 301 } 302 303 if (cur == null) { 304 return false; 305 } 306 307 return true; 308 } 309 310 @Override 311 public void close() { 312 if (closed) return; 313 cur = null; 314 this.hfs.close(); 315 if (this.reader != null) { 316 this.reader.readCompleted(); 317 } 318 closed = true; 319 } 320 321 /** Returns false if not found or if k is after the end. */ 322 public static boolean seekAtOrAfter(HFileScanner s, ExtendedCell k) throws IOException { 323 int result = s.seekTo(k); 324 if (result < 0) { 325 if (result == HConstants.INDEX_KEY_MAGIC) { 326 // using faked key 327 return true; 328 } 329 // Passed KV is smaller than first KV in file, work from start of file 330 return s.seekTo(); 331 } else if (result > 0) { 332 // Passed KV is larger than current KV in file, if there is a next 333 // it is the "after", if not then this scanner is done. 334 return s.next(); 335 } 336 // Seeked to the exact key 337 return true; 338 } 339 340 static boolean reseekAtOrAfter(HFileScanner s, ExtendedCell k) throws IOException { 341 // This function is similar to seekAtOrAfter function 342 int result = s.reseekTo(k); 343 if (result <= 0) { 344 if (result == HConstants.INDEX_KEY_MAGIC) { 345 // using faked key 346 return true; 347 } 348 // If up to now scanner is not seeked yet, this means passed KV is smaller 349 // than first KV in file, and it is the first time we seek on this file. 350 // So we also need to work from the start of file. 351 if (!s.isSeeked()) { 352 return s.seekTo(); 353 } 354 return true; 355 } 356 // passed KV is larger than current KV in file, if there is a next 357 // it is after, if not then this scanner is done. 358 return s.next(); 359 } 360 361 /** 362 * @see KeyValueScanner#getScannerOrder() 363 */ 364 @Override 365 public long getScannerOrder() { 366 return scannerOrder; 367 } 368 369 /** 370 * Pretend we have done a seek but don't do it yet, if possible. The hope is that we find 371 * requested columns in more recent files and won't have to seek in older files. Creates a fake 372 * key/value with the given row/column and the highest (most recent) possible timestamp we might 373 * get from this file. When users of such "lazy scanner" need to know the next KV precisely (e.g. 374 * when this scanner is at the top of the heap), they run {@link #enforceSeek()}. 375 * <p> 376 * Note that this function does guarantee that the current KV of this scanner will be advanced to 377 * at least the given KV. Because of this, it does have to do a real seek in cases when the seek 378 * timestamp is older than the highest timestamp of the file, e.g. when we are trying to seek to 379 * the next row/column and use OLDEST_TIMESTAMP in the seek key. 380 */ 381 @Override 382 public boolean requestSeek(ExtendedCell kv, boolean forward, boolean useBloom) 383 throws IOException { 384 if (kv.getFamilyLength() == 0) { 385 useBloom = false; 386 } 387 388 boolean haveToSeek = true; 389 if (useBloom) { 390 // check ROWCOL Bloom filter first. 391 if (reader.getBloomFilterType() == BloomType.ROWCOL) { 392 haveToSeek = reader.passesGeneralRowColBloomFilter(kv); 393 } else if ( 394 canOptimizeForNonNullColumn 395 && ((PrivateCellUtil.isDeleteFamily(kv) || PrivateCellUtil.isDeleteFamilyVersion(kv))) 396 ) { 397 // if there is no such delete family kv in the store file, 398 // then no need to seek. 399 haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getRowArray(), kv.getRowOffset(), 400 kv.getRowLength()); 401 } 402 } 403 404 delayedReseek = forward; 405 delayedSeekKV = kv; 406 407 if (haveToSeek) { 408 // This row/column might be in this store file (or we did not use the 409 // Bloom filter), so we still need to seek. 410 realSeekDone = false; 411 long maxTimestampInFile = reader.getMaxTimestamp(); 412 long seekTimestamp = kv.getTimestamp(); 413 if (seekTimestamp > maxTimestampInFile) { 414 // Create a fake key that is not greater than the real next key. 415 // (Lower timestamps correspond to higher KVs.) 416 // To understand this better, consider that we are asked to seek to 417 // a higher timestamp than the max timestamp in this file. We know that 418 // the next point when we have to consider this file again is when we 419 // pass the max timestamp of this file (with the same row/column). 420 setCurrentCell(PrivateCellUtil.createFirstOnRowColTS(kv, maxTimestampInFile)); 421 } else { 422 // This will be the case e.g. when we need to seek to the next 423 // row/column, and we don't know exactly what they are, so we set the 424 // seek key's timestamp to OLDEST_TIMESTAMP to skip the rest of this 425 // row/column. 426 enforceSeek(); 427 } 428 return cur != null; 429 } 430 431 // Multi-column Bloom filter optimization. 432 // Create a fake key/value, so that this scanner only bubbles up to the top 433 // of the KeyValueHeap in StoreScanner after we scanned this row/column in 434 // all other store files. The query matcher will then just skip this fake 435 // key/value and the store scanner will progress to the next column. This 436 // is obviously not a "real real" seek, but unlike the fake KV earlier in 437 // this method, we want this to be propagated to ScanQueryMatcher. 438 setCurrentCell(PrivateCellUtil.createLastOnRowCol(kv)); 439 440 realSeekDone = true; 441 return true; 442 } 443 444 StoreFileReader getReader() { 445 return reader; 446 } 447 448 CellComparator getComparator() { 449 return reader.getComparator(); 450 } 451 452 @Override 453 public boolean realSeekDone() { 454 return realSeekDone; 455 } 456 457 @Override 458 public void enforceSeek() throws IOException { 459 if (realSeekDone) return; 460 461 if (delayedReseek) { 462 reseek(delayedSeekKV); 463 } else { 464 seek(delayedSeekKV); 465 } 466 } 467 468 @Override 469 public boolean isFileScanner() { 470 return true; 471 } 472 473 @Override 474 public void recordBlockSize(IntConsumer blockSizeConsumer) { 475 hfs.recordBlockSize(blockSizeConsumer); 476 } 477 478 @Override 479 public Path getFilePath() { 480 return reader.getHFileReader().getPath(); 481 } 482 483 // Test methods 484 static final long getSeekCount() { 485 return seekCount.sum(); 486 } 487 488 static final void instrument() { 489 seekCount = new LongAdder(); 490 } 491 492 @Override 493 public boolean shouldUseScanner(Scan scan, HStore store, long oldestUnexpiredTS) { 494 // if the file has no entries, no need to validate or create a scanner. 495 byte[] cf = store.getColumnFamilyDescriptor().getName(); 496 TimeRange timeRange = scan.getColumnFamilyTimeRange().get(cf); 497 if (timeRange == null) { 498 timeRange = scan.getTimeRange(); 499 } 500 return reader.passesTimerangeFilter(timeRange, oldestUnexpiredTS) 501 && reader.passesKeyRangeFilter(scan) 502 && reader.passesBloomFilter(scan, scan.getFamilyMap().get(cf)); 503 } 504 505 @Override 506 public boolean seekToPreviousRow(ExtendedCell originalKey) throws IOException { 507 try { 508 if (isFastSeekingEncoding) { 509 return seekToPreviousRowStateless(originalKey); 510 } else if (previousRow == null || getComparator().compareRows(previousRow, originalKey) > 0) { 511 return seekToPreviousRowWithoutHint(originalKey); 512 } else { 513 return seekToPreviousRowWithHint(); 514 } 515 } catch (FileNotFoundException e) { 516 throw e; 517 } catch (IOException ioe) { 518 throw new IOException("Could not seekToPreviousRow " + this + " to key " + originalKey, ioe); 519 } finally { 520 this.realSeekDone = true; 521 } 522 } 523 524 /** 525 * This variant of the {@link StoreFileScanner#seekToPreviousRow(ExtendedCell)} method requires 526 * one seek and one reseek. This method maintains state in {@link StoreFileScanner#previousRow} 527 * which only makes sense in the context of a sequential row-by-row reverse scan. 528 * {@link StoreFileScanner#previousRow} should be reset if that is not the case. The reasoning for 529 * why this method is faster than 530 * {@link StoreFileScanner#seekToPreviousRowStateless(ExtendedCell)} is that seeks are slower as 531 * they need to start from the beginning of the file, while reseeks go forward from the current 532 * position. 533 */ 534 private boolean seekToPreviousRowWithHint() throws IOException { 535 do { 536 // Using our existing seek hint, set our next seek hint 537 ExtendedCell firstKeyOfPreviousRow = PrivateCellUtil.createFirstOnRow(previousRow); 538 seekBeforeAndSaveKeyToPreviousRow(firstKeyOfPreviousRow); 539 540 // Reseek back to our initial seek hint (i.e. what we think is the start of the 541 // previous row) 542 if (!reseekAtOrAfter(firstKeyOfPreviousRow)) { 543 return false; 544 } 545 546 // If after skipping newer Kvs, we're still in our seek hint row, then we're finished 547 if (isStillAtSeekTargetAfterSkippingNewerKvs(firstKeyOfPreviousRow)) { 548 return true; 549 } 550 551 // If the previousRow seek hint is missing, that means that we're at row after the first row 552 // in the storefile. Use the without-hint seek path to process the final row 553 if (previousRow == null) { 554 return seekToPreviousRowWithoutHint(firstKeyOfPreviousRow); 555 } 556 557 // Otherwise, use the previousRow seek hint to continue traversing backwards 558 } while (true); 559 } 560 561 /** 562 * This variant of the {@link StoreFileScanner#seekToPreviousRow(ExtendedCell)} method requires 563 * two seeks and one reseek. The extra expense/seek is with the intent of speeding up subsequent 564 * calls by using the {@link StoreFileScanner#seekToPreviousRowWithHint} which this method seeds 565 * the state for by setting {@link StoreFileScanner#previousRow} 566 */ 567 private boolean seekToPreviousRowWithoutHint(ExtendedCell originalKey) throws IOException { 568 // Rewind to the cell before the beginning of this row 569 ExtendedCell keyAtBeginningOfRow = PrivateCellUtil.createFirstOnRow(originalKey); 570 if (!seekBefore(keyAtBeginningOfRow)) { 571 return false; 572 } 573 574 // Rewind before this row and save what we find as a seek hint 575 ExtendedCell firstKeyOfPreviousRow = PrivateCellUtil.createFirstOnRow(hfs.getKey()); 576 seekBeforeAndSaveKeyToPreviousRow(firstKeyOfPreviousRow); 577 578 // Seek back to the start of the previous row 579 if (!reseekAtOrAfter(firstKeyOfPreviousRow)) { 580 return false; 581 } 582 583 // If after skipping newer Kvs, we're still in what we thought was the previous 584 // row, then we can exit 585 if (isStillAtSeekTargetAfterSkippingNewerKvs(firstKeyOfPreviousRow)) { 586 return true; 587 } 588 589 // Skipping newer kvs resulted in skipping the entire row that we thought was the 590 // previous row. If we've set a seek hint, then we can use that to go backwards 591 // further 592 if (previousRow != null) { 593 return seekToPreviousRowWithHint(); 594 } 595 596 // If we've made it here, then we weren't able to set a seek hint. This can happen 597 // only if we're at the beginning of the storefile i.e. there is no row before this 598 // one 599 return false; 600 } 601 602 /** 603 * This variant of the {@link StoreFileScanner#seekToPreviousRow(ExtendedCell)} method requires 604 * two seeks. It should be used if the cost for seeking is lower i.e. when using a fast seeking 605 * data block encoding like RIV1. 606 */ 607 private boolean seekToPreviousRowStateless(ExtendedCell originalKey) throws IOException { 608 ExtendedCell key = originalKey; 609 do { 610 ExtendedCell keyAtBeginningOfRow = PrivateCellUtil.createFirstOnRow(key); 611 if (!seekBefore(keyAtBeginningOfRow)) { 612 return false; 613 } 614 615 ExtendedCell firstKeyOfPreviousRow = PrivateCellUtil.createFirstOnRow(hfs.getKey()); 616 if (!seekAtOrAfter(firstKeyOfPreviousRow)) { 617 return false; 618 } 619 620 if (isStillAtSeekTargetAfterSkippingNewerKvs(firstKeyOfPreviousRow)) { 621 return true; 622 } 623 key = firstKeyOfPreviousRow; 624 } while (true); 625 } 626 627 private boolean seekBefore(ExtendedCell seekKey) throws IOException { 628 if (seekCount != null) { 629 seekCount.increment(); 630 } 631 if (!hfs.seekBefore(seekKey)) { 632 this.cur = null; 633 return false; 634 } 635 636 return true; 637 } 638 639 /** 640 * Seeks before the seek target cell and saves the location to {@link #previousRow}. If there 641 * doesn't exist a KV in this file before the seek target cell, reposition the scanner at the 642 * beginning of the storefile (in preparation to a reseek at or after the seek key) and set the 643 * {@link #previousRow} to null. If {@link #previousRow} is ever non-null and then transitions to 644 * being null again via this method, that's because there doesn't exist a row before the seek 645 * target in the storefile (i.e. we're at the beginning of the storefile) 646 */ 647 private void seekBeforeAndSaveKeyToPreviousRow(ExtendedCell seekKey) throws IOException { 648 if (seekCount != null) { 649 seekCount.increment(); 650 } 651 if (!hfs.seekBefore(seekKey)) { 652 // Since the above seek failed, we need to position ourselves back at the start of the 653 // block or else our reseek might fail. seekTo() cannot return false here as at least 654 // one seekBefore will have returned true by the time we get here 655 hfs.seekTo(); 656 this.previousRow = null; 657 } else { 658 this.previousRow = hfs.getKey(); 659 } 660 } 661 662 private boolean seekAtOrAfter(ExtendedCell seekKey) throws IOException { 663 if (seekCount != null) { 664 seekCount.increment(); 665 } 666 if (!seekAtOrAfter(hfs, seekKey)) { 667 this.cur = null; 668 return false; 669 } 670 671 return true; 672 } 673 674 private boolean reseekAtOrAfter(ExtendedCell seekKey) throws IOException { 675 if (seekCount != null) { 676 seekCount.increment(); 677 } 678 if (!reseekAtOrAfter(hfs, seekKey)) { 679 this.cur = null; 680 return false; 681 } 682 683 return true; 684 } 685 686 private boolean isStillAtSeekTargetAfterSkippingNewerKvs(Cell seekKey) throws IOException { 687 setCurrentCell(hfs.getCell()); 688 return skipKvsNewerThanReadpointReversed() && getComparator().compareRows(cur, seekKey) <= 0; 689 } 690 691 private boolean skipKvsNewerThanReadpointReversed() throws IOException { 692 this.stopSkippingKVsIfNextRow = true; 693 boolean resultOfSkipKVs; 694 try { 695 resultOfSkipKVs = skipKVsNewerThanReadpoint(); 696 } finally { 697 this.stopSkippingKVsIfNextRow = false; 698 } 699 700 return resultOfSkipKVs; 701 } 702 703 @Override 704 public boolean seekToLastRow() throws IOException { 705 Optional<byte[]> lastRow = reader.getLastRowKey(); 706 if (!lastRow.isPresent()) { 707 return false; 708 } 709 ExtendedCell seekKey = PrivateCellUtil.createFirstOnRow(lastRow.get()); 710 if (seek(seekKey)) { 711 return true; 712 } else { 713 return seekToPreviousRow(seekKey); 714 } 715 } 716 717 @Override 718 public boolean backwardSeek(ExtendedCell key) throws IOException { 719 seek(key); 720 if (cur == null || getComparator().compareRows(cur, key) > 0) { 721 return seekToPreviousRow(key); 722 } 723 return true; 724 } 725 726 @Override 727 public ExtendedCell getNextIndexedKey() { 728 return hfs.getNextIndexedKey(); 729 } 730 731 @Override 732 public void shipped() throws IOException { 733 this.hfs.shipped(); 734 } 735}