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