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