Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.NavigableSet;
import java.util.concurrent.CountDownLatch;
Expand All @@ -42,7 +41,6 @@
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
import org.apache.hadoop.hbase.regionserver.handler.ParallelSeekHandler;
Expand Down Expand Up @@ -552,8 +550,7 @@ public boolean next(List<Cell> outResult, ScannerContext scannerContext) throws
prevCell = cell;
topChanged = false;
ScanQueryMatcher.MatchCode qcode = matcher.match(cell);
qcode = optimize(qcode, cell);
switch(qcode) {
switch (qcode) {
case INCLUDE:
case INCLUDE_AND_SEEK_NEXT_ROW:
case INCLUDE_AND_SEEK_NEXT_COL:
Expand Down Expand Up @@ -606,9 +603,9 @@ public boolean next(List<Cell> outResult, ScannerContext scannerContext) throws
// the heap.peek() will any way be in the next row. So the SQM.match(cell) need do
// another compareRow to say the current row is DONE
matcher.row = null;
seekToNextRow(cell);
seekOrSkipToNextRow(cell);
} else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
seekAsDirection(matcher.getKeyForNextColumn(cell));
seekOrSkipToNextColumn(cell);
} else {
this.heap.next();
}
Expand Down Expand Up @@ -648,15 +645,15 @@ public boolean next(List<Cell> outResult, ScannerContext scannerContext) throws
// the heap.peek() will any way be in the next row. So the SQM.match(cell) need do
// another compareRow to say the current row is DONE
matcher.row = null;
seekToNextRow(cell);
seekOrSkipToNextRow(cell);
NextState stateAfterSeekNextRow = needToReturn(outResult);
if (stateAfterSeekNextRow != null) {
return scannerContext.setScannerState(stateAfterSeekNextRow).hasMoreValues();
}
break;

case SEEK_NEXT_COL:
seekAsDirection(matcher.getKeyForNextColumn(cell));
seekOrSkipToNextColumn(cell);
NextState stateAfterSeekNextColumn = needToReturn(outResult);
if (stateAfterSeekNextColumn != null) {
return scannerContext.setScannerState(stateAfterSeekNextColumn).hasMoreValues();
Expand Down Expand Up @@ -713,93 +710,6 @@ private NextState needToReturn(List<Cell> outResult) {
return null;
}

/**
* See if we should actually SEEK or rather just SKIP to the next Cell (see HBASE-13109).
* This method works together with ColumnTrackers and Filters. ColumnTrackers may issue SEEK
* hints, such as seek to next column, next row, or seek to an arbitrary seek key.
* This method intercepts these qcodes and decides whether a seek is the most efficient _actual_
* way to get us to the requested cell (SEEKs are more expensive than SKIP, SKIP, SKIP inside the
* current, loaded block).
* It does this by looking at the next indexed key of the current HFile. This key
* is then compared with the _SEEK_ key, where a SEEK key is an artificial 'last possible key
* on the row' (only in here, we avoid actually creating a SEEK key; in the compare we work with
* the current Cell but compare as though it were a seek key; see down in
* matcher.compareKeyForNextRow, etc). If the compare gets us onto the
* next block we *_SEEK, otherwise we just INCLUDE or SKIP, and let the ColumnTrackers or Filters
* go through the next Cell, and so on)
*
* <p>The ColumnTrackers and Filters must behave correctly in all cases, i.e. if they are past the
* Cells they care about they must issues a SKIP or SEEK.
*
* <p>Other notes:
* <ul>
* <li>Rows can straddle block boundaries</li>
* <li>Versions of columns can straddle block boundaries (i.e. column C1 at T1 might be in a
* different block than column C1 at T2)</li>
* <li>We want to SKIP and INCLUDE if the chance is high that we'll find the desired Cell after a
* few SKIPs...</li>
* <li>We want to INCLUDE_AND_SEEK and SEEK when the chance is high that we'll be able to seek
* past many Cells, especially if we know we need to go to the next block.</li>
* </ul>
* <p>A good proxy (best effort) to determine whether INCLUDE/SKIP is better than SEEK is whether
* we'll likely end up seeking to the next block (or past the next block) to get our next column.
* Example:
* <pre>
* | BLOCK 1 | BLOCK 2 |
* | r1/c1, r1/c2, r1/c3 | r1/c4, r1/c5, r2/c1 |
* ^ ^
* | |
* Next Index Key SEEK_NEXT_ROW (before r2/c1)
*
*
* | BLOCK 1 | BLOCK 2 |
* | r1/c1/t5, r1/c1/t4, r1/c1/t3 | r1/c1/t2, r1/c1/T1, r1/c2/T3 |
* ^ ^
* | |
* Next Index Key SEEK_NEXT_COL
* </pre>
* Now imagine we want columns c1 and c3 (see first diagram above), the 'Next Index Key' of r1/c4
* is > r1/c3 so we should seek to get to the c1 on the next row, r2. In second case, say we only
* want one version of c1, after we have it, a SEEK_COL will be issued to get to c2. Looking at
* the 'Next Index Key', it would land us in the next block, so we should SEEK. In other scenarios
* where the SEEK will not land us in the next block, it is very likely better to issues a series
* of SKIPs.
*/
@VisibleForTesting
protected ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
switch(qcode) {
case INCLUDE_AND_SEEK_NEXT_COL:
case SEEK_NEXT_COL:
{
Cell nextIndexedKey = getNextIndexedKey();
if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
&& matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0) {
return qcode == MatchCode.SEEK_NEXT_COL ? MatchCode.SKIP : MatchCode.INCLUDE;
}
break;
}
case INCLUDE_AND_SEEK_NEXT_ROW:
case SEEK_NEXT_ROW:
{
// If it is a Get Scan, then we know that we are done with this row; there are no more
// rows beyond the current one: don't try to optimize. We are DONE. Return the *_NEXT_ROW
// qcode as is. When the caller gets these flags on a Get Scan, it knows it can shut down the
// Scan.
if (!this.scan.isGetScan()) {
Cell nextIndexedKey = getNextIndexedKey();
if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
&& matcher.compareKeyForNextRow(nextIndexedKey, cell) > 0) {
return qcode == MatchCode.SEEK_NEXT_ROW ? MatchCode.SKIP : MatchCode.INCLUDE;
}
}
break;
}
default:
break;
}
return qcode;
}

