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}