HBASE-11544: [Ergonomics] hbase.client.scanner.caching is dogged and will try to return batch even if it means OOME

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
Jonathan Lawlor 2015-02-04 14:07:35 -08:00 committed by stack
parent 39425a8558
commit de9791e91e
65 changed files with 2735 additions and 549 deletions

View File

@ -19,7 +19,10 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.ExecutorService;
import org.apache.commons.logging.Log;
@ -44,8 +47,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.annotations.VisibleForTesting;
import static org.apache.hadoop.hbase.client.ReversedClientScanner.createClosestRowBefore;
/**
* Implements the scanner interface for the HBase client.
* If there are multiple regions in a table, this scanner will iterate
@ -54,6 +55,9 @@ import static org.apache.hadoop.hbase.client.ReversedClientScanner.createClosest
@InterfaceAudience.Private
public class ClientScanner extends AbstractClientScanner {
private final Log LOG = LogFactory.getLog(this.getClass());
// A byte array in which all elements are the max byte, and it is used to
// construct closest front row
static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);
protected Scan scan;
protected boolean closed = false;
// Current region scanner is against. Gets cleared if current region goes
@ -61,6 +65,12 @@ public class ClientScanner extends AbstractClientScanner {
protected HRegionInfo currentRegion = null;
protected ScannerCallableWithReplicas callable = null;
protected final LinkedList<Result> cache = new LinkedList<Result>();
/**
* A list of partial results that have been returned from the server. This list should only
* contain results if this scanner does not have enough partial results to form the complete
* result.
*/
protected final LinkedList<Result> partialResults = new LinkedList<Result>();
protected final int caching;
protected long lastNext;
// Keep lastResult returned successfully in case we have to reset scanner.
@ -337,21 +347,23 @@ public class ClientScanner extends AbstractClientScanner {
return null;
}
if (cache.size() == 0) {
Result [] values = null;
Result[] values = null;
long remainingResultSize = maxScannerResultSize;
int countdown = this.caching;
// We need to reset it if it's a new callable that was created
// with a countdown in nextScanner
callable.setCaching(this.caching);
// This flag is set when we want to skip the result returned. We do
// This flag is set when we want to skip the result returned. We do
// this when we reset scanner because it split under us.
boolean retryAfterOutOfOrderException = true;
boolean retryAfterOutOfOrderException = true;
do {
try {
// Server returns a null values if scanning is to stop. Else,
// Server returns a null values if scanning is to stop. Else,
// returns an empty array if scanning is to go on and we've just
// exhausted current region.
values = call(callable, caller, scannerTimeout);
// When the replica switch happens, we need to do certain operations
// again. The callable will openScanner with the right startkey
// but we need to pick up from there. Bypass the rest of the loop
@ -362,9 +374,13 @@ public class ClientScanner extends AbstractClientScanner {
this.currentRegion = callable.getHRegionInfo();
continue;
}
retryAfterOutOfOrderException = true;
retryAfterOutOfOrderException = true;
} catch (DoNotRetryIOException e) {
// DNRIOEs are thrown to make us break out of retries. Some types of DNRIOEs want us
// An exception was thrown which makes any partial results that we were collecting
// invalid. The scanner will need to be reset to the beginning of a row.
partialResults.clear();
// DNRIOEs are thrown to make us break out of retries. Some types of DNRIOEs want us
// to reset the scanner and come back in again.
if (e instanceof UnknownScannerException) {
long timeout = lastNext + scannerTimeout;
@ -373,9 +389,9 @@ public class ClientScanner extends AbstractClientScanner {
// id against the new region server; reset the scanner.
if (timeout < System.currentTimeMillis()) {
long elapsed = System.currentTimeMillis() - lastNext;
ScannerTimeoutException ex = new ScannerTimeoutException(
elapsed + "ms passed since the last invocation, " +
"timeout is currently set to " + scannerTimeout);
ScannerTimeoutException ex =
new ScannerTimeoutException(elapsed + "ms passed since the last invocation, "
+ "timeout is currently set to " + scannerTimeout);
ex.initCause(e);
throw ex;
}
@ -384,8 +400,8 @@ public class ClientScanner extends AbstractClientScanner {
// the scanner and retry.
Throwable cause = e.getCause();
if ((cause != null && cause instanceof NotServingRegionException) ||
(cause != null && cause instanceof RegionServerStoppedException) ||
e instanceof OutOfOrderScannerNextException) {
(cause != null && cause instanceof RegionServerStoppedException) ||
e instanceof OutOfOrderScannerNextException) {
// Pass
// It is easier writing the if loop test as list of what is allowed rather than
// as a list of what is not allowed... so if in here, it means we do not throw.
@ -401,9 +417,9 @@ public class ClientScanner extends AbstractClientScanner {
// scanner starts at the correct row. Otherwise we may see previously
// returned rows again.
// (ScannerCallable by now has "relocated" the correct region)
if(scan.isReversed()){
if (scan.isReversed()) {
scan.setStartRow(createClosestRowBefore(lastResult.getRow()));
}else {
} else {
scan.setStartRow(Bytes.add(lastResult.getRow(), new byte[1]));
}
}
@ -413,7 +429,7 @@ public class ClientScanner extends AbstractClientScanner {
} else {
// TODO: Why wrap this in a DNRIOE when it already is a DNRIOE?
throw new DoNotRetryIOException("Failed after retry of " +
"OutOfOrderScannerNextException: was there a rpc timeout?", e);
"OutOfOrderScannerNextException: was there a rpc timeout?", e);
}
}
// Clear region.
@ -421,16 +437,21 @@ public class ClientScanner extends AbstractClientScanner {
// Set this to zero so we don't try and do an rpc and close on remote server when
// the exception we got was UnknownScanner or the Server is going down.
callable = null;
// This continue will take us to while at end of loop where we will set up new scanner.
continue;
}
long currentTime = System.currentTimeMillis();
if (this.scanMetrics != null) {
this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime-lastNext);
this.scanMetrics.sumOfMillisSecBetweenNexts.addAndGet(currentTime - lastNext);
}
lastNext = currentTime;
if (values != null && values.length > 0) {
for (Result rs : values) {
// Groom the array of Results that we received back from the server before adding that
// Results to the scanner's cache. If partial results are not allowed to be seen by the
// caller, all book keeping will be performed within this method.
List<Result> resultsToAddToCache = getResultsToAddToCache(values);
if (!resultsToAddToCache.isEmpty()) {
for (Result rs : resultsToAddToCache) {
cache.add(rs);
// We don't make Iterator here
for (Cell cell : rs.rawCells()) {
@ -441,8 +462,11 @@ public class ClientScanner extends AbstractClientScanner {
}
}
// Values == null means server-side filter has determined we must STOP
} while (remainingResultSize > 0 && countdown > 0 &&
possiblyNextScanner(countdown, values == null));
// !partialResults.isEmpty() means that we are still accumulating partial Results for a
// row. We should not change scanners before we receive all the partial Results for that
// row.
} while (remainingResultSize > 0 && countdown > 0
&& (!partialResults.isEmpty() || possiblyNextScanner(countdown, values == null)));
}
if (cache.size() > 0) {
@ -454,6 +478,145 @@ public class ClientScanner extends AbstractClientScanner {
return null;
}
@VisibleForTesting
public int getCacheSize() {
return cache != null ? cache.size() : 0;
}
/**
* This method ensures all of our book keeping regarding partial results is kept up to date. This
* method should be called once we know that the results we received back from the RPC request do
* not contain errors. We return a list of results that should be added to the cache. In general,
* this list will contain all NON-partial results from the input array (unless the client has
* specified that they are okay with receiving partial results)
* @return the list of results that should be added to the cache.
* @throws IOException
*/
protected List<Result> getResultsToAddToCache(Result[] resultsFromServer) throws IOException {
int resultSize = resultsFromServer != null ? resultsFromServer.length : 0;
List<Result> resultsToAddToCache = new ArrayList<Result>(resultSize);
final boolean isBatchSet = scan != null && scan.getBatch() > 0;
final boolean allowPartials = scan != null && scan.getAllowPartialResults();
// If the caller has indicated in their scan that they are okay with seeing partial results,
// then simply add all results to the list. Note that since scan batching also returns results
// for a row in pieces we treat batch being set as equivalent to allowing partials. The
// implication of treating batching as equivalent to partial results is that it is possible
// the caller will receive a result back where the number of cells in the result is less than
// the batch size even though it may not be the last group of cells for that row.
if (allowPartials || isBatchSet) {
addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length);
return resultsToAddToCache;
}
// If no results were returned it indicates that we have the all the partial results necessary
// to construct the complete result.
if (resultsFromServer == null || resultsFromServer.length == 0) {
if (!partialResults.isEmpty()) {
resultsToAddToCache.add(Result.createCompleteResult(partialResults));
partialResults.clear();
}
return resultsToAddToCache;
}
// In every RPC response there should be at most a single partial result. Furthermore, if
// there is a partial result, it is guaranteed to be in the last position of the array.
Result last = resultsFromServer[resultsFromServer.length - 1];
Result partial = last.isPartial() ? last : null;
if (LOG.isTraceEnabled()) {
StringBuilder sb = new StringBuilder();
sb.append("number results from RPC: ").append(resultsFromServer.length).append(",");
sb.append("partial != null: ").append(partial != null).append(",");
sb.append("number of partials so far: ").append(partialResults.size());
LOG.trace(sb.toString());
}
// There are four possibilities cases that can occur while handling partial results
//
// 1. (partial != null && partialResults.isEmpty())
// This is the first partial result that we have received. It should be added to
// the list of partialResults and await the next RPC request at which point another
// portion of the complete result will be received
//
// 2. (partial != null && !partialResults.isEmpty())
// a. values.length == 1
// Since partialResults contains some elements, it means that we are expecting to receive
// the remainder of the complete result within this RPC response. The fact that a partial result
// was returned and it's the ONLY result returned indicates that we are still receiving
// fragments of the complete result. The Result can be completely formed only when we have
// received all of the fragments and thus in this case we simply add the partial result to
// our list.
//
// b. values.length > 1
// More than one result has been returned from the server. The fact that we are accumulating
// partials in partialList and we just received more than one result back from the server
// indicates that the FIRST result we received from the server must be the final fragment that
// can be used to complete our result. What this means is that the partial that we received is
// a partial result for a different row, and at this point we should combine the existing
// partials into a complete result, clear the partialList, and begin accumulating partials for
// a new row
//
// 3. (partial == null && !partialResults.isEmpty())
// No partial was received but we are accumulating partials in our list. That means the final
// fragment of the complete result will be the first Result in values[]. We use it to create the
// complete Result, clear the list, and add it to the list of Results that must be added to the
// cache. All other Results in values[] are added after the complete result to maintain proper
// ordering
//
// 4. (partial == null && partialResults.isEmpty())
// Business as usual. We are not accumulating partial results and there wasn't a partial result
// in the RPC response. This means that all of the results we received from the server are
// complete and can be added directly to the cache
if (partial != null && partialResults.isEmpty()) {
partialResults.add(partial);
// Exclude the last result, it's a partial
addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length - 1);
} else if (partial != null && !partialResults.isEmpty()) {
if (resultsFromServer.length > 1) {
Result finalResult = resultsFromServer[0];
partialResults.add(finalResult);
resultsToAddToCache.add(Result.createCompleteResult(partialResults));
partialResults.clear();
// Exclude first result, it was used to form our complete result
// Exclude last result, it's a partial result
addResultsToList(resultsToAddToCache, resultsFromServer, 1, resultsFromServer.length - 1);
}
partialResults.add(partial);
} else if (partial == null && !partialResults.isEmpty()) {
Result finalResult = resultsFromServer[0];
partialResults.add(finalResult);
resultsToAddToCache.add(Result.createCompleteResult(partialResults));
partialResults.clear();
// Exclude the first result, it was used to form our complete result
addResultsToList(resultsToAddToCache, resultsFromServer, 1, resultsFromServer.length);
} else { // partial == null && partialResults.isEmpty() -- business as usual
addResultsToList(resultsToAddToCache, resultsFromServer, 0, resultsFromServer.length);
}
return resultsToAddToCache;
}
/**
* Helper method for adding results between the indices [start, end) to the outputList
* @param outputList the list that results will be added to
* @param inputArray the array that results are taken from
* @param start beginning index (inclusive)
* @param end ending index (exclusive)
*/
private void addResultsToList(List<Result> outputList, Result[] inputArray, int start, int end) {
if (inputArray == null || start < 0 || end > inputArray.length) return;
for (int i = start; i < end; i++) {
outputList.add(inputArray[i]);
}
}
@Override
public void close() {
if (!scanMetricsPublished) writeScanMetrics();
@ -476,4 +639,26 @@ public class ClientScanner extends AbstractClientScanner {
}
closed = true;
}
/**
* Create the closest row before the specified row
* @param row
* @return a new byte array which is the closest front row of the specified one
*/
protected static byte[] createClosestRowBefore(byte[] row) {
if (row == null) {
throw new IllegalArgumentException("The passed row is empty");
}
if (Bytes.equals(row, HConstants.EMPTY_BYTE_ARRAY)) {
return MAX_BYTE_ARRAY;
}
if (row[row.length - 1] == 0) {
return Arrays.copyOf(row, row.length - 1);
} else {
byte[] closestFrontRow = Arrays.copyOf(row, row.length);
closestFrontRow[row.length - 1] = (byte) ((closestFrontRow[row.length - 1] & 0xff) - 1);
closestFrontRow = Bytes.add(closestFrontRow, MAX_BYTE_ARRAY);
return closestFrontRow;
}
}
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.nio.BufferOverflowException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@ -33,6 +34,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -80,6 +82,17 @@ public class Result implements CellScannable, CellScanner {
private Cell[] cells;
private Boolean exists; // if the query was just to check existence.
private boolean stale = false;
/**
* Partial results do not contain the full row's worth of cells. The result had to be returned in
* parts because the size of the cells in the row exceeded the RPC result size on the server.
* Partial results must be combined client side with results representing the remainder of the
* row's cells to form the complete result. Partial results and RPC result size allow us to avoid
* OOME on the server when servicing requests for large rows. The Scan configuration used to
* control the result size on the server is {@link Scan#setMaxResultSize(long)} and the default
* value can be seen here: {@link HConstants#DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE}
*/
private boolean partial = false;
// We're not using java serialization. Transient here is just a marker to say
// that this is where we cache row if we're ever asked for it.
private transient byte [] row = null;
@ -115,7 +128,7 @@ public class Result implements CellScannable, CellScanner {
* @param cells List of cells
*/
public static Result create(List<Cell> cells) {
return new Result(cells.toArray(new Cell[cells.size()]), null, false);
return create(cells, null);
}
public static Result create(List<Cell> cells, Boolean exists) {
@ -123,10 +136,14 @@ public class Result implements CellScannable, CellScanner {
}
public static Result create(List<Cell> cells, Boolean exists, boolean stale) {
return create(cells, exists, stale, false);
}
public static Result create(List<Cell> cells, Boolean exists, boolean stale, boolean partial) {
if (exists != null){
return new Result(null, exists, stale);
return new Result(null, exists, stale, partial);
}
return new Result(cells.toArray(new Cell[cells.size()]), null, stale);
return new Result(cells.toArray(new Cell[cells.size()]), null, stale, partial);
}
/**
@ -135,21 +152,26 @@ public class Result implements CellScannable, CellScanner {
* @param cells array of cells
*/
public static Result create(Cell[] cells) {
return new Result(cells, null, false);
return create(cells, null, false);
}
public static Result create(Cell[] cells, Boolean exists, boolean stale) {
return create(cells, exists, stale, false);
}
public static Result create(Cell[] cells, Boolean exists, boolean stale, boolean partial) {
if (exists != null){
return new Result(null, exists, stale);
return new Result(null, exists, stale, partial);
}
return new Result(cells, null, stale);
return new Result(cells, null, stale, partial);
}
/** Private ctor. Use {@link #create(Cell[])}. */
private Result(Cell[] cells, Boolean exists, boolean stale) {
private Result(Cell[] cells, Boolean exists, boolean stale, boolean partial) {
this.cells = cells;
this.exists = exists;
this.stale = stale;
this.partial = partial;
}
/**
@ -746,7 +768,59 @@ public class Result implements CellScannable, CellScanner {
}
/**
* Get total size of raw cells
* Forms a single result from the partial results in the partialResults list. This method is
* useful for reconstructing partial results on the client side.
* @param partialResults list of partial results
* @return The complete result that is formed by combining all of the partial results together
* @throws IOException A complete result cannot be formed because the results in the partial list
* come from different rows
*/
public static Result createCompleteResult(List<Result> partialResults)
throws IOException {
List<Cell> cells = new ArrayList<Cell>();
boolean stale = false;
byte[] prevRow = null;
byte[] currentRow = null;
if (partialResults != null && !partialResults.isEmpty()) {
for (int i = 0; i < partialResults.size(); i++) {
Result r = partialResults.get(i);
currentRow = r.getRow();
if (prevRow != null && !Bytes.equals(prevRow, currentRow)) {
throw new IOException(
"Cannot form complete result. Rows of partial results do not match." +
" Partial Results: " + partialResults);
}
// Ensure that all Results except the last one are marked as partials. The last result
// may not be marked as a partial because Results are only marked as partials when
// the scan on the server side must be stopped due to reaching the maxResultSize.
// Visualizing it makes it easier to understand:
// maxResultSize: 2 cells
// (-x-) represents cell number x in a row
// Example: row1: -1- -2- -3- -4- -5- (5 cells total)
// How row1 will be returned by the server as partial Results:
// Result1: -1- -2- (2 cells, size limit reached, mark as partial)
// Result2: -3- -4- (2 cells, size limit reached, mark as partial)
// Result3: -5- (1 cell, size limit NOT reached, NOT marked as partial)
if (i != (partialResults.size() - 1) && !r.isPartial()) {
throw new IOException(
"Cannot form complete result. Result is missing partial flag. " +
"Partial Results: " + partialResults);
}
prevRow = currentRow;
stale = stale || r.isStale();
for (Cell c : r.rawCells()) {
cells.add(c);
}
}
}
return Result.create(cells, null, stale);
}
/**
* Get total size of raw cells
* @param result
* @return Total size.
*/
@ -804,6 +878,16 @@ public class Result implements CellScannable, CellScanner {
return stale;
}
/**
* Whether or not the result is a partial result. Partial results contain a subset of the cells
* for a row and should be combined with a result representing the remaining cells in that row to
* form a complete (non-partial) result.
* @return Whether or not the result is a partial result
*/
public boolean isPartial() {
return partial;
}
/**
* Add load information about the region to the information about the result
* @param loadStats statistics about the current region from which this was returned

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.Arrays;
import java.util.concurrent.ExecutorService;
import org.apache.commons.logging.Log;
@ -38,9 +37,6 @@ import org.apache.hadoop.hbase.util.ExceptionUtil;
@InterfaceAudience.Private
public class ReversedClientScanner extends ClientScanner {
private static final Log LOG = LogFactory.getLog(ReversedClientScanner.class);
// A byte array in which all elements are the max byte, and it is used to
// construct closest front row
static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);
/**
* Create a new ReversibleClientScanner for the specified table Note that the
@ -139,9 +135,10 @@ public class ReversedClientScanner extends ClientScanner {
new ReversedScannerCallable(getConnection(), getTable(), scan, this.scanMetrics,
locateStartRow, this.rpcControllerFactory);
s.setCaching(nbRows);
ScannerCallableWithReplicas sr = new ScannerCallableWithReplicas(getTable(), getConnection(),
s, pool, primaryOperationTimeout, scan,
getRetries(), getScannerTimeout(), caching, getConf(), caller);
ScannerCallableWithReplicas sr =
new ScannerCallableWithReplicas(getTable(), getConnection(), s, pool,
primaryOperationTimeout, scan, getRetries(), getScannerTimeout(), caching, getConf(),
caller);
return sr;
}
@ -161,26 +158,4 @@ public class ReversedClientScanner extends ClientScanner {
}
return false; // unlikely.
}
/**
* Create the closest row before the specified row
* @param row
* @return a new byte array which is the closest front row of the specified one
*/
protected static byte[] createClosestRowBefore(byte[] row) {
if (row == null) {
throw new IllegalArgumentException("The passed row is empty");
}
if (Bytes.equals(row, HConstants.EMPTY_BYTE_ARRAY)) {
return MAX_BYTE_ARRAY;
}
if (row[row.length - 1] == 0) {
return Arrays.copyOf(row, row.length - 1);
} else {
byte[] closestFrontRow = Arrays.copyOf(row, row.length);
closestFrontRow[row.length - 1] = (byte) ((closestFrontRow[row.length - 1] & 0xff) - 1);
closestFrontRow = Bytes.add(closestFrontRow, MAX_BYTE_ARRAY);
return closestFrontRow;
}
}
}

View File

@ -98,6 +98,19 @@ public class Scan extends Query {
private int maxVersions = 1;
private int batch = -1;
/**
* Partial {@link Result}s are {@link Result}s must be combined to form a complete {@link Result}.
* The {@link Result}s had to be returned in fragments (i.e. as partials) because the size of the
* cells in the row exceeded max result size on the server. Typically partial results will be
* combined client side into complete results before being delivered to the caller. However, if
* this flag is set, the caller is indicating that they do not mind seeing partial results (i.e.
* they understand that the results returned from the Scanner may only represent part of a
* particular row). In such a case, any attempt to combine the partials into a complete result on
* the client side will be skipped, and the caller will be able to see the exact results returned
* from the server.
*/
private boolean allowPartialResults = false;
private int storeLimit = -1;
private int storeOffset = 0;
private boolean getScan;
@ -674,6 +687,27 @@ public class Scan extends Query {
return reversed;
}
/**
* Setting whether the caller wants to see the partial results that may be returned from the
* server. By default this value is false and the complete results will be assembled client side
* before being delivered to the caller.
* @param allowPartialResults
* @return this
*/
public Scan setAllowPartialResults(final boolean allowPartialResults) {
this.allowPartialResults = allowPartialResults;
return this;
}
/**
* @return true when the constructor of this scan understands that the results they will see may
* only represent a partial portion of a row. The entire row would be retrieved by
* subsequent calls to {@link ResultScanner#next()}
*/
public boolean getAllowPartialResults() {
return allowPartialResults;
}
/**
* Set the value indicating whether loading CFs on demand should be allowed (cluster
* default is false). On-demand CF loading doesn't load column families until necessary, e.g.

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.client.ClientScanner.createClosestRowBefore;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
@ -44,8 +46,6 @@ import org.apache.hadoop.hbase.util.Pair;
import com.google.common.annotations.VisibleForTesting;
import static org.apache.hadoop.hbase.client.ReversedClientScanner.createClosestRowBefore;
/**
* This class has the logic for handling scanners for regions with and without replicas.
* 1. A scan is attempted on the default (primary) region

View File

@ -37,7 +37,6 @@ import java.util.Map.Entry;
import java.util.NavigableSet;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
@ -53,6 +52,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Delete;
@ -68,7 +68,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
@ -1285,6 +1284,7 @@ public final class ProtobufUtil {
}
builder.setStale(result.isStale());
builder.setPartial(result.isPartial());
return builder.build();
}
@ -1343,7 +1343,7 @@ public final class ProtobufUtil {
for (CellProtos.Cell c : values) {
cells.add(toCell(c));
}
return Result.create(cells, null, proto.getStale());
return Result.create(cells, null, proto.getStale(), proto.getPartial());
}
/**

View File

@ -486,6 +486,7 @@ public final class RequestConverter {
builder.setCloseScanner(closeScanner);
builder.setRegion(region);
builder.setScan(ProtobufUtil.toScan(scan));
builder.setClientHandlesPartials(true);
return builder.build();
}
@ -503,6 +504,7 @@ public final class RequestConverter {
builder.setNumberOfRows(numberOfRows);
builder.setCloseScanner(closeScanner);
builder.setScannerId(scannerId);
builder.setClientHandlesPartials(true);
return builder.build();
}
@ -522,6 +524,7 @@ public final class RequestConverter {
builder.setCloseScanner(closeScanner);
builder.setScannerId(scannerId);
builder.setNextCallSeq(nextCallSeq);
builder.setClientHandlesPartials(true);
return builder.build();
}

View File

@ -339,6 +339,9 @@ public final class ResponseConverter {
// Cells are out in cellblocks. Group them up again as Results. How many to read at a
// time will be found in getCellsLength -- length here is how many Cells in the i'th Result
int noOfCells = response.getCellsPerResult(i);
boolean isPartial =
response.getPartialFlagPerResultCount() > i ?
response.getPartialFlagPerResult(i) : false;
List<Cell> cells = new ArrayList<Cell>(noOfCells);
for (int j = 0; j < noOfCells; j++) {
try {
@ -361,7 +364,7 @@ public final class ResponseConverter {
}
cells.add(cellScanner.current());
}
results[i] = Result.create(cells, null, response.getStale());
results[i] = Result.create(cells, null, response.getStale(), isPartial);
} else {
// Result is pure pb.
results[i] = ProtobufUtil.toResult(response.getResults(i));

View File

@ -698,7 +698,7 @@ public final class HConstants {
/**
* Default value for {@link #HBASE_CLIENT_SCANNER_CACHING}
*/
public static final int DEFAULT_HBASE_CLIENT_SCANNER_CACHING = 100;
public static final int DEFAULT_HBASE_CLIENT_SCANNER_CACHING = Integer.MAX_VALUE;
/**
* Parameter name for number of rows that will be fetched when calling next on

View File

@ -527,13 +527,19 @@ possible configurations would overwhelm and obscure the important.
</property>
<property>
<name>hbase.client.scanner.caching</name>
<value>100</value>
<description>Number of rows that will be fetched when calling next
on a scanner if it is not served from (local, client) memory. Higher
caching values will enable faster scanners but will eat up more memory
and some calls of next may take longer and longer times when the cache is empty.
Do not set this value such that the time between invocations is greater
than the scanner timeout; i.e. hbase.client.scanner.timeout.period</description>
<value>2147483647</value>
<description>Number of rows that we try to fetch when calling next
on a scanner if it is not served from (local, client) memory. This configuration
works together with hbase.client.scanner.max.result.size to try and use the
network efficiently. The default value is Integer.MAX_VALUE by default so that
the network will fill the chunk size defined by hbase.client.scanner.max.result.size
rather than be limited by a particular number of rows since the size of rows varies
table to table. If you know ahead of time that you will not require more than a certain
number of rows from a scan, this configuration should be set to that row limit via
Scan#setCaching. Higher caching values will enable faster scanners but will eat up more
memory and some calls of next may take longer and longer times when the cache is empty.
Do not set this value such that the time between invocations is greater than the scanner
timeout; i.e. hbase.client.scanner.timeout.period</description>
</property>
<property>
<name>hbase.client.keyvalue.maxsize</name>

View File

@ -26,11 +26,11 @@ import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Mutation;
@ -39,14 +39,15 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.Builder;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
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.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.OperationStatus;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.util.Bytes;
@ -136,7 +137,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 = scanner.next(results);
hasMore = NextState.hasMoreValues(scanner.next(results));
if (results.size() > 0) {
deleteRows.add(results);
}

View File

@ -34,6 +34,7 @@ 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;
@ -80,7 +81,7 @@ public class RowCountEndpoint extends ExampleProtos.RowCountService
byte[] lastRow = null;
long count = 0;
do {
hasMore = scanner.next(results);
hasMore = NextState.hasMoreValues(scanner.next(results));
for (Cell kv : results) {
byte[] currentRow = CellUtil.cloneRow(kv);
if (lastRow == null || !Bytes.equals(lastRow, currentRow)) {
@ -119,7 +120,7 @@ public class RowCountEndpoint extends ExampleProtos.RowCountService
boolean hasMore = false;
long count = 0;
do {
hasMore = scanner.next(results);
hasMore = NextState.hasMoreValues(scanner.next(results));
for (Cell kv : results) {
count++;
}

View File

@ -4247,6 +4247,30 @@ public final class ClientProtos {
* </pre>
*/
boolean getStale();
// optional bool partial = 5 [default = false];
/**
* <code>optional bool partial = 5 [default = false];</code>
*
* <pre>
* Whether or not the entire result could be returned. Results will be split when
* the RPC chunk size limit is reached. Partial results contain only a subset of the
* cells for a row and must be combined with a result containing the remaining cells
* to form a complete result
* </pre>
*/
boolean hasPartial();
/**
* <code>optional bool partial = 5 [default = false];</code>
*
* <pre>
* Whether or not the entire result could be returned. Results will be split when
* the RPC chunk size limit is reached. Partial results contain only a subset of the
* cells for a row and must be combined with a result containing the remaining cells
* to form a complete result
* </pre>
*/
boolean getPartial();
}
/**
* Protobuf type {@code Result}
@ -4322,6 +4346,11 @@ public final class ClientProtos {
stale_ = input.readBool();
break;
}
case 40: {
bitField0_ |= 0x00000008;
partial_ = input.readBool();
break;
}
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -4510,11 +4539,42 @@ public final class ClientProtos {
return stale_;
}
// optional bool partial = 5 [default = false];
public static final int PARTIAL_FIELD_NUMBER = 5;
private boolean partial_;
/**
* <code>optional bool partial = 5 [default = false];</code>
*
* <pre>
* Whether or not the entire result could be returned. Results will be split when
* the RPC chunk size limit is reached. Partial results contain only a subset of the
* cells for a row and must be combined with a result containing the remaining cells
* to form a complete result
* </pre>
*/
public boolean hasPartial() {
return ((bitField0_ & 0x00000008) == 0x00000008);
}
/**
* <code>optional bool partial = 5 [default = false];</code>
*
* <pre>
* Whether or not the entire result could be returned. Results will be split when
* the RPC chunk size limit is reached. Partial results contain only a subset of the
* cells for a row and must be combined with a result containing the remaining cells
* to form a complete result
* </pre>
*/
public boolean getPartial() {
return partial_;
}
private void initFields() {
cell_ = java.util.Collections.emptyList();
associatedCellCount_ = 0;
exists_ = false;
stale_ = false;
partial_ = false;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -4540,6 +4600,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000004) == 0x00000004)) {
output.writeBool(4, stale_);
}
if (((bitField0_ & 0x00000008) == 0x00000008)) {
output.writeBool(5, partial_);
}
getUnknownFields().writeTo(output);
}
@ -4565,6 +4628,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(4, stale_);
}
if (((bitField0_ & 0x00000008) == 0x00000008)) {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(5, partial_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@ -4605,6 +4672,11 @@ public final class ClientProtos {
result = result && (getStale()
== other.getStale());
}
result = result && (hasPartial() == other.hasPartial());
if (hasPartial()) {
result = result && (getPartial()
== other.getPartial());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@ -4634,6 +4706,10 @@ public final class ClientProtos {
hash = (37 * hash) + STALE_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getStale());
}
if (hasPartial()) {
hash = (37 * hash) + PARTIAL_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getPartial());
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@ -4756,6 +4832,8 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00000004);
stale_ = false;
bitField0_ = (bitField0_ & ~0x00000008);
partial_ = false;
bitField0_ = (bitField0_ & ~0x00000010);
return this;
}
@ -4805,6 +4883,10 @@ public final class ClientProtos {
to_bitField0_ |= 0x00000004;
}
result.stale_ = stale_;
if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
to_bitField0_ |= 0x00000008;
}
result.partial_ = partial_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -4856,6 +4938,9 @@ public final class ClientProtos {
if (other.hasStale()) {
setStale(other.getStale());
}
if (other.hasPartial()) {
setPartial(other.getPartial());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@ -5384,6 +5469,67 @@ public final class ClientProtos {
return this;
}
// optional bool partial = 5 [default = false];
private boolean partial_ ;
/**
* <code>optional bool partial = 5 [default = false];</code>
*
* <pre>
* Whether or not the entire result could be returned. Results will be split when
* the RPC chunk size limit is reached. Partial results contain only a subset of the
* cells for a row and must be combined with a result containing the remaining cells
* to form a complete result
* </pre>
*/
public boolean hasPartial() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
/**
* <code>optional bool partial = 5 [default = false];</code>
*
* <pre>
* Whether or not the entire result could be returned. Results will be split when
* the RPC chunk size limit is reached. Partial results contain only a subset of the
* cells for a row and must be combined with a result containing the remaining cells
* to form a complete result
* </pre>
*/
public boolean getPartial() {
return partial_;
}
/**
* <code>optional bool partial = 5 [default = false];</code>
*
* <pre>
* Whether or not the entire result could be returned. Results will be split when
* the RPC chunk size limit is reached. Partial results contain only a subset of the
* cells for a row and must be combined with a result containing the remaining cells
* to form a complete result
* </pre>
*/
public Builder setPartial(boolean value) {
bitField0_ |= 0x00000010;
partial_ = value;
onChanged();
return this;
}
/**
* <code>optional bool partial = 5 [default = false];</code>
*
* <pre>
* Whether or not the entire result could be returned. Results will be split when
* the RPC chunk size limit is reached. Partial results contain only a subset of the
* cells for a row and must be combined with a result containing the remaining cells
* to form a complete result
* </pre>
*/
public Builder clearPartial() {
bitField0_ = (bitField0_ & ~0x00000010);
partial_ = false;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:Result)
}
@ -16277,6 +16423,16 @@ public final class ClientProtos {
* <code>optional uint64 next_call_seq = 6;</code>
*/
long getNextCallSeq();
// optional bool client_handles_partials = 7;
/**
* <code>optional bool client_handles_partials = 7;</code>
*/
boolean hasClientHandlesPartials();
/**
* <code>optional bool client_handles_partials = 7;</code>
*/
boolean getClientHandlesPartials();
}
/**
* Protobuf type {@code ScanRequest}
@ -16388,6 +16544,11 @@ public final class ClientProtos {
nextCallSeq_ = input.readUInt64();
break;
}
case 56: {
bitField0_ |= 0x00000040;
clientHandlesPartials_ = input.readBool();
break;
}
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -16536,6 +16697,22 @@ public final class ClientProtos {
return nextCallSeq_;
}
// optional bool client_handles_partials = 7;
public static final int CLIENT_HANDLES_PARTIALS_FIELD_NUMBER = 7;
private boolean clientHandlesPartials_;
/**
* <code>optional bool client_handles_partials = 7;</code>
*/
public boolean hasClientHandlesPartials() {
return ((bitField0_ & 0x00000040) == 0x00000040);
}
/**
* <code>optional bool client_handles_partials = 7;</code>
*/
public boolean getClientHandlesPartials() {
return clientHandlesPartials_;
}
private void initFields() {
region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance();
scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance();
@ -16543,6 +16720,7 @@ public final class ClientProtos {
numberOfRows_ = 0;
closeScanner_ = false;
nextCallSeq_ = 0L;
clientHandlesPartials_ = false;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -16586,6 +16764,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000020) == 0x00000020)) {
output.writeUInt64(6, nextCallSeq_);
}
if (((bitField0_ & 0x00000040) == 0x00000040)) {
output.writeBool(7, clientHandlesPartials_);
}
getUnknownFields().writeTo(output);
}
@ -16619,6 +16800,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(6, nextCallSeq_);
}
if (((bitField0_ & 0x00000040) == 0x00000040)) {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(7, clientHandlesPartials_);
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@ -16672,6 +16857,11 @@ public final class ClientProtos {
result = result && (getNextCallSeq()
== other.getNextCallSeq());
}
result = result && (hasClientHandlesPartials() == other.hasClientHandlesPartials());
if (hasClientHandlesPartials()) {
result = result && (getClientHandlesPartials()
== other.getClientHandlesPartials());
}
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@ -16709,6 +16899,10 @@ public final class ClientProtos {
hash = (37 * hash) + NEXT_CALL_SEQ_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getNextCallSeq());
}
if (hasClientHandlesPartials()) {
hash = (37 * hash) + CLIENT_HANDLES_PARTIALS_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getClientHandlesPartials());
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@ -16853,6 +17047,8 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00000010);
nextCallSeq_ = 0L;
bitField0_ = (bitField0_ & ~0x00000020);
clientHandlesPartials_ = false;
bitField0_ = (bitField0_ & ~0x00000040);
return this;
}
@ -16913,6 +17109,10 @@ public final class ClientProtos {
to_bitField0_ |= 0x00000020;
}
result.nextCallSeq_ = nextCallSeq_;
if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
to_bitField0_ |= 0x00000040;
}
result.clientHandlesPartials_ = clientHandlesPartials_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -16947,6 +17147,9 @@ public final class ClientProtos {
if (other.hasNextCallSeq()) {
setNextCallSeq(other.getNextCallSeq());
}
if (other.hasClientHandlesPartials()) {
setClientHandlesPartials(other.getClientHandlesPartials());
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@ -17352,6 +17555,39 @@ public final class ClientProtos {
return this;
}
// optional bool client_handles_partials = 7;
private boolean clientHandlesPartials_ ;
/**
* <code>optional bool client_handles_partials = 7;</code>
*/
public boolean hasClientHandlesPartials() {
return ((bitField0_ & 0x00000040) == 0x00000040);
}
/**
* <code>optional bool client_handles_partials = 7;</code>
*/
public boolean getClientHandlesPartials() {
return clientHandlesPartials_;
}
/**
* <code>optional bool client_handles_partials = 7;</code>
*/
public Builder setClientHandlesPartials(boolean value) {
bitField0_ |= 0x00000040;
clientHandlesPartials_ = value;
onChanged();
return this;
}
/**
* <code>optional bool client_handles_partials = 7;</code>
*/
public Builder clearClientHandlesPartials() {
bitField0_ = (bitField0_ & ~0x00000040);
clientHandlesPartials_ = false;
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:ScanRequest)
}
@ -17504,6 +17740,50 @@ public final class ClientProtos {
* <code>optional bool stale = 6;</code>
*/
boolean getStale();
// repeated bool partial_flag_per_result = 7;
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
java.util.List<java.lang.Boolean> getPartialFlagPerResultList();
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
int getPartialFlagPerResultCount();
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
boolean getPartialFlagPerResult(int index);
}
/**
* Protobuf type {@code ScanResponse}
@ -17611,6 +17891,27 @@ public final class ClientProtos {
stale_ = input.readBool();
break;
}
case 56: {
if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
partialFlagPerResult_ = new java.util.ArrayList<java.lang.Boolean>();
mutable_bitField0_ |= 0x00000040;
}
partialFlagPerResult_.add(input.readBool());
break;
}
case 58: {
int length = input.readRawVarint32();
int limit = input.pushLimit(length);
if (!((mutable_bitField0_ & 0x00000040) == 0x00000040) && input.getBytesUntilLimit() > 0) {
partialFlagPerResult_ = new java.util.ArrayList<java.lang.Boolean>();
mutable_bitField0_ |= 0x00000040;
}
while (input.getBytesUntilLimit() > 0) {
partialFlagPerResult_.add(input.readBool());
}
input.popLimit(limit);
break;
}
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -17625,6 +17926,9 @@ public final class ClientProtos {
if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
results_ = java.util.Collections.unmodifiableList(results_);
}
if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
partialFlagPerResult_ = java.util.Collections.unmodifiableList(partialFlagPerResult_);
}
this.unknownFields = unknownFields.build();
makeExtensionsImmutable();
}
@ -17840,6 +18144,59 @@ public final class ClientProtos {
return stale_;
}
// repeated bool partial_flag_per_result = 7;
public static final int PARTIAL_FLAG_PER_RESULT_FIELD_NUMBER = 7;
private java.util.List<java.lang.Boolean> partialFlagPerResult_;
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
public java.util.List<java.lang.Boolean>
getPartialFlagPerResultList() {
return partialFlagPerResult_;
}
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
public int getPartialFlagPerResultCount() {
return partialFlagPerResult_.size();
}
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
public boolean getPartialFlagPerResult(int index) {
return partialFlagPerResult_.get(index);
}
private void initFields() {
cellsPerResult_ = java.util.Collections.emptyList();
scannerId_ = 0L;
@ -17847,6 +18204,7 @@ public final class ClientProtos {
ttl_ = 0;
results_ = java.util.Collections.emptyList();
stale_ = false;
partialFlagPerResult_ = java.util.Collections.emptyList();
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -17878,6 +18236,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000008) == 0x00000008)) {
output.writeBool(6, stale_);
}
for (int i = 0; i < partialFlagPerResult_.size(); i++) {
output.writeBool(7, partialFlagPerResult_.get(i));
}
getUnknownFields().writeTo(output);
}
@ -17916,6 +18277,12 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(6, stale_);
}
{
int dataSize = 0;
dataSize = 1 * getPartialFlagPerResultList().size();
size += dataSize;
size += 1 * getPartialFlagPerResultList().size();
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@ -17963,6 +18330,8 @@ public final class ClientProtos {
result = result && (getStale()
== other.getStale());
}
result = result && getPartialFlagPerResultList()
.equals(other.getPartialFlagPerResultList());
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@ -18000,6 +18369,10 @@ public final class ClientProtos {
hash = (37 * hash) + STALE_FIELD_NUMBER;
hash = (53 * hash) + hashBoolean(getStale());
}
if (getPartialFlagPerResultCount() > 0) {
hash = (37 * hash) + PARTIAL_FLAG_PER_RESULT_FIELD_NUMBER;
hash = (53 * hash) + getPartialFlagPerResultList().hashCode();
}
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@ -18132,6 +18505,8 @@ public final class ClientProtos {
}
stale_ = false;
bitField0_ = (bitField0_ & ~0x00000020);
partialFlagPerResult_ = java.util.Collections.emptyList();
bitField0_ = (bitField0_ & ~0x00000040);
return this;
}
@ -18190,6 +18565,11 @@ public final class ClientProtos {
to_bitField0_ |= 0x00000008;
}
result.stale_ = stale_;
if (((bitField0_ & 0x00000040) == 0x00000040)) {
partialFlagPerResult_ = java.util.Collections.unmodifiableList(partialFlagPerResult_);
bitField0_ = (bitField0_ & ~0x00000040);
}
result.partialFlagPerResult_ = partialFlagPerResult_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -18254,6 +18634,16 @@ public final class ClientProtos {
if (other.hasStale()) {
setStale(other.getStale());
}
if (!other.partialFlagPerResult_.isEmpty()) {
if (partialFlagPerResult_.isEmpty()) {
partialFlagPerResult_ = other.partialFlagPerResult_;
bitField0_ = (bitField0_ & ~0x00000040);
} else {
ensurePartialFlagPerResultIsMutable();
partialFlagPerResult_.addAll(other.partialFlagPerResult_);
}
onChanged();
}
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@ -18897,6 +19287,142 @@ public final class ClientProtos {
return this;
}
// repeated bool partial_flag_per_result = 7;
private java.util.List<java.lang.Boolean> partialFlagPerResult_ = java.util.Collections.emptyList();
private void ensurePartialFlagPerResultIsMutable() {
if (!((bitField0_ & 0x00000040) == 0x00000040)) {
partialFlagPerResult_ = new java.util.ArrayList<java.lang.Boolean>(partialFlagPerResult_);
bitField0_ |= 0x00000040;
}
}
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
public java.util.List<java.lang.Boolean>
getPartialFlagPerResultList() {
return java.util.Collections.unmodifiableList(partialFlagPerResult_);
}
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
public int getPartialFlagPerResultCount() {
return partialFlagPerResult_.size();
}
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
public boolean getPartialFlagPerResult(int index) {
return partialFlagPerResult_.get(index);
}
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
public Builder setPartialFlagPerResult(
int index, boolean value) {
ensurePartialFlagPerResultIsMutable();
partialFlagPerResult_.set(index, value);
onChanged();
return this;
}
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
public Builder addPartialFlagPerResult(boolean value) {
ensurePartialFlagPerResultIsMutable();
partialFlagPerResult_.add(value);
onChanged();
return this;
}
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
public Builder addAllPartialFlagPerResult(
java.lang.Iterable<? extends java.lang.Boolean> values) {
ensurePartialFlagPerResultIsMutable();
super.addAll(values, partialFlagPerResult_);
onChanged();
return this;
}
/**
* <code>repeated bool partial_flag_per_result = 7;</code>
*
* <pre>
* This field is filled in if we are doing cellblocks. In the event that a row
* could not fit all of its cells into a single RPC chunk, the results will be
* returned as partials, and reconstructed into a complete result on the client
* side. This field is a list of flags indicating whether or not the result
* that the cells belong to is a partial result. For example, if this field
* has false, false, true in it, then we know that on the client side, we need to
* make another RPC request since the last result was only a partial.
* </pre>
*/
public Builder clearPartialFlagPerResult() {
partialFlagPerResult_ = java.util.Collections.emptyList();
bitField0_ = (bitField0_ & ~0x00000040);
onChanged();
return this;
}
// @@protoc_insertion_point(builder_scope:ScanResponse)
}
@ -31980,105 +32506,108 @@ public final class ClientProtos {
"\024\n\014store_offset\030\t \001(\r\022\035\n\016existence_only\030" +
"\n \001(\010:\005false\022!\n\022closest_row_before\030\013 \001(\010" +
":\005false\022)\n\013consistency\030\014 \001(\0162\014.Consisten" +
"cy:\006STRONG\"b\n\006Result\022\023\n\004cell\030\001 \003(\0132\005.Cel" +
"cy:\006STRONG\"z\n\006Result\022\023\n\004cell\030\001 \003(\0132\005.Cel" +
"l\022\035\n\025associated_cell_count\030\002 \001(\005\022\016\n\006exis" +
"ts\030\003 \001(\010\022\024\n\005stale\030\004 \001(\010:\005false\"A\n\nGetReq" +
"uest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022\021" +
"\n\003get\030\002 \002(\0132\004.Get\"&\n\013GetResponse\022\027\n\006resu" +
"lt\030\001 \001(\0132\007.Result\"\200\001\n\tCondition\022\013\n\003row\030\001" +
" \002(\014\022\016\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022",
"\"\n\014compare_type\030\004 \002(\0162\014.CompareType\022\037\n\nc" +
"omparator\030\005 \002(\0132\013.Comparator\"\265\006\n\rMutatio" +
"nProto\022\013\n\003row\030\001 \001(\014\0220\n\013mutate_type\030\002 \001(\016" +
"2\033.MutationProto.MutationType\0220\n\014column_" +
"value\030\003 \003(\0132\032.MutationProto.ColumnValue\022" +
"\021\n\ttimestamp\030\004 \001(\004\022!\n\tattribute\030\005 \003(\0132\016." +
"NameBytesPair\022:\n\ndurability\030\006 \001(\0162\031.Muta" +
"tionProto.Durability:\013USE_DEFAULT\022\036\n\ntim" +
"e_range\030\007 \001(\0132\n.TimeRange\022\035\n\025associated_" +
"cell_count\030\010 \001(\005\022\r\n\005nonce\030\t \001(\004\032\347\001\n\013Colu",
"mnValue\022\016\n\006family\030\001 \002(\014\022B\n\017qualifier_val" +
"ue\030\002 \003(\0132).MutationProto.ColumnValue.Qua" +
"lifierValue\032\203\001\n\016QualifierValue\022\021\n\tqualif" +
"ier\030\001 \001(\014\022\r\n\005value\030\002 \001(\014\022\021\n\ttimestamp\030\003 " +
"\001(\004\022.\n\013delete_type\030\004 \001(\0162\031.MutationProto" +
".DeleteType\022\014\n\004tags\030\005 \001(\014\"W\n\nDurability\022" +
"\017\n\013USE_DEFAULT\020\000\022\014\n\010SKIP_WAL\020\001\022\r\n\tASYNC_" +
"WAL\020\002\022\014\n\010SYNC_WAL\020\003\022\r\n\tFSYNC_WAL\020\004\">\n\014Mu" +
"tationType\022\n\n\006APPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n" +
"\003PUT\020\002\022\n\n\006DELETE\020\003\"p\n\nDeleteType\022\026\n\022DELE",
"TE_ONE_VERSION\020\000\022\034\n\030DELETE_MULTIPLE_VERS" +
"IONS\020\001\022\021\n\rDELETE_FAMILY\020\002\022\031\n\025DELETE_FAMI" +
"LY_VERSION\020\003\"\207\001\n\rMutateRequest\022 \n\006region" +
"\030\001 \002(\0132\020.RegionSpecifier\022 \n\010mutation\030\002 \002" +
"(\0132\016.MutationProto\022\035\n\tcondition\030\003 \001(\0132\n." +
"Condition\022\023\n\013nonce_group\030\004 \001(\004\"<\n\016Mutate" +
"Response\022\027\n\006result\030\001 \001(\0132\007.Result\022\021\n\tpro" +
"cessed\030\002 \001(\010\"\271\003\n\004Scan\022\027\n\006column\030\001 \003(\0132\007." +
"Column\022!\n\tattribute\030\002 \003(\0132\016.NameBytesPai" +
"r\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_row\030\004 \001(\014\022\027",
"\n\006filter\030\005 \001(\0132\007.Filter\022\036\n\ntime_range\030\006 " +
"\001(\0132\n.TimeRange\022\027\n\014max_versions\030\007 \001(\r:\0011" +
"\022\032\n\014cache_blocks\030\010 \001(\010:\004true\022\022\n\nbatch_si" +
"ze\030\t \001(\r\022\027\n\017max_result_size\030\n \001(\004\022\023\n\013sto" +
"re_limit\030\013 \001(\r\022\024\n\014store_offset\030\014 \001(\r\022&\n\036" +
"load_column_families_on_demand\030\r \001(\010\022\r\n\005" +
"small\030\016 \001(\010\022\027\n\010reversed\030\017 \001(\010:\005false\022)\n\013" +
"consistency\030\020 \001(\0162\014.Consistency:\006STRONG\022" +
"\017\n\007caching\030\021 \001(\r\"\236\001\n\013ScanRequest\022 \n\006regi" +
"on\030\001 \001(\0132\020.RegionSpecifier\022\023\n\004scan\030\002 \001(\013",
"2\005.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016number_of" +
"_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005 \001(\010\022\025\n\rne" +
"xt_call_seq\030\006 \001(\004\"\210\001\n\014ScanResponse\022\030\n\020ce" +
"lls_per_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001(\004" +
"\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022\030\n\007r" +
"esults\030\005 \003(\0132\007.Result\022\r\n\005stale\030\006 \001(\010\"\263\001\n" +
"\024BulkLoadHFileRequest\022 \n\006region\030\001 \002(\0132\020." +
"RegionSpecifier\0225\n\013family_path\030\002 \003(\0132 .B" +
"ulkLoadHFileRequest.FamilyPath\022\026\n\016assign" +
"_seq_num\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006family\030\001",
" \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileRespo" +
"nse\022\016\n\006loaded\030\001 \002(\010\"a\n\026CoprocessorServic" +
"eCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(\t" +
"\022\023\n\013method_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"9" +
"\n\030CoprocessorServiceResult\022\035\n\005value\030\001 \001(" +
"\0132\016.NameBytesPair\"d\n\031CoprocessorServiceR" +
"equest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier" +
"\022%\n\004call\030\002 \002(\0132\027.CoprocessorServiceCall\"" +
"]\n\032CoprocessorServiceResponse\022 \n\006region\030" +
"\001 \002(\0132\020.RegionSpecifier\022\035\n\005value\030\002 \002(\0132\016",
".NameBytesPair\"{\n\006Action\022\r\n\005index\030\001 \001(\r\022" +
" \n\010mutation\030\002 \001(\0132\016.MutationProto\022\021\n\003get" +
"\030\003 \001(\0132\004.Get\022-\n\014service_call\030\004 \001(\0132\027.Cop" +
"rocessorServiceCall\"Y\n\014RegionAction\022 \n\006r" +
"egion\030\001 \002(\0132\020.RegionSpecifier\022\016\n\006atomic\030" +
"\002 \001(\010\022\027\n\006action\030\003 \003(\0132\007.Action\"D\n\017Region" +
"LoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010\022\030\n\rhe" +
"apOccupancy\030\002 \001(\005:\0010\"\266\001\n\021ResultOrExcepti" +
"on\022\r\n\005index\030\001 \001(\r\022\027\n\006result\030\002 \001(\0132\007.Resu" +
"lt\022!\n\texception\030\003 \001(\0132\016.NameBytesPair\0221\n",
"\016service_result\030\004 \001(\0132\031.CoprocessorServi" +
"ceResult\022#\n\tloadStats\030\005 \001(\0132\020.RegionLoad" +
"Stats\"f\n\022RegionActionResult\022-\n\021resultOrE" +
"xception\030\001 \003(\0132\022.ResultOrException\022!\n\tex" +
"ception\030\002 \001(\0132\016.NameBytesPair\"f\n\014MultiRe" +
"quest\022#\n\014regionAction\030\001 \003(\0132\r.RegionActi" +
"on\022\022\n\nnonceGroup\030\002 \001(\004\022\035\n\tcondition\030\003 \001(" +
"\0132\n.Condition\"S\n\rMultiResponse\022/\n\022region" +
"ActionResult\030\001 \003(\0132\023.RegionActionResult\022" +
"\021\n\tprocessed\030\002 \001(\010*\'\n\013Consistency\022\n\n\006STR",
"ONG\020\000\022\014\n\010TIMELINE\020\0012\205\003\n\rClientService\022 \n" +
"\003Get\022\013.GetRequest\032\014.GetResponse\022)\n\006Mutat" +
"e\022\016.MutateRequest\032\017.MutateResponse\022#\n\004Sc" +
"an\022\014.ScanRequest\032\r.ScanResponse\022>\n\rBulkL" +
"oadHFile\022\025.BulkLoadHFileRequest\032\026.BulkLo" +
"adHFileResponse\022F\n\013ExecService\022\032.Coproce" +
"ssorServiceRequest\032\033.CoprocessorServiceR" +
"esponse\022R\n\027ExecRegionServerService\022\032.Cop" +
"rocessorServiceRequest\032\033.CoprocessorServ" +
"iceResponse\022&\n\005Multi\022\r.MultiRequest\032\016.Mu",
"ltiResponseBB\n*org.apache.hadoop.hbase.p" +
"rotobuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001"
"ts\030\003 \001(\010\022\024\n\005stale\030\004 \001(\010:\005false\022\026\n\007partia" +
"l\030\005 \001(\010:\005false\"A\n\nGetRequest\022 \n\006region\030\001" +
" \002(\0132\020.RegionSpecifier\022\021\n\003get\030\002 \002(\0132\004.Ge" +
"t\"&\n\013GetResponse\022\027\n\006result\030\001 \001(\0132\007.Resul" +
"t\"\200\001\n\tCondition\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002",
" \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\"\n\014compare_type\030" +
"\004 \002(\0162\014.CompareType\022\037\n\ncomparator\030\005 \002(\0132" +
"\013.Comparator\"\265\006\n\rMutationProto\022\013\n\003row\030\001 " +
"\001(\014\0220\n\013mutate_type\030\002 \001(\0162\033.MutationProto" +
".MutationType\0220\n\014column_value\030\003 \003(\0132\032.Mu" +
"tationProto.ColumnValue\022\021\n\ttimestamp\030\004 \001" +
"(\004\022!\n\tattribute\030\005 \003(\0132\016.NameBytesPair\022:\n" +
"\ndurability\030\006 \001(\0162\031.MutationProto.Durabi" +
"lity:\013USE_DEFAULT\022\036\n\ntime_range\030\007 \001(\0132\n." +
"TimeRange\022\035\n\025associated_cell_count\030\010 \001(\005",
"\022\r\n\005nonce\030\t \001(\004\032\347\001\n\013ColumnValue\022\016\n\006famil" +
"y\030\001 \002(\014\022B\n\017qualifier_value\030\002 \003(\0132).Mutat" +
"ionProto.ColumnValue.QualifierValue\032\203\001\n\016" +
"QualifierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005val" +
"ue\030\002 \001(\014\022\021\n\ttimestamp\030\003 \001(\004\022.\n\013delete_ty" +
"pe\030\004 \001(\0162\031.MutationProto.DeleteType\022\014\n\004t" +
"ags\030\005 \001(\014\"W\n\nDurability\022\017\n\013USE_DEFAULT\020\000" +
"\022\014\n\010SKIP_WAL\020\001\022\r\n\tASYNC_WAL\020\002\022\014\n\010SYNC_WA" +
"L\020\003\022\r\n\tFSYNC_WAL\020\004\">\n\014MutationType\022\n\n\006AP" +
"PEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002\022\n\n\006DELETE",
"\020\003\"p\n\nDeleteType\022\026\n\022DELETE_ONE_VERSION\020\000" +
"\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001\022\021\n\rDELETE" +
"_FAMILY\020\002\022\031\n\025DELETE_FAMILY_VERSION\020\003\"\207\001\n" +
"\rMutateRequest\022 \n\006region\030\001 \002(\0132\020.RegionS" +
"pecifier\022 \n\010mutation\030\002 \002(\0132\016.MutationPro" +
"to\022\035\n\tcondition\030\003 \001(\0132\n.Condition\022\023\n\013non" +
"ce_group\030\004 \001(\004\"<\n\016MutateResponse\022\027\n\006resu" +
"lt\030\001 \001(\0132\007.Result\022\021\n\tprocessed\030\002 \001(\010\"\271\003\n" +
"\004Scan\022\027\n\006column\030\001 \003(\0132\007.Column\022!\n\tattrib" +
"ute\030\002 \003(\0132\016.NameBytesPair\022\021\n\tstart_row\030\003",
" \001(\014\022\020\n\010stop_row\030\004 \001(\014\022\027\n\006filter\030\005 \001(\0132\007" +
".Filter\022\036\n\ntime_range\030\006 \001(\0132\n.TimeRange\022" +
"\027\n\014max_versions\030\007 \001(\r:\0011\022\032\n\014cache_blocks" +
"\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017max_" +
"result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(\r\022\024" +
"\n\014store_offset\030\014 \001(\r\022&\n\036load_column_fami" +
"lies_on_demand\030\r \001(\010\022\r\n\005small\030\016 \001(\010\022\027\n\010r" +
"eversed\030\017 \001(\010:\005false\022)\n\013consistency\030\020 \001(" +
"\0162\014.Consistency:\006STRONG\022\017\n\007caching\030\021 \001(\r" +
"\"\277\001\n\013ScanRequest\022 \n\006region\030\001 \001(\0132\020.Regio",
"nSpecifier\022\023\n\004scan\030\002 \001(\0132\005.Scan\022\022\n\nscann" +
"er_id\030\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(\r\022\025\n\rc" +
"lose_scanner\030\005 \001(\010\022\025\n\rnext_call_seq\030\006 \001(" +
"\004\022\037\n\027client_handles_partials\030\007 \001(\010\"\251\001\n\014S" +
"canResponse\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n" +
"\nscanner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022" +
"\013\n\003ttl\030\004 \001(\r\022\030\n\007results\030\005 \003(\0132\007.Result\022\r" +
"\n\005stale\030\006 \001(\010\022\037\n\027partial_flag_per_result" +
"\030\007 \003(\010\"\263\001\n\024BulkLoadHFileRequest\022 \n\006regio" +
"n\030\001 \002(\0132\020.RegionSpecifier\0225\n\013family_path",
"\030\002 \003(\0132 .BulkLoadHFileRequest.FamilyPath" +
"\022\026\n\016assign_seq_num\030\003 \001(\010\032*\n\nFamilyPath\022\016" +
"\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoad" +
"HFileResponse\022\016\n\006loaded\030\001 \002(\010\"a\n\026Coproce" +
"ssorServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_" +
"name\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007reque" +
"st\030\004 \002(\014\"9\n\030CoprocessorServiceResult\022\035\n\005" +
"value\030\001 \001(\0132\016.NameBytesPair\"d\n\031Coprocess" +
"orServiceRequest\022 \n\006region\030\001 \002(\0132\020.Regio" +
"nSpecifier\022%\n\004call\030\002 \002(\0132\027.CoprocessorSe",
"rviceCall\"]\n\032CoprocessorServiceResponse\022" +
" \n\006region\030\001 \002(\0132\020.RegionSpecifier\022\035\n\005val" +
"ue\030\002 \002(\0132\016.NameBytesPair\"{\n\006Action\022\r\n\005in" +
"dex\030\001 \001(\r\022 \n\010mutation\030\002 \001(\0132\016.MutationPr" +
"oto\022\021\n\003get\030\003 \001(\0132\004.Get\022-\n\014service_call\030\004" +
" \001(\0132\027.CoprocessorServiceCall\"Y\n\014RegionA" +
"ction\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022" +
"\016\n\006atomic\030\002 \001(\010\022\027\n\006action\030\003 \003(\0132\007.Action" +
"\"D\n\017RegionLoadStats\022\027\n\014memstoreLoad\030\001 \001(" +
"\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\"\266\001\n\021Resul",
"tOrException\022\r\n\005index\030\001 \001(\r\022\027\n\006result\030\002 " +
"\001(\0132\007.Result\022!\n\texception\030\003 \001(\0132\016.NameBy" +
"tesPair\0221\n\016service_result\030\004 \001(\0132\031.Coproc" +
"essorServiceResult\022#\n\tloadStats\030\005 \001(\0132\020." +
"RegionLoadStats\"f\n\022RegionActionResult\022-\n" +
"\021resultOrException\030\001 \003(\0132\022.ResultOrExcep" +
"tion\022!\n\texception\030\002 \001(\0132\016.NameBytesPair\"" +
"f\n\014MultiRequest\022#\n\014regionAction\030\001 \003(\0132\r." +
"RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022\035\n\tcond" +
"ition\030\003 \001(\0132\n.Condition\"S\n\rMultiResponse",
"\022/\n\022regionActionResult\030\001 \003(\0132\023.RegionAct" +
"ionResult\022\021\n\tprocessed\030\002 \001(\010*\'\n\013Consiste" +
"ncy\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\205\003\n\rClient" +
"Service\022 \n\003Get\022\013.GetRequest\032\014.GetRespons" +
"e\022)\n\006Mutate\022\016.MutateRequest\032\017.MutateResp" +
"onse\022#\n\004Scan\022\014.ScanRequest\032\r.ScanRespons" +
"e\022>\n\rBulkLoadHFile\022\025.BulkLoadHFileReques" +
"t\032\026.BulkLoadHFileResponse\022F\n\013ExecService" +
"\022\032.CoprocessorServiceRequest\032\033.Coprocess" +
"orServiceResponse\022R\n\027ExecRegionServerSer",
"vice\022\032.CoprocessorServiceRequest\032\033.Copro" +
"cessorServiceResponse\022&\n\005Multi\022\r.MultiRe" +
"quest\032\016.MultiResponseBB\n*org.apache.hado" +
"op.hbase.protobuf.generatedB\014ClientProto" +
"sH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -32114,7 +32643,7 @@ public final class ClientProtos {
internal_static_Result_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_Result_descriptor,
new java.lang.String[] { "Cell", "AssociatedCellCount", "Exists", "Stale", });
new java.lang.String[] { "Cell", "AssociatedCellCount", "Exists", "Stale", "Partial", });
internal_static_GetRequest_descriptor =
getDescriptor().getMessageTypes().get(5);
internal_static_GetRequest_fieldAccessorTable = new
@ -32174,13 +32703,13 @@ public final class ClientProtos {
internal_static_ScanRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_ScanRequest_descriptor,
new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", });
new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", });
internal_static_ScanResponse_descriptor =
getDescriptor().getMessageTypes().get(13);
internal_static_ScanResponse_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_ScanResponse_descriptor,
new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", });
new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", "PartialFlagPerResult", });
internal_static_BulkLoadHFileRequest_descriptor =
getDescriptor().getMessageTypes().get(14);
internal_static_BulkLoadHFileRequest_fieldAccessorTable = new

View File

@ -105,6 +105,12 @@ message Result {
// Whether or not the results are coming from possibly stale data
optional bool stale = 4 [default = false];
// Whether or not the entire result could be returned. Results will be split when
// the RPC chunk size limit is reached. Partial results contain only a subset of the
// cells for a row and must be combined with a result containing the remaining cells
// to form a complete result
optional bool partial = 5 [default = false];
}
/**
@ -268,6 +274,7 @@ message ScanRequest {
optional uint32 number_of_rows = 4;
optional bool close_scanner = 5;
optional uint64 next_call_seq = 6;
optional bool client_handles_partials = 7;
}
/**
@ -283,6 +290,7 @@ message ScanResponse {
// has 3, 3, 3 in it, then we know that on the client, we are to make
// three Results each of three Cells each.
repeated uint32 cells_per_result = 1;
optional uint64 scanner_id = 2;
optional bool more_results = 3;
optional uint32 ttl = 4;
@ -291,6 +299,15 @@ message ScanResponse {
// be inside the pb'd Result)
repeated Result results = 5;
optional bool stale = 6;
// This field is filled in if we are doing cellblocks. In the event that a row
// could not fit all of its cells into a single RPC chunk, the results will be
// returned as partials, and reconstructed into a complete result on the client
// side. This field is a list of flags indicating whether or not the result
// that the cells belong to is a partial result. For example, if this field
// has false, false, true in it, then we know that on the client side, we need to
// make another RPC request since the last result was only a partial.
repeated bool partial_flag_per_result = 7;
}
/**

View File

@ -72,7 +72,10 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
public Result next() throws IOException {
values.clear();
scanner.nextRaw(values, -1); // pass -1 as limit so that we see the whole row.
// negative values indicate no limits
final long remainingResultSize = -1;
final int batchLimit = -1;
scanner.nextRaw(values, batchLimit, remainingResultSize);
if (values.isEmpty()) {
//we are done
return null;

View File

@ -26,14 +26,14 @@ import java.util.UUID;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
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.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;

View File

@ -26,10 +26,10 @@ import java.util.NavigableSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -38,6 +38,7 @@ 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;
@ -91,7 +92,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
// qualifier can be null.
boolean hasMoreRows = false;
do {
hasMoreRows = scanner.next(results);
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
int listSize = results.size();
for (int i = 0; i < listSize; i++) {
temp = ci.getValue(colFamily, qualifier, results.get(i));
@ -145,7 +146,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
}
boolean hasMoreRows = false;
do {
hasMoreRows = scanner.next(results);
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
int listSize = results.size();
for (int i = 0; i < listSize; i++) {
temp = ci.getValue(colFamily, qualifier, results.get(i));
@ -199,7 +200,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
List<Cell> results = new ArrayList<Cell>();
boolean hasMoreRows = false;
do {
hasMoreRows = scanner.next(results);
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
int listSize = results.size();
for (int i = 0; i < listSize; i++) {
temp = ci.getValue(colFamily, qualifier, results.get(i));
@ -253,7 +254,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
scanner = env.getRegion().getScanner(scan);
boolean hasMoreRows = false;
do {
hasMoreRows = scanner.next(results);
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
if (results.size() > 0) {
counter++;
}
@ -312,7 +313,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
do {
results.clear();
hasMoreRows = scanner.next(results);
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
int listSize = results.size();
for (int i = 0; i < listSize; i++) {
sumVal = ci.add(sumVal, ci.castToReturnType(ci.getValue(colFamily,
@ -373,7 +374,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
do {
tempVal = null;
hasMoreRows = scanner.next(results);
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
int listSize = results.size();
for (int i = 0; i < listSize; i++) {
tempVal = ci.add(tempVal, ci.castToReturnType(ci.getValue(colFamily,
@ -440,7 +441,7 @@ extends AggregateService implements CoprocessorService, Coprocessor {
do {
tempVal = null;
tempWeight = null;
hasMoreRows = scanner.next(results);
hasMoreRows = NextState.hasMoreValues(scanner.next(results));
int listSize = results.size();
for (int i = 0; i < listSize; i++) {
Cell kv = results.get(i);

View File

@ -138,10 +138,11 @@ 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.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.ReplayHLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@ -5177,8 +5178,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
* If the joined heap data gathering is interrupted due to scan limits, this will
* contain the row for which we are populating the values.*/
protected Cell joinedContinuationRow = null;
// KeyValue indicating that limit is reached when scanning
private final KeyValue KV_LIMIT = new KeyValue();
protected final byte[] stopRow;
private final FilterWrapper filter;
private int batch;
@ -5263,6 +5262,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
return this.readPt;
}
@Override
public int getBatch() {
return this.batch;
}
/**
* Reset both the filter and the old filter.
*
@ -5275,14 +5279,20 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
}
@Override
public boolean next(List<Cell> outResults)
public NextState next(List<Cell> outResults)
throws IOException {
// apply the batching limit by default
return next(outResults, batch);
}
@Override
public synchronized boolean next(List<Cell> outResults, int limit) throws IOException {
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)
throws IOException {
if (this.filterClosed) {
throw new UnknownScannerException("Scanner was closed (timed out?) " +
"after we renewed it. Could be caused by a very slow scanner " +
@ -5291,77 +5301,157 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
startRegionOperation(Operation.SCAN);
readRequestsCount.increment();
try {
return nextRaw(outResults, limit);
return nextRaw(outResults, limit, remainingResultSize);
} finally {
closeRegionOperation(Operation.SCAN);
}
}
@Override
public boolean nextRaw(List<Cell> outResults)
throws IOException {
public NextState nextRaw(List<Cell> outResults) throws IOException {
return nextRaw(outResults, batch);
}
@Override
public boolean nextRaw(List<Cell> outResults, int limit) throws IOException {
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)
throws IOException {
if (storeHeap == null) {
// scanner is closed
throw new UnknownScannerException("Scanner was closed");
}
boolean returnResult;
NextState state;
if (outResults.isEmpty()) {
// Usually outResults is empty. This is true when next is called
// to handle scan or get operation.
returnResult = nextInternal(outResults, limit);
state = nextInternal(outResults, batchLimit, remainingResultSize);
} else {
List<Cell> tmpList = new ArrayList<Cell>();
returnResult = nextInternal(tmpList, limit);
state = nextInternal(tmpList, batchLimit, remainingResultSize);
outResults.addAll(tmpList);
}
// State should never be null, this is a precautionary measure
if (state == null) {
if (LOG.isTraceEnabled()) LOG.trace("State was null. Defaulting to no more values state");
state = NextState.makeState(NextState.State.NO_MORE_VALUES);
}
resetFilters();
if (isFilterDoneInternal()) {
returnResult = false;
state = NextState.makeState(NextState.State.NO_MORE_VALUES, state.getResultSize());
}
return returnResult;
return state;
}
private void populateFromJoinedHeap(List<Cell> results, int limit)
throws IOException {
/**
* @return the state the joinedHeap returned on the call to
* {@link KeyValueHeap#next(List, int, long)}
*/
private NextState populateFromJoinedHeap(List<Cell> results, int limit, long resultSize)
throws IOException {
assert joinedContinuationRow != null;
Cell kv = populateResult(results, this.joinedHeap, limit,
NextState state =
populateResult(results, this.joinedHeap, limit, resultSize,
joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(),
joinedContinuationRow.getRowLength());
if (kv != KV_LIMIT) {
if (state != null && !state.batchLimitReached() && !state.sizeLimitReached()) {
// 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;
}
/**
* Fetches records with currentRow into results list, until next row or limit (if not -1).
* 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 limit Max amount of KVs to place in result list, -1 means no limit.
* @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 currentRow Byte array with key we are fetching.
* @param offset offset for currentRow
* @param length length for currentRow
* @return KV_LIMIT if limit reached, next KeyValue otherwise.
* @return state of last call to {@link KeyValueHeap#next()}
*/
private Cell populateResult(List<Cell> results, KeyValueHeap heap, int limit,
byte[] currentRow, int offset, short length) throws IOException {
private NextState populateResult(List<Cell> results, KeyValueHeap heap, int batchLimit,
long remainingResultSize, byte[] currentRow, int offset, short length) throws IOException {
Cell nextKv;
boolean moreCellsInRow = false;
long accumulatedResultSize = 0;
List<Cell> tmpResults = new ArrayList<Cell>();
do {
heap.next(results, limit - results.size());
if (limit > 0 && results.size() == limit) {
return KV_LIMIT;
}
nextKv = heap.peek();
} while (nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length));
int remainingBatchLimit = batchLimit - results.size();
NextState heapState =
heap.next(tmpResults, remainingBatchLimit, remainingResultSize - accumulatedResultSize);
results.addAll(tmpResults);
accumulatedResultSize += calculateResultSize(tmpResults, heapState);
tmpResults.clear();
return nextKv;
if (batchLimit > 0 && results.size() == batchLimit) {
return NextState.makeState(NextState.State.BATCH_LIMIT_REACHED, accumulatedResultSize);
}
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);
}
} while (moreCellsInRow);
if (nextKv != null) {
return NextState.makeState(NextState.State.MORE_VALUES, accumulatedResultSize);
} else {
return NextState.makeState(NextState.State.NO_MORE_VALUES, accumulatedResultSize);
}
}
/**
* Based on the nextKv in the heap, and the current row, decide whether or not there are more
* cells to be read in the heap. If the row of the nextKv in the heap matches the current row
* then there are more cells to be read in the row.
* @param nextKv
* @param currentRow
* @param offset
* @param length
* @return true When there are more cells in the row to be read
*/
private boolean moreCellsInRow(final Cell nextKv, byte[] currentRow, int offset,
short length) {
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.estimatedHeapSizeOf(c);
}
return size;
}
/*
@ -5376,11 +5466,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
return this.filter != null && this.filter.filterAllRemaining();
}
private boolean nextInternal(List<Cell> results, int limit)
throws IOException {
private NextState nextInternal(List<Cell> results, int batchLimit, long remainingResultSize)
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;
RpcCallContext rpcCall = RpcServer.getCurrentCall();
// 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.
@ -5413,38 +5505,74 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
offset = current.getRowOffset();
length = current.getRowLength();
}
boolean stopRow = isStopRow(currentRow, offset, length);
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;
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.");
}
}
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) {
// First, check if we are at a stop row. If so, there are no more results.
if (stopRow) {
if (filter != null && filter.hasFilterRow()) {
if (hasFilterRow) {
filter.filterRowCells(results);
}
return false;
return NextState.makeState(NextState.State.NO_MORE_VALUES, resultSize);
}
// 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 false;
if (!moreRows) return NextState.makeState(NextState.State.NO_MORE_VALUES, resultSize);
results.clear();
continue;
}
Cell nextKv = populateResult(results, this.storeHeap, limit, currentRow, offset,
length);
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 (nextKv == KV_LIMIT) {
if (this.filter != null && filter.hasFilterRow()) {
if (storeHeapState.batchLimitReached()) {
if (hasFilterRow) {
throw new IncompatibleFilterException(
"Filter whose hasFilterRow() returns true is incompatible with scan with limit!");
}
return true; // We hit the limit.
// 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);
}
Cell nextKv = this.storeHeap.peek();
stopRow = nextKv == null ||
isStopRow(nextKv.getRowArray(), nextKv.getRowOffset(), nextKv.getRowLength());
// save that the row was empty before filters applied to it.
@ -5453,19 +5581,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
// We have the part of the row necessary for filtering (all of it, usually).
// First filter with the filterRow(List).
FilterWrapper.FilterRowRetCode ret = FilterWrapper.FilterRowRetCode.NOT_CALLED;
if (filter != null && filter.hasFilterRow()) {
if (hasFilterRow) {
ret = filter.filterRowCellsWithRet(results);
}
if ((isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE) || filterRow()) {
results.clear();
boolean moreRows = nextRow(currentRow, offset, length);
if (!moreRows) return false;
if (!moreRows) return NextState.makeState(NextState.State.NO_MORE_VALUES, 0);
// 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 false;
return NextState.makeState(NextState.State.NO_MORE_VALUES, 0);
}
// Ok, we are done with storeHeap for this row.
@ -5483,18 +5611,31 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
currentRow, offset, length));
if (mayHaveData) {
joinedContinuationRow = current;
populateFromJoinedHeap(results, limit);
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);
}
}
}
} else {
// Populating from the joined heap was stopped by limits, populate some more.
populateFromJoinedHeap(results, limit);
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);
}
}
// 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 true;
return NextState.makeState(NextState.State.MORE_VALUES, resultSize);
}
// Finally, we are done with both joinedHeap and storeHeap.
@ -5502,12 +5643,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
// the case when SingleColumnValueExcludeFilter is used.
if (results.isEmpty()) {
boolean moreRows = nextRow(currentRow, offset, length);
if (!moreRows) return false;
if (!moreRows) return NextState.makeState(NextState.State.NO_MORE_VALUES, 0);
if (!stopRow) continue;
}
// We are done. Return the result.
return !stopRow;
if (stopRow) {
return NextState.makeState(NextState.State.NO_MORE_VALUES, resultSize);
} else {
return NextState.makeState(NextState.State.MORE_VALUES, resultSize);
}
}
}
@ -7141,7 +7286,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
boolean done;
do {
kvs.clear();
done = scanner.next(kvs);
done = NextState.hasMoreValues(scanner.next(kvs));
if (kvs.size() > 0) LOG.info(kvs);
} while (done);
} finally {

View File

@ -22,8 +22,8 @@ import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Internal scanners differ from client-side scanners in that they operate on
@ -42,22 +42,213 @@ import org.apache.hadoop.hbase.Cell;
@InterfaceAudience.Private
public interface InternalScanner extends Closeable {
/**
* 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
* 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.
*/
boolean next(List<Cell> results) throws IOException;
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;
}
/**
* 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();
}
}
/**
* 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 true if more rows exist after this one, false if scanner is done
* 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
*/
boolean next(List<Cell> result, int limit) throws IOException;
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
* @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;
/**
* Closes the scanner and releases any resources it has allocated

View File

@ -24,9 +24,9 @@ import java.util.Comparator;
import java.util.List;
import java.util.PriorityQueue;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Implements a heap merge across any number of KeyValueScanners.
@ -125,18 +125,29 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
* <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}).
* 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 true if there are more keys, false if all scanners are done
* @return state where NextState#hasMoreValues() is true if more keys exist after this
* one, false if scanner is done
*/
public boolean next(List<Cell> result, int limit) throws IOException {
public NextState next(List<Cell> result, int limit) throws IOException {
return next(result, limit, -1);
}
public NextState next(List<Cell> result, int limit, long remainingResultSize) throws IOException {
if (this.current == null) {
return false;
return NextState.makeState(NextState.State.NO_MORE_VALUES);
}
InternalScanner currentAsInternal = (InternalScanner)this.current;
boolean mayContainMoreRows = currentAsInternal.next(result, limit);
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);
Cell pee = this.current.peek();
/*
* By definition, any InternalScanner must return false only when it has no
@ -151,7 +162,10 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
this.heap.add(this.current);
}
this.current = pollRealKV();
return (this.current != null);
if (this.current == null) {
state = NextState.makeState(NextState.State.NO_MORE_VALUES, state.getResultSize());
}
return state;
}
/**
@ -159,12 +173,13 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
* <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}).
* This can ONLY be called when you are using Scanners that implement InternalScanner as well as
* KeyValueScanner (a {@link StoreScanner}).
* @param result
* @return true if there are more keys, false if all scanners are done
* @return state where NextState#hasMoreValues() is true if more keys exist after this
* one, false if scanner is done
*/
public boolean next(List<Cell> result) throws IOException {
public NextState next(List<Cell> result) throws IOException {
return next(result, -1);
}

View File

@ -150,6 +150,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescripto
import org.apache.hadoop.hbase.quotas.OperationQuota;
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
@ -339,6 +340,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
if (isClientCellBlockSupport()) {
for (Result res : results) {
builder.addCellsPerResult(res.size());
builder.addPartialFlagPerResult(res.isPartial());
}
((PayloadCarryingRpcController)controller).
setCellScanner(CellUtil.createCellScanner(results));
@ -2040,6 +2042,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
RegionScannerHolder rsh = null;
boolean moreResults = true;
boolean closeScanner = false;
boolean isSmallScan = false;
ScanResponse.Builder builder = ScanResponse.newBuilder();
if (request.hasCloseScanner()) {
closeScanner = request.getCloseScanner();
@ -2071,6 +2074,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
if (!isLoadingCfsOnDemandSet) {
scan.setLoadColumnFamiliesOnDemand(region.isLoadingCfsOnDemandDefault());
}
isSmallScan = scan.isSmall();
region.prepareScanner(scan);
if (region.getCoprocessorHost() != null) {
scanner = region.getCoprocessorHost().preScannerOpen(scan);
@ -2111,9 +2116,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
// Remove lease while its being processed in server; protects against case
// where processing of request takes > lease expiration time.
lease = regionServer.leases.removeLease(scannerName);
List<Result> results = new ArrayList<Result>(rows);
long currentScanResultSize = 0;
List<Result> results = new ArrayList<Result>();
long totalCellSize = 0;
long currentScanResultSize = 0;
boolean done = false;
// Call coprocessor. Get region info from scanner.
@ -2123,8 +2128,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
if (!results.isEmpty()) {
for (Result r : results) {
for (Cell cell : r.rawCells()) {
currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
totalCellSize += CellUtil.estimatedSerializedSizeOf(cell);
currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
}
}
}
@ -2144,23 +2149,60 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
int i = 0;
synchronized(scanner) {
boolean stale = (region.getRegionInfo().getReplicaId() != 0);
boolean clientHandlesPartials =
request.hasClientHandlesPartials() && request.getClientHandlesPartials();
// On the server side we must ensure that the correct ordering of partial results is
// returned to the client to allow them to properly reconstruct the partial results.
// If the coprocessor host is adding to the result list, we cannot guarantee the
// correct ordering of partial results and so we prevent partial results from being
// formed.
boolean serverGuaranteesOrderOfPartials = currentScanResultSize == 0;
boolean enforceMaxResultSizeAtCellLevel =
clientHandlesPartials && serverGuaranteesOrderOfPartials && !isSmallScan;
while (i < rows) {
// Stop collecting results if maxScannerResultSize is set and we have exceeded it
if ((maxScannerResultSize < Long.MAX_VALUE) &&
(currentScanResultSize >= maxResultSize)) {
// Stop collecting results if we have exceeded maxResultSize
if (currentScanResultSize >= maxResultSize) {
break;
}
// A negative remainingResultSize communicates that there is no limit on the size
// of the results.
final long remainingResultSize =
enforceMaxResultSizeAtCellLevel ? maxResultSize - currentScanResultSize
: -1;
// Collect values to be returned here
boolean moreRows = scanner.nextRaw(values);
NextState 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();
for (Cell cell : values) {
currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
totalCellSize += CellUtil.estimatedSerializedSizeOf(cell);
// If the calculation can't be skipped, then do it now.
if (!skipResultSizeCalculation) {
currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
}
}
results.add(Result.create(values, null, stale));
// 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();
results.add(Result.create(values, null, stale, partial));
i++;
}
if (!moreRows) {
if (!NextState.hasMoreValues(state)) {
break;
}
values.clear();

View File

@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* RegionScanner describes iterators over rows in an HRegion.
@ -68,25 +68,43 @@ public interface RegionScanner extends InternalScanner {
long getMvccReadPoint();
/**
* 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)}
* @param result return output array
* @return true if more rows exist after this one, false if scanner is done
* @throws IOException e
* @return The limit on the number of cells to retrieve on each call to next(). See
* {@link org.apache.hadoop.hbase.client.Scan#setBatch(int)}
*/
boolean nextRaw(List<Cell> result) throws IOException;
int getBatch();
/**
* Grab the next row's worth of values with a limit on the number of values
* to return.
* 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.
* Example:
* <code><pre>
* 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
* @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) 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
* 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 = ...;
* RegionScanner scanner = ...
* MultiVersionConsistencyControl.setThreadReadPoint(scanner.getMvccReadPoint());
@ -103,8 +121,12 @@ public interface RegionScanner extends InternalScanner {
* </pre></code>
* @param result return output array
* @param limit limit on row count to get
* @return true if more rows exist after this one, false if scanner is done
* @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.
* @throws IOException e
*/
boolean nextRaw(List<Cell> result, int limit) throws IOException;
NextState nextRaw(List<Cell> result, int limit, final long remainingResultSize)
throws IOException;
}

View File

@ -23,13 +23,14 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
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;
/**
@ -112,7 +113,7 @@ abstract class StoreFlusher {
List<Cell> kvs = new ArrayList<Cell>();
boolean hasMore;
do {
hasMore = scanner.next(kvs, compactionKVMax);
hasMore = NextState.hasMoreValues(scanner.next(kvs, compactionKVMax));
if (!kvs.isEmpty()) {
for (Cell c : kvs) {
// If we know that this KV is going to be included always, then let us

View File

@ -30,7 +30,6 @@ import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.IsolationLevel;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.executor.ExecutorService;
@ -449,24 +449,38 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* @return true if there are more rows, false if scanner is done
*/
@Override
public boolean next(List<Cell> outResult, int limit) throws IOException {
public NextState next(List<Cell> outResult, int limit) throws IOException {
// -1 means no limit
return next(outResult, limit, -1);
}
/**
* Get the next row of values from this Store.
* @param outResult
* @param limit
* @param remainingResultSize
* @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 {
lock.lock();
try {
if (checkReseek()) {
return true;
return NextState.makeState(NextState.State.MORE_VALUES, 0);
}
// if the heap was left null, then the scanners had previously run out anyways, close and
// return.
if (this.heap == null) {
close();
return false;
return NextState.makeState(NextState.State.NO_MORE_VALUES, 0);
}
Cell peeked = this.heap.peek();
if (peeked == null) {
close();
return false;
return NextState.makeState(NextState.State.NO_MORE_VALUES, 0);
}
// only call setRow if the row changes; avoids confusing the query matcher
@ -474,10 +488,15 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
byte[] row = peeked.getRowArray();
int offset = peeked.getRowOffset();
short length = peeked.getRowLength();
if (limit < 0 || matcher.row == null || !Bytes.equals(row, offset, length, matcher.row,
matcher.rowOffset, matcher.rowLength)) {
this.countPerRow = 0;
matcher.setRow(row, offset, length);
// 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);
}
Cell cell;
@ -488,6 +507,7 @@ 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.
@ -512,7 +532,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
this.countPerRow > (storeLimit + storeOffset)) {
// do what SEEK_NEXT_ROW does.
if (!matcher.moreRowsMayExistAfter(cell)) {
return false;
return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize);
}
seekToNextRow(cell);
break LOOP;
@ -524,6 +544,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
outResult.add(cell);
count++;
totalBytesRead += CellUtil.estimatedSerializedSizeOf(cell);
totalHeapSize += CellUtil.estimatedHeapSizeOf(cell);
if (totalBytesRead > maxRowSize) {
throw new RowTooBigException("Max row size allowed: " + maxRowSize
+ ", but the row is bigger than that.");
@ -532,7 +553,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {
if (!matcher.moreRowsMayExistAfter(cell)) {
return false;
return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize);
}
seekToNextRow(cell);
} else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {
@ -544,20 +565,23 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
if (limit > 0 && (count == limit)) {
break LOOP;
}
if (remainingResultSize > 0 && (totalHeapSize >= remainingResultSize)) {
break LOOP;
}
continue;
case DONE:
return true;
return NextState.makeState(NextState.State.MORE_VALUES, totalHeapSize);
case DONE_SCAN:
close();
return false;
return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize);
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 false;
return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize);
}
seekToNextRow(cell);
@ -587,12 +611,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
}
if (count > 0) {
return true;
return NextState.makeState(NextState.State.MORE_VALUES, totalHeapSize);
}
// No more keys
close();
return false;
return NextState.makeState(NextState.State.NO_MORE_VALUES, totalHeapSize);
} finally {
lock.unlock();
}
@ -631,7 +655,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
}
@Override
public boolean next(List<Cell> outResult) throws IOException {
public NextState next(List<Cell> outResult) throws IOException {
return next(outResult, -1);
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
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.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
@ -248,7 +249,7 @@ public abstract class Compactor {
throughputController.start(compactionName);
try {
do {
hasMore = scanner.next(cells, compactionKVMax);
hasMore = NextState.hasMoreValues(scanner.next(cells, compactionKVMax));
if (LOG.isDebugEnabled()) {
now = EnvironmentEdgeManager.currentTime();
}

View File

@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
import org.apache.hadoop.hbase.regionserver.BloomType;
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.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
@ -369,7 +370,7 @@ public class AccessControlLists {
while (true) {
List<Cell> row = new ArrayList<Cell>();
boolean hasNext = iScanner.next(row);
boolean hasNext = NextState.hasMoreValues(iScanner.next(row));
ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
byte[] entry = null;
for (Cell kv : row) {

View File

@ -88,6 +88,7 @@ import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBu
import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest;
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.MiniBatchOperationInProgress;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.regionserver.ScanType;
@ -739,7 +740,7 @@ public class AccessController extends BaseMasterAndRegionObserver
do {
cells.clear();
// scan with limit as 1 to hold down memory use on wide rows
more = scanner.next(cells, 1);
more = NextState.hasMoreValues(scanner.next(cells, 1));
for (Cell cell: cells) {
if (LOG.isTraceEnabled()) {
LOG.trace("Found cell " + cell);

View File

@ -42,6 +42,7 @@ 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.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
@ -563,7 +564,7 @@ public abstract class HBaseTestCase extends TestCase {
@Override
public boolean next(List<Cell> results)
throws IOException {
return scanner.next(results);
return NextState.hasMoreValues(scanner.next(results));
}
@Override

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@ -45,7 +44,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;

View File

@ -0,0 +1,787 @@
/**
* 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;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.ClientScanner;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.RandomRowFilter;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* These tests are focused on testing how partial results appear to a client. Partial results are
* {@link Result}s that contain only a portion of a row's complete list of cells. Partial results
* are formed when the server breaches its maximum result size when trying to service a client's RPC
* request. It is the responsibility of the scanner on the client side to recognize when partial
* results have been returned and to take action to form the complete results.
* <p>
* Unless the flag {@link Scan#setAllowPartialResults(boolean)} has been set to true, the caller of
* {@link ResultScanner#next()} should never see partial results.
*/
@Category(MediumTests.class)
public class TestPartialResultsFromClientSide {
private static final Log LOG = LogFactory.getLog(TestPartialResultsFromClientSide.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static Table TABLE = null;
/**
* Table configuration
*/
private static TableName TABLE_NAME = TableName.valueOf("testTable");
private static int NUM_ROWS = 5;
private static byte[] ROW = Bytes.toBytes("testRow");
private static byte[][] ROWS = HTestConst.makeNAscii(ROW, NUM_ROWS);
// Should keep this value below 10 to keep generation of expected kv's simple. If above 10 then
// table/row/cf1/... will be followed by table/row/cf10/... instead of table/row/cf2/... which
// breaks the simple generation of expected kv's
private static int NUM_FAMILIES = 10;
private static byte[] FAMILY = Bytes.toBytes("testFamily");
private static byte[][] FAMILIES = HTestConst.makeNAscii(FAMILY, NUM_FAMILIES);
private static int NUM_QUALIFIERS = 10;
private static byte[] QUALIFIER = Bytes.toBytes("testQualifier");
private static byte[][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, NUM_QUALIFIERS);
private static int VALUE_SIZE = 1024;
private static byte[] VALUE = Bytes.createMaxByteArray(VALUE_SIZE);
private static int NUM_COLS = NUM_FAMILIES * NUM_QUALIFIERS;
// Approximation of how large the heap size of cells in our table. Should be accessed through
// getCellHeapSize().
private static long CELL_HEAP_SIZE = -1;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TABLE = createTestTable(TABLE_NAME, ROWS, FAMILIES, QUALIFIERS, VALUE);
}
static Table createTestTable(TableName name, byte[][] rows, byte[][] families,
byte[][] qualifiers, byte[] cellValue) throws IOException {
Table ht = TEST_UTIL.createTable(name, families);
List<Put> puts = createPuts(rows, families, qualifiers, cellValue);
ht.put(puts);
return ht;
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
/**
* Ensure that the expected key values appear in a result returned from a scanner that is
* combining partial results into complete results
* @throws Exception
*/
@Test
public void testExpectedValuesOfPartialResults() throws Exception {
testExpectedValuesOfPartialResults(false);
testExpectedValuesOfPartialResults(true);
}
public void testExpectedValuesOfPartialResults(boolean reversed) throws Exception {
Scan partialScan = new Scan();
partialScan.setMaxVersions();
// Max result size of 1 ensures that each RPC request will return a single cell. The scanner
// will need to reconstruct the results into a complete result before returning to the caller
partialScan.setMaxResultSize(1);
partialScan.setReversed(reversed);
ResultScanner partialScanner = TABLE.getScanner(partialScan);
final int startRow = reversed ? ROWS.length - 1 : 0;
final int endRow = reversed ? -1 : ROWS.length;
final int loopDelta = reversed ? -1 : 1;
String message;
for (int row = startRow; row != endRow; row = row + loopDelta) {
message = "Ensuring the expected keyValues are present for row " + row;
List<Cell> expectedKeyValues = createKeyValuesForRow(ROWS[row], FAMILIES, QUALIFIERS, VALUE);
Result result = partialScanner.next();
assertFalse(result.isPartial());
verifyResult(result, expectedKeyValues, message);
}
partialScanner.close();
}
/**
* Ensure that we only see Results marked as partial when the allowPartial flag is set
* @throws Exception
*/
@Test
public void testAllowPartialResults() throws Exception {
Scan scan = new Scan();
scan.setAllowPartialResults(true);
scan.setMaxResultSize(1);
ResultScanner scanner = TABLE.getScanner(scan);
Result result = scanner.next();
assertTrue(result != null);
assertTrue(result.isPartial());
assertTrue(result.rawCells() != null);
assertTrue(result.rawCells().length == 1);
scanner.close();
scan.setAllowPartialResults(false);
scanner = TABLE.getScanner(scan);
result = scanner.next();
assertTrue(result != null);
assertTrue(!result.isPartial());
assertTrue(result.rawCells() != null);
assertTrue(result.rawCells().length == NUM_COLS);
scanner.close();
}
/**
* Ensure that the results returned from a scanner that retrieves all results in a single RPC call
* matches the results that are returned from a scanner that must incrementally combine partial
* results into complete results. A variety of scan configurations can be tested
* @throws Exception
*/
@Test
public void testEquivalenceOfScanResults() throws Exception {
Scan oneShotScan = new Scan();
oneShotScan.setMaxResultSize(Long.MAX_VALUE);
Scan partialScan = new Scan(oneShotScan);
partialScan.setMaxResultSize(1);
testEquivalenceOfScanResults(TABLE, oneShotScan, partialScan);
}
public void testEquivalenceOfScanResults(Table table, Scan scan1, Scan scan2) throws Exception {
ResultScanner scanner1 = table.getScanner(scan1);
ResultScanner scanner2 = table.getScanner(scan2);
Result r1 = null;
Result r2 = null;
int count = 0;
while ((r1 = scanner1.next()) != null) {
r2 = scanner2.next();
assertTrue(r2 != null);
compareResults(r1, r2, "Comparing result #" + count);
count++;
}
assertTrue(scanner2.next() == null);
scanner1.close();
scanner2.close();
}
/**
* Order of cells in partial results matches the ordering of cells from complete results
* @throws Exception
*/
@Test
public void testOrderingOfCellsInPartialResults() throws Exception {
Scan scan = new Scan();
for (int col = 1; col <= NUM_COLS; col++) {
scan.setMaxResultSize(getResultSizeForNumberOfCells(col));
testOrderingOfCellsInPartialResults(scan);
// Test again with a reversed scanner
scan.setReversed(true);
testOrderingOfCellsInPartialResults(scan);
}
}
public void testOrderingOfCellsInPartialResults(final Scan basePartialScan) throws Exception {
// Scan that retrieves results in pieces (partials). By setting allowPartialResults to be true
// the results will NOT be reconstructed and instead the caller will see the partial results
// returned by the server
Scan partialScan = new Scan(basePartialScan);
partialScan.setAllowPartialResults(true);
ResultScanner partialScanner = TABLE.getScanner(partialScan);
// Scan that retrieves all table results in single RPC request
Scan oneShotScan = new Scan(basePartialScan);
oneShotScan.setMaxResultSize(Long.MAX_VALUE);
oneShotScan.setCaching(ROWS.length);
ResultScanner oneShotScanner = TABLE.getScanner(oneShotScan);
Result oneShotResult = oneShotScanner.next();
Result partialResult = null;
int iterationCount = 0;
while (oneShotResult != null && oneShotResult.rawCells() != null) {
List<Cell> aggregatePartialCells = new ArrayList<Cell>();
do {
partialResult = partialScanner.next();
assertTrue("Partial Result is null. iteration: " + iterationCount, partialResult != null);
assertTrue("Partial cells are null. iteration: " + iterationCount,
partialResult.rawCells() != null);
for (Cell c : partialResult.rawCells()) {
aggregatePartialCells.add(c);
}
} while (partialResult.isPartial());
assertTrue("Number of cells differs. iteration: " + iterationCount,
oneShotResult.rawCells().length == aggregatePartialCells.size());
final Cell[] oneShotCells = oneShotResult.rawCells();
for (int cell = 0; cell < oneShotCells.length; cell++) {
Cell oneShotCell = oneShotCells[cell];
Cell partialCell = aggregatePartialCells.get(cell);
assertTrue("One shot cell was null", oneShotCell != null);
assertTrue("Partial cell was null", partialCell != null);
assertTrue("Cell differs. oneShotCell:" + oneShotCell + " partialCell:" + partialCell,
oneShotCell.equals(partialCell));
}
oneShotResult = oneShotScanner.next();
iterationCount++;
}
assertTrue(partialScanner.next() == null);
partialScanner.close();
oneShotScanner.close();
}
/**
* Setting the max result size allows us to control how many cells we expect to see on each call
* to next on the scanner. Test a variety of different sizes for correctness
* @throws Exception
*/
@Test
public void testExpectedNumberOfCellsPerPartialResult() throws Exception {
Scan scan = new Scan();
testExpectedNumberOfCellsPerPartialResult(scan);
scan.setReversed(true);
testExpectedNumberOfCellsPerPartialResult(scan);
}
public void testExpectedNumberOfCellsPerPartialResult(Scan baseScan) throws Exception {
for (int expectedCells = 1; expectedCells <= NUM_COLS; expectedCells++) {
testExpectedNumberOfCellsPerPartialResult(baseScan, expectedCells);
}
}
public void testExpectedNumberOfCellsPerPartialResult(Scan baseScan, int expectedNumberOfCells)
throws Exception {
if (LOG.isInfoEnabled()) LOG.info("groupSize:" + expectedNumberOfCells);
// Use the cellHeapSize to set maxResultSize such that we know how many cells to expect back
// from the call. The returned results should NOT exceed expectedNumberOfCells but may be less
// than it in cases where expectedNumberOfCells is not an exact multiple of the number of
// columns in the table.
Scan scan = new Scan(baseScan);
scan.setAllowPartialResults(true);
scan.setMaxResultSize(getResultSizeForNumberOfCells(expectedNumberOfCells));
ResultScanner scanner = TABLE.getScanner(scan);
Result result = null;
byte[] prevRow = null;
while ((result = scanner.next()) != null) {
assertTrue(result.rawCells() != null);
// Cases when cell count won't equal expectedNumberOfCells:
// 1. Returned result is the final result needed to form the complete result for that row
// 2. It is the first result we have seen for that row and thus may have been fetched as
// the last group of cells that fit inside the maxResultSize
assertTrue(
"Result's cell count differed from expected number. result: " + result,
result.rawCells().length == expectedNumberOfCells || !result.isPartial()
|| !Bytes.equals(prevRow, result.getRow()));
prevRow = result.getRow();
}
scanner.close();
}
/**
* @return The approximate heap size of a cell in the test table. All cells should have
* approximately the same heap size, so the value is cached to avoid repeating the
* calculation
* @throws Exception
*/
private long getCellHeapSize() throws Exception {
if (CELL_HEAP_SIZE == -1) {
// Do a partial scan that will return a single result with a single cell
Scan scan = new Scan();
scan.setMaxResultSize(1);
scan.setAllowPartialResults(true);
ResultScanner scanner = TABLE.getScanner(scan);
Result result = scanner.next();
assertTrue(result != null);
assertTrue(result.rawCells() != null);
assertTrue(result.rawCells().length == 1);
CELL_HEAP_SIZE = CellUtil.estimatedHeapSizeOf(result.rawCells()[0]);
if (LOG.isInfoEnabled()) LOG.info("Cell heap size: " + CELL_HEAP_SIZE);
scanner.close();
}
return CELL_HEAP_SIZE;
}
/**
* @param numberOfCells
* @return the result size that should be used in {@link Scan#setMaxResultSize(long)} if you want
* the server to return exactly numberOfCells cells
* @throws Exception
*/
private long getResultSizeForNumberOfCells(int numberOfCells) throws Exception {
return getCellHeapSize() * numberOfCells;
}
/**
* Test various combinations of batching and partial results for correctness
*/
@Test
public void testPartialResultsAndBatch() throws Exception {
for (int batch = 1; batch <= NUM_COLS / 4; batch++) {
for (int cellsPerPartial = 1; cellsPerPartial <= NUM_COLS / 4; cellsPerPartial++) {
testPartialResultsAndBatch(batch, cellsPerPartial);
}
}
}
public void testPartialResultsAndBatch(final int batch, final int cellsPerPartialResult)
throws Exception {
if (LOG.isInfoEnabled()) {
LOG.info("batch: " + batch + " cellsPerPartialResult: " + cellsPerPartialResult);
}
Scan scan = new Scan();
scan.setMaxResultSize(getResultSizeForNumberOfCells(cellsPerPartialResult));
scan.setBatch(batch);
ResultScanner scanner = TABLE.getScanner(scan);
Result result = scanner.next();
while ((result = scanner.next()) != null) {
assertTrue(result.rawCells() != null);
if (result.isPartial()) {
final String error =
"Cells:" + result.rawCells().length + " Batch size:" + batch
+ " cellsPerPartialResult:" + cellsPerPartialResult;
assertTrue(error, result.rawCells().length <= Math.min(batch, cellsPerPartialResult));
} else {
assertTrue(result.rawCells().length <= batch);
}
}
scanner.close();
}
/**
* Test the method {@link Result#createCompleteResult(List, Result)}
* @throws Exception
*/
@Test
public void testPartialResultsReassembly() throws Exception {
Scan scan = new Scan();
testPartialResultsReassembly(scan);
scan.setReversed(true);
testPartialResultsReassembly(scan);
}
public void testPartialResultsReassembly(Scan scanBase) throws Exception {
Scan partialScan = new Scan(scanBase);
partialScan.setMaxResultSize(1);
partialScan.setAllowPartialResults(true);
ResultScanner partialScanner = TABLE.getScanner(partialScan);
Scan oneShotScan = new Scan(scanBase);
oneShotScan.setMaxResultSize(Long.MAX_VALUE);
ResultScanner oneShotScanner = TABLE.getScanner(oneShotScan);
ArrayList<Result> partials = new ArrayList<>();
for (int i = 0; i < NUM_ROWS; i++) {
Result partialResult = null;
Result completeResult = null;
Result oneShotResult = null;
partials.clear();
do {
partialResult = partialScanner.next();
partials.add(partialResult);
} while (partialResult.isPartial());
completeResult = Result.createCompleteResult(partials);
oneShotResult = oneShotScanner.next();
compareResults(completeResult, oneShotResult, null);
}
assertTrue(oneShotScanner.next() == null);
assertTrue(partialScanner.next() == null);
oneShotScanner.close();
partialScanner.close();
}
/**
* When reconstructing the complete result from its partials we ensure that the row of each
* partial result is the same. If one of the rows differs, an exception is thrown.
*/
@Test
public void testExceptionThrownOnMismatchedPartialResults() throws IOException {
assertTrue(NUM_ROWS >= 2);
ArrayList<Result> partials = new ArrayList<>();
Scan scan = new Scan();
scan.setMaxResultSize(Long.MAX_VALUE);
ResultScanner scanner = TABLE.getScanner(scan);
Result r1 = scanner.next();
partials.add(r1);
Result r2 = scanner.next();
partials.add(r2);
assertFalse(Bytes.equals(r1.getRow(), r2.getRow()));
try {
Result.createCompleteResult(partials);
fail("r1 and r2 are from different rows. It should not be possible to combine them into"
+ " a single result");
} catch (IOException e) {
}
scanner.close();
}
/**
* When a scan has a filter where {@link org.apache.hadoop.hbase.filter.Filter#hasFilterRow()} is
* true, the scanner should not return partial results. The scanner cannot return partial results
* because the entire row needs to be read for the include/exclude decision to be made
*/
@Test
public void testNoPartialResultsWhenFilterPresent() throws Exception {
Scan scan = new Scan();
scan.setMaxResultSize(1);
scan.setAllowPartialResults(true);
// If a filter hasFilter() is true then partial results should not be returned else filter
// application server side would break.
scan.setFilter(new RandomRowFilter(1.0f));
ResultScanner scanner = TABLE.getScanner(scan);
Result r = null;
while ((r = scanner.next()) != null) {
assertFalse(r.isPartial());
}
scanner.close();
}
/**
* Examine the interaction between the maxResultSize and caching. If the caching limit is reached
* before the maxResultSize limit, we should not see partial results. On the other hand, if the
* maxResultSize limit is reached before the caching limit, it is likely that partial results will
* be seen.
* @throws Exception
*/
@Test
public void testPartialResultsAndCaching() throws Exception {
for (int caching = 1; caching <= NUM_ROWS; caching++) {
for (int maxResultRows = 0; maxResultRows <= NUM_ROWS; maxResultRows++) {
testPartialResultsAndCaching(maxResultRows, caching);
}
}
}
/**
* @param resultSizeRowLimit The row limit that will be enforced through maxResultSize
* @param cachingRowLimit The row limit that will be enforced through caching
* @throws Exception
*/
public void testPartialResultsAndCaching(int resultSizeRowLimit, int cachingRowLimit)
throws Exception {
Scan scan = new Scan();
scan.setAllowPartialResults(true);
// The number of cells specified in the call to getResultSizeForNumberOfCells is offset to
// ensure that the result size we specify is not an exact multiple of the number of cells
// in a row. This ensures that partial results will be returned when the result size limit
// is reached before the caching limit.
int cellOffset = NUM_COLS / 3;
long maxResultSize = getResultSizeForNumberOfCells(resultSizeRowLimit * NUM_COLS + cellOffset);
scan.setMaxResultSize(maxResultSize);
scan.setCaching(cachingRowLimit);
ResultScanner scanner = TABLE.getScanner(scan);
ClientScanner clientScanner = (ClientScanner) scanner;
Result r = null;
// Approximate the number of rows we expect will fit into the specified max rsult size. If this
// approximation is less than caching, then we expect that the max result size limit will be
// hit before the caching limit and thus partial results may be seen
boolean expectToSeePartialResults = resultSizeRowLimit < cachingRowLimit;
while ((r = clientScanner.next()) != null) {
assertTrue(!r.isPartial() || expectToSeePartialResults);
}
scanner.close();
}
/**
* Small scans should not return partial results because it would prevent small scans from
* retrieving all of the necessary results in a single RPC request which is what makese small
* scans useful. Thus, ensure that even when {@link Scan#getAllowPartialResults()} is true, small
* scans do not return partial results
* @throws Exception
*/
@Test
public void testSmallScansDoNotAllowPartials() throws Exception {
Scan scan = new Scan();
testSmallScansDoNotAllowPartials(scan);
scan.setReversed(true);
testSmallScansDoNotAllowPartials(scan);
}
public void testSmallScansDoNotAllowPartials(Scan baseScan) throws Exception {
Scan scan = new Scan(baseScan);
scan.setAllowPartialResults(true);
scan.setSmall(true);
scan.setMaxResultSize(1);
ResultScanner scanner = TABLE.getScanner(scan);
Result r = null;
while ((r = scanner.next()) != null) {
assertFalse(r.isPartial());
}
scanner.close();
}
/**
* Make puts to put the input value into each combination of row, family, and qualifier
* @param rows
* @param families
* @param qualifiers
* @param value
* @return
* @throws IOException
*/
static ArrayList<Put> createPuts(byte[][] rows, byte[][] families, byte[][] qualifiers,
byte[] value) throws IOException {
Put put;
ArrayList<Put> puts = new ArrayList<>();
for (int row = 0; row < rows.length; row++) {
put = new Put(rows[row]);
for (int fam = 0; fam < families.length; fam++) {
for (int qual = 0; qual < qualifiers.length; qual++) {
KeyValue kv = new KeyValue(rows[row], families[fam], qualifiers[qual], qual, value);
put.add(kv);
}
}
puts.add(put);
}
return puts;
}
/**
* Make key values to represent each possible combination of family and qualifier in the specified
* row.
* @param row
* @param families
* @param qualifiers
* @param value
* @return
*/
static ArrayList<Cell> createKeyValuesForRow(byte[] row, byte[][] families, byte[][] qualifiers,
byte[] value) {
ArrayList<Cell> outList = new ArrayList<>();
for (int fam = 0; fam < families.length; fam++) {
for (int qual = 0; qual < qualifiers.length; qual++) {
outList.add(new KeyValue(row, families[fam], qualifiers[qual], qual, value));
}
}
return outList;
}
/**
* Verifies that result contains all the key values within expKvList. Fails the test otherwise
* @param result
* @param expKvList
* @param msg
*/
static void verifyResult(Result result, List<Cell> expKvList, String msg) {
if (LOG.isInfoEnabled()) {
LOG.info(msg);
LOG.info("Expected count: " + expKvList.size());
LOG.info("Actual count: " + result.size());
}
if (expKvList.size() == 0) return;
int i = 0;
for (Cell kv : result.rawCells()) {
if (i >= expKvList.size()) {
break; // we will check the size later
}
Cell kvExp = expKvList.get(i++);
assertTrue("Not equal. get kv: " + kv.toString() + " exp kv: " + kvExp.toString(),
kvExp.equals(kv));
}
assertEquals(expKvList.size(), result.size());
}
/**
* Compares two results and fails the test if the results are different
* @param r1
* @param r2
* @param message
*/
static void compareResults(Result r1, Result r2, final String message) {
if (LOG.isInfoEnabled()) {
if (message != null) LOG.info(message);
LOG.info("r1: " + r1);
LOG.info("r2: " + r2);
}
final String failureMessage = "Results r1:" + r1 + " r2:" + r2 + " are not equivalent";
if (r1 == null && r2 == null) fail(failureMessage);
else if (r1 == null || r2 == null) fail(failureMessage);
try {
Result.compareResults(r1, r2);
} catch (Exception e) {
fail(failureMessage);
}
}
@Test
public void testReadPointAndPartialResults() throws Exception {
TableName testName = TableName.valueOf("testReadPointAndPartialResults");
int numRows = 5;
int numFamilies = 5;
int numQualifiers = 5;
byte[][] rows = HTestConst.makeNAscii(Bytes.toBytes("testRow"), numRows);
byte[][] families = HTestConst.makeNAscii(Bytes.toBytes("testFamily"), numFamilies);
byte[][] qualifiers = HTestConst.makeNAscii(Bytes.toBytes("testQualifier"), numQualifiers);
byte[] value = Bytes.createMaxByteArray(100);
Table tmpTable = createTestTable(testName, rows, families, qualifiers, value);
Scan scan = new Scan();
scan.setMaxResultSize(1);
scan.setAllowPartialResults(true);
// Open scanner before deletes
ResultScanner scanner = tmpTable.getScanner(scan);
Delete delete1 = new Delete(rows[0]);
delete1.addColumn(families[0], qualifiers[0], 0);
tmpTable.delete(delete1);
Delete delete2 = new Delete(rows[1]);
delete2.addColumn(families[1], qualifiers[1], 1);
tmpTable.delete(delete2);
// Should see all cells because scanner was opened prior to deletes
int scannerCount = countCellsFromScanner(scanner);
int expectedCount = numRows * numFamilies * numQualifiers;
assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
scannerCount == expectedCount);
// Minus 2 for the two cells that were deleted
scanner = tmpTable.getScanner(scan);
scannerCount = countCellsFromScanner(scanner);
expectedCount = numRows * numFamilies * numQualifiers - 2;
assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
scannerCount == expectedCount);
scanner = tmpTable.getScanner(scan);
// Put in 2 new rows. The timestamps differ from the deleted rows
Put put1 = new Put(rows[0]);
put1.add(new KeyValue(rows[0], families[0], qualifiers[0], 1, value));
tmpTable.put(put1);
Put put2 = new Put(rows[1]);
put2.add(new KeyValue(rows[1], families[1], qualifiers[1], 2, value));
tmpTable.put(put2);
// Scanner opened prior to puts. Cell count shouldn't have changed
scannerCount = countCellsFromScanner(scanner);
expectedCount = numRows * numFamilies * numQualifiers - 2;
assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
scannerCount == expectedCount);
// Now the scanner should see the cells that were added by puts
scanner = tmpTable.getScanner(scan);
scannerCount = countCellsFromScanner(scanner);
expectedCount = numRows * numFamilies * numQualifiers;
assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
scannerCount == expectedCount);
TEST_UTIL.deleteTable(testName);
}
/**
* Exhausts the scanner by calling next repetitively. Once completely exhausted, close scanner and
* return total cell count
* @param scanner
* @return
* @throws Exception
*/
private int countCellsFromScanner(ResultScanner scanner) throws Exception {
Result result = null;
int numCells = 0;
while ((result = scanner.next()) != null) {
numCells += result.rawCells().length;
}
scanner.close();
return numCells;
}
}

View File

@ -28,6 +28,7 @@ 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;
@ -93,7 +94,7 @@ public class TestIntraRowPagination {
RegionScanner scanner = region.getScanner(scan);
List<Cell> kvListScan = new ArrayList<Cell>();
List<Cell> results = new ArrayList<Cell>();
while (scanner.next(results) || !results.isEmpty()) {
while (NextState.hasMoreValues(scanner.next(results)) || !results.isEmpty()) {
kvListScan.addAll(results);
results.clear();
}

View File

@ -28,13 +28,13 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.ColumnAggregationService;
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest;
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 +89,7 @@ implements Coprocessor, CoprocessorService {
boolean hasMore = false;
do {
curVals.clear();
hasMore = scanner.next(curVals);
hasMore = NextState.hasMoreValues(scanner.next(curVals));
for (Cell kv : curVals) {
if (CellUtil.matchingQualifier(kv, qualifier)) {
sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset());

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
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.util.Bytes;
import com.google.protobuf.RpcCallback;
@ -97,7 +98,7 @@ implements Coprocessor, CoprocessorService {
boolean hasMore = false;
do {
curVals.clear();
hasMore = scanner.next(curVals);
hasMore = NextState.hasMoreValues(scanner.next(curVals));
for (Cell kv : curVals) {
if (CellUtil.matchingQualifier(kv, qualifier)) {
sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset());

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationW
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
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.util.Bytes;
import com.google.protobuf.RpcCallback;
@ -97,7 +98,7 @@ implements Coprocessor, CoprocessorService {
boolean hasMore = false;
do {
curVals.clear();
hasMore = scanner.next(curVals);
hasMore = NextState.hasMoreValues(scanner.next(curVals));
for (Cell kv : curVals) {
if (CellUtil.matchingQualifier(kv, qualifier)) {
sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset());

View File

@ -85,27 +85,38 @@ public class TestCoprocessorInterface {
}
@Override
public boolean next(List<Cell> results) throws IOException {
public NextState next(List<Cell> results) throws IOException {
return delegate.next(results);
}
@Override
public boolean next(List<Cell> result, int limit) throws IOException {
public NextState next(List<Cell> result, int limit) throws IOException {
return delegate.next(result, limit);
}
@Override
public boolean nextRaw(List<Cell> result)
public NextState next(List<Cell> result, int limit, long remainingResultSize)
throws IOException {
return delegate.next(result, limit, remainingResultSize);
}
@Override
public NextState nextRaw(List<Cell> result)
throws IOException {
return delegate.nextRaw(result);
}
@Override
public boolean nextRaw(List<Cell> result, int limit)
throws IOException {
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)
throws IOException {
return delegate.nextRaw(result, limit, remainingResultSize);
}
@Override
public void close() throws IOException {
delegate.close();
@ -135,6 +146,12 @@ public class TestCoprocessorInterface {
public long getMvccReadPoint() {
return delegate.getMvccReadPoint();
}
@Override
public int getBatch() {
return delegate.getBatch();
}
}
public static class CoprocessorImpl extends BaseRegionObserver {

View File

@ -433,17 +433,24 @@ public class TestRegionObserverInterface {
Store store, final InternalScanner scanner, final ScanType scanType) {
return new InternalScanner() {
@Override
public boolean next(List<Cell> results) throws IOException {
public NextState next(List<Cell> results) throws IOException {
return next(results, -1);
}
@Override
public boolean next(List<Cell> results, int limit)
throws IOException{
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)
throws IOException {
List<Cell> internalResults = new ArrayList<Cell>();
boolean hasMore;
NextState state;
do {
hasMore = scanner.next(internalResults, limit);
state = scanner.next(internalResults, limit, remainingResultSize);
hasMore = state != null && state.hasMoreValues();
if (!internalResults.isEmpty()) {
long row = Bytes.toLong(CellUtil.cloneValue(internalResults.get(0)));
if (row % 2 == 0) {
@ -458,7 +465,7 @@ public class TestRegionObserverInterface {
if (!internalResults.isEmpty()) {
results.addAll(internalResults);
}
return hasMore;
return state;
}
@Override
@ -772,4 +779,4 @@ public class TestRegionObserverInterface {
writer.close();
}
}
}
}

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.filter;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.ArrayList;
@ -27,12 +27,20 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
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.client.Durability;
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;
@ -97,7 +105,7 @@ public class TestColumnPrefixFilter {
InternalScanner scanner = region.getScanner(scan);
List<Cell> results = new ArrayList<Cell>();
while(scanner.next(results));
while (NextState.hasMoreValues(scanner.next(results)));
assertEquals(prefixMap.get(s).size(), results.size());
}
} finally {
@ -162,7 +170,7 @@ public class TestColumnPrefixFilter {
InternalScanner scanner = region.getScanner(scan);
List<Cell> results = new ArrayList<Cell>();
while(scanner.next(results));
while (NextState.hasMoreValues(scanner.next(results)));
assertEquals(prefixMap.get(s).size(), results.size());
}
} finally {

View File

@ -19,6 +19,10 @@
package org.apache.hadoop.hbase.filter;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -26,24 +30,26 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
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;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import org.junit.experimental.categories.Category;
@Category({FilterTests.class, SmallTests.class})
@ -145,7 +151,7 @@ public class TestDependentColumnFilter {
int i = 0;
int cells = 0;
for (boolean done = true; done; i++) {
done = scanner.next(results);
done = NextState.hasMoreValues(scanner.next(results));
Arrays.sort(results.toArray(new KeyValue[results.size()]),
KeyValue.COMPARATOR);
LOG.info("counter=" + i + ", " + results);

View File

@ -47,11 +47,12 @@ 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.RegionScanner;
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;
@ -501,7 +502,7 @@ public class TestFilter {
InternalScanner scanner = this.region.getScanner(s);
int scannerCounter = 0;
while (true) {
boolean isMoreResults = scanner.next(new ArrayList<Cell>());
boolean isMoreResults = NextState.hasMoreValues(scanner.next(new ArrayList<Cell>()));
scannerCounter++;
if (scannerCounter >= pageSize) {
@ -530,7 +531,7 @@ public class TestFilter {
InternalScanner scanner = this.region.getScanner(s);
while (true) {
ArrayList<Cell> values = new ArrayList<Cell>();
boolean isMoreResults = scanner.next(values);
boolean isMoreResults = NextState.hasMoreValues(scanner.next(values));
if (!isMoreResults
|| !Bytes.toString(values.get(0).getRow()).startsWith(prefix)) {
Assert.assertTrue(
@ -564,7 +565,7 @@ public class TestFilter {
InternalScanner scanner = this.region.getScanner(s);
int scannerCounter = 0;
while (true) {
boolean isMoreResults = scanner.next(new ArrayList<Cell>());
boolean isMoreResults = NextState.hasMoreValues(scanner.next(new ArrayList<Cell>()));
scannerCounter++;
if (scannerCounter >= pageSize) {
@ -642,7 +643,7 @@ public class TestFilter {
InternalScanner scanner = this.region.getScanner(s);
while (true) {
ArrayList<Cell> values = new ArrayList<Cell>();
boolean isMoreResults = scanner.next(values);
boolean isMoreResults = NextState.hasMoreValues(scanner.next(values));
if (!isMoreResults || !Bytes.toString(CellUtil.cloneRow(values.get(0))).startsWith(prefix)) {
assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining());
}
@ -671,7 +672,7 @@ public class TestFilter {
InternalScanner scanner = this.region.getScanner(s);
while (true) {
ArrayList<Cell> values = new ArrayList<Cell>();
boolean isMoreResults = scanner.next(values);
boolean isMoreResults = NextState.hasMoreValues(scanner.next(values));
assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining());
if (!isMoreResults) {
break;
@ -1474,7 +1475,7 @@ public class TestFilter {
InternalScanner scanner = testRegion.getScanner(s1);
List<Cell> results = new ArrayList<Cell>();
int resultCount = 0;
while(scanner.next(results)) {
while (NextState.hasMoreValues(scanner.next(results))) {
resultCount++;
byte[] row = CellUtil.cloneRow(results.get(0));
LOG.debug("Found row: " + Bytes.toStringBinary(row));
@ -1616,7 +1617,7 @@ public class TestFilter {
List<Cell> results = new ArrayList<Cell>();
int i = 0;
for (boolean done = true; done; i++) {
done = scanner.next(results);
done = NextState.hasMoreValues(scanner.next(results));
Arrays.sort(results.toArray(new KeyValue[results.size()]),
KeyValue.COMPARATOR);
LOG.info("counter=" + i + ", " + results);
@ -1638,7 +1639,7 @@ public class TestFilter {
List<Cell> results = new ArrayList<Cell>();
int i = 0;
for (boolean done = true; done; i++) {
done = scanner.next(results);
done = NextState.hasMoreValues(scanner.next(results));
Arrays.sort(results.toArray(new KeyValue[results.size()]),
KeyValue.COMPARATOR);
LOG.info("counter=" + i + ", " + results);
@ -1660,7 +1661,7 @@ public class TestFilter {
int row = 0;
int idx = 0;
for (boolean done = true; done; row++) {
done = scanner.next(results);
done = NextState.hasMoreValues(scanner.next(results));
Arrays.sort(results.toArray(new KeyValue[results.size()]),
KeyValue.COMPARATOR);
if(results.isEmpty()) break;
@ -1691,7 +1692,7 @@ public class TestFilter {
int row = 0;
int idx = 0;
for (boolean more = true; more; row++) {
more = scanner.next(results);
more = NextState.hasMoreValues(scanner.next(results));
Arrays.sort(results.toArray(new KeyValue[results.size()]),
KeyValue.COMPARATOR);
if(results.isEmpty()) break;
@ -2027,7 +2028,7 @@ public class TestFilter {
List<Cell> results = new ArrayList<Cell>();
int i = 5;
for (boolean done = true; done; i++) {
done = scanner.next(results);
done = NextState.hasMoreValues(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();
@ -2050,7 +2051,7 @@ public class TestFilter {
assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
results.clear();
}
assertFalse(scanner.next(results));
assertFalse(NextState.hasMoreValues(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);
@ -2072,7 +2073,7 @@ public class TestFilter {
assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
results.clear();
}
assertFalse(scanner.next(results));
assertFalse(NextState.hasMoreValues(scanner.next(results)));
// 3.2 MAGIC here! add subFilterList first, then add rowFilter
filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
filterList.addFilter(subFilterList);
@ -2093,7 +2094,7 @@ public class TestFilter {
assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
results.clear();
}
assertFalse(scanner.next(results));
assertFalse(NextState.hasMoreValues(scanner.next(results)));
WAL wal = testRegion.getWAL();
testRegion.close();
wal.close();

View File

@ -34,10 +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.wal.WAL;
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;
import org.apache.hadoop.hbase.wal.WAL;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -138,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 (scanner.next(temp)) {
while (NextState.hasMoreValues(scanner.next(temp))) {
actualValues.addAll(temp);
temp.clear();
}

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.filter;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.ArrayList;
@ -27,12 +27,20 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
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.client.Durability;
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;
@ -102,7 +110,8 @@ public class TestMultipleColumnPrefixFilter {
scan.setFilter(filter);
List<Cell> results = new ArrayList<Cell>();
InternalScanner scanner = region.getScanner(scan);
while(scanner.next(results));
while (NextState.hasMoreValues(scanner.next(results)))
;
assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
HBaseTestingUtility.closeRegionAndWAL(region);
@ -174,7 +183,8 @@ public class TestMultipleColumnPrefixFilter {
scan.setFilter(filter);
List<Cell> results = new ArrayList<Cell>();
InternalScanner scanner = region.getScanner(scan);
while(scanner.next(results));
while (NextState.hasMoreValues(scanner.next(results)))
;
assertEquals(prefixMap.get("p").size() + prefixMap.get("q").size(), results.size());
HBaseTestingUtility.closeRegionAndWAL(region);
@ -218,7 +228,8 @@ public class TestMultipleColumnPrefixFilter {
scan1.setFilter(multiplePrefixFilter);
List<Cell> results1 = new ArrayList<Cell>();
InternalScanner scanner1 = region.getScanner(scan1);
while(scanner1.next(results1));
while (NextState.hasMoreValues(scanner1.next(results1)))
;
ColumnPrefixFilter singlePrefixFilter;
Scan scan2 = new Scan();
@ -228,7 +239,8 @@ public class TestMultipleColumnPrefixFilter {
scan2.setFilter(singlePrefixFilter);
List<Cell> results2 = new ArrayList<Cell>();
InternalScanner scanner2 = region.getScanner(scan1);
while(scanner2.next(results2));
while (NextState.hasMoreValues(scanner2.next(results2)))
;
assertEquals(results1.size(), results2.size());

View File

@ -38,6 +38,7 @@ 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.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@ -116,7 +117,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, scanner.next(cells));
assertEquals(i < 2, NextState.hasMoreValues(scanner.next(cells)));
CellScanner cellScanner = Result.create(cells).cellScanner();
while (cellScanner.advance()) {
assertEquals(rows[i], Bytes.toString(cellScanner.current().getRowArray(), cellScanner
@ -153,7 +154,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, scanner.next(cells));
assertEquals(i < 2, NextState.hasMoreValues(scanner.next(cells)));
CellScanner cellScanner = Result.create(cells).cellScanner();
while (cellScanner.advance()) {
assertEquals(rows[i], Bytes.toString(cellScanner.current().getRowArray(), cellScanner
@ -168,7 +169,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(scanner.next(cells));
assertFalse(NextState.hasMoreValues(scanner.next(cells)));
assertFalse(cells.isEmpty());
scanner.close();
}
@ -186,7 +187,7 @@ public class TestPrefixTree {
Scan scan = new Scan(Bytes.toBytes("obj29995"));
RegionScanner scanner = region.getScanner(scan);
List<Cell> cells = new ArrayList<Cell>();
assertFalse(scanner.next(cells));
assertFalse(NextState.hasMoreValues(scanner.next(cells)));
assertArrayEquals(Bytes.toBytes("obj3"), Result.create(cells).getRow());
}
}

View File

@ -32,14 +32,15 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.TableName;
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.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.Test;
@ -121,7 +122,7 @@ public class TestScannerSelectionUsingKeyRange {
cache.clearCache();
InternalScanner scanner = region.getScanner(scan);
List<Cell> results = new ArrayList<Cell>();
while (scanner.next(results)) {
while (NextState.hasMoreValues(scanner.next(results))) {
}
scanner.close();
assertEquals(0, results.size());

View File

@ -28,16 +28,17 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
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.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -141,7 +142,7 @@ public class TestScannerSelectionUsingTTL {
final int expectedKVsPerRow = numFreshFiles * NUM_COLS_PER_ROW;
int numReturnedRows = 0;
LOG.info("Scanning the entire table");
while (scanner.next(results) || results.size() > 0) {
while (NextState.hasMoreValues(scanner.next(results)) || results.size() > 0) {
assertEquals(expectedKVsPerRow, results.size());
++numReturnedRows;
results.clear();

View File

@ -61,6 +61,7 @@ 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;
@ -466,7 +467,7 @@ public class TestAtomicOperation {
Scan s = new Scan(row);
RegionScanner rs = region.getScanner(s);
List<Cell> r = new ArrayList<Cell>();
while(rs.next(r));
while(NextState.hasMoreValues(rs.next(r)));
rs.close();
if (r.size() != 1) {
LOG.debug(r);

View File

@ -25,6 +25,7 @@ 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;
@ -103,7 +104,7 @@ public class TestBlocksScanned extends HBaseTestCase {
InternalScanner s = r.getScanner(scan);
List<Cell> results = new ArrayList<Cell>();
while (s.next(results));
while (NextState.hasMoreValues(s.next(results)));
s.close();
int expectResultSize = 'z' - 'a';

View File

@ -19,7 +19,8 @@
package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
@ -31,10 +32,18 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
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.client.Durability;
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;
@ -152,7 +161,7 @@ public class TestColumnSeeking {
}
InternalScanner scanner = region.getScanner(scan);
List<Cell> results = new ArrayList<Cell>();
while (scanner.next(results))
while (NextState.hasMoreValues(scanner.next(results)))
;
assertEquals(kvSet.size(), results.size());
assertTrue(KeyValueTestUtil.containsIgnoreMvccVersion(results, kvSet));
@ -264,7 +273,7 @@ public class TestColumnSeeking {
}
InternalScanner scanner = region.getScanner(scan);
List<Cell> results = new ArrayList<Cell>();
while (scanner.next(results))
while (NextState.hasMoreValues(scanner.next(results)))
;
assertEquals(kvSet.size(), results.size());
assertTrue(KeyValueTestUtil.containsIgnoreMvccVersion(results, kvSet));

View File

@ -48,9 +48,10 @@ import org.apache.hadoop.hbase.KeyValueTestUtil;
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.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -107,7 +108,7 @@ public class TestDefaultMemStore extends TestCase {
StoreScanner s = new StoreScanner(scan, scanInfo, scanType, null, memstorescanners);
int count = 0;
try {
while (s.next(result)) {
while (NextState.hasMoreValues(s.next(result))) {
LOG.info(result);
count++;
// Row count is same as column count.
@ -127,7 +128,7 @@ public class TestDefaultMemStore extends TestCase {
s = new StoreScanner(scan, scanInfo, scanType, null, memstorescanners);
count = 0;
try {
while (s.next(result)) {
while (NextState.hasMoreValues(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 +155,7 @@ public class TestDefaultMemStore extends TestCase {
count = 0;
int snapshotIndex = 5;
try {
while (s.next(result)) {
while (NextState.hasMoreValues(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 +529,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; scanner.next(results); i++) {
for (int i = 0; NextState.hasMoreValues(scanner.next(results)); i++) {
int rowId = startRowId + i;
Cell left = results.get(0);
byte[] row1 = Bytes.toBytes(rowId);

View File

@ -34,15 +34,16 @@ 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.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Delete;
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;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -95,7 +96,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
InternalScanner s = mr.getScanner(new Scan());
try {
List<Cell> keys = new ArrayList<Cell>();
while(s.next(keys)) {
while (NextState.hasMoreValues(s.next(keys))) {
LOG.info(keys);
keys.clear();
}
@ -119,7 +120,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
s = mr.getScanner(scan);
try {
List<Cell> keys = new ArrayList<Cell>();
while (s.next(keys)) {
while (NextState.hasMoreValues(s.next(keys))) {
mr.delete(new Delete(CellUtil.cloneRow(keys.get(0))));
keys.clear();
}

View File

@ -79,8 +79,6 @@ import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
@ -91,8 +89,10 @@ import org.apache.hadoop.hbase.MultithreadedTestUtil;
import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagType;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Append;
@ -122,26 +122,20 @@ import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
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.HRegion.RowLock;
import org.apache.hadoop.hbase.regionserver.InternalScanner.NextState;
import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem;
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.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.FaultyFSLog;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALProvider;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -153,6 +147,13 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.FaultyFSLog;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALProvider;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -1262,7 +1263,7 @@ public class TestHRegion {
boolean more = false;
List<Cell> results = new ArrayList<Cell>();
do {
more = scanner.next(results);
more = NextState.hasMoreValues(scanner.next(results));
if (results != null && !results.isEmpty())
count++;
else
@ -1281,7 +1282,7 @@ public class TestHRegion {
List<Cell> results = new ArrayList<Cell>();
boolean more = false;
do {
more = resultScanner.next(results);
more = NextState.hasMoreValues(resultScanner.next(results));
if (results != null && !results.isEmpty())
numberOfResults++;
else
@ -2271,7 +2272,7 @@ public class TestHRegion {
InternalScanner s = region.getScanner(scan);
List<Cell> results = new ArrayList<Cell>();
assertEquals(false, s.next(results));
assertEquals(false, NextState.hasMoreValues(s.next(results)));
assertEquals(1, results.size());
Cell kv = results.get(0);
@ -2762,7 +2763,7 @@ public class TestHRegion {
List<Cell> actual = new ArrayList<Cell>();
InternalScanner scanner = region.getScanner(scan);
boolean hasNext = scanner.next(actual);
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
assertEquals(false, hasNext);
// Verify result
@ -2825,7 +2826,7 @@ public class TestHRegion {
List<Cell> actual = new ArrayList<Cell>();
InternalScanner scanner = region.getScanner(scan);
boolean hasNext = scanner.next(actual);
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
assertEquals(false, hasNext);
// Verify result
@ -2907,7 +2908,7 @@ public class TestHRegion {
List<Cell> actual = new ArrayList<Cell>();
InternalScanner scanner = region.getScanner(scan);
boolean hasNext = scanner.next(actual);
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
assertEquals(false, hasNext);
// Verify result
@ -2968,7 +2969,7 @@ public class TestHRegion {
List<Cell> actual = new ArrayList<Cell>();
InternalScanner scanner = region.getScanner(scan);
boolean hasNext = scanner.next(actual);
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
assertEquals(false, hasNext);
// Verify result
@ -3029,7 +3030,7 @@ public class TestHRegion {
List<Cell> actual = new ArrayList<Cell>();
InternalScanner scanner = region.getScanner(scan);
boolean hasNext = scanner.next(actual);
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
assertEquals(false, hasNext);
// Verify result
@ -3082,7 +3083,7 @@ public class TestHRegion {
InternalScanner s = region.getScanner(scan);
List<Cell> results = new ArrayList<Cell>();
assertEquals(false, s.next(results));
assertEquals(false, NextState.hasMoreValues(s.next(results)));
assertEquals(0, results.size());
} finally {
HBaseTestingUtility.closeRegionAndWAL(this.region);
@ -3156,7 +3157,7 @@ public class TestHRegion {
List<Cell> actual = new ArrayList<Cell>();
InternalScanner scanner = region.getScanner(scan);
boolean hasNext = scanner.next(actual);
boolean hasNext = NextState.hasMoreValues(scanner.next(actual));
assertEquals(false, hasNext);
// Verify result
@ -3219,18 +3220,18 @@ public class TestHRegion {
InternalScanner s = region.getScanner(scan);
List<Cell> results = new ArrayList<Cell>();
assertTrue(s.next(results));
assertTrue(NextState.hasMoreValues(s.next(results)));
assertEquals(results.size(), 1);
results.clear();
assertTrue(s.next(results));
assertTrue(NextState.hasMoreValues(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(s.next(results));
assertFalse(NextState.hasMoreValues(s.next(results)));
assertEquals(results.size(), 0);
} finally {
HBaseTestingUtility.closeRegionAndWAL(this.region);
@ -3314,7 +3315,7 @@ public class TestHRegion {
List<Cell> results = new ArrayList<Cell>();
int index = 0;
while (true) {
boolean more = s.next(results, 3);
boolean more = NextState.hasMoreValues(s.next(results, 3));
if ((index >> 1) < 5) {
if (index % 2 == 0)
assertEquals(results.size(), 3);
@ -3587,7 +3588,7 @@ public class TestHRegion {
if (toggle) {
flushThread.flush();
}
while (scanner.next(res))
while (NextState.hasMoreValues(scanner.next(res)))
;
if (!toggle) {
flushThread.flush();
@ -3712,7 +3713,7 @@ public class TestHRegion {
boolean previousEmpty = res.isEmpty();
res.clear();
InternalScanner scanner = region.getScanner(scan);
while (scanner.next(res))
while (NextState.hasMoreValues(scanner.next(res)))
;
if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
assertEquals("i=" + i, expectedCount, res.size());
@ -3994,7 +3995,7 @@ public class TestHRegion {
InternalScanner scanner = region.getScanner(idxScan);
List<Cell> res = new ArrayList<Cell>();
while (scanner.next(res))
while (NextState.hasMoreValues(scanner.next(res)))
;
assertEquals(1L, res.size());
} finally {
@ -4890,7 +4891,7 @@ public class TestHRegion {
try {
List<Cell> curVals = new ArrayList<Cell>();
boolean first = true;
OUTER_LOOP: while (s.next(curVals)) {
OUTER_LOOP: while (NextState.hasMoreValues(s.next(curVals))) {
for (Cell kv : curVals) {
byte[] val = CellUtil.cloneValue(kv);
byte[] curval = val;
@ -5086,17 +5087,17 @@ public class TestHRegion {
scan.setReversed(true);
InternalScanner scanner = region.getScanner(scan);
List<Cell> currRow = new ArrayList<Cell>();
boolean hasNext = scanner.next(currRow);
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(2, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
assertFalse(hasNext);
@ -5143,17 +5144,17 @@ public class TestHRegion {
scan.setReversed(true);
scan.setMaxVersions(5);
InternalScanner scanner = region.getScanner(scan);
boolean hasNext = scanner.next(currRow);
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(2, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
assertFalse(hasNext);
@ -5197,17 +5198,17 @@ public class TestHRegion {
List<Cell> currRow = new ArrayList<Cell>();
scan.setReversed(true);
InternalScanner scanner = region.getScanner(scan);
boolean hasNext = scanner.next(currRow);
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
assertFalse(hasNext);
@ -5265,17 +5266,17 @@ public class TestHRegion {
scan.setReversed(true);
List<Cell> currRow = new ArrayList<Cell>();
InternalScanner scanner = region.getScanner(scan);
boolean hasNext = scanner.next(currRow);
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
assertFalse(hasNext);
@ -5286,7 +5287,7 @@ public class TestHRegion {
scan.setReversed(true);
currRow.clear();
scanner = region.getScanner(scan);
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
scanner.close();
@ -5345,17 +5346,17 @@ public class TestHRegion {
scan.setReversed(true);
List<Cell> currRow = new ArrayList<Cell>();
InternalScanner scanner = region.getScanner(scan);
boolean hasNext = scanner.next(currRow);
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
assertFalse(hasNext);
@ -5366,7 +5367,7 @@ public class TestHRegion {
scan.setReversed(true);
currRow.clear();
scanner = region.getScanner(scan);
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
scanner.close();
@ -5490,42 +5491,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 = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(3, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(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 = scanner.next(currRow);
hasNext = NextState.hasMoreValues(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 = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(3, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(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 = scanner.next(currRow);
hasNext = NextState.hasMoreValues(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 = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), row0));
assertFalse(hasNext);
@ -5586,22 +5587,22 @@ public class TestHRegion {
scan.setBatch(10);
InternalScanner scanner = region.getScanner(scan);
List<Cell> currRow = new ArrayList<Cell>();
boolean hasNext = scanner.next(currRow);
boolean hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), row3));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
assertTrue(hasNext);
currRow.clear();
hasNext = scanner.next(currRow);
hasNext = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(1, currRow.size());
assertTrue(Bytes.equals(currRow.get(0).getRow(), row1));
assertFalse(hasNext);
@ -5654,7 +5655,7 @@ public class TestHRegion {
boolean more = false;
int verify = startRow + 2 * numRows - 1;
do {
more = scanner.next(currRow);
more = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
verify--;
currRow.clear();
@ -5667,7 +5668,7 @@ public class TestHRegion {
scanner = regions[1].getScanner(scan);
verify = startRow + 2 * numRows - 1;
do {
more = scanner.next(currRow);
more = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
verify--;
currRow.clear();
@ -5680,7 +5681,7 @@ public class TestHRegion {
scanner = regions[0].getScanner(scan);
verify = startRow + numRows - 1;
do {
more = scanner.next(currRow);
more = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
verify--;
currRow.clear();
@ -5693,7 +5694,7 @@ public class TestHRegion {
scanner = regions[0].getScanner(scan);
verify = startRow + numRows - 1;
do {
more = scanner.next(currRow);
more = NextState.hasMoreValues(scanner.next(currRow));
assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
verify--;
currRow.clear();

View File

@ -33,13 +33,14 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeepDeletedCells;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.client.Delete;
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;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
@ -263,7 +264,7 @@ public class TestKeepDeletes {
s.setTimeRange(0L, ts+1);
InternalScanner scanner = region.getScanner(s);
List<Cell> kvs = new ArrayList<Cell>();
while(scanner.next(kvs));
while (NextState.hasMoreValues(scanner.next(kvs)));
assertTrue(kvs.isEmpty());
// flushing and minor compaction keep delete markers
@ -945,7 +946,7 @@ public class TestKeepDeletes {
int res = 0;
boolean hasMore;
do {
hasMore = scan.next(kvs);
hasMore = NextState.hasMoreValues(scan.next(kvs));
for (Cell kv : kvs) {
if(CellUtil.isDelete(kv)) res++;
}

View File

@ -44,8 +44,6 @@ import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result;
@ -54,11 +52,14 @@ 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;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
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;
@ -131,7 +132,7 @@ public class TestMajorCompaction {
InternalScanner s = r.getScanner(new Scan());
do {
List<Cell> results = new ArrayList<Cell>();
boolean result = s.next(results);
boolean result = NextState.hasMoreValues(s.next(results));
r.delete(new Delete(CellUtil.cloneRow(results.get(0))));
if (!result) break;
} while(true);
@ -144,7 +145,7 @@ public class TestMajorCompaction {
int counter = 0;
do {
List<Cell> results = new ArrayList<Cell>();
boolean result = s.next(results);
boolean result = NextState.hasMoreValues(s.next(results));
if (!result) break;
counter++;
} while(true);
@ -458,7 +459,7 @@ public class TestMajorCompaction {
InternalScanner s = r.getScanner(scan);
do {
List<Cell> results = new ArrayList<Cell>();
boolean result = s.next(results);
boolean result = NextState.hasMoreValues(s.next(results));
assertTrue(!results.isEmpty());
r.delete(new Delete(results.get(0).getRow()));
if (!result) break;
@ -474,7 +475,7 @@ public class TestMajorCompaction {
int counter = 0;
do {
List<Cell> results = new ArrayList<Cell>();
boolean result = s.next(results);
boolean result = NextState.hasMoreValues(s.next(results));
if (!result) break;
counter++;
} while (true);

View File

@ -44,13 +44,14 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
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.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;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -249,7 +250,7 @@ public class TestMultiColumnScanner {
String queryInfo = "columns queried: " + qualSet + " (columnBitMask="
+ columnBitMask + "), maxVersions=" + maxVersions;
while (scanner.next(results) || results.size() > 0) {
while (NextState.hasMoreValues(scanner.next(results)) || results.size() > 0) {
for (Cell kv : results) {
while (kvPos < kvs.size()
&& !matchesQuery(kvs.get(kvPos), qualSet, maxVersions,

View File

@ -44,11 +44,12 @@ 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.wal.WALFactory;
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;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Before;
@ -426,7 +427,7 @@ public class TestRegionMergeTransaction {
List<Cell> kvs = new ArrayList<Cell>();
boolean hasNext = true;
while (hasNext) {
hasNext = scanner.next(kvs);
hasNext = NextState.hasMoreValues(scanner.next(kvs));
if (!kvs.isEmpty())
rowcount++;
}

View File

@ -42,8 +42,6 @@ import org.apache.hadoop.hbase.KeepDeletedCells;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@ -57,6 +55,9 @@ 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;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.Test;
@ -433,7 +434,7 @@ public class TestReversibleScanners {
int rowCount = 0;
int kvCount = 0;
try {
while (scanner.next(kvList)) {
while (NextState.hasMoreValues(scanner.next(kvList))) {
if (kvList.isEmpty()) continue;
rowCount++;
kvCount += kvList.size();

View File

@ -42,13 +42,14 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
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.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;
import org.junit.Before;
import org.junit.Test;
@ -164,7 +165,7 @@ public class TestScanWithBloomError {
{ // Limit the scope of results.
List<Cell> results = new ArrayList<Cell>();
while (scanner.next(results) || results.size() > 0) {
while (NextState.hasMoreValues(scanner.next(results)) || results.size() > 0) {
allResults.addAll(results);
results.clear();
}

View File

@ -43,8 +43,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
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.UnknownScannerException;
import org.apache.hadoop.hbase.client.Delete;
@ -56,6 +54,9 @@ 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;
import org.junit.Rule;
import org.junit.Test;
@ -136,7 +137,7 @@ public class TestScanner {
InternalScanner s = r.getScanner(scan);
int count = 0;
while (s.next(results)) {
while (NextState.hasMoreValues(s.next(results))) {
count++;
}
s.close();
@ -149,7 +150,7 @@ public class TestScanner {
count = 0;
Cell kv = null;
results = new ArrayList<Cell>();
for (boolean first = true; s.next(results);) {
for (boolean first = true; NextState.hasMoreValues(s.next(results));) {
kv = results.get(0);
if (first) {
assertTrue(CellUtil.matchingRow(kv, startrow));
@ -172,7 +173,7 @@ public class TestScanner {
InternalScanner s = r.getScanner(scan);
boolean hasMore = true;
while (hasMore) {
hasMore = s.next(results);
hasMore = NextState.hasMoreValues(s.next(results));
for (Cell kv : results) {
assertEquals((byte)'a', CellUtil.cloneRow(kv)[0]);
assertEquals((byte)'b', CellUtil.cloneRow(kv)[1]);
@ -188,7 +189,7 @@ public class TestScanner {
InternalScanner s = r.getScanner(scan);
boolean hasMore = true;
while (hasMore) {
hasMore = s.next(results);
hasMore = NextState.hasMoreValues(s.next(results));
for (Cell kv : results) {
assertTrue(Bytes.compareTo(CellUtil.cloneRow(kv), stopRow) <= 0);
}
@ -388,7 +389,7 @@ public class TestScanner {
scan.addColumn(COLS[0], EXPLICIT_COLS[ii]);
}
scanner = r.getScanner(scan);
while (scanner.next(results)) {
while (NextState.hasMoreValues(scanner.next(results))) {
assertTrue(hasColumn(results, HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER));
byte [] val = CellUtil.cloneValue(getColumn(results, HConstants.CATALOG_FAMILY,

View File

@ -40,12 +40,13 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
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;
import org.junit.After;
import org.junit.Before;
@ -224,7 +225,7 @@ public class TestSeekOptimizations {
// result, not to the one already returned in results.
boolean hasNext;
do {
hasNext = scanner.next(results);
hasNext = NextState.hasMoreValues(scanner.next(results));
actualKVs.addAll(results);
results.clear();
} while (hasNext);

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doNothing;
@ -44,19 +43,20 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.wal.WALFactory;
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;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Before;
@ -358,7 +358,7 @@ public class TestSplitTransaction {
List<Cell> kvs = new ArrayList<Cell>();
boolean hasNext = true;
while (hasNext) {
hasNext = scanner.next(kvs);
hasNext = NextState.hasMoreValues(scanner.next(kvs));
if (!kvs.isEmpty()) rowcount++;
}
} finally {

View File

@ -35,9 +35,10 @@ import org.apache.hadoop.hbase.HConstants;
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.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
@ -90,7 +91,7 @@ public class TestStoreScanner extends TestCase {
StoreScanner scan = new StoreScanner(scanSpec, scanInfo, scanType,
getCols("a"), scanners);
List<Cell> results = new ArrayList<Cell>();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(5, results.size());
assertEquals(kvs[kvs.length - 1], results.get(0));
// Scan limited TimeRange
@ -100,7 +101,7 @@ public class TestStoreScanner extends TestCase {
scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"),
scanners);
results = new ArrayList<Cell>();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(2, results.size());
// Another range.
scanSpec = new Scan(Bytes.toBytes(r1));
@ -109,7 +110,7 @@ public class TestStoreScanner extends TestCase {
scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"),
scanners);
results = new ArrayList<Cell>();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(1, results.size());
// See how TimeRange and Versions interact.
// Another range.
@ -119,7 +120,7 @@ public class TestStoreScanner extends TestCase {
scan = new StoreScanner(scanSpec, scanInfo, scanType, getCols("a"),
scanners);
results = new ArrayList<Cell>();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(3, results.size());
}
@ -140,7 +141,7 @@ public class TestStoreScanner extends TestCase {
getCols("a"), scanners);
List<Cell> results = new ArrayList<Cell>();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(1, results.size());
assertEquals(kvs[0], results.get(0));
}
@ -195,7 +196,7 @@ public class TestStoreScanner extends TestCase {
getCols("a"), scanners);
List<Cell> results = new ArrayList<Cell>();
assertFalse(scan.next(results));
assertFalse(NextState.hasMoreValues(scan.next(results)));
assertEquals(0, results.size());
}
@ -215,14 +216,14 @@ public class TestStoreScanner extends TestCase {
getCols("a"), scanners);
List<Cell> results = new ArrayList<Cell>();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(0, results.size());
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(1, results.size());
assertEquals(kvs[2], results.get(0));
assertEquals(false, scan.next(results));
assertEquals(false, NextState.hasMoreValues(scan.next(results)));
}
public void testDeleteVersionMaskingMultiplePuts() throws IOException {
@ -244,7 +245,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, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(1, results.size());
assertEquals(kvs2[1], results.get(0));
}
@ -266,7 +267,7 @@ public class TestStoreScanner extends TestCase {
StoreScanner scan = new StoreScanner(scanSpec, scanInfo, scanType,
getCols("a"), scanners);
List<Cell> results = new ArrayList<Cell>();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(2, results.size());
assertEquals(kvs2[1], results.get(0));
assertEquals(kvs2[0], results.get(1));
@ -282,7 +283,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, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(2, results.size());
assertEquals(kvs[0], results.get(0));
assertEquals(kvs[1], results.get(1));
@ -311,7 +312,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, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(5, results.size());
assertEquals(kvs[0], results.get(0));
assertEquals(kvs[2], results.get(1));
@ -340,13 +341,13 @@ public class TestStoreScanner extends TestCase {
new Scan().setMaxVersions(Integer.MAX_VALUE), scanInfo, scanType, null,
scanners);
List<Cell> results = new ArrayList<Cell>();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(0, results.size());
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(1, results.size());
assertEquals(kvs[kvs.length-1], results.get(0));
assertEquals(false, scan.next(results));
assertEquals(false, NextState.hasMoreValues(scan.next(results)));
}
public void testDeleteColumn() throws IOException {
@ -360,7 +361,7 @@ public class TestStoreScanner extends TestCase {
StoreScanner scan = new StoreScanner(new Scan(), scanInfo, scanType, null,
scanners);
List<Cell> results = new ArrayList<Cell>();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(1, results.size());
assertEquals(kvs[3], results.get(0));
}
@ -384,18 +385,18 @@ public class TestStoreScanner extends TestCase {
getCols("a", "d"), scanners);
List<Cell> results = new ArrayList<Cell>();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(2, results.size());
assertEquals(kvs[0], results.get(0));
assertEquals(kvs[3], results.get(1));
results.clear();
assertEquals(true, scan.next(results));
assertEquals(true, NextState.hasMoreValues(scan.next(results)));
assertEquals(1, results.size());
assertEquals(kvs[kvs.length-1], results.get(0));
results.clear();
assertEquals(false, scan.next(results));
assertEquals(false, NextState.hasMoreValues(scan.next(results)));
}
/*
@ -425,20 +426,20 @@ public class TestStoreScanner extends TestCase {
null, scanners);
List<Cell> results = new ArrayList<Cell>();
assertEquals(true, scanner.next(results));
assertEquals(true, NextState.hasMoreValues(scanner.next(results)));
assertEquals(2, results.size());
assertEquals(kvs[1], results.get(0));
assertEquals(kvs[2], results.get(1));
results.clear();
assertEquals(true, scanner.next(results));
assertEquals(true, NextState.hasMoreValues(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, scanner.next(results));
assertEquals(false, NextState.hasMoreValues(scanner.next(results)));
}
public void testScannerReseekDoesntNPE() throws Exception {
@ -495,12 +496,12 @@ public class TestStoreScanner extends TestCase {
new StoreScanner(scan, scanInfo, scanType, null, scanners);
List<Cell> results = new ArrayList<Cell>();
assertEquals(true, scanner.next(results));
assertEquals(true, NextState.hasMoreValues(scanner.next(results)));
assertEquals(1, results.size());
assertEquals(kvs[1], results.get(0));
results.clear();
assertEquals(false, scanner.next(results));
assertEquals(false, NextState.hasMoreValues(scanner.next(results)));
}
public void testDeleteMarkerLongevity() throws Exception {
@ -560,7 +561,7 @@ public class TestStoreScanner extends TestCase {
HConstants.OLDEST_TIMESTAMP);
List<Cell> results = new ArrayList<Cell>();
results = new ArrayList<Cell>();
assertEquals(true, scanner.next(results));
assertEquals(true, NextState.hasMoreValues(scanner.next(results)));
assertEquals(kvs[0], results.get(0));
assertEquals(kvs[2], results.get(1));
assertEquals(kvs[3], results.get(2));

View File

@ -17,11 +17,20 @@
*/
package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.*;
import static org.mockito.Mockito.*;
import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_START_KEY;
import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_END_KEY;
import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_END_KEY;
import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_START_KEY;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.ArrayList;
@ -38,14 +47,14 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
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.io.compress.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -228,15 +237,26 @@ public class TestStripeCompactor {
}
@Override
public boolean next(List<Cell> results) throws IOException {
if (kvs.isEmpty()) return false;
public NextState next(List<Cell> results) throws IOException {
if (kvs.isEmpty()) return NextState.makeState(NextState.State.NO_MORE_VALUES);
results.add(kvs.remove(0));
return !kvs.isEmpty();
if (!kvs.isEmpty()) {
return NextState.makeState(NextState.State.MORE_VALUES);
} else {
return NextState.makeState(NextState.State.NO_MORE_VALUES);
}
}
@Override
public boolean next(List<Cell> result, int limit) throws IOException {
public NextState next(List<Cell> result, int limit) throws IOException {
return next(result);
}
@Override
public NextState next(List<Cell> result, int limit, long remainingResultSize)
throws IOException {
return next(result);
}
@Override
public void close() throws IOException {}
}

View File

@ -33,12 +33,13 @@ import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
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.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;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -105,7 +106,7 @@ public class TestWideScanner extends HBaseTestCase {
int i = 0;
boolean more;
do {
more = s.next(results);
more = NextState.hasMoreValues(s.next(results));
i++;
LOG.info("iteration #" + i + ", results.size=" + results.size());

View File

@ -49,8 +49,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.regionserver.BloomType;
@ -66,6 +64,8 @@ import org.apache.hadoop.hbase.regionserver.StripeStoreFileManager;
import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher;
import org.apache.hadoop.hbase.regionserver.TestStripeCompactor.StoreFileWritersCapture;
import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy.StripeInformationProvider;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConcatenatedLists;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -774,14 +774,25 @@ public class TestStripeCompactionPolicy {
}
@Override
public boolean next(List<Cell> results) throws IOException {
if (kvs.isEmpty()) return false;
public NextState next(List<Cell> results) throws IOException {
if (kvs.isEmpty()) return NextState.makeState(NextState.State.NO_MORE_VALUES);
results.add(kvs.remove(0));
return !kvs.isEmpty();
if (!kvs.isEmpty()) {
return NextState.makeState(NextState.State.MORE_VALUES);
} else {
return NextState.makeState(NextState.State.NO_MORE_VALUES);
}
}
@Override
public boolean next(List<Cell> result, int limit) throws IOException {
public NextState next(List<Cell> result, int limit) throws IOException {
return next(result);
}
@Override
public NextState next(List<Cell> result, int limit, long remainingResultSize)
throws IOException {
return next(result);
}

View File

@ -71,6 +71,7 @@ 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.RegionScanner;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@ -86,17 +87,17 @@ import org.apache.hadoop.hbase.util.HFileTestUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.Rule;
import org.junit.rules.TestName;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.mockito.Mockito;
/**
@ -749,7 +750,7 @@ public class TestWALReplay {
int scannedCount = 0;
List<Cell> results = new ArrayList<Cell>();
while (true) {
boolean existMore = scanner.next(results);
boolean existMore = NextState.hasMoreValues(scanner.next(results));
if (!results.isEmpty())
scannedCount++;
if (!existMore)

View File

@ -40,10 +40,11 @@ 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.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
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;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.util.ToolRunner;
import org.junit.After;
@ -231,7 +232,7 @@ public class TestMergeTool extends HBaseTestCase {
List<Cell> testRes = null;
while (true) {
testRes = new ArrayList<Cell>();
boolean hasNext = scanner.next(testRes);
boolean hasNext = NextState.hasMoreValues(scanner.next(testRes));
if (!hasNext) {
break;
}