@Override
public long getReadPoint() {
return readPt;
Expand Down Expand Up @@ -836,7 +746,7 @@ public void updateReaders(List<StoreFile> sfs, List<KeyValueScanner> memStoreSca
// these scanners are properly closed() whether or not the scan is completed successfully
// Eagerly creating scanners so that we have the ref counting ticking on the newly created
// store files. In case of stream scanners this eager creation does not induce performance
// penalty because in scans (that uses stream scanners) the next() call is bound to happen.
// penalty because in scans (that uses stream scanners) the next() call is bound to happen.
List<KeyValueScanner> scanners = store.getScanners(sfs, cacheBlocks, get, usePread,
isCompaction, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt, false);
flushedstoreFileScanners.addAll(scanners);
Expand Down Expand Up @@ -872,6 +782,130 @@ protected void nullifyCurrentHeap() throws IOException {
// Let the next() call handle re-creating and seeking
}

private void seekOrSkipToNextRow(Cell cell) throws IOException {
// If it is a Get Scan, then we know that we are done with this row; there are no more
// rows beyond the current one: don't try to optimize.
if (!get) {
if (trySkipToNextRow(cell)) {
return;
}
}
seekToNextRow(cell);
}

private void seekOrSkipToNextColumn(Cell cell) throws IOException {
if (!trySkipToNextColumn(cell)) {
seekAsDirection(matcher.getKeyForNextColumn(cell));
}
}

/**
* See if we should actually SEEK or rather just SKIP to the next Cell (see HBASE-13109).
* ScanQueryMatcher may issue SEEK hints, such as seek to next column, next row, or seek to an
* arbitrary seek key. This method decides whether a seek is the most efficient _actual_ way to
* get us to the requested cell (SEEKs are more expensive than SKIP, SKIP, SKIP inside the
* current, loaded block). It does this by looking at the next indexed key of the current HFile.
* This key is then compared with the _SEEK_ key, where a SEEK key is an artificial 'last possible
* key on the row' (only in here, we avoid actually creating a SEEK key; in the compare we work
* with the current Cell but compare as though it were a seek key; see down in
* matcher.compareKeyForNextRow, etc). If the compare gets us onto the next block we *_SEEK,
* otherwise we just SKIP to the next requested cell.
* <p>
* Other notes:
* <ul>
* <li>Rows can straddle block boundaries</li>
* <li>Versions of columns can straddle block boundaries (i.e. column C1 at T1 might be in a
* different block than column C1 at T2)</li>
* <li>We want to SKIP if the chance is high that we'll find the desired Cell after a few
* SKIPs...</li>
* <li>We want to SEEK when the chance is high that we'll be able to seek past many Cells,
* especially if we know we need to go to the next block.</li>
* </ul>
* <p>
* A good proxy (best effort) to determine whether SKIP is better than SEEK is whether we'll
* likely end up seeking to the next block (or past the next block) to get our next column.
* Example:
*
* <pre>
* | BLOCK 1 | BLOCK 2 |
* | r1/c1, r1/c2, r1/c3 | r1/c4, r1/c5, r2/c1 |
* ^ ^
* | |
* Next Index Key SEEK_NEXT_ROW (before r2/c1)
*
*
* | BLOCK 1 | BLOCK 2 |
* | r1/c1/t5, r1/c1/t4, r1/c1/t3 | r1/c1/t2, r1/c1/T1, r1/c2/T3 |
* ^ ^
* | |
* Next Index Key SEEK_NEXT_COL
* </pre>
*
* Now imagine we want columns c1 and c3 (see first diagram above), the 'Next Index Key' of r1/c4
* is > r1/c3 so we should seek to get to the c1 on the next row, r2. In second case, say we only
* want one version of c1, after we have it, a SEEK_COL will be issued to get to c2. Looking at
* the 'Next Index Key', it would land us in the next block, so we should SEEK. In other scenarios
* where the SEEK will not land us in the next block, it is very likely better to issues a series
* of SKIPs.
* @param cell current cell
* @return true means skip to next row, false means not
*/
@VisibleForTesting
protected boolean trySkipToNextRow(Cell cell) throws IOException {
Cell nextCell = null;
// used to guard against a changed next indexed key by doing a identity comparison
// when the identity changes we need to compare the bytes again
Cell previousIndexedKey = null;
do {
Cell nextIndexedKey = getNextIndexedKey();
if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
&& (nextIndexedKey == previousIndexedKey
|| matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0)) {
this.heap.next();
++kvsScanned;
previousIndexedKey = nextIndexedKey;
} else {
return false;
}
nextCell = this.heap.peek();
} while (nextCell != null && CellUtil.matchingRow(cell, nextCell));
return true;
}

/**
* See {@link org.apache.hadoop.hbase.regionserver.StoreScanner#trySkipToNextRow(Cell)}
* @param cell current cell
* @return true means skip to next column, false means not
*/
@VisibleForTesting
protected boolean trySkipToNextColumn(Cell cell) throws IOException {
Cell nextCell = null;
// used to guard against a changed next indexed key by doing a identity comparison
// when the identity changes we need to compare the bytes again
Cell previousIndexedKey = null;
do {
Cell nextIndexedKey = getNextIndexedKey();
if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
&& (nextIndexedKey == previousIndexedKey
|| matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0)) {
this.heap.next();
++kvsScanned;
previousIndexedKey = nextIndexedKey;
} else {
return false;
}
nextCell = this.heap.peek();
} while (nextCell != null && CellUtil.matchingRow(cell, nextCell)
&& CellUtil.matchingColumn(cell, nextCell));
// We need this check because it may happen that the new scanner that we get
// during heap.next() is requiring reseek due of fake KV previously generated for
// ROWCOL bloom filter optimization. See HBASE-19863 for more details
if (nextCell != null && matcher.compareKeyForNextColumn(nextCell, cell) < 0) {
return false;
}
return true;
}

