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