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