/**
* @param flushed indicates if there was a flush
* @return true if top of heap has changed (and KeyValueHeap has to try the
Expand Down Expand Up @@ -1087,4 +1121,3 @@ public Cell getNextIndexedKey() {
return this.heap.getNextIndexedKey();
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -1504,6 +1504,35 @@ public HTable createTable(HTableDescriptor htd, byte[][] families, byte[][] spli
return (HTable) getConnection().getTable(htd.getTableName());
}

/**
* Create a table.
* @param htd table descriptor
* @param families array of column families
* @param splitKeys array of split keys
* @param type Bloom type
* @param blockSize block size
* @param c Configuration to use
* @return An HTable instance for the created table.
* @throws IOException if getAdmin or createTable fails
*/
public HTable createTable(HTableDescriptor htd, byte[][] families, byte[][] splitKeys,
BloomType type, int blockSize, Configuration c) throws IOException {
for (byte[] family : families) {
HColumnDescriptor hcd = new HColumnDescriptor(family);
// Disable blooms (they are on by default as of 0.95) but we disable them here because
// tests have hard coded counts of what to expect in block cache, etc., and blooms being
// on is interfering.
hcd.setBloomFilterType(type);
hcd.setBlocksize(blockSize);
htd.addFamily(hcd);
}
getHBaseAdmin().createTable(htd, splitKeys);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
// assigned
waitUntilAllRegionsAssigned(htd.getTableName());
return (HTable) getConnection().getTable(htd.getTableName());
}

/**
* Create a table.
* @param htd
Expand Down
Loading