001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.regionserver;
020
021import java.io.IOException;
022import java.io.InterruptedIOException;
023import java.util.ArrayList;
024import java.util.List;
025import java.util.NavigableSet;
026import java.util.OptionalInt;
027import java.util.concurrent.CountDownLatch;
028import java.util.concurrent.locks.ReentrantLock;
029
030import org.apache.hadoop.hbase.Cell;
031import org.apache.hadoop.hbase.CellComparator;
032import org.apache.hadoop.hbase.CellUtil;
033import org.apache.hadoop.hbase.DoNotRetryIOException;
034import org.apache.hadoop.hbase.HConstants;
035import org.apache.hadoop.hbase.PrivateCellUtil;
036import org.apache.hadoop.hbase.KeyValue;
037import org.apache.hadoop.hbase.KeyValueUtil;
038import org.apache.hadoop.hbase.client.IsolationLevel;
039import org.apache.hadoop.hbase.client.Scan;
040import org.apache.hadoop.hbase.executor.ExecutorService;
041import org.apache.hadoop.hbase.filter.Filter;
042import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
043import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
044import org.apache.hadoop.hbase.regionserver.handler.ParallelSeekHandler;
045import org.apache.hadoop.hbase.regionserver.querymatcher.CompactionScanQueryMatcher;
046import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
047import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher;
048import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
049import org.apache.yetus.audience.InterfaceAudience;
050
051import org.slf4j.Logger;
052import org.slf4j.LoggerFactory;
053
054import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
055import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
056import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
057
058/**
059 * Scanner scans both the memstore and the Store. Coalesce KeyValue stream into List<KeyValue>
060 * for a single row.
061 * <p>
062 * The implementation is not thread safe. So there will be no race between next and close. The only
063 * exception is updateReaders, it will be called in the memstore flush thread to indicate that there
064 * is a flush.
065 */
066@InterfaceAudience.Private
067public class StoreScanner extends NonReversedNonLazyKeyValueScanner
068    implements KeyValueScanner, InternalScanner, ChangedReadersObserver {
069  private static final Logger LOG = LoggerFactory.getLogger(StoreScanner.class);
070  // In unit tests, the store could be null
071  protected final HStore store;
072  private final CellComparator comparator;
073  private ScanQueryMatcher matcher;
074  protected KeyValueHeap heap;
075  private boolean cacheBlocks;
076
077  private long countPerRow = 0;
078  private int storeLimit = -1;
079  private int storeOffset = 0;
080
081  // Used to indicate that the scanner has closed (see HBASE-1107)
082  // Do not need to be volatile because it's always accessed via synchronized methods
083  private boolean closing = false;
084  private final boolean get;
085  private final boolean explicitColumnQuery;
086  private final boolean useRowColBloom;
087  /**
088   * A flag that enables StoreFileScanner parallel-seeking
089   */
090  private boolean parallelSeekEnabled = false;
091  private ExecutorService executor;
092  private final Scan scan;
093  private final long oldestUnexpiredTS;
094  private final long now;
095  private final int minVersions;
096  private final long maxRowSize;
097  private final long cellsPerHeartbeatCheck;
098
099  // 1) Collects all the KVHeap that are eagerly getting closed during the
100  //    course of a scan
101  // 2) Collects the unused memstore scanners. If we close the memstore scanners
102  //    before sending data to client, the chunk may be reclaimed by other
103  //    updates and the data will be corrupt.
104  private final List<KeyValueScanner> scannersForDelayedClose = new ArrayList<>();
105
106  /**
107   * The number of KVs seen by the scanner. Includes explicitly skipped KVs, but not
108   * KVs skipped via seeking to next row/column. TODO: estimate them?
109   */
110  private long kvsScanned = 0;
111  private Cell prevCell = null;
112
113  private final long preadMaxBytes;
114  private long bytesRead;
115
116  /** We don't ever expect to change this, the constant is just for clarity. */
117  static final boolean LAZY_SEEK_ENABLED_BY_DEFAULT = true;
118  public static final String STORESCANNER_PARALLEL_SEEK_ENABLE =
119      "hbase.storescanner.parallel.seek.enable";
120
121  /** Used during unit testing to ensure that lazy seek does save seek ops */
122  private static boolean lazySeekEnabledGlobally = LAZY_SEEK_ENABLED_BY_DEFAULT;
123
124  /**
125   * The number of cells scanned in between timeout checks. Specifying a larger value means that
126   * timeout checks will occur less frequently. Specifying a small value will lead to more frequent
127   * timeout checks.
128   */
129  public static final String HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK =
130      "hbase.cells.scanned.per.heartbeat.check";
131
132  /**
133   * Default value of {@link #HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK}.
134   */
135  public static final long DEFAULT_HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK = 10000;
136
137  /**
138   * If the read type if Scan.ReadType.DEFAULT, we will start with pread, and if the kvs we scanned
139   * reaches this limit, we will reopen the scanner with stream. The default value is 4 times of
140   * block size for this store.
141   */
142  public static final String STORESCANNER_PREAD_MAX_BYTES = "hbase.storescanner.pread.max.bytes";
143
144  private final Scan.ReadType readType;
145
146  // A flag whether use pread for scan
147  // it maybe changed if we use Scan.ReadType.DEFAULT and we have read lots of data.
148  private boolean scanUsePread;
149  // Indicates whether there was flush during the course of the scan
150  private volatile boolean flushed = false;
151  // generally we get one file from a flush
152  private final List<KeyValueScanner> flushedstoreFileScanners = new ArrayList<>(1);
153  // Since CompactingMemstore is now default, we get three memstore scanners from a flush
154  private final List<KeyValueScanner> memStoreScannersAfterFlush = new ArrayList<>(3);
155  // The current list of scanners
156  @VisibleForTesting
157  final List<KeyValueScanner> currentScanners = new ArrayList<>();
158  // flush update lock
159  private final ReentrantLock flushLock = new ReentrantLock();
160
161  protected final long readPt;
162  private boolean topChanged = false;
163
164  /** An internal constructor. */
165  private StoreScanner(HStore store, Scan scan, ScanInfo scanInfo,
166      int numColumns, long readPt, boolean cacheBlocks, ScanType scanType) {
167    this.readPt = readPt;
168    this.store = store;
169    this.cacheBlocks = cacheBlocks;
170    this.comparator = Preconditions.checkNotNull(scanInfo.getComparator());
171    get = scan.isGetScan();
172    explicitColumnQuery = numColumns > 0;
173    this.scan = scan;
174    this.now = EnvironmentEdgeManager.currentTime();
175    this.oldestUnexpiredTS = scan.isRaw() ? 0L : now - scanInfo.getTtl();
176    this.minVersions = scanInfo.getMinVersions();
177
178    // We look up row-column Bloom filters for multi-column queries as part of
179    // the seek operation. However, we also look the row-column Bloom filter
180    // for multi-row (non-"get") scans because this is not done in
181    // StoreFile.passesBloomFilter(Scan, SortedSet<byte[]>).
182    this.useRowColBloom = numColumns > 1 || (!get && numColumns == 1);
183    this.maxRowSize = scanInfo.getTableMaxRowSize();
184    if (get) {
185      this.readType = Scan.ReadType.PREAD;
186      this.scanUsePread = true;
187    } else if (scanType != ScanType.USER_SCAN) {
188      // For compaction scanners never use Pread as already we have stream based scanners on the
189      // store files to be compacted
190      this.readType = Scan.ReadType.STREAM;
191      this.scanUsePread = false;
192    } else {
193      if (scan.getReadType() == Scan.ReadType.DEFAULT) {
194        this.readType = scanInfo.isUsePread() ? Scan.ReadType.PREAD : Scan.ReadType.DEFAULT;
195      } else {
196        this.readType = scan.getReadType();
197      }
198      // Always start with pread unless user specific stream. Will change to stream later if
199      // readType is default if the scan keeps running for a long time.
200      this.scanUsePread = this.readType != Scan.ReadType.STREAM;
201    }
202    this.preadMaxBytes = scanInfo.getPreadMaxBytes();
203    this.cellsPerHeartbeatCheck = scanInfo.getCellsPerTimeoutCheck();
204    // Parallel seeking is on if the config allows and more there is more than one store file.
205    if (store != null && store.getStorefilesCount() > 1) {
206      RegionServerServices rsService = store.getHRegion().getRegionServerServices();
207      if (rsService != null && scanInfo.isParallelSeekEnabled()) {
208        this.parallelSeekEnabled = true;
209        this.executor = rsService.getExecutorService();
210      }
211    }
212  }
213
214  private void addCurrentScanners(List<? extends KeyValueScanner> scanners) {
215    this.currentScanners.addAll(scanners);
216  }
217
218  /**
219   * Opens a scanner across memstore, snapshot, and all StoreFiles. Assumes we
220   * are not in a compaction.
221   *
222   * @param store who we scan
223   * @param scan the spec
224   * @param columns which columns we are scanning
225   * @throws IOException
226   */
227  public StoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
228      long readPt) throws IOException {
229    this(store, scan, scanInfo, columns != null ? columns.size() : 0, readPt,
230        scan.getCacheBlocks(), ScanType.USER_SCAN);
231    if (columns != null && scan.isRaw()) {
232      throw new DoNotRetryIOException("Cannot specify any column for a raw scan");
233    }
234    matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now,
235      store.getCoprocessorHost());
236
237    store.addChangedReaderObserver(this);
238
239    try {
240      // Pass columns to try to filter out unnecessary StoreFiles.
241      List<KeyValueScanner> scanners = selectScannersFrom(store,
242        store.getScanners(cacheBlocks, scanUsePread, false, matcher, scan.getStartRow(),
243          scan.includeStartRow(), scan.getStopRow(), scan.includeStopRow(), this.readPt));
244
245      // Seek all scanners to the start of the Row (or if the exact matching row
246      // key does not exist, then to the start of the next matching Row).
247      // Always check bloom filter to optimize the top row seek for delete
248      // family marker.
249      seekScanners(scanners, matcher.getStartKey(), explicitColumnQuery && lazySeekEnabledGlobally,
250        parallelSeekEnabled);
251
252      // set storeLimit
253      this.storeLimit = scan.getMaxResultsPerColumnFamily();
254
255      // set rowOffset
256      this.storeOffset = scan.getRowOffsetPerColumnFamily();
257      addCurrentScanners(scanners);
258      // Combine all seeked scanners with a heap
259      resetKVHeap(scanners, comparator);
260    } catch (IOException e) {
261      // remove us from the HStore#changedReaderObservers here or we'll have no chance to
262      // and might cause memory leak
263      store.deleteChangedReaderObserver(this);
264      throw e;
265    }
266  }
267
268  // a dummy scan instance for compaction.
269  private static final Scan SCAN_FOR_COMPACTION = new Scan();
270
271  /**
272   * Used for store file compaction and memstore compaction.
273   * <p>
274   * Opens a scanner across specified StoreFiles/MemStoreSegments.
275   * @param store who we scan
276   * @param scanners ancillary scanners
277   * @param smallestReadPoint the readPoint that we should use for tracking versions
278   */
279  public StoreScanner(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners,
280      ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
281    this(store, scanInfo, scanners, scanType, smallestReadPoint, earliestPutTs, null, null);
282  }
283
284  /**
285   * Used for compactions that drop deletes from a limited range of rows.
286   * <p>
287   * Opens a scanner across specified StoreFiles.
288   * @param store who we scan
289   * @param scanners ancillary scanners
290   * @param smallestReadPoint the readPoint that we should use for tracking versions
291   * @param dropDeletesFromRow The inclusive left bound of the range; can be EMPTY_START_ROW.
292   * @param dropDeletesToRow The exclusive right bound of the range; can be EMPTY_END_ROW.
293   */
294  public StoreScanner(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners,
295      long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
296      byte[] dropDeletesToRow) throws IOException {
297    this(store, scanInfo, scanners, ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint,
298        earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
299  }
300
301  private StoreScanner(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners,
302      ScanType scanType, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
303      byte[] dropDeletesToRow) throws IOException {
304    this(store, SCAN_FOR_COMPACTION, scanInfo, 0,
305        store.getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED), false, scanType);
306    assert scanType != ScanType.USER_SCAN;
307    matcher =
308        CompactionScanQueryMatcher.create(scanInfo, scanType, smallestReadPoint, earliestPutTs,
309          oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow, store.getCoprocessorHost());
310
311    // Filter the list of scanners using Bloom filters, time range, TTL, etc.
312    scanners = selectScannersFrom(store, scanners);
313
314    // Seek all scanners to the initial key
315    seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);
316    addCurrentScanners(scanners);
317    // Combine all seeked scanners with a heap
318    resetKVHeap(scanners, comparator);
319  }
320
321  private void seekAllScanner(ScanInfo scanInfo, List<? extends KeyValueScanner> scanners)
322      throws IOException {
323    // Seek all scanners to the initial key
324    seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);
325    addCurrentScanners(scanners);
326    resetKVHeap(scanners, comparator);
327  }
328
329  // For mob compaction only as we do not have a Store instance when doing mob compaction.
330  public StoreScanner(ScanInfo scanInfo, ScanType scanType,
331      List<? extends KeyValueScanner> scanners) throws IOException {
332    this(null, SCAN_FOR_COMPACTION, scanInfo, 0, Long.MAX_VALUE, false, scanType);
333    assert scanType != ScanType.USER_SCAN;
334    this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE, 0L,
335      oldestUnexpiredTS, now, null, null, null);
336    seekAllScanner(scanInfo, scanners);
337  }
338
339  // Used to instantiate a scanner for user scan in test
340  @VisibleForTesting
341  StoreScanner(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
342      List<? extends KeyValueScanner> scanners) throws IOException {
343    // 0 is passed as readpoint because the test bypasses Store
344    this(null, scan, scanInfo, columns != null ? columns.size() : 0, 0L,
345        scan.getCacheBlocks(), ScanType.USER_SCAN);
346    this.matcher =
347        UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, null);
348    seekAllScanner(scanInfo, scanners);
349  }
350
351  // Used to instantiate a scanner for compaction in test
352  @VisibleForTesting
353  StoreScanner(ScanInfo scanInfo, OptionalInt maxVersions, ScanType scanType,
354      List<? extends KeyValueScanner> scanners) throws IOException {
355    // 0 is passed as readpoint because the test bypasses Store
356    this(null, maxVersions.isPresent() ? new Scan().readVersions(maxVersions.getAsInt())
357        : SCAN_FOR_COMPACTION, scanInfo, 0, 0L, false, scanType);
358    this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE,
359      HConstants.OLDEST_TIMESTAMP, oldestUnexpiredTS, now, null, null, null);
360    seekAllScanner(scanInfo, scanners);
361  }
362
363  @VisibleForTesting
364  boolean isScanUsePread() {
365    return this.scanUsePread;
366  }
367  /**
368   * Seek the specified scanners with the given key
369   * @param scanners
370   * @param seekKey
371   * @param isLazy true if using lazy seek
372   * @param isParallelSeek true if using parallel seek
373   * @throws IOException
374   */
375  protected void seekScanners(List<? extends KeyValueScanner> scanners,
376      Cell seekKey, boolean isLazy, boolean isParallelSeek)
377      throws IOException {
378    // Seek all scanners to the start of the Row (or if the exact matching row
379    // key does not exist, then to the start of the next matching Row).
380    // Always check bloom filter to optimize the top row seek for delete
381    // family marker.
382    if (isLazy) {
383      for (KeyValueScanner scanner : scanners) {
384        scanner.requestSeek(seekKey, false, true);
385      }
386    } else {
387      if (!isParallelSeek) {
388        long totalScannersSoughtBytes = 0;
389        for (KeyValueScanner scanner : scanners) {
390          if (matcher.isUserScan() && totalScannersSoughtBytes >= maxRowSize) {
391            throw new RowTooBigException("Max row size allowed: " + maxRowSize
392              + ", but row is bigger than that");
393          }
394          scanner.seek(seekKey);
395          Cell c = scanner.peek();
396          if (c != null) {
397            totalScannersSoughtBytes += PrivateCellUtil.estimatedSerializedSizeOf(c);
398          }
399        }
400      } else {
401        parallelSeek(scanners, seekKey);
402      }
403    }
404  }
405
406  @VisibleForTesting
407  protected void resetKVHeap(List<? extends KeyValueScanner> scanners,
408      CellComparator comparator) throws IOException {
409    // Combine all seeked scanners with a heap
410    heap = newKVHeap(scanners, comparator);
411  }
412
413  protected KeyValueHeap newKVHeap(List<? extends KeyValueScanner> scanners,
414      CellComparator comparator) throws IOException {
415    return new KeyValueHeap(scanners, comparator);
416  }
417
418  /**
419   * Filters the given list of scanners using Bloom filter, time range, and TTL.
420   * <p>
421   * Will be overridden by testcase so declared as protected.
422   */
423  @VisibleForTesting
424  protected List<KeyValueScanner> selectScannersFrom(HStore store,
425      List<? extends KeyValueScanner> allScanners) {
426    boolean memOnly;
427    boolean filesOnly;
428    if (scan instanceof InternalScan) {
429      InternalScan iscan = (InternalScan) scan;
430      memOnly = iscan.isCheckOnlyMemStore();
431      filesOnly = iscan.isCheckOnlyStoreFiles();
432    } else {
433      memOnly = false;
434      filesOnly = false;
435    }
436
437    List<KeyValueScanner> scanners = new ArrayList<>(allScanners.size());
438
439    // We can only exclude store files based on TTL if minVersions is set to 0.
440    // Otherwise, we might have to return KVs that have technically expired.
441    long expiredTimestampCutoff = minVersions == 0 ? oldestUnexpiredTS : Long.MIN_VALUE;
442
443    // include only those scan files which pass all filters
444    for (KeyValueScanner kvs : allScanners) {
445      boolean isFile = kvs.isFileScanner();
446      if ((!isFile && filesOnly) || (isFile && memOnly)) {
447        continue;
448      }
449
450      if (kvs.shouldUseScanner(scan, store, expiredTimestampCutoff)) {
451        scanners.add(kvs);
452      } else {
453        kvs.close();
454      }
455    }
456    return scanners;
457  }
458
459  @Override
460  public Cell peek() {
461    return heap != null ? heap.peek() : null;
462  }
463
464  @Override
465  public KeyValue next() {
466    // throw runtime exception perhaps?
467    throw new RuntimeException("Never call StoreScanner.next()");
468  }
469
470  @Override
471  public void close() {
472    close(true);
473  }
474
475  private void close(boolean withDelayedScannersClose) {
476    if (this.closing) {
477      return;
478    }
479    if (withDelayedScannersClose) {
480      this.closing = true;
481    }
482    // For mob compaction, we do not have a store.
483    if (this.store != null) {
484      this.store.deleteChangedReaderObserver(this);
485    }
486    if (withDelayedScannersClose) {
487      clearAndClose(scannersForDelayedClose);
488      clearAndClose(memStoreScannersAfterFlush);
489      clearAndClose(flushedstoreFileScanners);
490      if (this.heap != null) {
491        this.heap.close();
492        this.currentScanners.clear();
493        this.heap = null; // CLOSED!
494      }
495    } else {
496      if (this.heap != null) {
497        this.scannersForDelayedClose.add(this.heap);
498        this.currentScanners.clear();
499        this.heap = null;
500      }
501    }
502  }
503
504  @Override
505  public boolean seek(Cell key) throws IOException {
506    if (checkFlushed()) {
507      reopenAfterFlush();
508    }
509    return this.heap.seek(key);
510  }
511
512  /**
513   * Get the next row of values from this Store.
514   * @param outResult
515   * @param scannerContext
516   * @return true if there are more rows, false if scanner is done
517   */
518  @Override
519  public boolean next(List<Cell> outResult, ScannerContext scannerContext) throws IOException {
520    if (scannerContext == null) {
521      throw new IllegalArgumentException("Scanner context cannot be null");
522    }
523    if (checkFlushed() && reopenAfterFlush()) {
524      return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
525    }
526
527    // if the heap was left null, then the scanners had previously run out anyways, close and
528    // return.
529    if (this.heap == null) {
530      // By this time partial close should happened because already heap is null
531      close(false);// Do all cleanup except heap.close()
532      return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
533    }
534
535    Cell cell = this.heap.peek();
536    if (cell == null) {
537      close(false);// Do all cleanup except heap.close()
538      return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
539    }
540
541    // only call setRow if the row changes; avoids confusing the query matcher
542    // if scanning intra-row
543
544    // If no limits exists in the scope LimitScope.Between_Cells then we are sure we are changing
545    // rows. Else it is possible we are still traversing the same row so we must perform the row
546    // comparison.
547    if (!scannerContext.hasAnyLimit(LimitScope.BETWEEN_CELLS) || matcher.currentRow() == null) {
548      this.countPerRow = 0;
549      matcher.setToNewRow(cell);
550    }
551
552    // Clear progress away unless invoker has indicated it should be kept.
553    if (!scannerContext.getKeepProgress()) {
554      scannerContext.clearProgress();
555    }
556
557    int count = 0;
558    long totalBytesRead = 0;
559
560    LOOP: do {
561      // Update and check the time limit based on the configured value of cellsPerTimeoutCheck
562      // Or if the preadMaxBytes is reached and we may want to return so we can switch to stream in
563      // the shipped method below.
564      if (kvsScanned % cellsPerHeartbeatCheck == 0 || (scanUsePread &&
565        readType == Scan.ReadType.DEFAULT && bytesRead > preadMaxBytes)) {
566        if (scannerContext.checkTimeLimit(LimitScope.BETWEEN_CELLS)) {
567          return scannerContext.setScannerState(NextState.TIME_LIMIT_REACHED).hasMoreValues();
568        }
569      }
570      // Do object compare - we set prevKV from the same heap.
571      if (prevCell != cell) {
572        ++kvsScanned;
573      }
574      checkScanOrder(prevCell, cell, comparator);
575      int cellSize = PrivateCellUtil.estimatedSerializedSizeOf(cell);
576      bytesRead += cellSize;
577      if (scanUsePread && readType == Scan.ReadType.DEFAULT &&
578        bytesRead > preadMaxBytes) {
579        // return immediately if we want to switch from pread to stream. We need this because we can
580        // only switch in the shipped method, if user use a filter to filter out everything and rpc
581        // timeout is very large then the shipped method will never be called until the whole scan
582        // is finished, but at that time we have already scan all the data...
583        // See HBASE-20457 for more details.
584        // And there is still a scenario that can not be handled. If we have a very large row, which
585        // have millions of qualifiers, and filter.filterRow is used, then even if we set the flag
586        // here, we still need to scan all the qualifiers before returning...
587        scannerContext.returnImmediately();
588      }
589      prevCell = cell;
590      scannerContext.setLastPeekedCell(cell);
591      topChanged = false;
592      ScanQueryMatcher.MatchCode qcode = matcher.match(cell);
593      switch (qcode) {
594        case INCLUDE:
595        case INCLUDE_AND_SEEK_NEXT_ROW:
596        case INCLUDE_AND_SEEK_NEXT_COL:
597
598          Filter f = matcher.getFilter();
599          if (f != null) {
600            cell = f.transformCell(cell);
601          }
602
603          this.countPerRow++;
604          if (storeLimit > -1 && this.countPerRow > (storeLimit + storeOffset)) {
605            // do what SEEK_NEXT_ROW does.
606            if (!matcher.moreRowsMayExistAfter(cell)) {
607              close(false);// Do all cleanup except heap.close()
608              return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
609            }
610            matcher.clearCurrentRow();
611            seekToNextRow(cell);
612            break LOOP;
613          }
614
615          // add to results only if we have skipped #storeOffset kvs
616          // also update metric accordingly
617          if (this.countPerRow > storeOffset) {
618            outResult.add(cell);
619
620            // Update local tracking information
621            count++;
622            totalBytesRead += cellSize;
623
624            // Update the progress of the scanner context
625            scannerContext.incrementSizeProgress(cellSize,
626              PrivateCellUtil.estimatedSizeOfCell(cell));
627            scannerContext.incrementBatchProgress(1);
628
629            if (matcher.isUserScan() && totalBytesRead > maxRowSize) {
630              throw new RowTooBigException(
631                  "Max row size allowed: " + maxRowSize + ", but the row is bigger than that.");
632            }
633          }
634
635          if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
636            if (!matcher.moreRowsMayExistAfter(cell)) {
637              close(false);// Do all cleanup except heap.close()
638              return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
639            }
640            matcher.clearCurrentRow();
641            seekOrSkipToNextRow(cell);
642          } else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
643            seekOrSkipToNextColumn(cell);
644          } else {
645            this.heap.next();
646          }
647
648          if (scannerContext.checkBatchLimit(LimitScope.BETWEEN_CELLS)) {
649            break LOOP;
650          }
651          if (scannerContext.checkSizeLimit(LimitScope.BETWEEN_CELLS)) {
652            break LOOP;
653          }
654          continue;
655
656        case DONE:
657          // Optimization for Gets! If DONE, no more to get on this row, early exit!
658          if (get) {
659            // Then no more to this row... exit.
660            close(false);// Do all cleanup except heap.close()
661            return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
662          }
663          matcher.clearCurrentRow();
664          return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
665
666        case DONE_SCAN:
667          close(false);// Do all cleanup except heap.close()
668          return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
669
670        case SEEK_NEXT_ROW:
671          // This is just a relatively simple end of scan fix, to short-cut end
672          // us if there is an endKey in the scan.
673          if (!matcher.moreRowsMayExistAfter(cell)) {
674            close(false);// Do all cleanup except heap.close()
675            return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
676          }
677          matcher.clearCurrentRow();
678          seekOrSkipToNextRow(cell);
679          NextState stateAfterSeekNextRow = needToReturn(outResult);
680          if (stateAfterSeekNextRow != null) {
681            return scannerContext.setScannerState(stateAfterSeekNextRow).hasMoreValues();
682          }
683          break;
684
685        case SEEK_NEXT_COL:
686          seekOrSkipToNextColumn(cell);
687          NextState stateAfterSeekNextColumn = needToReturn(outResult);
688          if (stateAfterSeekNextColumn != null) {
689            return scannerContext.setScannerState(stateAfterSeekNextColumn).hasMoreValues();
690          }
691          break;
692
693        case SKIP:
694          this.heap.next();
695          break;
696
697        case SEEK_NEXT_USING_HINT:
698          Cell nextKV = matcher.getNextKeyHint(cell);
699          if (nextKV != null && comparator.compare(nextKV, cell) > 0) {
700            seekAsDirection(nextKV);
701            NextState stateAfterSeekByHint = needToReturn(outResult);
702            if (stateAfterSeekByHint != null) {
703              return scannerContext.setScannerState(stateAfterSeekByHint).hasMoreValues();
704            }
705          } else {
706            heap.next();
707          }
708          break;
709
710        default:
711          throw new RuntimeException("UNEXPECTED");
712      }
713    } while ((cell = this.heap.peek()) != null);
714
715    if (count > 0) {
716      return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
717    }
718
719    // No more keys
720    close(false);// Do all cleanup except heap.close()
721    return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
722  }
723
724  /**
725   * If the top cell won't be flushed into disk, the new top cell may be
726   * changed after #reopenAfterFlush. Because the older top cell only exist
727   * in the memstore scanner but the memstore scanner is replaced by hfile
728   * scanner after #reopenAfterFlush. If the row of top cell is changed,
729   * we should return the current cells. Otherwise, we may return
730   * the cells across different rows.
731   * @param outResult the cells which are visible for user scan
732   * @return null is the top cell doesn't change. Otherwise, the NextState
733   *         to return
734   */
735  private NextState needToReturn(List<Cell> outResult) {
736    if (!outResult.isEmpty() && topChanged) {
737      return heap.peek() == null ? NextState.NO_MORE_VALUES : NextState.MORE_VALUES;
738    }
739    return null;
740  }
741
742  private void seekOrSkipToNextRow(Cell cell) throws IOException {
743    // If it is a Get Scan, then we know that we are done with this row; there are no more
744    // rows beyond the current one: don't try to optimize.
745    if (!get) {
746      if (trySkipToNextRow(cell)) {
747        return;
748      }
749    }
750    seekToNextRow(cell);
751  }
752
753  private void seekOrSkipToNextColumn(Cell cell) throws IOException {
754    if (!trySkipToNextColumn(cell)) {
755      seekAsDirection(matcher.getKeyForNextColumn(cell));
756    }
757  }
758
759  /**
760   * See if we should actually SEEK or rather just SKIP to the next Cell (see HBASE-13109).
761   * ScanQueryMatcher may issue SEEK hints, such as seek to next column, next row,
762   * or seek to an arbitrary seek key. This method decides whether a seek is the most efficient
763   * _actual_ way to get us to the requested cell (SEEKs are more expensive than SKIP, SKIP,
764   * SKIP inside the current, loaded block).
765   * It does this by looking at the next indexed key of the current HFile. This key
766   * is then compared with the _SEEK_ key, where a SEEK key is an artificial 'last possible key
767   * on the row' (only in here, we avoid actually creating a SEEK key; in the compare we work with
768   * the current Cell but compare as though it were a seek key; see down in
769   * matcher.compareKeyForNextRow, etc). If the compare gets us onto the
770   * next block we *_SEEK, otherwise we just SKIP to the next requested cell.
771   *
772   * <p>Other notes:
773   * <ul>
774   * <li>Rows can straddle block boundaries</li>
775   * <li>Versions of columns can straddle block boundaries (i.e. column C1 at T1 might be in a
776   * different block than column C1 at T2)</li>
777   * <li>We want to SKIP if the chance is high that we'll find the desired Cell after a
778   * few SKIPs...</li>
779   * <li>We want to SEEK when the chance is high that we'll be able to seek
780   * past many Cells, especially if we know we need to go to the next block.</li>
781   * </ul>
782   * <p>A good proxy (best effort) to determine whether SKIP is better than SEEK is whether
783   * we'll likely end up seeking to the next block (or past the next block) to get our next column.
784   * Example:
785   * <pre>
786   * |    BLOCK 1              |     BLOCK 2                   |
787   * |  r1/c1, r1/c2, r1/c3    |    r1/c4, r1/c5, r2/c1        |
788   *                                   ^         ^
789   *                                   |         |
790   *                           Next Index Key   SEEK_NEXT_ROW (before r2/c1)
791   *
792   *
793   * |    BLOCK 1                       |     BLOCK 2                      |
794   * |  r1/c1/t5, r1/c1/t4, r1/c1/t3    |    r1/c1/t2, r1/c1/T1, r1/c2/T3  |
795   *                                            ^              ^
796   *                                            |              |
797   *                                    Next Index Key        SEEK_NEXT_COL
798   * </pre>
799   * Now imagine we want columns c1 and c3 (see first diagram above), the 'Next Index Key' of r1/c4
800   * is > r1/c3 so we should seek to get to the c1 on the next row, r2. In second case, say we only
801   * want one version of c1, after we have it, a SEEK_COL will be issued to get to c2. Looking at
802   * the 'Next Index Key', it would land us in the next block, so we should SEEK. In other scenarios
803   * where the SEEK will not land us in the next block, it is very likely better to issues a series
804   * of SKIPs.
805   * @param cell current cell
806   * @return true means skip to next row, false means not
807   */
808  @VisibleForTesting
809  protected boolean trySkipToNextRow(Cell cell) throws IOException {
810    Cell nextCell = null;
811    // used to guard against a changed next indexed key by doing a identity comparison
812    // when the identity changes we need to compare the bytes again
813    Cell previousIndexedKey = null;
814    do {
815      Cell nextIndexedKey = getNextIndexedKey();
816      if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY &&
817          (nextIndexedKey == previousIndexedKey ||
818          matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0)) {
819        this.heap.next();
820        ++kvsScanned;
821        previousIndexedKey = nextIndexedKey;
822      } else {
823        return false;
824      }
825    } while ((nextCell = this.heap.peek()) != null && CellUtil.matchingRows(cell, nextCell));
826    return true;
827  }
828
829  /**
830   * See {@link org.apache.hadoop.hbase.regionserver.StoreScanner#trySkipToNextRow(Cell)}
831   * @param cell current cell
832   * @return true means skip to next column, false means not
833   */
834  @VisibleForTesting
835  protected boolean trySkipToNextColumn(Cell cell) throws IOException {
836    Cell nextCell = null;
837    // used to guard against a changed next indexed key by doing a identity comparison
838    // when the identity changes we need to compare the bytes again
839    Cell previousIndexedKey = null;
840    do {
841      Cell nextIndexedKey = getNextIndexedKey();
842      if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY &&
843          (nextIndexedKey == previousIndexedKey ||
844          matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0)) {
845        this.heap.next();
846        ++kvsScanned;
847        previousIndexedKey = nextIndexedKey;
848      } else {
849        return false;
850      }
851    } while ((nextCell = this.heap.peek()) != null && CellUtil.matchingRowColumn(cell, nextCell));
852    // We need this check because it may happen that the new scanner that we get
853    // during heap.next() is requiring reseek due of fake KV previously generated for
854    // ROWCOL bloom filter optimization. See HBASE-19863 for more details
855    if (nextCell != null && matcher.compareKeyForNextColumn(nextCell, cell) < 0) {
856      return false;
857    }
858    return true;
859  }
860
861  @Override
862  public long getReadPoint() {
863    return this.readPt;
864  }
865
866  private static void clearAndClose(List<KeyValueScanner> scanners) {
867    for (KeyValueScanner s : scanners) {
868      s.close();
869    }
870    scanners.clear();
871  }
872
873  // Implementation of ChangedReadersObserver
874  @Override
875  public void updateReaders(List<HStoreFile> sfs, List<KeyValueScanner> memStoreScanners)
876      throws IOException {
877    if (CollectionUtils.isEmpty(sfs) && CollectionUtils.isEmpty(memStoreScanners)) {
878      return;
879    }
880    flushLock.lock();
881    try {
882      flushed = true;
883      final boolean isCompaction = false;
884      boolean usePread = get || scanUsePread;
885      // SEE HBASE-19468 where the flushed files are getting compacted even before a scanner
886      // calls next(). So its better we create scanners here rather than next() call. Ensure
887      // these scanners are properly closed() whether or not the scan is completed successfully
888      // Eagerly creating scanners so that we have the ref counting ticking on the newly created
889      // store files. In case of stream scanners this eager creation does not induce performance
890      // penalty because in scans (that uses stream scanners) the next() call is bound to happen.
891      List<KeyValueScanner> scanners = store.getScanners(sfs, cacheBlocks, get, usePread,
892        isCompaction, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt, false);
893      flushedstoreFileScanners.addAll(scanners);
894      if (!CollectionUtils.isEmpty(memStoreScanners)) {
895        clearAndClose(memStoreScannersAfterFlush);
896        memStoreScannersAfterFlush.addAll(memStoreScanners);
897      }
898    } finally {
899      flushLock.unlock();
900    }
901    // Let the next() call handle re-creating and seeking
902  }
903
904  /**
905   * @return if top of heap has changed (and KeyValueHeap has to try the next KV)
906   */
907  protected final boolean reopenAfterFlush() throws IOException {
908    // here we can make sure that we have a Store instance so no null check on store.
909    Cell lastTop = heap.peek();
910    // When we have the scan object, should we not pass it to getScanners() to get a limited set of
911    // scanners? We did so in the constructor and we could have done it now by storing the scan
912    // object from the constructor
913    List<KeyValueScanner> scanners;
914    flushLock.lock();
915    try {
916      List<KeyValueScanner> allScanners =
917          new ArrayList<>(flushedstoreFileScanners.size() + memStoreScannersAfterFlush.size());
918      allScanners.addAll(flushedstoreFileScanners);
919      allScanners.addAll(memStoreScannersAfterFlush);
920      scanners = selectScannersFrom(store, allScanners);
921      // Clear the current set of flushed store files scanners so that they don't get added again
922      flushedstoreFileScanners.clear();
923      memStoreScannersAfterFlush.clear();
924    } finally {
925      flushLock.unlock();
926    }
927
928    // Seek the new scanners to the last key
929    seekScanners(scanners, lastTop, false, parallelSeekEnabled);
930    // remove the older memstore scanner
931    for (int i = currentScanners.size() - 1; i >=0; i--) {
932      if (!currentScanners.get(i).isFileScanner()) {
933        scannersForDelayedClose.add(currentScanners.remove(i));
934      } else {
935        // we add the memstore scanner to the end of currentScanners
936        break;
937      }
938    }
939    // add the newly created scanners on the flushed files and the current active memstore scanner
940    addCurrentScanners(scanners);
941    // Combine all seeked scanners with a heap
942    resetKVHeap(this.currentScanners, store.getComparator());
943    resetQueryMatcher(lastTop);
944    if (heap.peek() == null || store.getComparator().compareRows(lastTop, this.heap.peek()) != 0) {
945      LOG.info("Storescanner.peek() is changed where before = " + lastTop.toString() +
946          ",and after = " + heap.peek());
947      topChanged = true;
948    } else {
949      topChanged = false;
950    }
951    return topChanged;
952  }
953
954  private void resetQueryMatcher(Cell lastTopKey) {
955    // Reset the state of the Query Matcher and set to top row.
956    // Only reset and call setRow if the row changes; avoids confusing the
957    // query matcher if scanning intra-row.
958    Cell cell = heap.peek();
959    if (cell == null) {
960      cell = lastTopKey;
961    }
962    if ((matcher.currentRow() == null) || !CellUtil.matchingRows(cell, matcher.currentRow())) {
963      this.countPerRow = 0;
964      // The setToNewRow will call reset internally
965      matcher.setToNewRow(cell);
966    }
967  }
968
969  /**
970   * Check whether scan as expected order
971   * @param prevKV
972   * @param kv
973   * @param comparator
974   * @throws IOException
975   */
976  protected void checkScanOrder(Cell prevKV, Cell kv,
977      CellComparator comparator) throws IOException {
978    // Check that the heap gives us KVs in an increasing order.
979    assert prevKV == null || comparator == null
980        || comparator.compare(prevKV, kv) <= 0 : "Key " + prevKV
981        + " followed by a " + "smaller key " + kv + " in cf " + store;
982  }
983
984  protected boolean seekToNextRow(Cell c) throws IOException {
985    return reseek(PrivateCellUtil.createLastOnRow(c));
986  }
987
988  /**
989   * Do a reseek in a normal StoreScanner(scan forward)
990   * @param kv
991   * @return true if scanner has values left, false if end of scanner
992   * @throws IOException
993   */
994  protected boolean seekAsDirection(Cell kv)
995      throws IOException {
996    return reseek(kv);
997  }
998
999  @Override
1000  public boolean reseek(Cell kv) throws IOException {
1001    if (checkFlushed()) {
1002      reopenAfterFlush();
1003    }
1004    if (explicitColumnQuery && lazySeekEnabledGlobally) {
1005      return heap.requestSeek(kv, true, useRowColBloom);
1006    }
1007    return heap.reseek(kv);
1008  }
1009
1010  @VisibleForTesting
1011  void trySwitchToStreamRead() {
1012    if (readType != Scan.ReadType.DEFAULT || !scanUsePread || closing ||
1013        heap.peek() == null || bytesRead < preadMaxBytes) {
1014      return;
1015    }
1016    LOG.debug("Switch to stream read (scanned={} bytes) of {}", bytesRead,
1017        this.store.getColumnFamilyName());
1018    scanUsePread = false;
1019    Cell lastTop = heap.peek();
1020    List<KeyValueScanner> memstoreScanners = new ArrayList<>();
1021    List<KeyValueScanner> scannersToClose = new ArrayList<>();
1022    for (KeyValueScanner kvs : currentScanners) {
1023      if (!kvs.isFileScanner()) {
1024        // collect memstorescanners here
1025        memstoreScanners.add(kvs);
1026      } else {
1027        scannersToClose.add(kvs);
1028      }
1029    }
1030    List<KeyValueScanner> fileScanners = null;
1031    List<KeyValueScanner> newCurrentScanners;
1032    KeyValueHeap newHeap;
1033    try {
1034      // We must have a store instance here so no null check
1035      // recreate the scanners on the current file scanners
1036      fileScanners = store.recreateScanners(scannersToClose, cacheBlocks, false, false,
1037        matcher, scan.getStartRow(), scan.includeStartRow(), scan.getStopRow(),
1038        scan.includeStopRow(), readPt, false);
1039      if (fileScanners == null) {
1040        return;
1041      }
1042      seekScanners(fileScanners, lastTop, false, parallelSeekEnabled);
1043      newCurrentScanners = new ArrayList<>(fileScanners.size() + memstoreScanners.size());
1044      newCurrentScanners.addAll(fileScanners);
1045      newCurrentScanners.addAll(memstoreScanners);
1046      newHeap = newKVHeap(newCurrentScanners, comparator);
1047    } catch (Exception e) {
1048      LOG.warn("failed to switch to stream read", e);
1049      if (fileScanners != null) {
1050        fileScanners.forEach(KeyValueScanner::close);
1051      }
1052      return;
1053    }
1054    currentScanners.clear();
1055    addCurrentScanners(newCurrentScanners);
1056    this.heap = newHeap;
1057    resetQueryMatcher(lastTop);
1058    scannersToClose.forEach(KeyValueScanner::close);
1059  }
1060
1061  protected final boolean checkFlushed() {
1062    // check the var without any lock. Suppose even if we see the old
1063    // value here still it is ok to continue because we will not be resetting
1064    // the heap but will continue with the referenced memstore's snapshot. For compactions
1065    // any way we don't need the updateReaders at all to happen as we still continue with
1066    // the older files
1067    if (flushed) {
1068      // If there is a flush and the current scan is notified on the flush ensure that the
1069      // scan's heap gets reset and we do a seek on the newly flushed file.
1070      if (this.closing) {
1071        return false;
1072      }
1073      // reset the flag
1074      flushed = false;
1075      return true;
1076    }
1077    return false;
1078  }
1079
1080
1081  /**
1082   * Seek storefiles in parallel to optimize IO latency as much as possible
1083   * @param scanners the list {@link KeyValueScanner}s to be read from
1084   * @param kv the KeyValue on which the operation is being requested
1085   * @throws IOException
1086   */
1087  private void parallelSeek(final List<? extends KeyValueScanner>
1088      scanners, final Cell kv) throws IOException {
1089    if (scanners.isEmpty()) return;
1090    int storeFileScannerCount = scanners.size();
1091    CountDownLatch latch = new CountDownLatch(storeFileScannerCount);
1092    List<ParallelSeekHandler> handlers = new ArrayList<>(storeFileScannerCount);
1093    for (KeyValueScanner scanner : scanners) {
1094      if (scanner instanceof StoreFileScanner) {
1095        ParallelSeekHandler seekHandler = new ParallelSeekHandler(scanner, kv,
1096          this.readPt, latch);
1097        executor.submit(seekHandler);
1098        handlers.add(seekHandler);
1099      } else {
1100        scanner.seek(kv);
1101        latch.countDown();
1102      }
1103    }
1104
1105    try {
1106      latch.await();
1107    } catch (InterruptedException ie) {
1108      throw (InterruptedIOException)new InterruptedIOException().initCause(ie);
1109    }
1110
1111    for (ParallelSeekHandler handler : handlers) {
1112      if (handler.getErr() != null) {
1113        throw new IOException(handler.getErr());
1114      }
1115    }
1116  }
1117
1118  /**
1119   * Used in testing.
1120   * @return all scanners in no particular order
1121   */
1122  @VisibleForTesting
1123  List<KeyValueScanner> getAllScannersForTesting() {
1124    List<KeyValueScanner> allScanners = new ArrayList<>();
1125    KeyValueScanner current = heap.getCurrentForTesting();
1126    if (current != null)
1127      allScanners.add(current);
1128    for (KeyValueScanner scanner : heap.getHeap())
1129      allScanners.add(scanner);
1130    return allScanners;
1131  }
1132
1133  static void enableLazySeekGlobally(boolean enable) {
1134    lazySeekEnabledGlobally = enable;
1135  }
1136
1137  /**
1138   * @return The estimated number of KVs seen by this scanner (includes some skipped KVs).
1139   */
1140  public long getEstimatedNumberOfKvsScanned() {
1141    return this.kvsScanned;
1142  }
1143
1144  @Override
1145  public Cell getNextIndexedKey() {
1146    return this.heap.getNextIndexedKey();
1147  }
1148
1149  @Override
1150  public void shipped() throws IOException {
1151    if (prevCell != null) {
1152      // Do the copy here so that in case the prevCell ref is pointing to the previous
1153      // blocks we can safely release those blocks.
1154      // This applies to blocks that are got from Bucket cache, L1 cache and the blocks
1155      // fetched from HDFS. Copying this would ensure that we let go the references to these
1156      // blocks so that they can be GCed safely(in case of bucket cache)
1157      prevCell = KeyValueUtil.toNewKeyCell(this.prevCell);
1158    }
1159    matcher.beforeShipped();
1160    // There wont be further fetch of Cells from these scanners. Just close.
1161    clearAndClose(scannersForDelayedClose);
1162    if (this.heap != null) {
1163      this.heap.shipped();
1164      // When switching from pread to stream, we will open a new scanner for each store file, but
1165      // the old scanner may still track the HFileBlocks we have scanned but not sent back to client
1166      // yet. If we close the scanner immediately then the HFileBlocks may be messed up by others
1167      // before we serialize and send it back to client. The HFileBlocks will be released in shipped
1168      // method, so we here will also open new scanners and close old scanners in shipped method.
1169      // See HBASE-18055 for more details.
1170      trySwitchToStreamRead();
1171    }
1172  }
1173}
1174