HBASE-13421 Reduce the number of object creations introduced by HBASE-11544 in scan RPC hot code paths
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
cbc53a0ba5
commit
62d47e175c
|
@ -401,6 +401,9 @@ public class ClientScanner extends AbstractClientScanner {
|
|||
// happens for the cases where we see exceptions. Since only openScanner
|
||||
// would have happened, values would be null
|
||||
if (values == null && callable.switchedToADifferentReplica()) {
|
||||
// Any accumulated partial results are no longer valid since the callable will
|
||||
// openScanner with the correct startkey and we must pick up from there
|
||||
clearPartialResults();
|
||||
this.currentRegion = callable.getHRegionInfo();
|
||||
continue;
|
||||
}
|
||||
|
|
|
@ -292,14 +292,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
continue; //this was already scheduled earlier
|
||||
}
|
||||
ScannerCallable s = currentScannerCallable.getScannerCallableForReplica(id);
|
||||
|
||||
if (this.lastResult != null) {
|
||||
if(s.getScan().isReversed()){
|
||||
s.getScan().setStartRow(createClosestRowBefore(this.lastResult.getRow()));
|
||||
}else {
|
||||
s.getScan().setStartRow(Bytes.add(this.lastResult.getRow(), new byte[1]));
|
||||
}
|
||||
}
|
||||
setStartRowForReplicaCallable(s);
|
||||
outstandingCallables.add(s);
|
||||
RetryingRPC retryingOnReplica = new RetryingRPC(s);
|
||||
cs.submit(retryingOnReplica, scannerTimeout, id);
|
||||
|
@ -307,6 +300,31 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
|
|||
return max - min + 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the start row for the replica callable based on the state of the last result received.
|
||||
* @param callable The callable to set the start row on
|
||||
*/
|
||||
private void setStartRowForReplicaCallable(ScannerCallable callable) {
|
||||
if (this.lastResult == null || callable == null) return;
|
||||
|
||||
if (this.lastResult.isPartial()) {
|
||||
// The last result was a partial result which means we have not received all of the cells
|
||||
// for this row. Thus, use the last result's row as the start row. If a replica switch
|
||||
// occurs, the scanner will ensure that any accumulated partial results are cleared,
|
||||
// and the scan can resume from this row.
|
||||
callable.getScan().setStartRow(this.lastResult.getRow());
|
||||
} else {
|
||||
// The last result was not a partial result which means it contained all of the cells for
|
||||
// that row (we no longer need any information from it). Set the start row to the next
|
||||
// closest row that could be seen.
|
||||
if (callable.getScan().isReversed()) {
|
||||
callable.getScan().setStartRow(createClosestRowBefore(this.lastResult.getRow()));
|
||||
} else {
|
||||
callable.getScan().setStartRow(Bytes.add(this.lastResult.getRow(), new byte[1]));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
boolean isAnyRPCcancelled() {
|
||||
return someRPCcancelled;
|
||||
|
|
|
@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.Bu
|
|||
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.OperationStatus;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
|
@ -137,7 +136,7 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
|
|||
List<List<Cell>> deleteRows = new ArrayList<List<Cell>>(rowBatchSize);
|
||||
for (int i = 0; i < rowBatchSize; i++) {
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
hasMore = NextState.hasMoreValues(scanner.next(results));
|
||||
hasMore = scanner.next(results);
|
||||
if (results.size() > 0) {
|
||||
deleteRows.add(results);
|
||||
}
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos;
|
|||
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
|
@ -81,7 +80,7 @@ public class RowCountEndpoint extends ExampleProtos.RowCountService
|
|||
byte[] lastRow = null;
|
||||
long count = 0;
|
||||
do {
|
||||
hasMore = NextState.hasMoreValues(scanner.next(results));
|
||||
hasMore = scanner.next(results);
|
||||
for (Cell kv : results) {
|
||||
byte[] currentRow = CellUtil.cloneRow(kv);
|
||||
if (lastRow == null || !Bytes.equals(lastRow, currentRow)) {
|
||||
|
@ -120,7 +119,7 @@ public class RowCountEndpoint extends ExampleProtos.RowCountService
|
|||
boolean hasMore = false;
|
||||
long count = 0;
|
||||
do {
|
||||
hasMore = NextState.hasMoreValues(scanner.next(results));
|
||||
hasMore = scanner.next(results);
|
||||
for (Cell kv : results) {
|
||||
count++;
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -30,8 +29,10 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.NoLimitScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.mortbay.log.Log;
|
||||
|
||||
|
@ -72,10 +73,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
|
|||
public Result next() throws IOException {
|
||||
values.clear();
|
||||
|
||||
// negative values indicate no limits
|
||||
final long remainingResultSize = -1;
|
||||
final int batchLimit = -1;
|
||||
scanner.nextRaw(values, batchLimit, remainingResultSize);
|
||||
scanner.nextRaw(values, NoLimitScannerContext.getInstance());
|
||||
if (values.isEmpty()) {
|
||||
//we are done
|
||||
return null;
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateReque
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateService;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.Message;
|
||||
|
@ -92,7 +91,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
|
|||
// qualifier can be null.
|
||||
boolean hasMoreRows = false;
|
||||
do {
|
||||
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
|
||||
hasMoreRows = scanner.next(results);
|
||||
int listSize = results.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
temp = ci.getValue(colFamily, qualifier, results.get(i));
|
||||
|
@ -146,7 +145,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
|
|||
}
|
||||
boolean hasMoreRows = false;
|
||||
do {
|
||||
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
|
||||
hasMoreRows = scanner.next(results);
|
||||
int listSize = results.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
temp = ci.getValue(colFamily, qualifier, results.get(i));
|
||||
|
@ -200,7 +199,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
|
|||
List<Cell> results = new ArrayList<Cell>();
|
||||
boolean hasMoreRows = false;
|
||||
do {
|
||||
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
|
||||
hasMoreRows = scanner.next(results);
|
||||
int listSize = results.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
temp = ci.getValue(colFamily, qualifier, results.get(i));
|
||||
|
@ -254,7 +253,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
|
|||
scanner = env.getRegion().getScanner(scan);
|
||||
boolean hasMoreRows = false;
|
||||
do {
|
||||
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
|
||||
hasMoreRows = scanner.next(results);
|
||||
if (results.size() > 0) {
|
||||
counter++;
|
||||
}
|
||||
|
@ -313,7 +312,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
|
|||
|
||||
do {
|
||||
results.clear();
|
||||
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
|
||||
hasMoreRows = scanner.next(results);
|
||||
int listSize = results.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
sumVal = ci.add(sumVal, ci.castToReturnType(ci.getValue(colFamily,
|
||||
|
@ -374,7 +373,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
|
|||
|
||||
do {
|
||||
tempVal = null;
|
||||
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
|
||||
hasMoreRows = scanner.next(results);
|
||||
int listSize = results.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
tempVal = ci.add(tempVal, ci.castToReturnType(ci.getValue(colFamily,
|
||||
|
@ -441,7 +440,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
|
|||
do {
|
||||
tempVal = null;
|
||||
tempWeight = null;
|
||||
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
|
||||
hasMoreRows = scanner.next(results);
|
||||
int listSize = results.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
Cell kv = results.get(i);
|
||||
|
|
|
@ -141,8 +141,9 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.Stor
|
|||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl.WriteEntry;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputControllerFactory;
|
||||
|
@ -5175,7 +5176,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
protected Cell joinedContinuationRow = null;
|
||||
protected final byte[] stopRow;
|
||||
private final FilterWrapper filter;
|
||||
private int batch;
|
||||
private ScannerContext defaultScannerContext;
|
||||
protected int isScan;
|
||||
private boolean filterClosed = false;
|
||||
private long readPt;
|
||||
|
@ -5198,7 +5199,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
this.filter = null;
|
||||
}
|
||||
|
||||
this.batch = scan.getBatch();
|
||||
/**
|
||||
* By default, calls to next/nextRaw must enforce the batch limit. Thus, construct a default
|
||||
* scanner context that can be used to enforce the batch limit in the event that a
|
||||
* ScannerContext is not specified during an invocation of next/nextRaw
|
||||
*/
|
||||
defaultScannerContext = ScannerContext.newBuilder().setBatchLimit(scan.getBatch()).build();
|
||||
|
||||
if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW) && !scan.isGetScan()) {
|
||||
this.stopRow = null;
|
||||
} else {
|
||||
|
@ -5259,7 +5266,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
@Override
|
||||
public int getBatch() {
|
||||
return this.batch;
|
||||
return this.defaultScannerContext.getBatchLimit();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -5274,19 +5281,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> outResults)
|
||||
public boolean next(List<Cell> outResults)
|
||||
throws IOException {
|
||||
// apply the batching limit by default
|
||||
return next(outResults, batch);
|
||||
return next(outResults, defaultScannerContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> outResults, int limit) throws IOException {
|
||||
return next(outResults, limit, -1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized NextState next(List<Cell> outResults, int limit, long remainingResultSize)
|
||||
public synchronized boolean next(List<Cell> outResults, ScannerContext scannerContext)
|
||||
throws IOException {
|
||||
if (this.filterClosed) {
|
||||
throw new UnknownScannerException("Scanner was closed (timed out?) " +
|
||||
|
@ -5296,122 +5298,107 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
startRegionOperation(Operation.SCAN);
|
||||
readRequestsCount.increment();
|
||||
try {
|
||||
return nextRaw(outResults, limit, remainingResultSize);
|
||||
return nextRaw(outResults, scannerContext);
|
||||
} finally {
|
||||
closeRegionOperation(Operation.SCAN);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState nextRaw(List<Cell> outResults) throws IOException {
|
||||
return nextRaw(outResults, batch);
|
||||
public boolean nextRaw(List<Cell> outResults) throws IOException {
|
||||
// Use the RegionScanner's context by default
|
||||
return nextRaw(outResults, defaultScannerContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState nextRaw(List<Cell> outResults, int limit)
|
||||
throws IOException {
|
||||
return nextRaw(outResults, limit, -1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState nextRaw(List<Cell> outResults, int batchLimit, long remainingResultSize)
|
||||
public boolean nextRaw(List<Cell> outResults, ScannerContext scannerContext)
|
||||
throws IOException {
|
||||
if (storeHeap == null) {
|
||||
// scanner is closed
|
||||
throw new UnknownScannerException("Scanner was closed");
|
||||
}
|
||||
NextState state;
|
||||
boolean moreValues;
|
||||
if (outResults.isEmpty()) {
|
||||
// Usually outResults is empty. This is true when next is called
|
||||
// to handle scan or get operation.
|
||||
state = nextInternal(outResults, batchLimit, remainingResultSize);
|
||||
moreValues = nextInternal(outResults, scannerContext);
|
||||
} else {
|
||||
List<Cell> tmpList = new ArrayList<Cell>();
|
||||
state = nextInternal(tmpList, batchLimit, remainingResultSize);
|
||||
moreValues = nextInternal(tmpList, scannerContext);
|
||||
outResults.addAll(tmpList);
|
||||
}
|
||||
// Invalid states should never be returned. Receiving an invalid state means that we have
|
||||
// no clue how to proceed. Throw an exception.
|
||||
if (!NextState.isValidState(state)) {
|
||||
throw new IOException("Invalid state returned from nextInternal. state:" + state);
|
||||
}
|
||||
|
||||
// If the size limit was reached it means a partial Result is being returned. Returning a
|
||||
// partial Result means that we should not reset the filters; filters should only be reset in
|
||||
// between rows
|
||||
if (!state.sizeLimitReached()) resetFilters();
|
||||
if (!scannerContext.partialResultFormed()) resetFilters();
|
||||
|
||||
if (isFilterDoneInternal()) {
|
||||
state = NextState.makeState(NextState.State.NO_MORE_VALUES, state.getResultSize());
|
||||
moreValues = false;
|
||||
}
|
||||
return state;
|
||||
return moreValues;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the state the joinedHeap returned on the call to
|
||||
* {@link KeyValueHeap#next(List, int, long)}
|
||||
* @return true if more cells exist after this batch, false if scanner is done
|
||||
*/
|
||||
private NextState populateFromJoinedHeap(List<Cell> results, int limit, long resultSize)
|
||||
private boolean populateFromJoinedHeap(List<Cell> results, ScannerContext scannerContext)
|
||||
throws IOException {
|
||||
assert joinedContinuationRow != null;
|
||||
NextState state =
|
||||
populateResult(results, this.joinedHeap, limit, resultSize,
|
||||
boolean moreValues =
|
||||
populateResult(results, this.joinedHeap, scannerContext,
|
||||
joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(),
|
||||
joinedContinuationRow.getRowLength());
|
||||
if (state != null && !state.batchLimitReached() && !state.sizeLimitReached()) {
|
||||
|
||||
if (!scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
|
||||
// We are done with this row, reset the continuation.
|
||||
joinedContinuationRow = null;
|
||||
}
|
||||
// As the data is obtained from two independent heaps, we need to
|
||||
// ensure that result list is sorted, because Result relies on that.
|
||||
Collections.sort(results, comparator);
|
||||
return state;
|
||||
return moreValues;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches records with currentRow into results list, until next row, batchLimit (if not -1) is
|
||||
* reached, or remainingResultSize (if not -1) is reaced
|
||||
* @param heap KeyValueHeap to fetch data from.It must be positioned on correct row before call.
|
||||
* @param remainingResultSize The remaining space within our result size limit. A negative value
|
||||
* indicate no limit
|
||||
* @param batchLimit Max amount of KVs to place in result list, -1 means no limit.
|
||||
* @param scannerContext
|
||||
* @param currentRow Byte array with key we are fetching.
|
||||
* @param offset offset for currentRow
|
||||
* @param length length for currentRow
|
||||
* @return state of last call to {@link KeyValueHeap#next()}
|
||||
*/
|
||||
private NextState populateResult(List<Cell> results, KeyValueHeap heap, int batchLimit,
|
||||
long remainingResultSize, byte[] currentRow, int offset, short length) throws IOException {
|
||||
private boolean populateResult(List<Cell> results, KeyValueHeap heap,
|
||||
ScannerContext scannerContext, byte[] currentRow, int offset, short length)
|
||||
throws IOException {
|
||||
Cell nextKv;
|
||||
boolean moreCellsInRow = false;
|
||||
long accumulatedResultSize = 0;
|
||||
List<Cell> tmpResults = new ArrayList<Cell>();
|
||||
boolean tmpKeepProgress = scannerContext.getKeepProgress();
|
||||
// Scanning between column families and thus the scope is between cells
|
||||
LimitScope limitScope = LimitScope.BETWEEN_CELLS;
|
||||
do {
|
||||
int remainingBatchLimit = batchLimit - results.size();
|
||||
NextState heapState =
|
||||
heap.next(tmpResults, remainingBatchLimit, remainingResultSize - accumulatedResultSize);
|
||||
results.addAll(tmpResults);
|
||||
accumulatedResultSize += calculateResultSize(tmpResults, heapState);
|
||||
tmpResults.clear();
|
||||
|
||||
if (batchLimit > 0 && results.size() == batchLimit) {
|
||||
return NextState.makeState(NextState.State.BATCH_LIMIT_REACHED, accumulatedResultSize);
|
||||
}
|
||||
// We want to maintain any progress that is made towards the limits while scanning across
|
||||
// different column families. To do this, we toggle the keep progress flag on during calls
|
||||
// to the StoreScanner to ensure that any progress made thus far is not wiped away.
|
||||
scannerContext.setKeepProgress(true);
|
||||
heap.next(results, scannerContext);
|
||||
scannerContext.setKeepProgress(tmpKeepProgress);
|
||||
|
||||
nextKv = heap.peek();
|
||||
moreCellsInRow = moreCellsInRow(nextKv, currentRow, offset, length);
|
||||
boolean sizeLimitReached =
|
||||
remainingResultSize > 0 && accumulatedResultSize >= remainingResultSize;
|
||||
if (moreCellsInRow && sizeLimitReached) {
|
||||
return NextState.makeState(NextState.State.SIZE_LIMIT_REACHED, accumulatedResultSize);
|
||||
|
||||
if (scannerContext.checkBatchLimit(limitScope)) {
|
||||
return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues();
|
||||
} else if (scannerContext.checkSizeLimit(limitScope)) {
|
||||
ScannerContext.NextState state =
|
||||
moreCellsInRow ? NextState.SIZE_LIMIT_REACHED_MID_ROW : NextState.SIZE_LIMIT_REACHED;
|
||||
return scannerContext.setScannerState(state).hasMoreValues();
|
||||
}
|
||||
} while (moreCellsInRow);
|
||||
|
||||
if (nextKv != null) {
|
||||
return NextState.makeState(NextState.State.MORE_VALUES, accumulatedResultSize);
|
||||
} else {
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, accumulatedResultSize);
|
||||
}
|
||||
return nextKv != null;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -5429,30 +5416,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
return nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculates the size of the results. If the state of the scanner that these results came from
|
||||
* indicates that an estimate of the result size has already been generated, we can skip the
|
||||
* calculation and use that instead.
|
||||
* @param results List of cells we want to calculate size of
|
||||
* @param state The state returned from the scanner that generated these results
|
||||
* @return aggregate size of results
|
||||
*/
|
||||
private long calculateResultSize(List<Cell> results, NextState state) {
|
||||
if (results == null || results.isEmpty()) return 0;
|
||||
|
||||
// In general, the state should contain the estimate because the result size used to
|
||||
// determine when the scan has exceeded its size limit. If the estimate is contained in the
|
||||
// state then we can avoid an unnecesasry calculation.
|
||||
if (state != null && state.hasResultSizeEstimate()) return state.getResultSize();
|
||||
|
||||
long size = 0;
|
||||
for (Cell c : results) {
|
||||
size += CellUtil.estimatedHeapSizeOfWithoutTags(c);
|
||||
}
|
||||
|
||||
return size;
|
||||
}
|
||||
|
||||
/*
|
||||
* @return True if a filter rules the scanner is over, done.
|
||||
*/
|
||||
|
@ -5465,20 +5428,37 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
return this.filter != null && this.filter.filterAllRemaining();
|
||||
}
|
||||
|
||||
private NextState nextInternal(List<Cell> results, int batchLimit, long remainingResultSize)
|
||||
private boolean nextInternal(List<Cell> results, ScannerContext scannerContext)
|
||||
throws IOException {
|
||||
if (!results.isEmpty()) {
|
||||
throw new IllegalArgumentException("First parameter should be an empty list");
|
||||
}
|
||||
// Estimate of the size (heap size) of the results returned from this method
|
||||
long resultSize = 0;
|
||||
if (scannerContext == null) {
|
||||
throw new IllegalArgumentException("Scanner context cannot be null");
|
||||
}
|
||||
RpcCallContext rpcCall = RpcServer.getCurrentCall();
|
||||
|
||||
// Save the initial progress from the Scanner context in these local variables. The progress
|
||||
// may need to be reset a few times if rows are being filtered out so we save the initial
|
||||
// progress.
|
||||
int initialBatchProgress = scannerContext.getBatchProgress();
|
||||
long initialSizeProgress = scannerContext.getSizeProgress();
|
||||
|
||||
// The loop here is used only when at some point during the next we determine
|
||||
// that due to effects of filters or otherwise, we have an empty row in the result.
|
||||
// Then we loop and try again. Otherwise, we must get out on the first iteration via return,
|
||||
// "true" if there's more data to read, "false" if there isn't (storeHeap is at a stop row,
|
||||
// and joinedHeap has no more data to read for the last row (if set, joinedContinuationRow).
|
||||
while (true) {
|
||||
// Starting to scan a new row. Reset the scanner progress according to whether or not
|
||||
// progress should be kept.
|
||||
if (scannerContext.getKeepProgress()) {
|
||||
// Progress should be kept. Reset to initial values seen at start of method invocation.
|
||||
scannerContext.setProgress(initialBatchProgress, initialSizeProgress);
|
||||
} else {
|
||||
scannerContext.clearProgress();
|
||||
}
|
||||
|
||||
if (rpcCall != null) {
|
||||
// If a user specifies a too-restrictive or too-slow scanner, the
|
||||
// client might time out and disconnect while the server side
|
||||
|
@ -5506,21 +5486,24 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
|
||||
boolean stopRow = isStopRow(currentRow, offset, length);
|
||||
// When has filter row is true it means that the all the cells for a particular row must be
|
||||
// read before a filtering decision can be made. This means that filters where hasFilterRow
|
||||
// run the risk of encountering out of memory errors in the case that they are applied to a
|
||||
// table that has very large rows.
|
||||
boolean hasFilterRow = this.filter != null && this.filter.hasFilterRow();
|
||||
|
||||
// If filter#hasFilterRow is true, partial results are not allowed since allowing them
|
||||
// would prevent the filters from being evaluated. Thus, if it is true, change the
|
||||
// remainingResultSize to -1 so that the entire row's worth of cells are fetched.
|
||||
if (hasFilterRow && remainingResultSize > 0) {
|
||||
remainingResultSize = -1;
|
||||
// scope of any limits that could potentially create partial results to
|
||||
// LimitScope.BETWEEN_ROWS so that those limits are not reached mid-row
|
||||
if (hasFilterRow) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("filter#hasFilterRow is true which prevents partial results from being " +
|
||||
" formed. The remainingResultSize of: " + remainingResultSize + " will not " +
|
||||
" be considered when fetching the cells for this row.");
|
||||
LOG.trace("filter#hasFilterRow is true which prevents partial results from being "
|
||||
+ " formed. Changing scope of limits that may create partials");
|
||||
}
|
||||
scannerContext.setSizeLimitScope(LimitScope.BETWEEN_ROWS);
|
||||
}
|
||||
|
||||
NextState joinedHeapState;
|
||||
// Check if we were getting data from the joinedHeap and hit the limit.
|
||||
// If not, then it's main path - getting results from storeHeap.
|
||||
if (joinedContinuationRow == null) {
|
||||
|
@ -5529,47 +5512,30 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
if (hasFilterRow) {
|
||||
filter.filterRowCells(results);
|
||||
}
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, resultSize);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
// Check if rowkey filter wants to exclude this row. If so, loop to next.
|
||||
// Technically, if we hit limits before on this row, we don't need this call.
|
||||
if (filterRowKey(currentRow, offset, length)) {
|
||||
boolean moreRows = nextRow(currentRow, offset, length);
|
||||
if (!moreRows) return NextState.makeState(NextState.State.NO_MORE_VALUES, resultSize);
|
||||
if (!moreRows) {
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
results.clear();
|
||||
continue;
|
||||
}
|
||||
|
||||
NextState storeHeapState =
|
||||
populateResult(results, this.storeHeap, batchLimit, remainingResultSize, currentRow,
|
||||
offset, length);
|
||||
resultSize += calculateResultSize(results, storeHeapState);
|
||||
// Invalid states should never be returned. If one is seen, throw exception
|
||||
// since we have no way of telling how we should proceed
|
||||
if (!NextState.isValidState(storeHeapState)) {
|
||||
throw new IOException("NextState returned from call storeHeap was invalid");
|
||||
}
|
||||
|
||||
// Ok, we are good, let's try to get some results from the main heap.
|
||||
if (storeHeapState.batchLimitReached()) {
|
||||
populateResult(results, this.storeHeap, scannerContext, currentRow, offset, length);
|
||||
|
||||
if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
|
||||
if (hasFilterRow) {
|
||||
throw new IncompatibleFilterException(
|
||||
"Filter whose hasFilterRow() returns true is incompatible with scan with limit!");
|
||||
"Filter whose hasFilterRow() returns true is incompatible with scans that must "
|
||||
+ " stop mid-row because of a limit. ScannerContext:" + scannerContext);
|
||||
}
|
||||
// We hit the batch limit.
|
||||
return NextState.makeState(NextState.State.BATCH_LIMIT_REACHED, resultSize);
|
||||
} else if (storeHeapState.sizeLimitReached()) {
|
||||
if (hasFilterRow) {
|
||||
// We try to guard against this case above when remainingResultSize is set to -1 if
|
||||
// hasFilterRow is true. In the even that the guard doesn't work, an exception must be
|
||||
// thrown
|
||||
throw new IncompatibleFilterException(
|
||||
"Filter whose hasFilterRows() returns true is incompatible with scans that"
|
||||
+ " return partial results");
|
||||
}
|
||||
// We hit the size limit.
|
||||
return NextState.makeState(NextState.State.SIZE_LIMIT_REACHED, resultSize);
|
||||
return true;
|
||||
}
|
||||
Cell nextKv = this.storeHeap.peek();
|
||||
stopRow = nextKv == null ||
|
||||
|
@ -5582,17 +5548,31 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
FilterWrapper.FilterRowRetCode ret = FilterWrapper.FilterRowRetCode.NOT_CALLED;
|
||||
if (hasFilterRow) {
|
||||
ret = filter.filterRowCellsWithRet(results);
|
||||
|
||||
// We don't know how the results have changed after being filtered. Must set progress
|
||||
// according to contents of results now.
|
||||
if (scannerContext.getKeepProgress()) {
|
||||
scannerContext.setProgress(initialBatchProgress, initialSizeProgress);
|
||||
} else {
|
||||
scannerContext.clearProgress();
|
||||
}
|
||||
scannerContext.incrementBatchProgress(results.size());
|
||||
for (Cell cell : results) {
|
||||
scannerContext.incrementSizeProgress(CellUtil.estimatedHeapSizeOfWithoutTags(cell));
|
||||
}
|
||||
}
|
||||
|
||||
if ((isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE) || filterRow()) {
|
||||
results.clear();
|
||||
boolean moreRows = nextRow(currentRow, offset, length);
|
||||
if (!moreRows) return NextState.makeState(NextState.State.NO_MORE_VALUES, 0);
|
||||
if (!moreRows) {
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
// This row was totally filtered out, if this is NOT the last row,
|
||||
// we should continue on. Otherwise, nothing else to do.
|
||||
if (!stopRow) continue;
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, 0);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
// Ok, we are done with storeHeap for this row.
|
||||
|
@ -5610,31 +5590,24 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
currentRow, offset, length));
|
||||
if (mayHaveData) {
|
||||
joinedContinuationRow = current;
|
||||
joinedHeapState =
|
||||
populateFromJoinedHeap(results, batchLimit, remainingResultSize - resultSize);
|
||||
resultSize +=
|
||||
joinedHeapState != null && joinedHeapState.hasResultSizeEstimate() ?
|
||||
joinedHeapState.getResultSize() : 0;
|
||||
if (joinedHeapState != null && joinedHeapState.sizeLimitReached()) {
|
||||
return NextState.makeState(NextState.State.SIZE_LIMIT_REACHED, resultSize);
|
||||
populateFromJoinedHeap(results, scannerContext);
|
||||
|
||||
if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Populating from the joined heap was stopped by limits, populate some more.
|
||||
joinedHeapState =
|
||||
populateFromJoinedHeap(results, batchLimit, remainingResultSize - resultSize);
|
||||
resultSize +=
|
||||
joinedHeapState != null && joinedHeapState.hasResultSizeEstimate() ?
|
||||
joinedHeapState.getResultSize() : 0;
|
||||
if (joinedHeapState != null && joinedHeapState.sizeLimitReached()) {
|
||||
return NextState.makeState(NextState.State.SIZE_LIMIT_REACHED, resultSize);
|
||||
populateFromJoinedHeap(results, scannerContext);
|
||||
if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
// We may have just called populateFromJoinedMap and hit the limits. If that is
|
||||
// the case, we need to call it again on the next next() invocation.
|
||||
if (joinedContinuationRow != null) {
|
||||
return NextState.makeState(NextState.State.MORE_VALUES, resultSize);
|
||||
return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
// Finally, we are done with both joinedHeap and storeHeap.
|
||||
|
@ -5642,15 +5615,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
// the case when SingleColumnValueExcludeFilter is used.
|
||||
if (results.isEmpty()) {
|
||||
boolean moreRows = nextRow(currentRow, offset, length);
|
||||
if (!moreRows) return NextState.makeState(NextState.State.NO_MORE_VALUES, 0);
|
||||
if (!moreRows) {
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
if (!stopRow) continue;
|
||||
}
|
||||
|
||||
// We are done. Return the result.
|
||||
if (stopRow) {
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, resultSize);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
} else {
|
||||
return NextState.makeState(NextState.State.MORE_VALUES, resultSize);
|
||||
return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -7269,7 +7244,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
boolean done;
|
||||
do {
|
||||
kvs.clear();
|
||||
done = NextState.hasMoreValues(scanner.next(kvs));
|
||||
done = scanner.next(kvs);
|
||||
if (kvs.size() > 0) LOG.info(kvs);
|
||||
} while (done);
|
||||
} finally {
|
||||
|
|
|
@ -42,218 +42,21 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
@InterfaceAudience.Private
|
||||
public interface InternalScanner extends Closeable {
|
||||
/**
|
||||
* This class encapsulates all the meaningful state information that we would like the know about
|
||||
* after a call to {@link InternalScanner#next(List)}. While this is not an enum, a restriction on
|
||||
* the possible states is implied through the exposed {@link #makeState(State)} method.
|
||||
* Grab the next row's worth of values.
|
||||
* @param results return output array
|
||||
* @return true if more rows exist after this one, false if scanner is done
|
||||
* @throws IOException e
|
||||
*/
|
||||
public static class NextState {
|
||||
/**
|
||||
* The possible states we want to restrict ourselves to. This enum is not sufficient to
|
||||
* encapsulate all of the state information since some of the fields of the state must be
|
||||
* dynamic (e.g. resultSize).
|
||||
*/
|
||||
public enum State {
|
||||
MORE_VALUES(true),
|
||||
NO_MORE_VALUES(false),
|
||||
SIZE_LIMIT_REACHED(true),
|
||||
BATCH_LIMIT_REACHED(true);
|
||||
|
||||
private boolean moreValues;
|
||||
|
||||
private State(final boolean moreValues) {
|
||||
this.moreValues = moreValues;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true when the state indicates that more values may follow those that have been
|
||||
* returned
|
||||
*/
|
||||
public boolean hasMoreValues() {
|
||||
return this.moreValues;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* state variables
|
||||
*/
|
||||
private final State state;
|
||||
private long resultSize;
|
||||
|
||||
/**
|
||||
* Value to use for resultSize when the size has not been calculated. Must be a negative number
|
||||
* so that {@link NextState#hasResultSizeEstimate()} returns false.
|
||||
*/
|
||||
private static final long DEFAULT_RESULT_SIZE = -1;
|
||||
|
||||
private NextState(State state, long resultSize) {
|
||||
this.state = state;
|
||||
this.resultSize = resultSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param state
|
||||
* @return An instance of {@link NextState} where the size of the results returned from the call
|
||||
* to {@link InternalScanner#next(List)} is unknown. It it the responsibility of the
|
||||
* caller of {@link InternalScanner#next(List)} to calculate the result size if needed
|
||||
*/
|
||||
public static NextState makeState(final State state) {
|
||||
return makeState(state, DEFAULT_RESULT_SIZE);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param state
|
||||
* @param resultSize
|
||||
* @return An instance of {@link NextState} where the size of the values returned from the call
|
||||
* to {@link InternalScanner#next(List)} is known. The caller can avoid recalculating
|
||||
* the result size by using the cached value retrievable via {@link #getResultSize()}
|
||||
*/
|
||||
public static NextState makeState(final State state, long resultSize) {
|
||||
switch (state) {
|
||||
case MORE_VALUES:
|
||||
return createMoreValuesState(resultSize);
|
||||
case NO_MORE_VALUES:
|
||||
return createNoMoreValuesState(resultSize);
|
||||
case BATCH_LIMIT_REACHED:
|
||||
return createBatchLimitReachedState(resultSize);
|
||||
case SIZE_LIMIT_REACHED:
|
||||
return createSizeLimitReachedState(resultSize);
|
||||
default:
|
||||
// If the state is not recognized, default to no more value state
|
||||
return createNoMoreValuesState(resultSize);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convenience method for creating a state that indicates that more values can be scanned
|
||||
* @param resultSize estimate of the size (heap size) of the values returned from the call to
|
||||
* {@link InternalScanner#next(List)}
|
||||
*/
|
||||
private static NextState createMoreValuesState(long resultSize) {
|
||||
return new NextState(State.MORE_VALUES, resultSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convenience method for creating a state that indicates that no more values can be scanned.
|
||||
* @param resultSize estimate of the size (heap size) of the values returned from the call to
|
||||
* {@link InternalScanner#next(List)}
|
||||
*/
|
||||
private static NextState createNoMoreValuesState(long resultSize) {
|
||||
return new NextState(State.NO_MORE_VALUES, resultSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convenience method for creating a state that indicates that the scan stopped because the
|
||||
* batch limit was exceeded
|
||||
* @param resultSize estimate of the size (heap size) of the values returned from the call to
|
||||
* {@link InternalScanner#next(List)}
|
||||
*/
|
||||
private static NextState createBatchLimitReachedState(long resultSize) {
|
||||
return new NextState(State.BATCH_LIMIT_REACHED, resultSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convenience method for creating a state that indicates that the scan stopped due to the size
|
||||
* limit
|
||||
* @param resultSize estimate of the size (heap size) of the values returned from the call to
|
||||
* {@link InternalScanner#next(List)}
|
||||
*/
|
||||
private static NextState createSizeLimitReachedState(long resultSize) {
|
||||
return new NextState(State.SIZE_LIMIT_REACHED, resultSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true when the scanner has more values to be scanned following the values returned by
|
||||
* the call to {@link InternalScanner#next(List)}
|
||||
*/
|
||||
public boolean hasMoreValues() {
|
||||
return this.state.hasMoreValues();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true when the scanner had to stop scanning because it reached the batch limit
|
||||
*/
|
||||
public boolean batchLimitReached() {
|
||||
return this.state == State.BATCH_LIMIT_REACHED;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true when the scanner had to stop scanning because it reached the size limit
|
||||
*/
|
||||
public boolean sizeLimitReached() {
|
||||
return this.state == State.SIZE_LIMIT_REACHED;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The size (heap size) of the values that were returned from the call to
|
||||
* {@link InternalScanner#next(List)}. This value should only be used if
|
||||
* {@link #hasResultSizeEstimate()} returns true.
|
||||
*/
|
||||
public long getResultSize() {
|
||||
return resultSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true when an estimate for the size of the values returned by
|
||||
* {@link InternalScanner#next(List)} was provided. If false, it is the responsibility
|
||||
* of the caller to calculate the result size
|
||||
*/
|
||||
public boolean hasResultSizeEstimate() {
|
||||
return resultSize >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "State: " + state + " resultSize: " + resultSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to centralize all checks as to whether or not the state is valid.
|
||||
* @param state
|
||||
* @return true when the state is valid
|
||||
*/
|
||||
public static boolean isValidState(NextState state) {
|
||||
return state != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param state
|
||||
* @return true when the state is non null and indicates that more values exist
|
||||
*/
|
||||
public static boolean hasMoreValues(NextState state) {
|
||||
return state != null && state.hasMoreValues();
|
||||
}
|
||||
}
|
||||
boolean next(List<Cell> results) throws IOException;
|
||||
|
||||
/**
|
||||
* Grab the next row's worth of values.
|
||||
* @param results return output array
|
||||
* @return state where {@link NextState#hasMoreValues()} is true if more rows exist after this
|
||||
* one, false if scanner is done
|
||||
* @throws IOException e
|
||||
*/
|
||||
NextState next(List<Cell> results) throws IOException;
|
||||
|
||||
/**
|
||||
* Grab the next row's worth of values with a limit on the number of values to return.
|
||||
* @param result return output array
|
||||
* @param limit limit on row count to get
|
||||
* @return state where {@link NextState#hasMoreValues()} is true if more rows exist after this
|
||||
* one, false if scanner is done
|
||||
* @param scannerContext
|
||||
* @return true if more rows exist after this one, false if scanner is done
|
||||
* @throws IOException e
|
||||
*/
|
||||
NextState next(List<Cell> result, int limit) throws IOException;
|
||||
|
||||
/**
|
||||
* Grab the next row's worth of values with a limit on the number of values to return as well as a
|
||||
* restriction on the size of the list of values that are returned.
|
||||
* @param result return output array
|
||||
* @param limit limit on row count to get
|
||||
* @param remainingResultSize limit on the size of the result being returned
|
||||
* @return state where {@link NextState#hasMoreValues()} is true if more rows exist after this
|
||||
* one, false if scanner is done
|
||||
* @throws IOException e
|
||||
*/
|
||||
NextState next(List<Cell> result, int limit, long remainingResultSize) throws IOException;
|
||||
boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException;
|
||||
|
||||
/**
|
||||
* Closes the scanner and releases any resources it has allocated
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.util.PriorityQueue;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
|
||||
|
||||
/**
|
||||
* Implements a heap merge across any number of KeyValueScanners.
|
||||
|
@ -128,26 +129,20 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
|
|||
* This can ONLY be called when you are using Scanners that implement InternalScanner as well as
|
||||
* KeyValueScanner (a {@link StoreScanner}).
|
||||
* @param result
|
||||
* @param limit
|
||||
* @return state where NextState#hasMoreValues() is true if more keys exist after this
|
||||
* one, false if scanner is done
|
||||
* @return true if more rows exist after this one, false if scanner is done
|
||||
*/
|
||||
public NextState next(List<Cell> result, int limit) throws IOException {
|
||||
return next(result, limit, -1);
|
||||
@Override
|
||||
public boolean next(List<Cell> result) throws IOException {
|
||||
return next(result, NoLimitScannerContext.getInstance());
|
||||
}
|
||||
|
||||
public NextState next(List<Cell> result, int limit, long remainingResultSize) throws IOException {
|
||||
@Override
|
||||
public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
|
||||
if (this.current == null) {
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
InternalScanner currentAsInternal = (InternalScanner)this.current;
|
||||
NextState state = currentAsInternal.next(result, limit, remainingResultSize);
|
||||
// Invalid states should never be returned. Receiving an invalid state means that we have
|
||||
// no clue how to proceed. Throw an exception.
|
||||
if (!NextState.isValidState(state)) {
|
||||
throw new IOException("Invalid state returned from InternalScanner#next");
|
||||
}
|
||||
boolean mayContainMoreRows = NextState.hasMoreValues(state);
|
||||
boolean moreCells = currentAsInternal.next(result, scannerContext);
|
||||
Cell pee = this.current.peek();
|
||||
/*
|
||||
* By definition, any InternalScanner must return false only when it has no
|
||||
|
@ -156,31 +151,16 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
|
|||
* more efficient to close scanners which are not needed than keep them in
|
||||
* the heap. This is also required for certain optimizations.
|
||||
*/
|
||||
if (pee == null || !mayContainMoreRows) {
|
||||
if (pee == null || !moreCells) {
|
||||
this.current.close();
|
||||
} else {
|
||||
this.heap.add(this.current);
|
||||
}
|
||||
this.current = pollRealKV();
|
||||
if (this.current == null) {
|
||||
state = NextState.makeState(NextState.State.NO_MORE_VALUES, state.getResultSize());
|
||||
moreCells = scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
return state;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the next row of keys from the top-most scanner.
|
||||
* <p>
|
||||
* This method takes care of updating the heap.
|
||||
* <p>
|
||||
* This can ONLY be called when you are using Scanners that implement InternalScanner as well as
|
||||
* KeyValueScanner (a {@link StoreScanner}).
|
||||
* @param result
|
||||
* @return state where NextState#hasMoreValues() is true if more keys exist after this
|
||||
* one, false if scanner is done
|
||||
*/
|
||||
public NextState next(List<Cell> result) throws IOException {
|
||||
return next(result, -1);
|
||||
return moreCells;
|
||||
}
|
||||
|
||||
protected static class KVScannerComparator implements Comparator<KeyValueScanner> {
|
||||
|
|
|
@ -0,0 +1,102 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* This is a special {@link ScannerContext} subclass that is designed to be used globally when
|
||||
* limits should not be enforced during invocations of {@link InternalScanner#next(java.util.List)}
|
||||
* or {@link RegionScanner#next(java.util.List)}.
|
||||
* <p>
|
||||
* Instances of {@link NoLimitScannerContext} are immutable after construction. Any attempt to
|
||||
* change the limits or progress of a {@link NoLimitScannerContext} will fail silently. The net
|
||||
* effect is that all limit checks will return false, thus indicating that a limit has not been
|
||||
* reached.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public class NoLimitScannerContext extends ScannerContext {
|
||||
|
||||
public NoLimitScannerContext() {
|
||||
super(false, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Use this instance whenever limits do not need to be enforced.
|
||||
*/
|
||||
private static final ScannerContext NO_LIMIT = new NoLimitScannerContext();
|
||||
|
||||
/**
|
||||
* @return The static, immutable instance of {@link NoLimitScannerContext} to be used whenever
|
||||
* limits should not be enforced
|
||||
*/
|
||||
public static final ScannerContext getInstance() {
|
||||
return NO_LIMIT;
|
||||
}
|
||||
|
||||
@Override
|
||||
void setKeepProgress(boolean keepProgress) {
|
||||
// Do nothing. NoLimitScannerContext instances are immutable post-construction
|
||||
}
|
||||
|
||||
@Override
|
||||
void setBatchProgress(int batchProgress) {
|
||||
// Do nothing. NoLimitScannerContext instances are immutable post-construction
|
||||
}
|
||||
|
||||
@Override
|
||||
void setSizeProgress(long sizeProgress) {
|
||||
// Do nothing. NoLimitScannerContext instances are immutable post-construction
|
||||
}
|
||||
|
||||
@Override
|
||||
void setProgress(int batchProgress, long sizeProgress) {
|
||||
// Do nothing. NoLimitScannerContext instances are immutable post-construction
|
||||
}
|
||||
|
||||
@Override
|
||||
void setSizeLimitScope(LimitScope scope) {
|
||||
// Do nothing. NoLimitScannerContext instances are immutable post-construction
|
||||
}
|
||||
|
||||
@Override
|
||||
NextState setScannerState(NextState state) {
|
||||
// Do nothing. NoLimitScannerContext instances are immutable post-construction
|
||||
return state;
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean checkBatchLimit(LimitScope checkerScope) {
|
||||
// No limits can be specified, thus return false to indicate no limit has been reached.
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean checkSizeLimit(LimitScope checkerScope) {
|
||||
// No limits can be specified, thus return false to indicate no limit has been reached.
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean checkAnyLimitReached(LimitScope checkerScope) {
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -105,8 +105,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
|
||||
|
@ -120,6 +118,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfiguratio
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath;
|
||||
|
@ -151,10 +151,10 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
import org.apache.hadoop.hbase.quotas.OperationQuota;
|
||||
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
|
||||
import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
|
||||
import org.apache.hadoop.hbase.regionserver.Region.Operation;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
@ -2236,61 +2236,53 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
// correct ordering of partial results and so we prevent partial results from being
|
||||
// formed.
|
||||
boolean serverGuaranteesOrderOfPartials = currentScanResultSize == 0;
|
||||
boolean enforceMaxResultSizeAtCellLevel =
|
||||
boolean allowPartialResults =
|
||||
clientHandlesPartials && serverGuaranteesOrderOfPartials && !isSmallScan;
|
||||
NextState state = null;
|
||||
boolean moreRows = false;
|
||||
|
||||
final LimitScope sizeScope =
|
||||
allowPartialResults ? LimitScope.BETWEEN_CELLS : LimitScope.BETWEEN_ROWS;
|
||||
|
||||
// Configure with limits for this RPC. Set keep progress true since size progress
|
||||
// towards size limit should be kept between calls to nextRaw
|
||||
ScannerContext.Builder contextBuilder = ScannerContext.newBuilder(true);
|
||||
contextBuilder.setSizeLimit(sizeScope, maxResultSize);
|
||||
contextBuilder.setBatchLimit(scanner.getBatch());
|
||||
ScannerContext scannerContext = contextBuilder.build();
|
||||
|
||||
while (i < rows) {
|
||||
// Stop collecting results if we have exceeded maxResultSize
|
||||
if (currentScanResultSize >= maxResultSize) {
|
||||
if (scannerContext.checkSizeLimit(LimitScope.BETWEEN_ROWS)) {
|
||||
builder.setMoreResultsInRegion(true);
|
||||
break;
|
||||
}
|
||||
|
||||
// A negative remainingResultSize communicates that there is no limit on the size
|
||||
// of the results.
|
||||
final long remainingResultSize =
|
||||
enforceMaxResultSizeAtCellLevel ? maxResultSize - currentScanResultSize
|
||||
: -1;
|
||||
// Reset the batch progress to 0 before every call to RegionScanner#nextRaw. The
|
||||
// batch limit is a limit on the number of cells per Result. Thus, if progress is
|
||||
// being tracked (i.e. scannerContext.keepProgress() is true) then we need to
|
||||
// reset the batch progress between nextRaw invocations since we don't want the
|
||||
// batch progress from previous calls to affect future calls
|
||||
scannerContext.setBatchProgress(0);
|
||||
|
||||
// Collect values to be returned here
|
||||
state = scanner.nextRaw(values, scanner.getBatch(), remainingResultSize);
|
||||
// Invalid states should never be returned. If one is seen, throw exception
|
||||
// to stop the scan -- We have no way of telling how we should proceed
|
||||
if (!NextState.isValidState(state)) {
|
||||
throw new IOException("NextState returned from call to nextRaw was invalid");
|
||||
}
|
||||
if (!values.isEmpty()) {
|
||||
// The state should always contain an estimate of the result size because that
|
||||
// estimate must be used to decide when partial results are formed.
|
||||
boolean skipResultSizeCalculation = state.hasResultSizeEstimate();
|
||||
if (skipResultSizeCalculation) currentScanResultSize += state.getResultSize();
|
||||
moreRows = scanner.nextRaw(values, scannerContext);
|
||||
|
||||
if (!values.isEmpty()) {
|
||||
for (Cell cell : values) {
|
||||
totalCellSize += CellUtil.estimatedSerializedSizeOf(cell);
|
||||
|
||||
// If the calculation can't be skipped, then do it now.
|
||||
if (!skipResultSizeCalculation) {
|
||||
currentScanResultSize += CellUtil.estimatedHeapSizeOfWithoutTags(cell);
|
||||
}
|
||||
}
|
||||
// The size limit was reached. This means there are more cells remaining in
|
||||
// the row but we had to stop because we exceeded our max result size. This
|
||||
// indicates that we are returning a partial result
|
||||
final boolean partial = state != null && state.sizeLimitReached();
|
||||
final boolean partial = scannerContext.partialResultFormed();
|
||||
results.add(Result.create(values, null, stale, partial));
|
||||
i++;
|
||||
}
|
||||
if (!NextState.hasMoreValues(state)) {
|
||||
if (!moreRows) {
|
||||
break;
|
||||
}
|
||||
values.clear();
|
||||
}
|
||||
// currentScanResultSize >= maxResultSize should be functionally equivalent to
|
||||
// state.sizeLimitReached()
|
||||
if (null != state
|
||||
&& (currentScanResultSize >= maxResultSize || i >= rows || state
|
||||
.hasMoreValues())) {
|
||||
|
||||
if (scannerContext.checkSizeLimit(LimitScope.BETWEEN_ROWS) || i >= rows ||
|
||||
moreRows) {
|
||||
// We stopped prematurely
|
||||
builder.setMoreResultsInRegion(true);
|
||||
} else {
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
|
|||
* RegionScanner describes iterators over rows in an HRegion.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Stable
|
||||
@InterfaceStability.Evolving
|
||||
public interface RegionScanner extends InternalScanner {
|
||||
/**
|
||||
* @return The RegionInfo for this scanner.
|
||||
|
@ -74,35 +74,22 @@ public interface RegionScanner extends InternalScanner {
|
|||
int getBatch();
|
||||
|
||||
/**
|
||||
* Grab the next row's worth of values with the default limit on the number of values to return.
|
||||
* This is a special internal method to be called from coprocessor hooks to avoid expensive setup.
|
||||
* Caller must set the thread's readpoint, start and close a region operation, an synchronize on
|
||||
* the scanner object. Caller should maintain and update metrics. See
|
||||
* {@link #nextRaw(List, int, long)}
|
||||
* Grab the next row's worth of values. This is a special internal method to be called from
|
||||
* coprocessor hooks to avoid expensive setup. Caller must set the thread's readpoint, start and
|
||||
* close a region operation, an synchronize on the scanner object. Caller should maintain and
|
||||
* update metrics. See {@link #nextRaw(List, ScannerContext)}
|
||||
* @param result return output array
|
||||
* @return a state where NextState#hasMoreValues() is true when more rows exist, false when
|
||||
* scanner is done.
|
||||
* @return true if more rows exist after this one, false if scanner is done
|
||||
* @throws IOException e
|
||||
*/
|
||||
NextState nextRaw(List<Cell> result) throws IOException;
|
||||
boolean nextRaw(List<Cell> result) throws IOException;
|
||||
|
||||
/**
|
||||
* Grab the next row's worth of values with the default limit on the number of values to return.
|
||||
* This is a special internal method to be called from coprocessor hooks to avoid expensive setup.
|
||||
* Caller must set the thread's readpoint, start and close a region operation, an synchronize on
|
||||
* the scanner object. Caller should maintain and update metrics. See
|
||||
* {@link #nextRaw(List, int, long)}
|
||||
* @param result return output array
|
||||
* @param limit limit on row count to get
|
||||
* @return a state where NextState#hasMoreValues() is true when more rows exist, false when
|
||||
* scanner is done.
|
||||
* @throws IOException e
|
||||
*/
|
||||
NextState nextRaw(List<Cell> result, int limit) throws IOException;
|
||||
|
||||
/**
|
||||
* Grab the next row's worth of values with a limit on the number of values to return as well as a
|
||||
* limit on the heap size of those values. This is a special internal method to be called from
|
||||
* Grab the next row's worth of values. The {@link ScannerContext} is used to enforce and track
|
||||
* any limits associated with this call. Any progress that exists in the {@link ScannerContext}
|
||||
* prior to calling this method will be LOST if {@link ScannerContext#getKeepProgress()} is false.
|
||||
* Upon returning from this method, the {@link ScannerContext} will contain information about the
|
||||
* progress made towards the limits. This is a special internal method to be called from
|
||||
* coprocessor hooks to avoid expensive setup. Caller must set the thread's readpoint, start and
|
||||
* close a region operation, an synchronize on the scanner object. Example: <code><pre>
|
||||
* HRegion region = ...;
|
||||
|
@ -120,13 +107,12 @@ public interface RegionScanner extends InternalScanner {
|
|||
* }
|
||||
* </pre></code>
|
||||
* @param result return output array
|
||||
* @param limit limit on row count to get
|
||||
* @param remainingResultSize the space remaining within the restriction on the result size.
|
||||
* Negative values indicate no limit
|
||||
* @return a state where NextState#hasMoreValues() is true when more rows exist, false when
|
||||
* scanner is done.
|
||||
* @param scannerContext The {@link ScannerContext} instance encapsulating all limits that should
|
||||
* be tracked during calls to this method. The progress towards these limits can be
|
||||
* tracked within this instance.
|
||||
* @return true if more rows exist after this one, false if scanner is done
|
||||
* @throws IOException e
|
||||
*/
|
||||
NextState nextRaw(List<Cell> result, int limit, final long remainingResultSize)
|
||||
boolean nextRaw(List<Cell> result, ScannerContext scannerContext)
|
||||
throws IOException;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,527 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* ScannerContext instances encapsulate limit tracking AND progress towards those limits during
|
||||
* invocations of {@link InternalScanner#next(java.util.List)} and
|
||||
* {@link RegionScanner#next(java.util.List)}.
|
||||
* <p>
|
||||
* A ScannerContext instance should be updated periodically throughout execution whenever progress
|
||||
* towards a limit has been made. Each limit can be checked via the appropriate checkLimit method.
|
||||
* <p>
|
||||
* Once a limit has been reached, the scan will stop. The invoker of
|
||||
* {@link InternalScanner#next(java.util.List)} or {@link RegionScanner#next(java.util.List)} can
|
||||
* use the appropriate check*Limit methods to see exactly which limits have been reached.
|
||||
* Alternatively, {@link #checkAnyLimitReached(LimitScope)} is provided to see if ANY limit was
|
||||
* reached
|
||||
* <p>
|
||||
* {@link NoLimitScannerContext#NO_LIMIT} is an immutable static definition that can be used
|
||||
* whenever a {@link ScannerContext} is needed but limits do not need to be enforced.
|
||||
* <p>
|
||||
* NOTE: It is important that this class only ever expose setter methods that can be safely skipped
|
||||
* when limits should be NOT enforced. This is because of the necessary immutability of the class
|
||||
* {@link NoLimitScannerContext}. If a setter cannot be safely skipped, the immutable nature of
|
||||
* {@link NoLimitScannerContext} will lead to incorrect behavior.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public class ScannerContext {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass());
|
||||
|
||||
/**
|
||||
* Two sets of the same fields. One for the limits, another for the progress towards those limits
|
||||
*/
|
||||
LimitFields limits;
|
||||
LimitFields progress;
|
||||
|
||||
/**
|
||||
* The state of the scanner after the invocation of {@link InternalScanner#next(java.util.List)}
|
||||
* or {@link RegionScanner#next(java.util.List)}.
|
||||
*/
|
||||
NextState scannerState;
|
||||
private static final NextState DEFAULT_STATE = NextState.MORE_VALUES;
|
||||
|
||||
/**
|
||||
* Used as an indication to invocations of {@link InternalScanner#next(java.util.List)} and
|
||||
* {@link RegionScanner#next(java.util.List)} that, if true, the progress tracked within this
|
||||
* {@link ScannerContext} instance should be considered while evaluating the limits. Useful for
|
||||
* enforcing a set of limits across multiple calls (i.e. the limit may not be reached in a single
|
||||
* invocation, but any progress made should be considered in future invocations)
|
||||
* <p>
|
||||
* Defaulting this value to false means that, by default, any tracked progress will be wiped clean
|
||||
* on invocations to {@link InternalScanner#next(java.util.List)} and
|
||||
* {@link RegionScanner#next(java.util.List)} and the call will be treated as though no progress
|
||||
* has been made towards the limits so far.
|
||||
* <p>
|
||||
* This is an important mechanism. Users of Internal/Region scanners expect that they can define
|
||||
* some limits and then repeatedly invoke {@link InternalScanner#next(List)} or
|
||||
* {@link RegionScanner#next(List)} where each invocation respects these limits separately.
|
||||
* <p>
|
||||
* For example: <code><pre>
|
||||
* ScannerContext context = new ScannerContext.newBuilder().setBatchLimit(5).build();
|
||||
* RegionScanner scanner = ...
|
||||
* List<Cell> results = new ArrayList<Cell>();
|
||||
* while(scanner.next(results, context)) {
|
||||
* // Do something with a batch of 5 cells
|
||||
* }
|
||||
* </pre></code> However, in the case of RPCs, the server wants to be able to define a set of
|
||||
* limits for a particular RPC request and have those limits respected across multiple
|
||||
* invocations. This means that the progress made towards the limits in earlier calls will be
|
||||
* saved and considered in future invocations
|
||||
*/
|
||||
boolean keepProgress;
|
||||
private static boolean DEFAULT_KEEP_PROGRESS = false;
|
||||
|
||||
ScannerContext(boolean keepProgress, LimitFields limitsToCopy) {
|
||||
this.limits = new LimitFields();
|
||||
if (limitsToCopy != null) this.limits.copy(limitsToCopy);
|
||||
|
||||
// Progress fields are initialized to 0
|
||||
progress = new LimitFields(0, LimitFields.DEFAULT_SCOPE, 0);
|
||||
|
||||
this.keepProgress = keepProgress;
|
||||
this.scannerState = DEFAULT_STATE;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the progress tracked so far in this instance will be considered during an
|
||||
* invocation of {@link InternalScanner#next(java.util.List)} or
|
||||
* {@link RegionScanner#next(java.util.List)}. false when the progress tracked so far
|
||||
* should not be considered and should instead be wiped away via {@link #clearProgress()}
|
||||
*/
|
||||
boolean getKeepProgress() {
|
||||
return keepProgress;
|
||||
}
|
||||
|
||||
void setKeepProgress(boolean keepProgress) {
|
||||
this.keepProgress = keepProgress;
|
||||
}
|
||||
|
||||
/**
|
||||
* Progress towards the batch limit has been made. Increment internal tracking of batch progress
|
||||
*/
|
||||
void incrementBatchProgress(int batch) {
|
||||
int currentBatch = progress.getBatch();
|
||||
progress.setBatch(currentBatch + batch);
|
||||
}
|
||||
|
||||
/**
|
||||
* Progress towards the size limit has been made. Increment internal tracking of size progress
|
||||
*/
|
||||
void incrementSizeProgress(long size) {
|
||||
long currentSize = progress.getSize();
|
||||
progress.setSize(currentSize + size);
|
||||
}
|
||||
|
||||
int getBatchProgress() {
|
||||
return progress.getBatch();
|
||||
}
|
||||
|
||||
long getSizeProgress() {
|
||||
return progress.getSize();
|
||||
}
|
||||
|
||||
void setProgress(int batchProgress, long sizeProgress) {
|
||||
setBatchProgress(batchProgress);
|
||||
setSizeProgress(sizeProgress);
|
||||
}
|
||||
|
||||
void setSizeProgress(long sizeProgress) {
|
||||
progress.setSize(sizeProgress);
|
||||
}
|
||||
|
||||
void setBatchProgress(int batchProgress) {
|
||||
progress.setBatch(batchProgress);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear away any progress that has been made so far. All progress fields are reset to initial
|
||||
* values
|
||||
*/
|
||||
void clearProgress() {
|
||||
progress.setFields(0, LimitFields.DEFAULT_SCOPE, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Note that this is not a typical setter. This setter returns the {@link NextState} that was
|
||||
* passed in so that methods can be invoked against the new state. Furthermore, this pattern
|
||||
* allows the {@link NoLimitScannerContext} to cleanly override this setter and simply return the
|
||||
* new state, thus preserving the immutability of {@link NoLimitScannerContext}
|
||||
* @param state
|
||||
* @return The state that
|
||||
*/
|
||||
NextState setScannerState(NextState state) {
|
||||
if (!NextState.isValidState(state)) {
|
||||
throw new IllegalArgumentException("Cannot set to invalid state: " + state);
|
||||
}
|
||||
|
||||
this.scannerState = state;
|
||||
return state;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true when a partial result is formed. A partial result is formed when a limit is
|
||||
* reached in the middle of a row.
|
||||
*/
|
||||
boolean partialResultFormed() {
|
||||
return scannerState == NextState.SIZE_LIMIT_REACHED_MID_ROW;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param checkerScope
|
||||
* @return true if the batch limit can be enforced in the checker's scope
|
||||
*/
|
||||
boolean hasBatchLimit(LimitScope checkerScope) {
|
||||
return limits.canEnforceBatchLimitFromScope(checkerScope) && limits.getBatch() > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param checkerScope
|
||||
* @return true if the size limit can be enforced in the checker's scope
|
||||
*/
|
||||
boolean hasSizeLimit(LimitScope checkerScope) {
|
||||
return limits.canEnforceSizeLimitFromScope(checkerScope) && limits.getSize() > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param checkerScope
|
||||
* @return true if any limit can be enforced within the checker's scope
|
||||
*/
|
||||
boolean hasAnyLimit(LimitScope checkerScope) {
|
||||
return hasBatchLimit(checkerScope) || hasSizeLimit(checkerScope);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param scope The scope in which the size limit will be enforced
|
||||
*/
|
||||
void setSizeLimitScope(LimitScope scope) {
|
||||
limits.setSizeScope(scope);
|
||||
}
|
||||
|
||||
int getBatchLimit() {
|
||||
return limits.getBatch();
|
||||
}
|
||||
|
||||
long getSizeLimit() {
|
||||
return limits.getSize();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param checkerScope The scope that the limit is being checked from
|
||||
* @return true when the limit is enforceable from the checker's scope and it has been reached
|
||||
*/
|
||||
boolean checkBatchLimit(LimitScope checkerScope) {
|
||||
return hasBatchLimit(checkerScope) && progress.getBatch() >= limits.getBatch();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param checkerScope The scope that the limit is being checked from
|
||||
* @return true when the limit is enforceable from the checker's scope and it has been reached
|
||||
*/
|
||||
boolean checkSizeLimit(LimitScope checkerScope) {
|
||||
return hasSizeLimit(checkerScope) && progress.getSize() >= limits.getSize();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param checkerScope The scope that the limits are being checked from
|
||||
* @return true when some limit is enforceable from the checker's scope and it has been reached
|
||||
*/
|
||||
boolean checkAnyLimitReached(LimitScope checkerScope) {
|
||||
return checkSizeLimit(checkerScope) || checkBatchLimit(checkerScope);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("{");
|
||||
|
||||
sb.append("limits:");
|
||||
sb.append(limits);
|
||||
|
||||
sb.append(", progress:");
|
||||
sb.append(progress);
|
||||
|
||||
sb.append(", keepProgress:");
|
||||
sb.append(keepProgress);
|
||||
|
||||
sb.append(", state:");
|
||||
sb.append(scannerState);
|
||||
|
||||
sb.append("}");
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public static Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static Builder newBuilder(boolean keepProgress) {
|
||||
return new Builder(keepProgress);
|
||||
}
|
||||
|
||||
public static final class Builder {
|
||||
boolean keepProgress = DEFAULT_KEEP_PROGRESS;
|
||||
LimitFields limits = new LimitFields();
|
||||
|
||||
private Builder() {
|
||||
}
|
||||
|
||||
private Builder(boolean keepProgress) {
|
||||
this.keepProgress = keepProgress;
|
||||
}
|
||||
|
||||
public Builder setKeepProgress(boolean keepProgress) {
|
||||
this.keepProgress = keepProgress;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setSizeLimit(LimitScope sizeScope, long sizeLimit) {
|
||||
limits.setSize(sizeLimit);
|
||||
limits.setSizeScope(sizeScope);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setBatchLimit(int batchLimit) {
|
||||
limits.setBatch(batchLimit);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ScannerContext build() {
|
||||
return new ScannerContext(keepProgress, limits);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The possible states a scanner may be in following a call to {@link InternalScanner#next(List)}
|
||||
*/
|
||||
public enum NextState {
|
||||
MORE_VALUES(true, false),
|
||||
NO_MORE_VALUES(false, false),
|
||||
SIZE_LIMIT_REACHED(true, true),
|
||||
|
||||
/**
|
||||
* Special case of size limit reached to indicate that the size limit was reached in the middle
|
||||
* of a row and thus a partial results was formed
|
||||
*/
|
||||
SIZE_LIMIT_REACHED_MID_ROW(true, true),
|
||||
BATCH_LIMIT_REACHED(true, true);
|
||||
|
||||
private boolean moreValues;
|
||||
private boolean limitReached;
|
||||
|
||||
private NextState(boolean moreValues, boolean limitReached) {
|
||||
this.moreValues = moreValues;
|
||||
this.limitReached = limitReached;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true when the state indicates that more values may follow those that have been
|
||||
* returned
|
||||
*/
|
||||
public boolean hasMoreValues() {
|
||||
return this.moreValues;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true when the state indicates that a limit has been reached and scan should stop
|
||||
*/
|
||||
public boolean limitReached() {
|
||||
return this.limitReached;
|
||||
}
|
||||
|
||||
public static boolean isValidState(NextState state) {
|
||||
return state != null;
|
||||
}
|
||||
|
||||
public static boolean hasMoreValues(NextState state) {
|
||||
return isValidState(state) && state.hasMoreValues();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The various scopes where a limit can be enforced. Used to differentiate when a limit should be
|
||||
* enforced or not.
|
||||
*/
|
||||
public enum LimitScope {
|
||||
/**
|
||||
* Enforcing a limit between rows means that the limit will not be considered until all the
|
||||
* cells for a particular row have been retrieved
|
||||
*/
|
||||
BETWEEN_ROWS(0),
|
||||
|
||||
/**
|
||||
* Enforcing a limit between cells means that the limit will be considered after each full cell
|
||||
* has been retrieved
|
||||
*/
|
||||
BETWEEN_CELLS(1);
|
||||
|
||||
/**
|
||||
* When enforcing a limit, we must check that the scope is appropriate for enforcement.
|
||||
* <p>
|
||||
* To communicate this concept, each scope has a depth. A limit will be enforced if the depth of
|
||||
* the checker's scope is less than or equal to the limit's scope. This means that when checking
|
||||
* limits, the checker must know their own scope (i.e. are they checking the limits between
|
||||
* rows, between cells, etc...)
|
||||
*/
|
||||
int depth;
|
||||
|
||||
LimitScope(int depth) {
|
||||
this.depth = depth;
|
||||
}
|
||||
|
||||
int depth() {
|
||||
return depth;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param checkerScope The scope in which the limit is being checked
|
||||
* @return true when the checker is in a scope that indicates the limit can be enforced. Limits
|
||||
* can be enforced from "higher or equal" scopes (i.e. the checker's scope is at a
|
||||
* lesser depth than the limit)
|
||||
*/
|
||||
boolean canEnforceLimitFromScope(LimitScope checkerScope) {
|
||||
return checkerScope != null && checkerScope.depth() <= depth;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The different fields that can be used as limits in calls to
|
||||
* {@link InternalScanner#next(java.util.List)} and {@link RegionScanner#next(java.util.List)}
|
||||
*/
|
||||
private static class LimitFields {
|
||||
/**
|
||||
* Default values of the limit fields. Defined such that if a field does NOT change from its
|
||||
* default, it will not be enforced
|
||||
*/
|
||||
private static int DEFAULT_BATCH = -1;
|
||||
private static long DEFAULT_SIZE = -1L;
|
||||
|
||||
/**
|
||||
* Default scope that is assigned to a limit if a scope is not specified.
|
||||
*/
|
||||
private static final LimitScope DEFAULT_SCOPE = LimitScope.BETWEEN_ROWS;
|
||||
|
||||
// The batch limit will always be enforced between cells, thus, there isn't a field to hold the
|
||||
// batch scope
|
||||
int batch = DEFAULT_BATCH;
|
||||
|
||||
LimitScope sizeScope = DEFAULT_SCOPE;
|
||||
long size = DEFAULT_SIZE;
|
||||
|
||||
/**
|
||||
* Fields keep their default values.
|
||||
*/
|
||||
LimitFields() {
|
||||
}
|
||||
|
||||
LimitFields(int batch, LimitScope sizeScope, long size) {
|
||||
setFields(batch, sizeScope, size);
|
||||
}
|
||||
|
||||
void copy(LimitFields limitsToCopy) {
|
||||
if (limitsToCopy != null) {
|
||||
setFields(limitsToCopy.getBatch(), limitsToCopy.getSizeScope(), limitsToCopy.getSize());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Set all fields together.
|
||||
* @param batch
|
||||
* @param sizeScope
|
||||
* @param size
|
||||
*/
|
||||
void setFields(int batch, LimitScope sizeScope, long size) {
|
||||
setBatch(batch);
|
||||
setSizeScope(sizeScope);
|
||||
setSize(size);
|
||||
}
|
||||
|
||||
int getBatch() {
|
||||
return this.batch;
|
||||
}
|
||||
|
||||
void setBatch(int batch) {
|
||||
this.batch = batch;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param checkerScope
|
||||
* @return true when the limit can be enforced from the scope of the checker
|
||||
*/
|
||||
boolean canEnforceBatchLimitFromScope(LimitScope checkerScope) {
|
||||
return LimitScope.BETWEEN_CELLS.canEnforceLimitFromScope(checkerScope);
|
||||
}
|
||||
|
||||
long getSize() {
|
||||
return this.size;
|
||||
}
|
||||
|
||||
void setSize(long size) {
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return {@link LimitScope} indicating scope in which the size limit is enforced
|
||||
*/
|
||||
LimitScope getSizeScope() {
|
||||
return this.sizeScope;
|
||||
}
|
||||
|
||||
/**
|
||||
* Change the scope in which the size limit is enforced
|
||||
*/
|
||||
void setSizeScope(LimitScope scope) {
|
||||
this.sizeScope = scope;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param checkerScope
|
||||
* @return true when the limit can be enforced from the scope of the checker
|
||||
*/
|
||||
boolean canEnforceSizeLimitFromScope(LimitScope checkerScope) {
|
||||
return this.sizeScope.canEnforceLimitFromScope(checkerScope);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("{");
|
||||
|
||||
sb.append("batch:");
|
||||
sb.append(batch);
|
||||
|
||||
sb.append(", size:");
|
||||
sb.append(size);
|
||||
|
||||
sb.append(", sizeScope:");
|
||||
sb.append(sizeScope);
|
||||
|
||||
sb.append("}");
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
|
||||
|
||||
/**
|
||||
|
@ -110,10 +109,14 @@ abstract class StoreFlusher {
|
|||
Compactor.CellSink sink, long smallestReadPoint) throws IOException {
|
||||
int compactionKVMax =
|
||||
conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
|
||||
|
||||
ScannerContext scannerContext =
|
||||
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
||||
|
||||
List<Cell> kvs = new ArrayList<Cell>();
|
||||
boolean hasMore;
|
||||
do {
|
||||
hasMore = NextState.hasMoreValues(scanner.next(kvs, compactionKVMax));
|
||||
hasMore = scanner.next(kvs, scannerContext);
|
||||
if (!kvs.isEmpty()) {
|
||||
for (Cell c : kvs) {
|
||||
// If we know that this KV is going to be included always, then let us
|
||||
|
|
|
@ -43,6 +43,8 @@ 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;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -442,45 +444,39 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the next row of values from this Store.
|
||||
* @param outResult
|
||||
* @param limit
|
||||
* @return true if there are more rows, false if scanner is done
|
||||
*/
|
||||
@Override
|
||||
public NextState next(List<Cell> outResult, int limit) throws IOException {
|
||||
// -1 means no limit
|
||||
return next(outResult, limit, -1);
|
||||
public boolean next(List<Cell> outResult) throws IOException {
|
||||
return next(outResult, NoLimitScannerContext.getInstance());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the next row of values from this Store.
|
||||
* @param outResult
|
||||
* @param limit
|
||||
* @param remainingResultSize
|
||||
* @param scannerContext
|
||||
* @return true if there are more rows, false if scanner is done
|
||||
*/
|
||||
@Override
|
||||
public NextState next(List<Cell> outResult, int limit, long remainingResultSize)
|
||||
throws IOException {
|
||||
public boolean next(List<Cell> outResult, ScannerContext scannerContext) throws IOException {
|
||||
lock.lock();
|
||||
try {
|
||||
if (scannerContext == null) {
|
||||
throw new IllegalArgumentException("Scanner context cannot be null");
|
||||
}
|
||||
if (checkReseek()) {
|
||||
return NextState.makeState(NextState.State.MORE_VALUES, 0);
|
||||
return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
// if the heap was left null, then the scanners had previously run out anyways, close and
|
||||
// return.
|
||||
if (this.heap == null) {
|
||||
close();
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, 0);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
Cell peeked = this.heap.peek();
|
||||
if (peeked == null) {
|
||||
close();
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, 0);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
// only call setRow if the row changes; avoids confusing the query matcher
|
||||
|
@ -489,16 +485,18 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
int offset = peeked.getRowOffset();
|
||||
short length = peeked.getRowLength();
|
||||
|
||||
// If limit < 0 and remainingResultSize < 0 we can skip the row comparison because we know
|
||||
// the row has changed. Else it is possible we are still traversing the same row so we
|
||||
// must perform the row comparison.
|
||||
if ((limit < 0 && remainingResultSize < 0) || matcher.row == null
|
||||
|| !Bytes.equals(row, offset, length, matcher.row, matcher.rowOffset,
|
||||
matcher.rowLength)) {
|
||||
this.countPerRow = 0;
|
||||
matcher.setRow(row, offset, length);
|
||||
// If no limits exists in the scope LimitScope.Between_Cells then we are sure we are changing
|
||||
// rows. Else it is possible we are still traversing the same row so we must perform the row
|
||||
// comparison.
|
||||
if (!scannerContext.hasAnyLimit(LimitScope.BETWEEN_CELLS) || matcher.row == null ||
|
||||
!Bytes.equals(row, offset, length, matcher.row, matcher.rowOffset, matcher.rowLength)) {
|
||||
this.countPerRow = 0;
|
||||
matcher.setRow(row, offset, length);
|
||||
}
|
||||
|
||||
// Clear progress away unless invoker has indicated it should be kept.
|
||||
if (!scannerContext.getKeepProgress()) scannerContext.clearProgress();
|
||||
|
||||
Cell cell;
|
||||
|
||||
// Only do a sanity-check if store and comparator are available.
|
||||
|
@ -507,7 +505,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
|
||||
int count = 0;
|
||||
long totalBytesRead = 0;
|
||||
long totalHeapSize = 0;
|
||||
|
||||
LOOP: while((cell = this.heap.peek()) != null) {
|
||||
if (prevCell != cell) ++kvsScanned; // Do object compare - we set prevKV from the same heap.
|
||||
|
@ -532,7 +529,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
this.countPerRow > (storeLimit + storeOffset)) {
|
||||
// do what SEEK_NEXT_ROW does.
|
||||
if (!matcher.moreRowsMayExistAfter(cell)) {
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
seekToNextRow(cell);
|
||||
break LOOP;
|
||||
|
@ -542,9 +539,15 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
// also update metric accordingly
|
||||
if (this.countPerRow > storeOffset) {
|
||||
outResult.add(cell);
|
||||
|
||||
// Update local tracking information
|
||||
count++;
|
||||
totalBytesRead += CellUtil.estimatedSerializedSizeOf(cell);
|
||||
totalHeapSize += CellUtil.estimatedHeapSizeOfWithoutTags(cell);
|
||||
|
||||
// Update the progress of the scanner context
|
||||
scannerContext.incrementSizeProgress(CellUtil.estimatedHeapSizeOfWithoutTags(cell));
|
||||
scannerContext.incrementBatchProgress(1);
|
||||
|
||||
if (totalBytesRead > maxRowSize) {
|
||||
throw new RowTooBigException("Max row size allowed: " + maxRowSize
|
||||
+ ", but the row is bigger than that.");
|
||||
|
@ -553,7 +556,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
|
||||
if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
|
||||
if (!matcher.moreRowsMayExistAfter(cell)) {
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
seekToNextRow(cell);
|
||||
} else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
|
||||
|
@ -562,26 +565,26 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
this.heap.next();
|
||||
}
|
||||
|
||||
if (limit > 0 && (count == limit)) {
|
||||
if (scannerContext.checkBatchLimit(LimitScope.BETWEEN_CELLS)) {
|
||||
break LOOP;
|
||||
}
|
||||
if (remainingResultSize > 0 && (totalHeapSize >= remainingResultSize)) {
|
||||
if (scannerContext.checkSizeLimit(LimitScope.BETWEEN_CELLS)) {
|
||||
break LOOP;
|
||||
}
|
||||
continue;
|
||||
|
||||
case DONE:
|
||||
return NextState.makeState(NextState.State.MORE_VALUES, totalHeapSize);
|
||||
return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
|
||||
|
||||
case DONE_SCAN:
|
||||
close();
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
|
||||
case SEEK_NEXT_ROW:
|
||||
// This is just a relatively simple end of scan fix, to short-cut end
|
||||
// us if there is an endKey in the scan.
|
||||
if (!matcher.moreRowsMayExistAfter(cell)) {
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
seekToNextRow(cell);
|
||||
|
@ -611,12 +614,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
}
|
||||
|
||||
if (count > 0) {
|
||||
return NextState.makeState(NextState.State.MORE_VALUES, totalHeapSize);
|
||||
return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
|
||||
}
|
||||
|
||||
// No more keys
|
||||
close();
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize);
|
||||
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
|
@ -655,11 +658,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
return qcode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> outResult) throws IOException {
|
||||
return next(outResult, -1);
|
||||
}
|
||||
|
||||
// Implementation of ChangedReadersObserver
|
||||
@Override
|
||||
public void updateReaders() throws IOException {
|
||||
|
|
|
@ -38,8 +38,8 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
|
|||
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||
|
@ -246,10 +246,13 @@ public abstract class Compactor {
|
|||
store.getRegionInfo().getRegionNameAsString() + "#" + store.getFamily().getNameAsString();
|
||||
long now = 0;
|
||||
boolean hasMore;
|
||||
ScannerContext scannerContext =
|
||||
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
||||
|
||||
throughputController.start(compactionName);
|
||||
try {
|
||||
do {
|
||||
hasMore = NextState.hasMoreValues(scanner.next(cells, compactionKVMax));
|
||||
hasMore = scanner.next(cells, scannerContext);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
now = EnvironmentEdgeManager.currentTime();
|
||||
}
|
||||
|
|
|
@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -369,7 +368,7 @@ public class AccessControlLists {
|
|||
while (true) {
|
||||
List<Cell> row = new ArrayList<Cell>();
|
||||
|
||||
boolean hasNext = NextState.hasMoreValues(iScanner.next(row));
|
||||
boolean hasNext = iScanner.next(row);
|
||||
ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
|
||||
byte[] entry = null;
|
||||
for (Cell kv : row) {
|
||||
|
|
|
@ -87,11 +87,11 @@ import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
|
||||
|
@ -807,10 +807,12 @@ public class AccessController extends BaseMasterAndRegionObserver
|
|||
boolean foundColumn = false;
|
||||
try {
|
||||
boolean more = false;
|
||||
ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(1).build();
|
||||
|
||||
do {
|
||||
cells.clear();
|
||||
// scan with limit as 1 to hold down memory use on wide rows
|
||||
more = NextState.hasMoreValues(scanner.next(cells, 1));
|
||||
more = scanner.next(cells, scannerContext);
|
||||
for (Cell cell: cells) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Found cell " + cell);
|
||||
|
|
|
@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSTableDescriptors;
|
||||
|
@ -568,7 +567,7 @@ public abstract class HBaseTestCase extends TestCase {
|
|||
@Override
|
||||
public boolean next(List<Cell> results)
|
||||
throws IOException {
|
||||
return NextState.hasMoreValues(scanner.next(results));
|
||||
return scanner.next(results);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -409,6 +409,7 @@ public class TestPartialResultsFromClientSide {
|
|||
scan.setBatch(batch);
|
||||
ResultScanner scanner = TABLE.getScanner(scan);
|
||||
Result result = scanner.next();
|
||||
int repCount = 0;
|
||||
|
||||
while ((result = scanner.next()) != null) {
|
||||
assertTrue(result.rawCells() != null);
|
||||
|
@ -416,11 +417,12 @@ public class TestPartialResultsFromClientSide {
|
|||
if (result.isPartial()) {
|
||||
final String error =
|
||||
"Cells:" + result.rawCells().length + " Batch size:" + batch
|
||||
+ " cellsPerPartialResult:" + cellsPerPartialResult;
|
||||
+ " cellsPerPartialResult:" + cellsPerPartialResult + " rep:" + repCount;
|
||||
assertTrue(error, result.rawCells().length <= Math.min(batch, cellsPerPartialResult));
|
||||
} else {
|
||||
assertTrue(result.rawCells().length <= batch);
|
||||
}
|
||||
repCount++;
|
||||
}
|
||||
|
||||
scanner.close();
|
||||
|
@ -458,7 +460,7 @@ public class TestPartialResultsFromClientSide {
|
|||
do {
|
||||
partialResult = partialScanner.next();
|
||||
partials.add(partialResult);
|
||||
} while (partialResult.isPartial());
|
||||
} while (partialResult != null && partialResult.isPartial());
|
||||
|
||||
completeResult = Result.createCompleteResult(partials);
|
||||
oneShotResult = oneShotScanner.next();
|
||||
|
@ -696,7 +698,7 @@ public class TestPartialResultsFromClientSide {
|
|||
LOG.info("r2: " + r2);
|
||||
}
|
||||
|
||||
final String failureMessage = "Results r1:" + r1 + " r2:" + r2 + " are not equivalent";
|
||||
final String failureMessage = "Results r1:" + r1 + " \nr2:" + r2 + " are not equivalent";
|
||||
if (r1 == null && r2 == null) fail(failureMessage);
|
||||
else if (r1 == null || r2 == null) fail(failureMessage);
|
||||
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.HTestConst;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -94,7 +93,7 @@ public class TestIntraRowPagination {
|
|||
RegionScanner scanner = region.getScanner(scan);
|
||||
List<Cell> kvListScan = new ArrayList<Cell>();
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(scanner.next(results)) || !results.isEmpty()) {
|
||||
while (scanner.next(results) || !results.isEmpty()) {
|
||||
kvListScan.addAll(results);
|
||||
results.clear();
|
||||
}
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.hadoop.hbase.RegionLocations;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
|
@ -659,25 +660,34 @@ public class TestReplicasClient {
|
|||
private void runMultipleScansOfOneType(boolean reversed, boolean small) throws Exception {
|
||||
openRegion(hriSecondary);
|
||||
int NUMROWS = 100;
|
||||
int NUMCOLS = 10;
|
||||
try {
|
||||
for (int i = 0; i < NUMROWS; i++) {
|
||||
byte[] b1 = Bytes.toBytes("testUseRegionWithReplica" + i);
|
||||
Put p = new Put(b1);
|
||||
p.add(f, b1, b1);
|
||||
table.put(p);
|
||||
for (int col = 0; col < NUMCOLS; col++) {
|
||||
Put p = new Put(b1);
|
||||
String qualifier = "qualifer" + col;
|
||||
KeyValue kv = new KeyValue(b1, f, qualifier.getBytes());
|
||||
p.add(kv);
|
||||
table.put(p);
|
||||
}
|
||||
}
|
||||
LOG.debug("PUT done");
|
||||
int caching = 20;
|
||||
long maxResultSize = Long.MAX_VALUE;
|
||||
|
||||
byte[] start;
|
||||
if (reversed) start = Bytes.toBytes("testUseRegionWithReplica" + (NUMROWS - 1));
|
||||
else start = Bytes.toBytes("testUseRegionWithReplica" + 0);
|
||||
|
||||
scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, start, NUMROWS, false, false);
|
||||
scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize,
|
||||
start, NUMROWS, NUMCOLS, false, false);
|
||||
|
||||
//Even if we were to slow the server down, unless we ask for stale
|
||||
//we won't get it
|
||||
// Even if we were to slow the server down, unless we ask for stale
|
||||
// we won't get it
|
||||
SlowMeCopro.sleepTime.set(5000);
|
||||
scanWithReplicas(reversed, small, Consistency.STRONG, caching, start, NUMROWS, false, false);
|
||||
scanWithReplicas(reversed, small, Consistency.STRONG, caching, maxResultSize, start, NUMROWS,
|
||||
NUMCOLS, false, false);
|
||||
SlowMeCopro.sleepTime.set(0);
|
||||
|
||||
flushRegion(hriPrimary);
|
||||
|
@ -686,13 +696,32 @@ public class TestReplicasClient {
|
|||
|
||||
//Now set the flag to get a response even if stale
|
||||
SlowMeCopro.sleepTime.set(5000);
|
||||
scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, start, NUMROWS, true, false);
|
||||
scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize,
|
||||
start, NUMROWS, NUMCOLS, true, false);
|
||||
SlowMeCopro.sleepTime.set(0);
|
||||
|
||||
// now make some 'next' calls slow
|
||||
SlowMeCopro.slowDownNext.set(true);
|
||||
SlowMeCopro.countOfNext.set(0);
|
||||
scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, start, NUMROWS, true, true);
|
||||
scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, start,
|
||||
NUMROWS, NUMCOLS, true, true);
|
||||
SlowMeCopro.slowDownNext.set(false);
|
||||
SlowMeCopro.countOfNext.set(0);
|
||||
|
||||
// Make sure we do not get stale data..
|
||||
SlowMeCopro.sleepTime.set(5000);
|
||||
scanWithReplicas(reversed, small, Consistency.STRONG, caching, maxResultSize,
|
||||
start, NUMROWS, NUMCOLS, false, false);
|
||||
SlowMeCopro.sleepTime.set(0);
|
||||
|
||||
// While the next calls are slow, set maxResultSize to 1 so that some partial results will be
|
||||
// returned from the server before the replica switch occurs.
|
||||
maxResultSize = 1;
|
||||
SlowMeCopro.slowDownNext.set(true);
|
||||
SlowMeCopro.countOfNext.set(0);
|
||||
scanWithReplicas(reversed, small, Consistency.TIMELINE, caching, maxResultSize, start,
|
||||
NUMROWS, NUMCOLS, true, true);
|
||||
maxResultSize = Long.MAX_VALUE;
|
||||
SlowMeCopro.slowDownNext.set(false);
|
||||
SlowMeCopro.countOfNext.set(0);
|
||||
} finally {
|
||||
|
@ -710,33 +739,60 @@ public class TestReplicasClient {
|
|||
}
|
||||
|
||||
private void scanWithReplicas(boolean reversed, boolean small, Consistency consistency,
|
||||
int caching, byte[] startRow, int numRows, boolean staleExpected, boolean slowNext)
|
||||
int caching, long maxResultSize, byte[] startRow, int numRows, int numCols,
|
||||
boolean staleExpected, boolean slowNext)
|
||||
throws Exception {
|
||||
Scan scan = new Scan(startRow);
|
||||
scan.setCaching(caching);
|
||||
scan.setMaxResultSize(maxResultSize);
|
||||
scan.setReversed(reversed);
|
||||
scan.setSmall(small);
|
||||
scan.setConsistency(consistency);
|
||||
ResultScanner scanner = table.getScanner(scan);
|
||||
Iterator<Result> iter = scanner.iterator();
|
||||
|
||||
// Maps of row keys that we have seen so far
|
||||
HashMap<String, Boolean> map = new HashMap<String, Boolean>();
|
||||
int count = 0;
|
||||
|
||||
// Tracked metrics
|
||||
int rowCount = 0;
|
||||
int cellCount = 0;
|
||||
int countOfStale = 0;
|
||||
|
||||
while (iter.hasNext()) {
|
||||
count++;
|
||||
rowCount++;
|
||||
Result r = iter.next();
|
||||
if (map.containsKey(new String(r.getRow()))) {
|
||||
String row = new String(r.getRow());
|
||||
|
||||
if (map.containsKey(row)) {
|
||||
throw new Exception("Unexpected scan result. Repeated row " + Bytes.toString(r.getRow()));
|
||||
}
|
||||
map.put(new String(r.getRow()), true);
|
||||
|
||||
map.put(row, true);
|
||||
|
||||
for (Cell cell : r.rawCells()) {
|
||||
cellCount++;
|
||||
}
|
||||
|
||||
if (!slowNext) Assert.assertTrue(r.isStale() == staleExpected);
|
||||
if (r.isStale()) countOfStale++;
|
||||
}
|
||||
LOG.debug("Count of rows " + count + " num rows expected " + numRows);
|
||||
Assert.assertTrue(count == numRows);
|
||||
Assert.assertTrue("Count of rows " + rowCount + " num rows expected " + numRows,
|
||||
rowCount == numRows);
|
||||
Assert.assertTrue("Count of cells: " + cellCount + " cells expected: " + numRows * numCols,
|
||||
cellCount == (numRows * numCols));
|
||||
|
||||
if (slowNext) {
|
||||
LOG.debug("Count of Stale " + countOfStale);
|
||||
Assert.assertTrue(countOfStale > 1 && countOfStale < numRows);
|
||||
Assert.assertTrue(countOfStale > 1);
|
||||
|
||||
// If the scan was configured in such a way that a full row was NOT retrieved before the
|
||||
// replica switch occurred, then it is possible that all rows were stale
|
||||
if (maxResultSize != Long.MAX_VALUE) {
|
||||
Assert.assertTrue(countOfStale <= numRows);
|
||||
} else {
|
||||
Assert.assertTrue(countOfStale < numRows);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationP
|
|||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
|
@ -89,7 +88,7 @@ implements Coprocessor, CoprocessorService {
|
|||
boolean hasMore = false;
|
||||
do {
|
||||
curVals.clear();
|
||||
hasMore = NextState.hasMoreValues(scanner.next(curVals));
|
||||
hasMore = scanner.next(curVals);
|
||||
for (Cell kv : curVals) {
|
||||
if (CellUtil.matchingQualifier(kv, qualifier)) {
|
||||
sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset());
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW
|
|||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
|
@ -98,7 +97,7 @@ implements Coprocessor, CoprocessorService {
|
|||
boolean hasMore = false;
|
||||
do {
|
||||
curVals.clear();
|
||||
hasMore = NextState.hasMoreValues(scanner.next(curVals));
|
||||
hasMore = scanner.next(curVals);
|
||||
for (Cell kv : curVals) {
|
||||
if (CellUtil.matchingQualifier(kv, qualifier)) {
|
||||
sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset());
|
||||
|
|
|
@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW
|
|||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
|
@ -98,7 +97,7 @@ implements Coprocessor, CoprocessorService {
|
|||
boolean hasMore = false;
|
||||
do {
|
||||
curVals.clear();
|
||||
hasMore = NextState.hasMoreValues(scanner.next(curVals));
|
||||
hasMore = scanner.next(curVals);
|
||||
for (Cell kv : curVals) {
|
||||
if (CellUtil.matchingQualifier(kv, qualifier)) {
|
||||
sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset());
|
||||
|
|
|
@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.regionserver.Region;
|
|||
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitTransaction;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitTransactionFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
|
@ -87,36 +88,26 @@ public class TestCoprocessorInterface {
|
|||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> results) throws IOException {
|
||||
public boolean next(List<Cell> results) throws IOException {
|
||||
return delegate.next(results);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> result, int limit) throws IOException {
|
||||
return delegate.next(result, limit);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> result, int limit, long remainingResultSize)
|
||||
public boolean next(List<Cell> result, ScannerContext scannerContext)
|
||||
throws IOException {
|
||||
return delegate.next(result, limit, remainingResultSize);
|
||||
return delegate.next(result, scannerContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState nextRaw(List<Cell> result)
|
||||
public boolean nextRaw(List<Cell> result)
|
||||
throws IOException {
|
||||
return delegate.nextRaw(result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState nextRaw(List<Cell> result, int limit) throws IOException {
|
||||
return delegate.nextRaw(result, limit);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState nextRaw(List<Cell> result, int limit, long remainingResultSize)
|
||||
public boolean nextRaw(List<Cell> result, ScannerContext context)
|
||||
throws IOException {
|
||||
return delegate.nextRaw(result, limit, remainingResultSize);
|
||||
return delegate.nextRaw(result, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -66,8 +66,10 @@ import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
|
|||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.NoLimitScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||
|
@ -433,24 +435,17 @@ public class TestRegionObserverInterface {
|
|||
Store store, final InternalScanner scanner, final ScanType scanType) {
|
||||
return new InternalScanner() {
|
||||
@Override
|
||||
public NextState next(List<Cell> results) throws IOException {
|
||||
return next(results, -1);
|
||||
public boolean next(List<Cell> results) throws IOException {
|
||||
return next(results, NoLimitScannerContext.getInstance());
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> results, int limit) throws IOException {
|
||||
return next(results, limit, -1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> results, int limit, long remainingResultSize)
|
||||
public boolean next(List<Cell> results, ScannerContext scannerContext)
|
||||
throws IOException {
|
||||
List<Cell> internalResults = new ArrayList<Cell>();
|
||||
boolean hasMore;
|
||||
NextState state;
|
||||
do {
|
||||
state = scanner.next(internalResults, limit, remainingResultSize);
|
||||
hasMore = state != null && state.hasMoreValues();
|
||||
hasMore = scanner.next(internalResults, scannerContext);
|
||||
if (!internalResults.isEmpty()) {
|
||||
long row = Bytes.toLong(CellUtil.cloneValue(internalResults.get(0)));
|
||||
if (row % 2 == 0) {
|
||||
|
@ -465,7 +460,7 @@ public class TestRegionObserverInterface {
|
|||
if (!internalResults.isEmpty()) {
|
||||
results.addAll(internalResults);
|
||||
}
|
||||
return state;
|
||||
return hasMore;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -105,7 +104,8 @@ public class TestColumnPrefixFilter {
|
|||
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(scanner.next(results)));
|
||||
while (scanner.next(results))
|
||||
;
|
||||
assertEquals(prefixMap.get(s).size(), results.size());
|
||||
}
|
||||
} finally {
|
||||
|
@ -170,7 +170,8 @@ public class TestColumnPrefixFilter {
|
|||
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(scanner.next(results)));
|
||||
while (scanner.next(results))
|
||||
;
|
||||
assertEquals(prefixMap.get(s).size(), results.size());
|
||||
}
|
||||
} finally {
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
|||
import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -151,7 +150,7 @@ public class TestDependentColumnFilter {
|
|||
int i = 0;
|
||||
int cells = 0;
|
||||
for (boolean done = true; done; i++) {
|
||||
done = NextState.hasMoreValues(scanner.next(results));
|
||||
done = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
LOG.info("counter=" + i + ", " + results);
|
||||
|
|
|
@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
|||
import org.apache.hadoop.hbase.filter.FilterList.Operator;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
|
@ -503,7 +502,7 @@ public class TestFilter {
|
|||
InternalScanner scanner = this.region.getScanner(s);
|
||||
int scannerCounter = 0;
|
||||
while (true) {
|
||||
boolean isMoreResults = NextState.hasMoreValues(scanner.next(new ArrayList<Cell>()));
|
||||
boolean isMoreResults = scanner.next(new ArrayList<Cell>());
|
||||
scannerCounter++;
|
||||
|
||||
if (scannerCounter >= pageSize) {
|
||||
|
@ -532,7 +531,7 @@ public class TestFilter {
|
|||
InternalScanner scanner = this.region.getScanner(s);
|
||||
while (true) {
|
||||
ArrayList<Cell> values = new ArrayList<Cell>();
|
||||
boolean isMoreResults = NextState.hasMoreValues(scanner.next(values));
|
||||
boolean isMoreResults = scanner.next(values);
|
||||
if (!isMoreResults
|
||||
|| !Bytes.toString(values.get(0).getRow()).startsWith(prefix)) {
|
||||
Assert.assertTrue(
|
||||
|
@ -566,7 +565,7 @@ public class TestFilter {
|
|||
InternalScanner scanner = this.region.getScanner(s);
|
||||
int scannerCounter = 0;
|
||||
while (true) {
|
||||
boolean isMoreResults = NextState.hasMoreValues(scanner.next(new ArrayList<Cell>()));
|
||||
boolean isMoreResults = scanner.next(new ArrayList<Cell>());
|
||||
scannerCounter++;
|
||||
|
||||
if (scannerCounter >= pageSize) {
|
||||
|
@ -644,7 +643,7 @@ public class TestFilter {
|
|||
InternalScanner scanner = this.region.getScanner(s);
|
||||
while (true) {
|
||||
ArrayList<Cell> values = new ArrayList<Cell>();
|
||||
boolean isMoreResults = NextState.hasMoreValues(scanner.next(values));
|
||||
boolean isMoreResults = scanner.next(values);
|
||||
if (!isMoreResults || !Bytes.toString(CellUtil.cloneRow(values.get(0))).startsWith(prefix)) {
|
||||
assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining());
|
||||
}
|
||||
|
@ -673,7 +672,7 @@ public class TestFilter {
|
|||
InternalScanner scanner = this.region.getScanner(s);
|
||||
while (true) {
|
||||
ArrayList<Cell> values = new ArrayList<Cell>();
|
||||
boolean isMoreResults = NextState.hasMoreValues(scanner.next(values));
|
||||
boolean isMoreResults = scanner.next(values);
|
||||
assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining());
|
||||
if (!isMoreResults) {
|
||||
break;
|
||||
|
@ -1476,7 +1475,7 @@ public class TestFilter {
|
|||
InternalScanner scanner = testRegion.getScanner(s1);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
int resultCount = 0;
|
||||
while (NextState.hasMoreValues(scanner.next(results))) {
|
||||
while (scanner.next(results)) {
|
||||
resultCount++;
|
||||
byte[] row = CellUtil.cloneRow(results.get(0));
|
||||
LOG.debug("Found row: " + Bytes.toStringBinary(row));
|
||||
|
@ -1618,7 +1617,7 @@ public class TestFilter {
|
|||
List<Cell> results = new ArrayList<Cell>();
|
||||
int i = 0;
|
||||
for (boolean done = true; done; i++) {
|
||||
done = NextState.hasMoreValues(scanner.next(results));
|
||||
done = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
LOG.info("counter=" + i + ", " + results);
|
||||
|
@ -1640,7 +1639,7 @@ public class TestFilter {
|
|||
List<Cell> results = new ArrayList<Cell>();
|
||||
int i = 0;
|
||||
for (boolean done = true; done; i++) {
|
||||
done = NextState.hasMoreValues(scanner.next(results));
|
||||
done = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
LOG.info("counter=" + i + ", " + results);
|
||||
|
@ -1662,7 +1661,7 @@ public class TestFilter {
|
|||
int row = 0;
|
||||
int idx = 0;
|
||||
for (boolean done = true; done; row++) {
|
||||
done = NextState.hasMoreValues(scanner.next(results));
|
||||
done = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
if(results.isEmpty()) break;
|
||||
|
@ -1693,7 +1692,7 @@ public class TestFilter {
|
|||
int row = 0;
|
||||
int idx = 0;
|
||||
for (boolean more = true; more; row++) {
|
||||
more = NextState.hasMoreValues(scanner.next(results));
|
||||
more = scanner.next(results);
|
||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||
KeyValue.COMPARATOR);
|
||||
if(results.isEmpty()) break;
|
||||
|
@ -2029,7 +2028,7 @@ public class TestFilter {
|
|||
List<Cell> results = new ArrayList<Cell>();
|
||||
int i = 5;
|
||||
for (boolean done = true; done; i++) {
|
||||
done = NextState.hasMoreValues(scanner.next(results));
|
||||
done = scanner.next(results);
|
||||
assertTrue(CellUtil.matchingRow(results.get(0), Bytes.toBytes("row" + i)));
|
||||
assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
|
||||
results.clear();
|
||||
|
@ -2052,7 +2051,7 @@ public class TestFilter {
|
|||
assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
|
||||
results.clear();
|
||||
}
|
||||
assertFalse(NextState.hasMoreValues(scanner.next(results)));
|
||||
assertFalse(scanner.next(results));
|
||||
// 3. let's begin to verify nested filter list
|
||||
// 3.1 add rowFilter, then add subFilterList
|
||||
FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
|
||||
|
@ -2074,7 +2073,7 @@ public class TestFilter {
|
|||
assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
|
||||
results.clear();
|
||||
}
|
||||
assertFalse(NextState.hasMoreValues(scanner.next(results)));
|
||||
assertFalse(scanner.next(results));
|
||||
// 3.2 MAGIC here! add subFilterList first, then add rowFilter
|
||||
filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
|
||||
filterList.addFilter(subFilterList);
|
||||
|
@ -2095,7 +2094,7 @@ public class TestFilter {
|
|||
assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
|
||||
results.clear();
|
||||
}
|
||||
assertFalse(NextState.hasMoreValues(scanner.next(results)));
|
||||
assertFalse(scanner.next(results));
|
||||
WAL wal = ((HRegion)testRegion).getWAL();
|
||||
((HRegion)testRegion).close();
|
||||
wal.close();
|
||||
|
|
|
@ -34,12 +34,11 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -140,7 +139,7 @@ public class TestInvocationRecordFilter {
|
|||
List<Cell> actualValues = new ArrayList<Cell>();
|
||||
List<Cell> temp = new ArrayList<Cell>();
|
||||
InternalScanner scanner = this.region.getScanner(scan);
|
||||
while (NextState.hasMoreValues(scanner.next(temp))) {
|
||||
while (scanner.next(temp)) {
|
||||
actualValues.addAll(temp);
|
||||
temp.clear();
|
||||
}
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -110,7 +109,7 @@ public class TestMultipleColumnPrefixFilter {
|
|||
scan.setFilter(filter);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
while (NextState.hasMoreValues(scanner.next(results)))
|
||||
while (scanner.next(results))
|
||||
;
|
||||
assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
|
||||
|
||||
|
@ -183,7 +182,7 @@ public class TestMultipleColumnPrefixFilter {
|
|||
scan.setFilter(filter);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
while (NextState.hasMoreValues(scanner.next(results)))
|
||||
while (scanner.next(results))
|
||||
;
|
||||
assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
|
||||
|
||||
|
@ -228,7 +227,7 @@ public class TestMultipleColumnPrefixFilter {
|
|||
scan1.setFilter(multiplePrefixFilter);
|
||||
List<Cell> results1 = new ArrayList<Cell>();
|
||||
InternalScanner scanner1 = region.getScanner(scan1);
|
||||
while (NextState.hasMoreValues(scanner1.next(results1)))
|
||||
while (scanner1.next(results1))
|
||||
;
|
||||
|
||||
ColumnPrefixFilter singlePrefixFilter;
|
||||
|
@ -239,7 +238,7 @@ public class TestMultipleColumnPrefixFilter {
|
|||
scan2.setFilter(singlePrefixFilter);
|
||||
List<Cell> results2 = new ArrayList<Cell>();
|
||||
InternalScanner scanner2 = region.getScanner(scan1);
|
||||
while (NextState.hasMoreValues(scanner2.next(results2)))
|
||||
while (scanner2.next(results2))
|
||||
;
|
||||
|
||||
assertEquals(results1.size(), results2.size());
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||
|
@ -117,7 +116,7 @@ public class TestPrefixTree {
|
|||
RegionScanner scanner = region.getScanner(scan);
|
||||
List<Cell> cells = new ArrayList<Cell>();
|
||||
for (int i = 0; i < 3; i++) {
|
||||
assertEquals(i < 2, NextState.hasMoreValues(scanner.next(cells)));
|
||||
assertEquals(i < 2, scanner.next(cells));
|
||||
CellScanner cellScanner = Result.create(cells).cellScanner();
|
||||
while (cellScanner.advance()) {
|
||||
assertEquals(rows[i], Bytes.toString(cellScanner.current().getRowArray(), cellScanner
|
||||
|
@ -136,7 +135,7 @@ public class TestPrefixTree {
|
|||
scan.setStopRow(Bytes.toBytes("a-b-A-1:"));
|
||||
scanner = region.getScanner(scan);
|
||||
for (int i = 1; i < 3; i++) {
|
||||
assertEquals(i < 2, NextState.hasMoreValues(scanner.next(cells)));
|
||||
assertEquals(i < 2, scanner.next(cells));
|
||||
CellScanner cellScanner = Result.create(cells).cellScanner();
|
||||
while (cellScanner.advance()) {
|
||||
assertEquals(rows[i], Bytes.toString(cellScanner.current().getRowArray(), cellScanner
|
||||
|
@ -152,7 +151,7 @@ public class TestPrefixTree {
|
|||
scan.setStopRow(Bytes.toBytes("a-b-A-1:"));
|
||||
scanner = region.getScanner(scan);
|
||||
for (int i = 1; i < 3; i++) {
|
||||
assertEquals(i < 2, NextState.hasMoreValues(scanner.next(cells)));
|
||||
assertEquals(i < 2, scanner.next(cells));
|
||||
CellScanner cellScanner = Result.create(cells).cellScanner();
|
||||
while (cellScanner.advance()) {
|
||||
assertEquals(rows[i], Bytes.toString(cellScanner.current().getRowArray(), cellScanner
|
||||
|
@ -167,7 +166,7 @@ public class TestPrefixTree {
|
|||
scan.setStartRow(Bytes.toBytes("a-b-A-1-140239"));
|
||||
scan.setStopRow(Bytes.toBytes("a-b-A-1:"));
|
||||
scanner = region.getScanner(scan);
|
||||
assertFalse(NextState.hasMoreValues(scanner.next(cells)));
|
||||
assertFalse(scanner.next(cells));
|
||||
assertFalse(cells.isEmpty());
|
||||
scanner.close();
|
||||
}
|
||||
|
@ -186,7 +185,7 @@ public class TestPrefixTree {
|
|||
Scan scan = new Scan(Bytes.toBytes("obj29995"));
|
||||
RegionScanner scanner = region.getScanner(scan);
|
||||
List<Cell> cells = new ArrayList<Cell>();
|
||||
assertFalse(NextState.hasMoreValues(scanner.next(cells)));
|
||||
assertFalse(scanner.next(cells));
|
||||
assertArrayEquals(Bytes.toBytes("obj3"), Result.create(cells).getRow());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,10 +37,9 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Test;
|
||||
|
@ -122,7 +121,7 @@ public class TestScannerSelectionUsingKeyRange {
|
|||
cache.clearCache();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(scanner.next(results))) {
|
||||
while (scanner.next(results)) {
|
||||
}
|
||||
scanner.close();
|
||||
assertEquals(0, results.size());
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -140,7 +139,7 @@ public class TestScannerSelectionUsingTTL {
|
|||
final int expectedKVsPerRow = numFreshFiles * NUM_COLS_PER_ROW;
|
||||
int numReturnedRows = 0;
|
||||
LOG.info("Scanning the entire table");
|
||||
while (NextState.hasMoreValues(scanner.next(results)) || results.size() > 0) {
|
||||
while (scanner.next(results) || results.size() > 0) {
|
||||
assertEquals(expectedKVsPerRow, results.size());
|
||||
++numReturnedRows;
|
||||
results.clear();
|
||||
|
|
|
@ -60,11 +60,10 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
|
@ -467,7 +466,8 @@ public class TestAtomicOperation {
|
|||
Scan s = new Scan(row);
|
||||
RegionScanner rs = region.getScanner(s);
|
||||
List<Cell> r = new ArrayList<Cell>();
|
||||
while(NextState.hasMoreValues(rs.next(r)));
|
||||
while (rs.next(r))
|
||||
;
|
||||
rs.close();
|
||||
if (r.size() != 1) {
|
||||
LOG.debug(r);
|
||||
|
@ -561,7 +561,8 @@ public class TestAtomicOperation {
|
|||
Scan s = new Scan();
|
||||
RegionScanner scanner = region.getScanner(s);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
scanner.next(results, 2);
|
||||
ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(2).build();
|
||||
scanner.next(results, scannerContext);
|
||||
for (Cell keyValue : results) {
|
||||
assertEquals("50",Bytes.toString(CellUtil.cloneValue(keyValue)));
|
||||
}
|
||||
|
|
|
@ -25,15 +25,14 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
|
|||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheStats;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -103,7 +102,8 @@ public class TestBlocksScanned extends HBaseTestCase {
|
|||
|
||||
InternalScanner s = r.getScanner(scan);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(s.next(results)));
|
||||
while (s.next(results))
|
||||
;
|
||||
s.close();
|
||||
|
||||
int expectResultSize = 'z' - 'a';
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -161,7 +160,7 @@ public class TestColumnSeeking {
|
|||
}
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(scanner.next(results)))
|
||||
while (scanner.next(results))
|
||||
;
|
||||
assertEquals(kvSet.size(), results.size());
|
||||
assertTrue(KeyValueTestUtil.containsIgnoreMvccVersion(results, kvSet));
|
||||
|
@ -273,7 +272,7 @@ public class TestColumnSeeking {
|
|||
}
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(scanner.next(results)))
|
||||
while (scanner.next(results))
|
||||
;
|
||||
assertEquals(kvSet.size(), results.size());
|
||||
assertTrue(KeyValueTestUtil.containsIgnoreMvccVersion(results, kvSet));
|
||||
|
|
|
@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.KeyValueUtil;
|
|||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -107,7 +106,7 @@ public class TestDefaultMemStore extends TestCase {
|
|||
StoreScanner s = new StoreScanner(scan, scanInfo, scanType, null, memstorescanners);
|
||||
int count = 0;
|
||||
try {
|
||||
while (NextState.hasMoreValues(s.next(result))) {
|
||||
while (s.next(result)) {
|
||||
LOG.info(result);
|
||||
count++;
|
||||
// Row count is same as column count.
|
||||
|
@ -127,7 +126,7 @@ public class TestDefaultMemStore extends TestCase {
|
|||
s = new StoreScanner(scan, scanInfo, scanType, null, memstorescanners);
|
||||
count = 0;
|
||||
try {
|
||||
while (NextState.hasMoreValues(s.next(result))) {
|
||||
while (s.next(result)) {
|
||||
LOG.info(result);
|
||||
// Assert the stuff is coming out in right order.
|
||||
assertTrue(CellUtil.matchingRow(result.get(0), Bytes.toBytes(count)));
|
||||
|
@ -154,7 +153,7 @@ public class TestDefaultMemStore extends TestCase {
|
|||
count = 0;
|
||||
int snapshotIndex = 5;
|
||||
try {
|
||||
while (NextState.hasMoreValues(s.next(result))) {
|
||||
while (s.next(result)) {
|
||||
LOG.info(result);
|
||||
// Assert the stuff is coming out in right order.
|
||||
assertTrue(CellUtil.matchingRow(result.get(0), Bytes.toBytes(count)));
|
||||
|
@ -528,7 +527,7 @@ public class TestDefaultMemStore extends TestCase {
|
|||
Bytes.toBytes(startRowId)), scanInfo, scanType, null,
|
||||
memstore.getScanners(0));
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
for (int i = 0; NextState.hasMoreValues(scanner.next(results)); i++) {
|
||||
for (int i = 0; scanner.next(results); i++) {
|
||||
int rowId = startRowId + i;
|
||||
Cell left = results.get(0);
|
||||
byte[] row1 = Bytes.toBytes(rowId);
|
||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.client.Durability;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -97,7 +96,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
|
|||
InternalScanner s = mr.getScanner(new Scan());
|
||||
try {
|
||||
List<Cell> keys = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(s.next(keys))) {
|
||||
while (s.next(keys)) {
|
||||
LOG.info(keys);
|
||||
keys.clear();
|
||||
}
|
||||
|
@ -121,7 +120,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
|
|||
s = mr.getScanner(scan);
|
||||
try {
|
||||
List<Cell> keys = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(s.next(keys))) {
|
||||
while (s.next(keys)) {
|
||||
mr.delete(new Delete(CellUtil.cloneRow(keys.get(0))));
|
||||
keys.clear();
|
||||
}
|
||||
|
|
|
@ -128,11 +128,10 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.Stor
|
|||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.Region.RowLock;
|
||||
import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
|
@ -1265,7 +1264,7 @@ public class TestHRegion {
|
|||
boolean more = false;
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
do {
|
||||
more = NextState.hasMoreValues(scanner.next(results));
|
||||
more = scanner.next(results);
|
||||
if (results != null && !results.isEmpty())
|
||||
count++;
|
||||
else
|
||||
|
@ -1284,7 +1283,7 @@ public class TestHRegion {
|
|||
List<Cell> results = new ArrayList<Cell>();
|
||||
boolean more = false;
|
||||
do {
|
||||
more = NextState.hasMoreValues(resultScanner.next(results));
|
||||
more = resultScanner.next(results);
|
||||
if (results != null && !results.isEmpty())
|
||||
numberOfResults++;
|
||||
else
|
||||
|
@ -2274,7 +2273,7 @@ public class TestHRegion {
|
|||
InternalScanner s = region.getScanner(scan);
|
||||
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(false, NextState.hasMoreValues(s.next(results)));
|
||||
assertEquals(false, s.next(results));
|
||||
assertEquals(1, results.size());
|
||||
Cell kv = results.get(0);
|
||||
|
||||
|
@ -2766,7 +2765,7 @@ public class TestHRegion {
|
|||
List<Cell> actual = new ArrayList<Cell>();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
|
||||
boolean hasNext = scanner.next(actual);
|
||||
assertEquals(false, hasNext);
|
||||
|
||||
// Verify result
|
||||
|
@ -2829,7 +2828,7 @@ public class TestHRegion {
|
|||
List<Cell> actual = new ArrayList<Cell>();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
|
||||
boolean hasNext = scanner.next(actual);
|
||||
assertEquals(false, hasNext);
|
||||
|
||||
// Verify result
|
||||
|
@ -2911,7 +2910,7 @@ public class TestHRegion {
|
|||
List<Cell> actual = new ArrayList<Cell>();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
|
||||
boolean hasNext = scanner.next(actual);
|
||||
assertEquals(false, hasNext);
|
||||
|
||||
// Verify result
|
||||
|
@ -2972,7 +2971,7 @@ public class TestHRegion {
|
|||
List<Cell> actual = new ArrayList<Cell>();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
|
||||
boolean hasNext = scanner.next(actual);
|
||||
assertEquals(false, hasNext);
|
||||
|
||||
// Verify result
|
||||
|
@ -3033,7 +3032,7 @@ public class TestHRegion {
|
|||
List<Cell> actual = new ArrayList<Cell>();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
|
||||
boolean hasNext = scanner.next(actual);
|
||||
assertEquals(false, hasNext);
|
||||
|
||||
// Verify result
|
||||
|
@ -3086,7 +3085,7 @@ public class TestHRegion {
|
|||
InternalScanner s = region.getScanner(scan);
|
||||
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(false, NextState.hasMoreValues(s.next(results)));
|
||||
assertEquals(false, s.next(results));
|
||||
assertEquals(0, results.size());
|
||||
} finally {
|
||||
HBaseTestingUtility.closeRegionAndWAL(this.region);
|
||||
|
@ -3160,7 +3159,7 @@ public class TestHRegion {
|
|||
List<Cell> actual = new ArrayList<Cell>();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
|
||||
boolean hasNext = scanner.next(actual);
|
||||
assertEquals(false, hasNext);
|
||||
|
||||
// Verify result
|
||||
|
@ -3223,18 +3222,18 @@ public class TestHRegion {
|
|||
InternalScanner s = region.getScanner(scan);
|
||||
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertTrue(NextState.hasMoreValues(s.next(results)));
|
||||
assertTrue(s.next(results));
|
||||
assertEquals(results.size(), 1);
|
||||
results.clear();
|
||||
|
||||
assertTrue(NextState.hasMoreValues(s.next(results)));
|
||||
assertTrue(s.next(results));
|
||||
assertEquals(results.size(), 3);
|
||||
assertTrue("orderCheck", CellUtil.matchingFamily(results.get(0), cf_alpha));
|
||||
assertTrue("orderCheck", CellUtil.matchingFamily(results.get(1), cf_essential));
|
||||
assertTrue("orderCheck", CellUtil.matchingFamily(results.get(2), cf_joined));
|
||||
results.clear();
|
||||
|
||||
assertFalse(NextState.hasMoreValues(s.next(results)));
|
||||
assertFalse(s.next(results));
|
||||
assertEquals(results.size(), 0);
|
||||
} finally {
|
||||
HBaseTestingUtility.closeRegionAndWAL(this.region);
|
||||
|
@ -3317,8 +3316,9 @@ public class TestHRegion {
|
|||
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
int index = 0;
|
||||
ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(3).build();
|
||||
while (true) {
|
||||
boolean more = NextState.hasMoreValues(s.next(results, 3));
|
||||
boolean more = s.next(results, scannerContext);
|
||||
if ((index >> 1) < 5) {
|
||||
if (index % 2 == 0)
|
||||
assertEquals(results.size(), 3);
|
||||
|
@ -3592,7 +3592,7 @@ public class TestHRegion {
|
|||
if (toggle) {
|
||||
flushThread.flush();
|
||||
}
|
||||
while (NextState.hasMoreValues(scanner.next(res)))
|
||||
while (scanner.next(res))
|
||||
;
|
||||
if (!toggle) {
|
||||
flushThread.flush();
|
||||
|
@ -3717,7 +3717,7 @@ public class TestHRegion {
|
|||
boolean previousEmpty = res.isEmpty();
|
||||
res.clear();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
while (NextState.hasMoreValues(scanner.next(res)))
|
||||
while (scanner.next(res))
|
||||
;
|
||||
if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
|
||||
assertEquals("i=" + i, expectedCount, res.size());
|
||||
|
@ -3999,7 +3999,7 @@ public class TestHRegion {
|
|||
InternalScanner scanner = region.getScanner(idxScan);
|
||||
List<Cell> res = new ArrayList<Cell>();
|
||||
|
||||
while (NextState.hasMoreValues(scanner.next(res)))
|
||||
while (scanner.next(res))
|
||||
;
|
||||
assertEquals(1L, res.size());
|
||||
} finally {
|
||||
|
@ -4895,7 +4895,7 @@ public class TestHRegion {
|
|||
try {
|
||||
List<Cell> curVals = new ArrayList<Cell>();
|
||||
boolean first = true;
|
||||
OUTER_LOOP: while (NextState.hasMoreValues(s.next(curVals))) {
|
||||
OUTER_LOOP: while (s.next(curVals)) {
|
||||
for (Cell kv : curVals) {
|
||||
byte[] val = CellUtil.cloneValue(kv);
|
||||
byte[] curval = val;
|
||||
|
@ -5091,17 +5091,17 @@ public class TestHRegion {
|
|||
scan.setReversed(true);
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
List<Cell> currRow = new ArrayList<Cell>();
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
boolean hasNext = scanner.next(currRow);
|
||||
assertEquals(2, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
|
||||
assertFalse(hasNext);
|
||||
|
@ -5148,17 +5148,17 @@ public class TestHRegion {
|
|||
scan.setReversed(true);
|
||||
scan.setMaxVersions(5);
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
boolean hasNext = scanner.next(currRow);
|
||||
assertEquals(2, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
|
||||
assertFalse(hasNext);
|
||||
|
@ -5202,17 +5202,17 @@ public class TestHRegion {
|
|||
List<Cell> currRow = new ArrayList<Cell>();
|
||||
scan.setReversed(true);
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
boolean hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
|
||||
assertFalse(hasNext);
|
||||
|
@ -5270,17 +5270,17 @@ public class TestHRegion {
|
|||
scan.setReversed(true);
|
||||
List<Cell> currRow = new ArrayList<Cell>();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
boolean hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
|
||||
assertFalse(hasNext);
|
||||
|
@ -5291,7 +5291,7 @@ public class TestHRegion {
|
|||
scan.setReversed(true);
|
||||
currRow.clear();
|
||||
scanner = region.getScanner(scan);
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
|
||||
scanner.close();
|
||||
|
@ -5350,17 +5350,17 @@ public class TestHRegion {
|
|||
scan.setReversed(true);
|
||||
List<Cell> currRow = new ArrayList<Cell>();
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
boolean hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
|
||||
assertFalse(hasNext);
|
||||
|
@ -5371,7 +5371,7 @@ public class TestHRegion {
|
|||
scan.setReversed(true);
|
||||
currRow.clear();
|
||||
scanner = region.getScanner(scan);
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
|
||||
scanner.close();
|
||||
|
@ -5495,42 +5495,42 @@ public class TestHRegion {
|
|||
// 1. scan out "row4" (5 kvs), "row5" can't be scanned out since not
|
||||
// included in scan range
|
||||
// "row4" takes 2 next() calls since batch=3
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(3, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(2, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
|
||||
assertTrue(hasNext);
|
||||
// 2. scan out "row3" (2 kv)
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(2, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row3));
|
||||
assertTrue(hasNext);
|
||||
// 3. scan out "row2" (4 kvs)
|
||||
// "row2" takes 2 next() calls since batch=3
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(3, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
|
||||
assertTrue(hasNext);
|
||||
// 4. scan out "row1" (2 kv)
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(2, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row1));
|
||||
assertTrue(hasNext);
|
||||
// 5. scan out "row0" (1 kv)
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row0));
|
||||
assertFalse(hasNext);
|
||||
|
@ -5591,22 +5591,22 @@ public class TestHRegion {
|
|||
scan.setBatch(10);
|
||||
InternalScanner scanner = region.getScanner(scan);
|
||||
List<Cell> currRow = new ArrayList<Cell>();
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
boolean hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row3));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
|
||||
assertTrue(hasNext);
|
||||
currRow.clear();
|
||||
hasNext = NextState.hasMoreValues(scanner.next(currRow));
|
||||
hasNext = scanner.next(currRow);
|
||||
assertEquals(1, currRow.size());
|
||||
assertTrue(Bytes.equals(currRow.get(0).getRow(), row1));
|
||||
assertFalse(hasNext);
|
||||
|
@ -5659,7 +5659,7 @@ public class TestHRegion {
|
|||
boolean more = false;
|
||||
int verify = startRow + 2 * numRows - 1;
|
||||
do {
|
||||
more = NextState.hasMoreValues(scanner.next(currRow));
|
||||
more = scanner.next(currRow);
|
||||
assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
|
||||
verify--;
|
||||
currRow.clear();
|
||||
|
@ -5672,7 +5672,7 @@ public class TestHRegion {
|
|||
scanner = regions[1].getScanner(scan);
|
||||
verify = startRow + 2 * numRows - 1;
|
||||
do {
|
||||
more = NextState.hasMoreValues(scanner.next(currRow));
|
||||
more = scanner.next(currRow);
|
||||
assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
|
||||
verify--;
|
||||
currRow.clear();
|
||||
|
@ -5685,7 +5685,7 @@ public class TestHRegion {
|
|||
scanner = regions[0].getScanner(scan);
|
||||
verify = startRow + numRows - 1;
|
||||
do {
|
||||
more = NextState.hasMoreValues(scanner.next(currRow));
|
||||
more = scanner.next(currRow);
|
||||
assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
|
||||
verify--;
|
||||
currRow.clear();
|
||||
|
@ -5698,7 +5698,7 @@ public class TestHRegion {
|
|||
scanner = regions[0].getScanner(scan);
|
||||
verify = startRow + numRows - 1;
|
||||
do {
|
||||
more = NextState.hasMoreValues(scanner.next(currRow));
|
||||
more = scanner.next(currRow);
|
||||
assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
|
||||
verify--;
|
||||
currRow.clear();
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.client.Get;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -264,7 +263,8 @@ public class TestKeepDeletes {
|
|||
s.setTimeRange(0L, ts+1);
|
||||
InternalScanner scanner = region.getScanner(s);
|
||||
List<Cell> kvs = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(scanner.next(kvs)));
|
||||
while (scanner.next(kvs))
|
||||
;
|
||||
assertTrue(kvs.isEmpty());
|
||||
|
||||
// flushing and minor compaction keep delete markers
|
||||
|
@ -946,7 +946,7 @@ public class TestKeepDeletes {
|
|||
int res = 0;
|
||||
boolean hasMore;
|
||||
do {
|
||||
hasMore = NextState.hasMoreValues(scan.next(kvs));
|
||||
hasMore = scan.next(kvs);
|
||||
for (Cell kv : kvs) {
|
||||
if(CellUtil.isDelete(kv)) res++;
|
||||
}
|
||||
|
|
|
@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
|||
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
|
||||
|
@ -132,7 +131,7 @@ public class TestMajorCompaction {
|
|||
InternalScanner s = r.getScanner(new Scan());
|
||||
do {
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
boolean result = NextState.hasMoreValues(s.next(results));
|
||||
boolean result = s.next(results);
|
||||
r.delete(new Delete(CellUtil.cloneRow(results.get(0))));
|
||||
if (!result) break;
|
||||
} while(true);
|
||||
|
@ -145,7 +144,7 @@ public class TestMajorCompaction {
|
|||
int counter = 0;
|
||||
do {
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
boolean result = NextState.hasMoreValues(s.next(results));
|
||||
boolean result = s.next(results);
|
||||
if (!result) break;
|
||||
counter++;
|
||||
} while(true);
|
||||
|
@ -456,7 +455,7 @@ public class TestMajorCompaction {
|
|||
InternalScanner s = r.getScanner(scan);
|
||||
do {
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
boolean result = NextState.hasMoreValues(s.next(results));
|
||||
boolean result = s.next(results);
|
||||
assertTrue(!results.isEmpty());
|
||||
r.delete(new Delete(results.get(0).getRow()));
|
||||
if (!result) break;
|
||||
|
@ -472,7 +471,7 @@ public class TestMajorCompaction {
|
|||
int counter = 0;
|
||||
do {
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
boolean result = NextState.hasMoreValues(s.next(results));
|
||||
boolean result = s.next(results);
|
||||
if (!result) break;
|
||||
counter++;
|
||||
} while (true);
|
||||
|
|
|
@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -250,7 +249,7 @@ public class TestMultiColumnScanner {
|
|||
String queryInfo = "columns queried: " + qualSet + " (columnBitMask="
|
||||
+ columnBitMask + "), maxVersions=" + maxVersions;
|
||||
|
||||
while (NextState.hasMoreValues(scanner.next(results)) || results.size() > 0) {
|
||||
while (scanner.next(results) || results.size() > 0) {
|
||||
for (Cell kv : results) {
|
||||
while (kvPos < kvs.size()
|
||||
&& !matchesQuery(kvs.get(kvPos), qualSet, maxVersions,
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -427,7 +426,7 @@ public class TestRegionMergeTransaction {
|
|||
List<Cell> kvs = new ArrayList<Cell>();
|
||||
boolean hasNext = true;
|
||||
while (hasNext) {
|
||||
hasNext = NextState.hasMoreValues(scanner.next(kvs));
|
||||
hasNext = scanner.next(kvs);
|
||||
if (!kvs.isEmpty())
|
||||
rowcount++;
|
||||
}
|
||||
|
|
|
@ -55,7 +55,6 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
|||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -434,7 +433,7 @@ public class TestReversibleScanners {
|
|||
int rowCount = 0;
|
||||
int kvCount = 0;
|
||||
try {
|
||||
while (NextState.hasMoreValues(scanner.next(kvList))) {
|
||||
while (scanner.next(kvList)) {
|
||||
if (kvList.isEmpty()) continue;
|
||||
rowCount++;
|
||||
kvCount += kvList.size();
|
||||
|
|
|
@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -165,7 +164,7 @@ public class TestScanWithBloomError {
|
|||
|
||||
{ // Limit the scope of results.
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
while (NextState.hasMoreValues(scanner.next(results)) || results.size() > 0) {
|
||||
while (scanner.next(results) || results.size() > 0) {
|
||||
allResults.addAll(results);
|
||||
results.clear();
|
||||
}
|
||||
|
|
|
@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.filter.Filter;
|
|||
import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
|
||||
import org.apache.hadoop.hbase.filter.PrefixFilter;
|
||||
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -137,7 +136,7 @@ public class TestScanner {
|
|||
|
||||
InternalScanner s = r.getScanner(scan);
|
||||
int count = 0;
|
||||
while (NextState.hasMoreValues(s.next(results))) {
|
||||
while (s.next(results)) {
|
||||
count++;
|
||||
}
|
||||
s.close();
|
||||
|
@ -150,7 +149,7 @@ public class TestScanner {
|
|||
count = 0;
|
||||
Cell kv = null;
|
||||
results = new ArrayList<Cell>();
|
||||
for (boolean first = true; NextState.hasMoreValues(s.next(results));) {
|
||||
for (boolean first = true; s.next(results);) {
|
||||
kv = results.get(0);
|
||||
if (first) {
|
||||
assertTrue(CellUtil.matchingRow(kv, startrow));
|
||||
|
@ -173,7 +172,7 @@ public class TestScanner {
|
|||
InternalScanner s = r.getScanner(scan);
|
||||
boolean hasMore = true;
|
||||
while (hasMore) {
|
||||
hasMore = NextState.hasMoreValues(s.next(results));
|
||||
hasMore = s.next(results);
|
||||
for (Cell kv : results) {
|
||||
assertEquals((byte)'a', CellUtil.cloneRow(kv)[0]);
|
||||
assertEquals((byte)'b', CellUtil.cloneRow(kv)[1]);
|
||||
|
@ -189,7 +188,7 @@ public class TestScanner {
|
|||
InternalScanner s = r.getScanner(scan);
|
||||
boolean hasMore = true;
|
||||
while (hasMore) {
|
||||
hasMore = NextState.hasMoreValues(s.next(results));
|
||||
hasMore = s.next(results);
|
||||
for (Cell kv : results) {
|
||||
assertTrue(Bytes.compareTo(CellUtil.cloneRow(kv), stopRow) <= 0);
|
||||
}
|
||||
|
@ -389,7 +388,7 @@ public class TestScanner {
|
|||
scan.addColumn(COLS[0], EXPLICIT_COLS[ii]);
|
||||
}
|
||||
scanner = r.getScanner(scan);
|
||||
while (NextState.hasMoreValues(scanner.next(results))) {
|
||||
while (scanner.next(results)) {
|
||||
assertTrue(hasColumn(results, HConstants.CATALOG_FAMILY,
|
||||
HConstants.REGIONINFO_QUALIFIER));
|
||||
byte [] val = CellUtil.cloneValue(getColumn(results, HConstants.CATALOG_FAMILY,
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.client.Delete;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -225,7 +224,7 @@ public class TestSeekOptimizations {
|
|||
// result, not to the one already returned in results.
|
||||
boolean hasNext;
|
||||
do {
|
||||
hasNext = NextState.hasMoreValues(scanner.next(results));
|
||||
hasNext = scanner.next(results);
|
||||
actualKVs.addAll(results);
|
||||
results.clear();
|
||||
} while (hasNext);
|
||||
|
|
|
@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
|||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -361,7 +360,7 @@ public class TestSplitTransaction {
|
|||
List<Cell> kvs = new ArrayList<Cell>();
|
||||
boolean hasNext = true;
|
||||
while (hasNext) {
|
||||
hasNext = NextState.hasMoreValues(scanner.next(kvs));
|
||||
hasNext = scanner.next(kvs);
|
||||
if (!kvs.isEmpty()) rowcount++;
|
||||
}
|
||||
} finally {
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.KeepDeletedCells;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -91,7 +90,7 @@ public class TestStoreScanner extends TestCase {
|
|||
StoreScanner scan = new StoreScanner(scanSpec, scanInfo, scanType,
|
||||
getCols("a"), scanners);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(5, results.size());
|
||||
assertEquals(kvs[kvs.length - 1], results.get(0));
|
||||
// Scan limited TimeRange
|
||||
|
@ -101,7 +100,7 @@ public class TestStoreScanner extends TestCase {
|
|||
scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"),
|
||||
scanners);
|
||||
results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(2, results.size());
|
||||
// Another range.
|
||||
scanSpec = new Scan(Bytes.toBytes(r1));
|
||||
|
@ -110,7 +109,7 @@ public class TestStoreScanner extends TestCase {
|
|||
scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"),
|
||||
scanners);
|
||||
results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(1, results.size());
|
||||
// See how TimeRange and Versions interact.
|
||||
// Another range.
|
||||
|
@ -120,7 +119,7 @@ public class TestStoreScanner extends TestCase {
|
|||
scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"),
|
||||
scanners);
|
||||
results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(3, results.size());
|
||||
}
|
||||
|
||||
|
@ -141,7 +140,7 @@ public class TestStoreScanner extends TestCase {
|
|||
getCols("a"), scanners);
|
||||
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(1, results.size());
|
||||
assertEquals(kvs[0], results.get(0));
|
||||
}
|
||||
|
@ -196,7 +195,7 @@ public class TestStoreScanner extends TestCase {
|
|||
getCols("a"), scanners);
|
||||
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertFalse(NextState.hasMoreValues(scan.next(results)));
|
||||
assertFalse(scan.next(results));
|
||||
assertEquals(0, results.size());
|
||||
}
|
||||
|
||||
|
@ -216,14 +215,14 @@ public class TestStoreScanner extends TestCase {
|
|||
getCols("a"), scanners);
|
||||
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(0, results.size());
|
||||
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(1, results.size());
|
||||
assertEquals(kvs[2], results.get(0));
|
||||
|
||||
assertEquals(false, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(false, scan.next(results));
|
||||
}
|
||||
|
||||
public void testDeleteVersionMaskingMultiplePuts() throws IOException {
|
||||
|
@ -245,7 +244,7 @@ public class TestStoreScanner extends TestCase {
|
|||
// the two put at ts=now will be masked by the 1 delete, and
|
||||
// since the scan default returns 1 version we'll return the newest
|
||||
// key, which is kvs[2], now-100.
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(1, results.size());
|
||||
assertEquals(kvs2[1], results.get(0));
|
||||
}
|
||||
|
@ -267,7 +266,7 @@ public class TestStoreScanner extends TestCase {
|
|||
StoreScanner scan = new StoreScanner(scanSpec, scanInfo, scanType,
|
||||
getCols("a"), scanners);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(2, results.size());
|
||||
assertEquals(kvs2[1], results.get(0));
|
||||
assertEquals(kvs2[0], results.get(1));
|
||||
|
@ -283,7 +282,7 @@ public class TestStoreScanner extends TestCase {
|
|||
StoreScanner scan = new StoreScanner(new Scan(Bytes.toBytes("R1")),
|
||||
scanInfo, scanType, null, scanners);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(2, results.size());
|
||||
assertEquals(kvs[0], results.get(0));
|
||||
assertEquals(kvs[1], results.get(1));
|
||||
|
@ -312,7 +311,7 @@ public class TestStoreScanner extends TestCase {
|
|||
StoreScanner scan = new StoreScanner(new Scan().setMaxVersions(2),
|
||||
scanInfo, scanType, null, scanners);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(5, results.size());
|
||||
assertEquals(kvs[0], results.get(0));
|
||||
assertEquals(kvs[2], results.get(1));
|
||||
|
@ -341,13 +340,13 @@ public class TestStoreScanner extends TestCase {
|
|||
new Scan().setMaxVersions(Integer.MAX_VALUE), scanInfo, scanType, null,
|
||||
scanners);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(0, results.size());
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(1, results.size());
|
||||
assertEquals(kvs[kvs.length-1], results.get(0));
|
||||
|
||||
assertEquals(false, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(false, scan.next(results));
|
||||
}
|
||||
|
||||
public void testDeleteColumn() throws IOException {
|
||||
|
@ -361,7 +360,7 @@ public class TestStoreScanner extends TestCase {
|
|||
StoreScanner scan = new StoreScanner(new Scan(), scanInfo, scanType, null,
|
||||
scanners);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(1, results.size());
|
||||
assertEquals(kvs[3], results.get(0));
|
||||
}
|
||||
|
@ -385,18 +384,18 @@ public class TestStoreScanner extends TestCase {
|
|||
getCols("a", "d"), scanners);
|
||||
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(2, results.size());
|
||||
assertEquals(kvs[0], results.get(0));
|
||||
assertEquals(kvs[3], results.get(1));
|
||||
results.clear();
|
||||
|
||||
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(true, scan.next(results));
|
||||
assertEquals(1, results.size());
|
||||
assertEquals(kvs[kvs.length-1], results.get(0));
|
||||
|
||||
results.clear();
|
||||
assertEquals(false, NextState.hasMoreValues(scan.next(results)));
|
||||
assertEquals(false, scan.next(results));
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -426,20 +425,20 @@ public class TestStoreScanner extends TestCase {
|
|||
null, scanners);
|
||||
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scanner.next(results)));
|
||||
assertEquals(true, scanner.next(results));
|
||||
assertEquals(2, results.size());
|
||||
assertEquals(kvs[1], results.get(0));
|
||||
assertEquals(kvs[2], results.get(1));
|
||||
results.clear();
|
||||
|
||||
assertEquals(true, NextState.hasMoreValues(scanner.next(results)));
|
||||
assertEquals(true, scanner.next(results));
|
||||
assertEquals(3, results.size());
|
||||
assertEquals(kvs[4], results.get(0));
|
||||
assertEquals(kvs[5], results.get(1));
|
||||
assertEquals(kvs[6], results.get(2));
|
||||
results.clear();
|
||||
|
||||
assertEquals(false, NextState.hasMoreValues(scanner.next(results)));
|
||||
assertEquals(false, scanner.next(results));
|
||||
}
|
||||
|
||||
public void testScannerReseekDoesntNPE() throws Exception {
|
||||
|
@ -496,12 +495,12 @@ public class TestStoreScanner extends TestCase {
|
|||
new StoreScanner(scan, scanInfo, scanType, null, scanners);
|
||||
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scanner.next(results)));
|
||||
assertEquals(true, scanner.next(results));
|
||||
assertEquals(1, results.size());
|
||||
assertEquals(kvs[1], results.get(0));
|
||||
results.clear();
|
||||
|
||||
assertEquals(false, NextState.hasMoreValues(scanner.next(results)));
|
||||
assertEquals(false, scanner.next(results));
|
||||
}
|
||||
|
||||
public void testDeleteMarkerLongevity() throws Exception {
|
||||
|
@ -561,7 +560,7 @@ public class TestStoreScanner extends TestCase {
|
|||
HConstants.OLDEST_TIMESTAMP);
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
results = new ArrayList<Cell>();
|
||||
assertEquals(true, NextState.hasMoreValues(scanner.next(results)));
|
||||
assertEquals(true, scanner.next(results));
|
||||
assertEquals(kvs[0], results.get(0));
|
||||
assertEquals(kvs[2], results.get(1));
|
||||
assertEquals(kvs[3], results.get(2));
|
||||
|
|
|
@ -237,22 +237,14 @@ public class TestStripeCompactor {
|
|||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> results) throws IOException {
|
||||
if (kvs.isEmpty()) return NextState.makeState(NextState.State.NO_MORE_VALUES);
|
||||
public boolean next(List<Cell> results) throws IOException {
|
||||
if (kvs.isEmpty()) return false;
|
||||
results.add(kvs.remove(0));
|
||||
if (!kvs.isEmpty()) {
|
||||
return NextState.makeState(NextState.State.MORE_VALUES);
|
||||
} else {
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES);
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public NextState next(List<Cell> result, int limit) throws IOException {
|
||||
return next(result);
|
||||
return !kvs.isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> result, int limit, long remainingResultSize)
|
||||
public boolean next(List<Cell> result, ScannerContext scannerContext)
|
||||
throws IOException {
|
||||
return next(result);
|
||||
}
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -106,7 +105,7 @@ public class TestWideScanner extends HBaseTestCase {
|
|||
int i = 0;
|
||||
boolean more;
|
||||
do {
|
||||
more = NextState.hasMoreValues(s.next(results));
|
||||
more = s.next(results);
|
||||
i++;
|
||||
LOG.info("iteration #" + i + ", results.size=" + results.size());
|
||||
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
|
|||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
@ -774,24 +775,15 @@ public class TestStripeCompactionPolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> results) throws IOException {
|
||||
if (kvs.isEmpty()) return NextState.makeState(NextState.State.NO_MORE_VALUES);
|
||||
public boolean next(List<Cell> results) throws IOException {
|
||||
if (kvs.isEmpty()) return false;
|
||||
results.add(kvs.remove(0));
|
||||
|
||||
if (!kvs.isEmpty()) {
|
||||
return NextState.makeState(NextState.State.MORE_VALUES);
|
||||
} else {
|
||||
return NextState.makeState(NextState.State.NO_MORE_VALUES);
|
||||
}
|
||||
return !kvs.isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> result, int limit) throws IOException {
|
||||
return next(result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NextState next(List<Cell> result, int limit, long remainingResultSize)
|
||||
public boolean next(List<Cell> result, ScannerContext scannerContext)
|
||||
throws IOException {
|
||||
return next(result);
|
||||
}
|
||||
|
|
|
@ -71,7 +71,6 @@ import org.apache.hadoop.hbase.regionserver.FlushRequestListener;
|
|||
import org.apache.hadoop.hbase.regionserver.FlushRequester;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
|
@ -751,7 +750,7 @@ public class TestWALReplay {
|
|||
int scannedCount = 0;
|
||||
List<Cell> results = new ArrayList<Cell>();
|
||||
while (true) {
|
||||
boolean existMore = NextState.hasMoreValues(scanner.next(results));
|
||||
boolean existMore = scanner.next(results);
|
||||
if (!results.isEmpty())
|
||||
scannedCount++;
|
||||
if (!existMore)
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
|
@ -232,7 +231,7 @@ public class TestMergeTool extends HBaseTestCase {
|
|||
List<Cell> testRes = null;
|
||||
while (true) {
|
||||
testRes = new ArrayList<Cell>();
|
||||
boolean hasNext = NextState.hasMoreValues(scanner.next(testRes));
|
||||
boolean hasNext = scanner.next(testRes);
|
||||
if (!hasNext) {
|
||||
break;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue