View Javadoc

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