HBASE-17958 Avoid passing unexpected cell to ScanQueryMatcher when optimize SEEK to SKIP
This commit is contained in:
parent
58c504e701
commit
9f25836d99
|
@ -474,6 +474,24 @@ public final class CellUtil {
|
|||
return CellComparator.compareTimestamps(a, b) == 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the row and column of two keyvalues for equality
|
||||
* @param left
|
||||
* @param right
|
||||
* @return True if same row and column.
|
||||
*/
|
||||
public static boolean matchingRowColumn(final Cell left, final Cell right) {
|
||||
if ((left.getRowLength() + left.getFamilyLength() + left.getQualifierLength()) != (right
|
||||
.getRowLength() + right.getFamilyLength() + right.getQualifierLength())) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!matchingRow(left, right)) {
|
||||
return false;
|
||||
}
|
||||
return matchingColumn(left, right);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return True if a delete type, a {@link KeyValue.Type#Delete} or a
|
||||
* {KeyValue.Type#DeleteFamily} or a
|
||||
|
|
|
@ -539,7 +539,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
prevCell = cell;
|
||||
|
||||
ScanQueryMatcher.MatchCode qcode = matcher.match(cell);
|
||||
qcode = optimize(qcode, cell);
|
||||
switch (qcode) {
|
||||
case INCLUDE:
|
||||
case INCLUDE_AND_SEEK_NEXT_ROW:
|
||||
|
@ -592,9 +591,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
// 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.clearCurrentRow();
|
||||
seekToNextRow(cell);
|
||||
seekOrSkipToNextRow(cell);
|
||||
} else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
|
||||
seekAsDirection(matcher.getKeyForNextColumn(cell));
|
||||
seekOrSkipToNextColumn(cell);
|
||||
} else {
|
||||
this.heap.next();
|
||||
}
|
||||
|
@ -634,11 +633,11 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
// 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.clearCurrentRow();
|
||||
seekToNextRow(cell);
|
||||
seekOrSkipToNextRow(cell);
|
||||
break;
|
||||
|
||||
case SEEK_NEXT_COL:
|
||||
seekAsDirection(matcher.getKeyForNextColumn(cell));
|
||||
seekOrSkipToNextColumn(cell);
|
||||
break;
|
||||
|
||||
case SKIP:
|
||||
|
@ -668,35 +667,47 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
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).
|
||||
* 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).
|
||||
* 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 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.
|
||||
* 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 and INCLUDE if the chance is high that we'll find the desired Cell after a
|
||||
* <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 INCLUDE_AND_SEEK and SEEK when the chance is high that we'll be able to seek
|
||||
* <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 INCLUDE/SKIP is better than SEEK is whether
|
||||
* <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>
|
||||
|
@ -719,40 +730,44 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
* 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 ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
|
||||
switch(qcode) {
|
||||
case INCLUDE_AND_SEEK_NEXT_COL:
|
||||
case SEEK_NEXT_COL:
|
||||
{
|
||||
protected boolean trySkipToNextRow(Cell cell) throws IOException {
|
||||
Cell nextCell = null;
|
||||
do {
|
||||
Cell nextIndexedKey = getNextIndexedKey();
|
||||
if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
|
||||
&& matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0) {
|
||||
this.heap.next();
|
||||
++kvsScanned;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
} while ((nextCell = this.heap.peek()) != 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;
|
||||
do {
|
||||
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;
|
||||
this.heap.next();
|
||||
++kvsScanned;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
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;
|
||||
} while ((nextCell = this.heap.peek()) != null && CellUtil.matchingRowColumn(cell, nextCell));
|
||||
return true;
|
||||
}
|
||||
|
||||
// Implementation of ChangedReadersObserver
|
||||
|
|
|
@ -255,7 +255,6 @@ public class LegacyScanQueryMatcher extends ScanQueryMatcher {
|
|||
return columns.getNextRowOrNextColumn(cell.getQualifierArray(),
|
||||
qualifierOffset, qualifierLength);
|
||||
case NEXT_ROW:
|
||||
stickyNextRow = true;
|
||||
return MatchCode.SEEK_NEXT_ROW;
|
||||
case SEEK_NEXT_USING_HINT:
|
||||
return MatchCode.SEEK_NEXT_USING_HINT;
|
||||
|
@ -287,14 +286,10 @@ public class LegacyScanQueryMatcher extends ScanQueryMatcher {
|
|||
columns.checkVersions(cell.getQualifierArray(), qualifierOffset,
|
||||
qualifierLength, timestamp, typeByte,
|
||||
mvccVersion > maxReadPointToTrackVersions);
|
||||
// Optimize with stickyNextRow
|
||||
stickyNextRow = colChecker == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW ? true : stickyNextRow;
|
||||
return (filterResponse == ReturnCode.INCLUDE_AND_NEXT_COL &&
|
||||
colChecker == MatchCode.INCLUDE) ? MatchCode.INCLUDE_AND_SEEK_NEXT_COL
|
||||
: colChecker;
|
||||
}
|
||||
stickyNextRow = (colChecker == MatchCode.SEEK_NEXT_ROW) ? true
|
||||
: stickyNextRow;
|
||||
return colChecker;
|
||||
}
|
||||
|
||||
|
|
|
@ -117,8 +117,6 @@ public abstract class ScanQueryMatcher {
|
|||
/** Row the query is on */
|
||||
protected Cell currentRow;
|
||||
|
||||
protected boolean stickyNextRow;
|
||||
|
||||
protected ScanQueryMatcher(Cell startKey, ScanInfo scanInfo, ColumnTracker columns,
|
||||
long oldestUnexpiredTS, long now) {
|
||||
this.rowComparator = scanInfo.getComparator();
|
||||
|
@ -146,13 +144,8 @@ public abstract class ScanQueryMatcher {
|
|||
if (rowComparator.compareRows(currentRow, cell) != 0) {
|
||||
return MatchCode.DONE;
|
||||
}
|
||||
// optimize case.
|
||||
if (this.stickyNextRow) {
|
||||
return MatchCode.SEEK_NEXT_ROW;
|
||||
}
|
||||
|
||||
if (this.columns.done()) {
|
||||
stickyNextRow = true;
|
||||
return MatchCode.SEEK_NEXT_ROW;
|
||||
}
|
||||
|
||||
|
@ -243,7 +236,6 @@ public abstract class ScanQueryMatcher {
|
|||
this.currentRow = currentRow;
|
||||
columns.reset();
|
||||
reset();
|
||||
stickyNextRow = false;
|
||||
}
|
||||
|
||||
public abstract boolean isUserScan();
|
||||
|
|
|
@ -113,9 +113,6 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
|
|||
MatchCode colChecker = columns.checkColumn(cell.getQualifierArray(), qualifierOffset,
|
||||
qualifierLength, typeByte);
|
||||
if (colChecker != MatchCode.INCLUDE) {
|
||||
if (colChecker == MatchCode.SEEK_NEXT_ROW) {
|
||||
stickyNextRow = true;
|
||||
}
|
||||
return colChecker;
|
||||
}
|
||||
ReturnCode filterResponse = ReturnCode.SKIP;
|
||||
|
@ -130,7 +127,6 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
|
|||
return columns.getNextRowOrNextColumn(cell.getQualifierArray(), qualifierOffset,
|
||||
qualifierLength);
|
||||
case NEXT_ROW:
|
||||
stickyNextRow = true;
|
||||
return MatchCode.SEEK_NEXT_ROW;
|
||||
case SEEK_NEXT_USING_HINT:
|
||||
return MatchCode.SEEK_NEXT_USING_HINT;
|
||||
|
@ -160,8 +156,6 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {
|
|||
*/
|
||||
colChecker = columns.checkVersions(cell.getQualifierArray(), qualifierOffset, qualifierLength,
|
||||
timestamp, typeByte, false);
|
||||
// Optimize with stickyNextRow
|
||||
stickyNextRow = colChecker == MatchCode.INCLUDE_AND_SEEK_NEXT_ROW ? true : stickyNextRow;
|
||||
return (filterResponse == ReturnCode.INCLUDE_AND_NEXT_COL && colChecker == MatchCode.INCLUDE)
|
||||
? MatchCode.INCLUDE_AND_SEEK_NEXT_COL : colChecker;
|
||||
}
|
||||
|
|
|
@ -27,15 +27,23 @@ import java.util.Arrays;
|
|||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdge;
|
||||
|
@ -47,8 +55,9 @@ import junit.framework.TestCase;
|
|||
// Can't be small as it plays with EnvironmentEdgeManager
|
||||
@Category(MediumTests.class)
|
||||
public class TestStoreScanner extends TestCase {
|
||||
private static final Log LOG = LogFactory.getLog(TestStoreScanner.class);
|
||||
private static final String CF_STR = "cf";
|
||||
final byte [] CF = Bytes.toBytes(CF_STR);
|
||||
final static byte [] CF = Bytes.toBytes(CF_STR);
|
||||
static Configuration CONF = HBaseConfiguration.create();
|
||||
private ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, Integer.MAX_VALUE,
|
||||
Long.MAX_VALUE, KeepDeletedCells.FALSE, 0, KeyValue.COMPARATOR);
|
||||
|
@ -58,6 +67,376 @@ public class TestStoreScanner extends TestCase {
|
|||
super.setUp();
|
||||
}
|
||||
|
||||
/**
|
||||
* From here on down, we have a bunch of defines and specific CELL_GRID of Cells. The
|
||||
* CELL_GRID then has a Scanner that can fake out 'block' transitions. All this elaborate
|
||||
* setup is for tests that ensure we don't overread, and that the
|
||||
* {@link StoreScanner#optimize(org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode,
|
||||
* Cell)} is not overly enthusiastic.
|
||||
*/
|
||||
private static final byte [] ZERO = new byte [] {'0'};
|
||||
private static final byte [] ZERO_POINT_ZERO = new byte [] {'0', '.', '0'};
|
||||
private static final byte [] ONE = new byte [] {'1'};
|
||||
private static final byte [] TWO = new byte [] {'2'};
|
||||
private static final byte [] TWO_POINT_TWO = new byte [] {'2', '.', '2'};
|
||||
private static final byte [] THREE = new byte [] {'3'};
|
||||
private static final byte [] FOUR = new byte [] {'4'};
|
||||
private static final byte [] FIVE = new byte [] {'5'};
|
||||
private static final byte [] VALUE = new byte [] {'v'};
|
||||
private static final int CELL_GRID_BLOCK2_BOUNDARY = 4;
|
||||
private static final int CELL_GRID_BLOCK3_BOUNDARY = 11;
|
||||
private static final int CELL_GRID_BLOCK4_BOUNDARY = 15;
|
||||
private static final int CELL_GRID_BLOCK5_BOUNDARY = 19;
|
||||
|
||||
/**
|
||||
* Five rows by four columns distinguished by column qualifier (column qualifier is one of the
|
||||
* four rows... ONE, TWO, etc.). Exceptions are a weird row after TWO; it is TWO_POINT_TWO.
|
||||
* And then row FOUR has five columns finishing w/ row FIVE having a single column.
|
||||
* We will use this to test scan does the right thing as it
|
||||
* we do Gets, StoreScanner#optimize, and what we do on (faked) block boundaries.
|
||||
*/
|
||||
private static final KeyValue [] CELL_GRID = new KeyValue [] {
|
||||
new KeyValue(ONE, CF, ONE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(ONE, CF, TWO, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(ONE, CF, THREE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(ONE, CF, FOUR, 1L, KeyValue.Type.Put, VALUE),
|
||||
// Offset 4 CELL_GRID_BLOCK2_BOUNDARY
|
||||
new KeyValue(TWO, CF, ONE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(TWO, CF, TWO, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(TWO, CF, THREE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(TWO, CF, FOUR, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(TWO_POINT_TWO, CF, ZERO, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(TWO_POINT_TWO, CF, ZERO_POINT_ZERO, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(TWO_POINT_TWO, CF, FIVE, 1L, KeyValue.Type.Put, VALUE),
|
||||
// Offset 11! CELL_GRID_BLOCK3_BOUNDARY
|
||||
new KeyValue(THREE, CF, ONE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(THREE, CF, TWO, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(THREE, CF, THREE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(THREE, CF, FOUR, 1L, KeyValue.Type.Put, VALUE),
|
||||
// Offset 15 CELL_GRID_BLOCK4_BOUNDARY
|
||||
new KeyValue(FOUR, CF, ONE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(FOUR, CF, TWO, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(FOUR, CF, THREE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(FOUR, CF, FOUR, 1L, KeyValue.Type.Put, VALUE),
|
||||
// Offset 19 CELL_GRID_BLOCK5_BOUNDARY
|
||||
new KeyValue(FOUR, CF, FIVE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(FIVE, CF, ZERO, 1L, KeyValue.Type.Put, VALUE),
|
||||
};
|
||||
|
||||
private static class KeyValueHeapWithCount extends KeyValueHeap {
|
||||
|
||||
final AtomicInteger count;
|
||||
|
||||
public KeyValueHeapWithCount(List<? extends KeyValueScanner> scanners,
|
||||
KVComparator comparator, AtomicInteger count) throws IOException {
|
||||
super(scanners, comparator);
|
||||
this.count = count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cell peek() {
|
||||
this.count.incrementAndGet();
|
||||
return super.peek();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A StoreScanner for our CELL_GRID above. Fakes the block transitions. Does counts of
|
||||
* calls to optimize and counts of when optimize actually did an optimize.
|
||||
*/
|
||||
private static class CellGridStoreScanner extends StoreScanner {
|
||||
// Count of how often optimize is called and of how often it does an optimize.
|
||||
AtomicInteger count;
|
||||
final AtomicInteger optimization = new AtomicInteger(0);
|
||||
|
||||
CellGridStoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType)
|
||||
throws IOException {
|
||||
super(scan, scanInfo, scanType, scan.getFamilyMap().get(CF),
|
||||
Arrays.<KeyValueScanner>asList(
|
||||
new KeyValueScanner[] {new KeyValueScanFixture(KeyValue.COMPARATOR, CELL_GRID)}));
|
||||
}
|
||||
|
||||
protected void resetKVHeap(List<? extends KeyValueScanner> scanners,
|
||||
KVComparator comparator) throws IOException {
|
||||
if (count == null) {
|
||||
count = new AtomicInteger(0);
|
||||
}
|
||||
heap = new KeyValueHeapWithCount(scanners, comparator, count);
|
||||
}
|
||||
|
||||
protected boolean trySkipToNextRow(Cell cell) throws IOException {
|
||||
boolean optimized = super.trySkipToNextRow(cell);
|
||||
LOG.info("Cell=" + cell + ", nextIndex=" + CellUtil.toString(getNextIndexedKey(), false)
|
||||
+ ", optimized=" + optimized);
|
||||
if (optimized) {
|
||||
optimization.incrementAndGet();
|
||||
}
|
||||
return optimized;
|
||||
}
|
||||
|
||||
protected boolean trySkipToNextColumn(Cell cell) throws IOException {
|
||||
boolean optimized = super.trySkipToNextColumn(cell);
|
||||
LOG.info("Cell=" + cell + ", nextIndex=" + CellUtil.toString(getNextIndexedKey(), false)
|
||||
+ ", optimized=" + optimized);
|
||||
if (optimized) {
|
||||
optimization.incrementAndGet();
|
||||
}
|
||||
return optimized;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cell getNextIndexedKey() {
|
||||
// Fake block boundaries by having index of next block change as we go through scan.
|
||||
return count.get() > CELL_GRID_BLOCK4_BOUNDARY?
|
||||
KeyValueUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK5_BOUNDARY].getRow()):
|
||||
count.get() > CELL_GRID_BLOCK3_BOUNDARY?
|
||||
KeyValueUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK4_BOUNDARY].getRow()):
|
||||
count.get() > CELL_GRID_BLOCK2_BOUNDARY?
|
||||
KeyValueUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK3_BOUNDARY].getRow()):
|
||||
KeyValueUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK2_BOUNDARY].getRow());
|
||||
}
|
||||
};
|
||||
|
||||
private static final int CELL_WITH_VERSIONS_BLOCK2_BOUNDARY = 4;
|
||||
|
||||
private static final KeyValue [] CELL_WITH_VERSIONS = new KeyValue [] {
|
||||
new KeyValue(ONE, CF, ONE, 2L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(ONE, CF, ONE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(ONE, CF, TWO, 2L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(ONE, CF, TWO, 1L, KeyValue.Type.Put, VALUE),
|
||||
// Offset 4 CELL_WITH_VERSIONS_BLOCK2_BOUNDARY
|
||||
new KeyValue(TWO, CF, ONE, 1L, KeyValue.Type.Put, VALUE),
|
||||
new KeyValue(TWO, CF, TWO, 1L, KeyValue.Type.Put, VALUE),
|
||||
};
|
||||
|
||||
private static class CellWithVersionsStoreScanner extends StoreScanner {
|
||||
// Count of how often optimize is called and of how often it does an optimize.
|
||||
final AtomicInteger optimization = new AtomicInteger(0);
|
||||
|
||||
CellWithVersionsStoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType)
|
||||
throws IOException {
|
||||
super(scan, scanInfo, scanType, scan.getFamilyMap().get(CF), Arrays
|
||||
.<KeyValueScanner> asList(new KeyValueScanner[] { new KeyValueScanFixture(
|
||||
KeyValue.COMPARATOR, CELL_WITH_VERSIONS) }));
|
||||
}
|
||||
|
||||
protected boolean trySkipToNextColumn(Cell cell) throws IOException {
|
||||
boolean optimized = super.trySkipToNextColumn(cell);
|
||||
LOG.info("Cell=" + cell + ", nextIndex=" + CellUtil.toString(getNextIndexedKey(), false)
|
||||
+ ", optimized=" + optimized);
|
||||
if (optimized) {
|
||||
optimization.incrementAndGet();
|
||||
}
|
||||
return optimized;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cell getNextIndexedKey() {
|
||||
// Fake block boundaries by having index of next block change as we go through scan.
|
||||
return KeyValueUtil.createFirstOnRow(CELL_WITH_VERSIONS[CELL_WITH_VERSIONS_BLOCK2_BOUNDARY].getRow());
|
||||
}
|
||||
};
|
||||
|
||||
private static class CellWithVersionsNoOptimizeStoreScanner extends StoreScanner {
|
||||
// Count of how often optimize is called and of how often it does an optimize.
|
||||
final AtomicInteger optimization = new AtomicInteger(0);
|
||||
|
||||
CellWithVersionsNoOptimizeStoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType)
|
||||
throws IOException {
|
||||
super(scan, scanInfo, scanType, scan.getFamilyMap().get(CF), Arrays
|
||||
.<KeyValueScanner> asList(new KeyValueScanner[] { new KeyValueScanFixture(
|
||||
KeyValue.COMPARATOR, CELL_WITH_VERSIONS) }));
|
||||
}
|
||||
|
||||
protected boolean trySkipToNextColumn(Cell cell) throws IOException {
|
||||
boolean optimized = super.trySkipToNextColumn(cell);
|
||||
LOG.info("Cell=" + cell + ", nextIndex=" + CellUtil.toString(getNextIndexedKey(), false)
|
||||
+ ", optimized=" + optimized);
|
||||
if (optimized) {
|
||||
optimization.incrementAndGet();
|
||||
}
|
||||
return optimized;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cell getNextIndexedKey() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
||||
public void testWithColumnCountGetFilter() throws Exception {
|
||||
Get get = new Get(ONE);
|
||||
get.setMaxVersions();
|
||||
get.addFamily(CF);
|
||||
get.setFilter(new ColumnCountGetFilter(2));
|
||||
|
||||
CellWithVersionsNoOptimizeStoreScanner scannerNoOptimize = new CellWithVersionsNoOptimizeStoreScanner(
|
||||
new Scan(get), this.scanInfo, this.scanType);
|
||||
try {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
while (scannerNoOptimize.next(results)) {
|
||||
continue;
|
||||
}
|
||||
assertEquals(2, results.size());
|
||||
assertTrue(CellUtil.matchingColumn(results.get(0), CELL_WITH_VERSIONS[0]));
|
||||
assertTrue(CellUtil.matchingColumn(results.get(1), CELL_WITH_VERSIONS[2]));
|
||||
assertTrue("Optimize should do some optimizations",
|
||||
scannerNoOptimize.optimization.get() == 0);
|
||||
} finally {
|
||||
scannerNoOptimize.close();
|
||||
}
|
||||
|
||||
get.setFilter(new ColumnCountGetFilter(2));
|
||||
CellWithVersionsStoreScanner scanner = new CellWithVersionsStoreScanner(new Scan(get),
|
||||
this.scanInfo, this.scanType);
|
||||
try {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
while (scanner.next(results)) {
|
||||
continue;
|
||||
}
|
||||
assertEquals(2, results.size());
|
||||
assertTrue(CellUtil.matchingColumn(results.get(0), CELL_WITH_VERSIONS[0]));
|
||||
assertTrue(CellUtil.matchingColumn(results.get(1), CELL_WITH_VERSIONS[2]));
|
||||
assertTrue("Optimize should do some optimizations", scanner.optimization.get() > 0);
|
||||
} finally {
|
||||
scanner.close();
|
||||
}
|
||||
}
|
||||
|
||||
public void testFullRowGetDoesNotOverreadWhenRowInsideOneBlock() throws IOException {
|
||||
// Do a Get against row two. Row two is inside a block that starts with row TWO but ends with
|
||||
// row TWO_POINT_TWO. We should read one block only.
|
||||
Get get = new Get(TWO);
|
||||
Scan scan = new Scan(get);
|
||||
CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
|
||||
try {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
while (scanner.next(results)) {
|
||||
continue;
|
||||
}
|
||||
// Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
|
||||
// TWO_POINT_TWO row does not have a a column ONE.
|
||||
assertEquals(4, results.size());
|
||||
// We should have gone the optimize route 5 times totally... an INCLUDE for the four cells
|
||||
// in the row plus the DONE on the end.
|
||||
assertEquals(5, scanner.count.get());
|
||||
// For a full row Get, there should be no opportunity for scanner optimization.
|
||||
assertEquals(0, scanner.optimization.get());
|
||||
} finally {
|
||||
scanner.close();
|
||||
}
|
||||
}
|
||||
|
||||
public void testFullRowSpansBlocks() throws IOException {
|
||||
// Do a Get against row FOUR. It spans two blocks.
|
||||
Get get = new Get(FOUR);
|
||||
Scan scan = new Scan(get);
|
||||
CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
|
||||
try {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
while (scanner.next(results)) {
|
||||
continue;
|
||||
}
|
||||
// Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
|
||||
// TWO_POINT_TWO row does not have a a column ONE.
|
||||
assertEquals(5, results.size());
|
||||
// We should have gone the optimize route 6 times totally... an INCLUDE for the five cells
|
||||
// in the row plus the DONE on the end.
|
||||
assertEquals(6, scanner.count.get());
|
||||
// For a full row Get, there should be no opportunity for scanner optimization.
|
||||
assertEquals(0, scanner.optimization.get());
|
||||
} finally {
|
||||
scanner.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test optimize in StoreScanner. Test that we skip to the next 'block' when we it makes sense
|
||||
* reading the block 'index'.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void testOptimize() throws IOException {
|
||||
Scan scan = new Scan();
|
||||
// A scan that just gets the first qualifier on each row of the CELL_GRID
|
||||
scan.addColumn(CF, ONE);
|
||||
CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
|
||||
try {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
while (scanner.next(results)) {
|
||||
continue;
|
||||
}
|
||||
// Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
|
||||
// TWO_POINT_TWO row does not have a a column ONE.
|
||||
assertEquals(4, results.size());
|
||||
for (Cell cell: results) {
|
||||
assertTrue(Bytes.equals(ONE, 0, ONE.length,
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()));
|
||||
}
|
||||
assertTrue("Optimize should do some optimizations", scanner.optimization.get() > 0);
|
||||
} finally {
|
||||
scanner.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure the optimize Scan method in StoreScanner does not get in the way of a Get doing minimum
|
||||
* work... seeking to start of block and then SKIPPING until we find the wanted Cell.
|
||||
* This 'simple' scenario mimics case of all Cells fitting inside a single HFileBlock.
|
||||
* See HBASE-15392. This test is a little cryptic. Takes a bit of staring to figure what it up to.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void testOptimizeAndGet() throws IOException {
|
||||
// First test a Get of two columns in the row R2. Every Get is a Scan. Get columns named
|
||||
// R2 and R3.
|
||||
Get get = new Get(TWO);
|
||||
get.addColumn(CF, TWO);
|
||||
get.addColumn(CF, THREE);
|
||||
Scan scan = new Scan(get);
|
||||
CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
|
||||
try {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
// For a Get there should be no more next's after the first call.
|
||||
assertEquals(false, scanner.next(results));
|
||||
// Should be one result only.
|
||||
assertEquals(2, results.size());
|
||||
// And we should have gone through optimize twice only.
|
||||
assertEquals("First qcode is SEEK_NEXT_COL and second INCLUDE_AND_SEEK_NEXT_ROW",
|
||||
3, scanner.count.get());
|
||||
} finally {
|
||||
scanner.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure that optimize does not cause the Get to do more seeking than required. Optimize
|
||||
* (see HBASE-15392) was causing us to seek all Cells in a block when a Get Scan if the next block
|
||||
* index/start key was a different row to the current one. A bug. We'd call next too often
|
||||
* because we had to exhaust all Cells in the current row making us load the next block just to
|
||||
* discard what we read there. This test is a little cryptic. Takes a bit of staring to figure
|
||||
* what it up to.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void testOptimizeAndGetWithFakedNextBlockIndexStart() throws IOException {
|
||||
// First test a Get of second column in the row R2. Every Get is a Scan. Second column has a
|
||||
// qualifier of R2.
|
||||
Get get = new Get(THREE);
|
||||
get.addColumn(CF, TWO);
|
||||
Scan scan = new Scan(get);
|
||||
CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
|
||||
try {
|
||||
List<Cell> results = new ArrayList<>();
|
||||
// For a Get there should be no more next's after the first call.
|
||||
assertEquals(false, scanner.next(results));
|
||||
// Should be one result only.
|
||||
assertEquals(1, results.size());
|
||||
// And we should have gone through optimize twice only.
|
||||
assertEquals("First qcode is SEEK_NEXT_COL and second INCLUDE_AND_SEEK_NEXT_ROW",
|
||||
2, scanner.count.get());
|
||||
} finally {
|
||||
scanner.close();
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Test utility for building a NavigableSet for scanners.
|
||||
* @param strCols
|
||||
|
|
Loading…
Reference in New Issue